tendermint/consensus/state.go

1158 lines
39 KiB
Go
Raw Normal View History

2014-10-30 03:32:09 -07:00
/*
Consensus State Machine Overview:
2015-06-05 14:15:40 -07:00
* NewHeight, NewRound, Propose, Prevote, Precommit represent state machine steps. (aka RoundStep).
* To "prevote/precommit" something means to broadcast a prevote/precommit vote for something.
* During NewHeight/NewRound/Propose/Prevote/Precommit:
2015-06-24 14:43:04 -07:00
* Nodes gossip the proposal block proposed by the designated proposer for that round.
2015-06-05 14:15:40 -07:00
* Nodes gossip prevotes/precommits for rounds [0...currentRound+1] (currentRound+1 for catch-up)
2015-06-24 14:43:04 -07:00
* Nodes also gossip prevotes for the proposal's POL (proof-of-lock) round if proposed.
2015-06-05 14:15:40 -07:00
* Upon each state transition, the height/round/step is broadcast to neighboring peers.
2015-06-24 14:43:04 -07:00
* The set of +2/3 of precommits at the same round for the same block is called a Commit, or Validation.
* A block contains the last block's Validation, which includes the Commit precommits.
While all the precommits in the Validation are from the same height & round (ordered by validator index),
some precommits may be nil (if the validator's precommit vote didn't reach the proposer in time),
or some precommits may be for different blockhashes for the last block hash (which is fine).
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* NewRound(height:H,round:R):
* Set up new round. --> goto Propose(H,R)
* NOTE: Not much happens in this step. It exists for clarity.
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* Propose(height:H,round:R):
2015-06-05 14:15:40 -07:00
* Upon entering Propose:
2015-06-24 14:43:04 -07:00
* The designated proposer proposes a block for (H,R).
2015-06-05 14:15:40 -07:00
* The Propose step ends:
2015-06-24 14:43:04 -07:00
* After `timeoutPropose` after entering Propose. --> goto Prevote(H,R)
* After receiving proposal block and all POL prevotes. --> goto Prevote(H,R)
* After any +2/3 prevotes received for (H,R+1). --> goto Prevote(H,R+1)
* After any +2/3 precommits received for (H,R+1). --> goto Precommit(H,R+1)
* After +2/3 precommits received for a particular block. --> goto Commit(H)
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* Prevote(height:H,round:R):
2015-06-05 14:15:40 -07:00
* Upon entering Prevote, each validator broadcasts its prevote vote.
* If the validator is locked on a block, it prevotes that.
2015-06-24 14:43:04 -07:00
* Else, if the proposed block from Propose(H,R) is good, it prevotes that.
2015-06-05 14:15:40 -07:00
* Else, if the proposal is invalid or wasn't received on time, it prevotes <nil>.
* The Prevote step ends:
2015-06-24 14:43:04 -07:00
* After +2/3 prevotes for a particular block or <nil>. --> goto Precommit(H,R)
* After `timeoutPrevote` after receiving any +2/3 prevotes. --> goto Precommit(H,R)
* After any +2/3 prevotes received for (H,R+1). --> goto Prevote(H,R+1)
* After any +2/3 precommits received for (H,R+1). --> goto Precommit(H,R+1)
* After +2/3 precommits received for a particular block. --> goto Commit(H)
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* Precommit(height:H,round:R):
2015-06-05 14:15:40 -07:00
* Upon entering Precommit, each validator broadcasts its precommit vote.
2015-06-24 14:43:04 -07:00
* If the validator had seen +2/3 of prevotes for a particular block from Prevote(H,R),
2015-06-05 14:15:40 -07:00
it locks (changes lock to) that block and precommits that block.
2015-06-24 14:43:04 -07:00
* Else, if the validator had seen +2/3 of prevotes for <nil>, it unlocks and precommits <nil>.
* Else, if +2/3 of prevotes for a particular block or <nil> is not received on time,
2015-06-05 14:15:40 -07:00
it precommits what it's locked on, or <nil>.
* The Precommit step ends:
2015-06-24 14:43:04 -07:00
* After +2/3 precommits for a particular block. --> goto Commit(H)
* After +2/3 precommits for <nil>. --> goto NewRound(H,R+1)
* After `timeoutPrecommit` after receiving any +2/3 precommits. --> goto NewRound(H,R+1)
* After any +2/3 prevotes received for (H,R+1). --> goto Prevote(H,R+1)
* After any +2/3 precommits received for (H,R+1). --> goto Precommit(H,R+1)
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* Commit(height:H):
2015-06-05 14:15:40 -07:00
* Set CommitTime = now
2015-06-24 14:43:04 -07:00
* Wait until block is received. --> goto NewHeight(H+1)
2015-06-05 14:15:40 -07:00
2015-06-24 14:43:04 -07:00
* NewHeight(height:H):
* Move Precommits to LastCommit and increment height.
* Set StartTime = CommitTime+timeoutCommit
* Wait until `StartTime` to receive straggler commits. --> goto NewRound(H,0)
2015-06-05 14:15:40 -07:00
* Proof of Safety:
If a good validator commits at round R, it's because it saw +2/3 of precommits for round R.
This implies that (assuming tolerance bounds) +1/3 of honest nodes are still locked at round R+1.
These locked validators will remain locked until they see +2/3 prevote for something
else, but this won't happen because +1/3 are locked and honest.
* Proof of Liveness:
Lemma 1: If +1/3 good nodes are locked on two different blocks, the proposers' POLRound will
eventually cause nodes locked from the earlier round to unlock.
-> `timeoutProposalR` increments with round R, while the block.size && POL prevote size
are fixed, so eventually we'll be able to "fully gossip" the block & POL.
TODO: cap the block.size at something reasonable.
Lemma 2: If a good node is at round R, neighboring good nodes will soon catch up to round R.
2014-10-30 03:32:09 -07:00
+-------------------------------------+
2015-06-05 14:15:40 -07:00
v |(Wait til `CommmitTime+timeoutCommit`)
2014-10-30 03:32:09 -07:00
+-----------+ +-----+-----+
+----------> | Propose +--------------+ | NewHeight |
| +-----------+ | +-----------+
| | ^
2015-06-05 14:15:40 -07:00
|(Else, after timeoutPrecommit) v |
2014-10-30 03:32:09 -07:00
+-----+-----+ +-----------+ |
| Precommit | <------------------------+ Prevote | |
+-----+-----+ +-----------+ |
2015-06-05 14:15:40 -07:00
|(When +2/3 Precommits for block found) |
v |
+--------------------------------------------------------------------+
| Commit |
| |
| * Set CommitTime = now; |
| * Wait for block, then stage/save/commit block; |
+--------------------------------------------------------------------+
2014-10-30 03:32:09 -07:00
*/
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-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/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
)
2015-04-13 21:43:13 -07:00
var (
2015-06-05 14:15:40 -07:00
timeoutPropose = 3000 * time.Millisecond // Maximum duration of RoundStepPropose
timeoutPrevote0 = 1000 * time.Millisecond // After any +2/3 prevotes received, wait this long for stragglers.
timeoutPrevoteDelta = 0500 * time.Millisecond // timeoutPrevoteN is timeoutPrevote0 + timeoutPrevoteDelta*N
timeoutPrecommit0 = 1000 * time.Millisecond // After any +2/3 precommits received, wait this long for stragglers.
timeoutPrecommitDelta = 0500 * time.Millisecond // timeoutPrecommitN is timeoutPrecommit0 + timeoutPrecommitDelta*N
timeoutCommit = 2000 * time.Millisecond // After +2/3 commits received for committed block, wait this long for stragglers in the next height's RoundStepNewHeight.
2014-12-30 17:14:54 -08:00
)
var (
ErrInvalidProposalSignature = errors.New("Error invalid proposal signature")
2015-06-22 19:04:31 -07:00
ErrInvalidProposalPOLRound = errors.New("Error invalid proposal POL round")
2014-12-30 17:14:54 -08:00
)
//-----------------------------------------------------------------------------
// RoundStepType enum type
2014-12-30 17:14:54 -08:00
type RoundStepType uint8 // These must be numeric, ordered.
2014-09-14 15:37:32 -07:00
const (
2015-06-05 14:15:40 -07:00
RoundStepNewHeight = RoundStepType(0x01) // Wait til CommitTime + timeoutCommit
RoundStepNewRound = RoundStepType(0x02) // Setup new round and go to RoundStepPropose
RoundStepPropose = RoundStepType(0x03) // Did propose, gossip proposal
RoundStepPrevote = RoundStepType(0x04) // Did prevote, gossip prevotes
RoundStepPrevoteWait = RoundStepType(0x05) // Did receive any +2/3 prevotes, start timeout
RoundStepPrecommit = RoundStepType(0x06) // Did precommit, gossip precommits
RoundStepPrecommitWait = RoundStepType(0x07) // Did receive any +2/3 precommits, start timeout
RoundStepCommit = RoundStepType(0x08) // Entered commit state machine
// NOTE: RoundStepNewHeight acts as RoundStepCommitWait.
2014-12-30 17:14:54 -08:00
)
func (rs RoundStepType) String() string {
2014-12-30 17:14:54 -08:00
switch rs {
case RoundStepNewHeight:
return "RoundStepNewHeight"
case RoundStepNewRound:
return "RoundStepNewRound"
case RoundStepPropose:
return "RoundStepPropose"
case RoundStepPrevote:
return "RoundStepPrevote"
2015-06-05 14:15:40 -07:00
case RoundStepPrevoteWait:
return "RoundStepPrevoteWait"
2014-12-30 17:14:54 -08:00
case RoundStepPrecommit:
return "RoundStepPrecommit"
2015-06-05 14:15:40 -07:00
case RoundStepPrecommitWait:
return "RoundStepPrecommitWait"
2014-12-30 17:14:54 -08:00
case RoundStepCommit:
return "RoundStepCommit"
default:
panic(Fmt("Unknown RoundStep %X", rs))
}
}
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
Step RoundStepType
2014-10-26 13:26:27 -07:00
StartTime time.Time
2015-06-05 14:15:40 -07:00
CommitTime time.Time // Subjective time when +2/3 precommits for Block at Round were found
Validators *sm.ValidatorSet
2014-10-26 13:26:27 -07:00
Proposal *Proposal
ProposalBlock *types.Block
ProposalBlockParts *types.PartSet
LockedBlock *types.Block
LockedBlockParts *types.PartSet
2015-06-05 14:15:40 -07:00
Votes *HeightVoteSet
2015-06-19 15:30:10 -07:00
LastCommit *VoteSet // Last precommits for Height-1
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 LockedBlock: %v %v
2015-06-05 14:15:40 -07:00
%s Votes: %v
2015-06-19 15:30:10 -07:00
%s LastCommit: %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.LockedBlockParts.StringShort(), rs.LockedBlock.StringShort(),
2015-06-05 14:15:40 -07:00
indent, rs.Votes.StringIndented(indent+" "),
2015-06-19 15:30:10 -07:00
indent, rs.LastCommit.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
privValidator *sm.PrivValidator
newStepCh chan *RoundState
2014-09-14 15:37:32 -07:00
2014-10-07 01:05:54 -07:00
mtx sync.Mutex
RoundState
2015-06-05 14:15:40 -07:00
state *sm.State // State until height-1.
stagedBlock *types.Block // Cache last staged block.
stagedState *sm.State // Cache result of staged block.
2015-04-15 23:40:27 -07:00
evsw events.Fireable
evc *events.EventCache // set in stageBlock and passed into state
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,
2015-06-05 14:15:40 -07:00
newStepCh: make(chan *RoundState, 10),
2014-09-14 15:37:32 -07:00
}
cs.updateToState(state, true)
2015-06-24 14:04:40 -07:00
// Don't call scheduleRound0 yet.
// We do that upon Start().
2015-06-05 14:15:40 -07:00
cs.maybeRebond()
2015-06-19 15:30:10 -07:00
cs.reconstructLastCommit(state)
return cs
2014-08-10 16:35:08 -07:00
}
2015-06-19 15:30:10 -07:00
// Reconstruct LastCommit from SeenValidation, which we saved along with the block,
// (which happens even before saving the state)
2015-06-19 15:30:10 -07:00
func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
if state.LastBlockHeight == 0 {
return
}
2015-06-05 14:15:40 -07:00
lastPrecommits := NewVoteSet(state.LastBlockHeight, 0, types.VoteTypePrecommit, state.LastBondedValidators)
seenValidation := cs.blockStore.LoadSeenValidation(state.LastBlockHeight)
2015-06-05 14:15:40 -07:00
for idx, precommit := range seenValidation.Precommits {
precommitVote := &types.Vote{
Height: state.LastBlockHeight,
2015-06-24 14:04:40 -07:00
Round: seenValidation.Round(),
2015-06-05 14:15:40 -07:00
Type: types.VoteTypePrecommit,
BlockHash: state.LastBlockHash,
BlockParts: state.LastBlockParts,
2015-06-05 14:15:40 -07:00
Signature: precommit.Signature,
}
2015-06-05 14:15:40 -07:00
added, _, err := lastPrecommits.AddByIndex(uint(idx), precommitVote)
if !added || err != nil {
2015-06-19 15:30:10 -07:00
panic(Fmt("Failed to reconstruct LastCommit: %v", err))
}
}
2015-06-05 14:15:40 -07:00
if !lastPrecommits.HasTwoThirdsMajority() {
2015-06-19 15:30:10 -07:00
panic("Failed to reconstruct LastCommit: Does not have +2/3 maj")
}
2015-06-19 15:30:10 -07:00
cs.LastCommit = lastPrecommits
}
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")
2015-06-24 14:04:40 -07:00
cs.scheduleRound0(cs.Height)
2014-10-30 03:32:09 -07:00
}
}
2015-06-05 14:15:40 -07:00
func (cs *ConsensusState) scheduleRound0(height uint) {
2015-06-24 14:04:40 -07:00
log.Debug("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
2015-06-05 14:15:40 -07:00
sleepDuration := cs.StartTime.Sub(time.Now())
go func() {
if sleepDuration > 0 {
time.Sleep(sleepDuration)
}
cs.EnterNewRound(height, 0)
}()
}
2014-10-30 03:32:09 -07:00
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
}
// Updates ConsensusState and increments height to match that of state.
2015-06-05 14:15:40 -07:00
// The round becomes 0 and cs.Step becomes RoundStepNewHeight.
func (cs *ConsensusState) updateToState(state *sm.State, contiguous bool) {
2015-06-05 14:15:40 -07:00
// SANITY CHECK
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
}
2015-06-05 14:15:40 -07:00
// END SANITY CHECK
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
2015-06-05 14:15:40 -07:00
lastPrecommits := (*VoteSet)(nil)
if contiguous && cs.Votes != nil {
lastPrecommits = cs.Votes.Precommits(cs.Round)
}
// 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() {
// "Now" makes it easier to sync up dev nodes.
2015-06-05 14:15:40 -07:00
// We add timeoutCommit to allow transactions
// to be gathered for the first block.
// And alternative solution that relies on clocks:
2015-06-05 14:15:40 -07:00
// cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
cs.StartTime = time.Now().Add(timeoutCommit)
} else {
2015-06-05 14:15:40 -07:00
cs.StartTime = cs.CommitTime.Add(timeoutCommit)
}
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.LockedBlock = nil
2014-10-26 13:26:27 -07:00
cs.LockedBlockParts = nil
2015-06-05 14:15:40 -07:00
cs.Votes = NewHeightVoteSet(height, validators)
2015-06-19 15:30:10 -07:00
cs.LastCommit = lastPrecommits
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
2015-06-19 15:30:10 -07:00
// Finally, broadcast RoundState
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
}
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// If we're unbonded, broadcast RebondTx.
func (cs *ConsensusState) maybeRebond() {
if cs.privValidator == nil || !cs.state.UnbondingValidators.HasAddress(cs.privValidator.Address) {
return
2014-09-14 15:37:32 -07:00
}
2015-06-05 14:15:40 -07:00
rebondTx := &types.RebondTx{
Address: cs.privValidator.Address,
Height: cs.Height,
}
err := cs.privValidator.SignRebondTx(cs.state.ChainID, rebondTx)
if err == nil {
err := cs.mempoolReactor.BroadcastTx(rebondTx)
if err != nil {
log.Error("Failed to broadcast RebondTx",
"height", cs.Height, "round", cs.Round, "tx", rebondTx, "error", err)
} else {
2015-06-05 14:15:40 -07:00
log.Info("Signed and broadcast RebondTx",
"height", cs.Height, "round", cs.Round, "tx", rebondTx)
}
2015-06-05 14:15:40 -07:00
} else {
log.Warn("Error signing RebondTx", "height", cs.Height, "round", cs.Round, "tx", rebondTx, "error", err)
}
2014-09-14 15:37:32 -07:00
}
func (cs *ConsensusState) SetPrivValidator(priv *sm.PrivValidator) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
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
2015-06-05 14:15:40 -07:00
// Enter: +2/3 precommits for nil from previous round
// Enter: `timeoutPrecommits` after any +2/3 precommits
// Enter: `commitTime+timeoutCommit` from NewHeight
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) EnterNewRound(height uint, round uint) {
2014-10-30 03:32:09 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-24 14:04:40 -07:00
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
2015-06-05 14:15:40 -07:00
log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
if now := time.Now(); cs.StartTime.After(now) {
log.Warn("Need to set a buffer and log.Warn() here for sanity.", "startTime", cs.StartTime, "now", now)
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
// Increment validators if necessary
2015-06-24 14:04:40 -07:00
validators := cs.Validators
2015-06-05 14:15:40 -07:00
if cs.Round < round {
2015-06-24 14:04:40 -07:00
validators = validators.Copy()
2015-06-05 14:15:40 -07:00
validators.IncrementAccum(round - cs.Round)
}
// Setup new round
cs.Round = round
cs.Step = RoundStepNewRound
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
cs.ProposalBlockParts = nil
cs.Votes.SetRound(round + 1) // track next round.
// Immediately go to EnterPropose.
go cs.EnterPropose(height, round)
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
// Enter: from NewRound.
func (cs *ConsensusState) EnterPropose(height uint, round uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Round > round || (cs.Round == round && cs.Step >= RoundStepPropose) {
log.Debug(Fmt("EnterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2014-10-21 01:18:46 -07:00
return
}
2015-06-05 14:15:40 -07:00
defer func() {
2015-06-05 14:15:40 -07:00
// Done EnterPropose:
cs.Round = round
cs.Step = RoundStepPropose
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
// If we already have the proposal + POL, then goto Prevote
if cs.isProposalComplete() {
go cs.EnterPrevote(height, round)
}
}()
// This step times out after `timeoutPropose`
go func() {
time.Sleep(timeoutPropose)
cs.EnterPrevote(height, round)
}()
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
}
2015-06-05 14:15:40 -07:00
// See if it is our turn to propose
if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.Address) {
2015-06-05 14:15:40 -07:00
log.Debug("EnterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
2014-08-10 16:35:08 -07:00
return
} else {
2015-06-05 14:15:40 -07:00
log.Debug("EnterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
2014-08-10 16:35:08 -07:00
}
2015-06-05 14:15:40 -07:00
// We are going to propose a block.
var block *types.Block
var blockParts *types.PartSet
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// Decide on block
2014-09-14 15:37:32 -07:00
if cs.LockedBlock != nil {
// If we're locked onto a block, just choose that.
2015-06-05 14:15:40 -07:00
block, blockParts = cs.LockedBlock, cs.LockedBlockParts
2014-08-10 16:35:08 -07:00
} else {
2015-06-05 14:15:40 -07:00
// Create a new proposal block from state/txs from the mempool.
block, blockParts = cs.createProposalBlock()
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Make proposal
2015-06-05 14:15:40 -07:00
proposal := NewProposal(cs.Height, cs.Round, blockParts.Header(), cs.Votes.POLRound())
2015-05-29 14:53:57 -07:00
err := cs.privValidator.SignProposal(cs.state.ChainID, 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
} else {
2015-06-05 14:15:40 -07:00
log.Warn("EnterPropose: Error signing proposal", "height", cs.Height, "round", cs.Round, "error", err)
}
2015-06-24 14:04:40 -07:00
2015-06-05 14:15:40 -07:00
}
func (cs *ConsensusState) isProposalComplete() bool {
if cs.Proposal == nil || cs.ProposalBlock == nil {
return false
}
2015-06-24 14:04:40 -07:00
if cs.Proposal.POLRound < 0 {
return true
} else {
return cs.Votes.Prevotes(uint(cs.Proposal.POLRound)).HasTwoThirdsMajority()
}
2015-06-05 14:15:40 -07:00
}
// Create the next block to propose and return it.
// NOTE: make it side-effect free for clarity.
2015-06-24 14:04:40 -07:00
func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts *types.PartSet) {
2015-06-05 14:15:40 -07:00
var validation *types.Validation
if cs.Height == 1 {
// We're creating a proposal for the first block.
2015-06-19 15:30:10 -07:00
// The validation is empty, but not nil.
2015-06-05 14:15:40 -07:00
validation = &types.Validation{}
2015-06-19 15:30:10 -07:00
} else if cs.LastCommit.HasTwoThirdsMajority() {
// Make the validation from LastCommit
validation = cs.LastCommit.MakeValidation()
2015-06-05 14:15:40 -07:00
} else {
// This shouldn't happen.
log.Error("EnterPropose: Cannot propose anything: No validation for the previous block.")
return
}
txs := cs.mempoolReactor.Mempool.GetProposalTxs()
block = &types.Block{
Header: &types.Header{
ChainID: cs.state.ChainID,
Height: cs.Height,
Time: time.Now(),
Fees: 0, // TODO fees
NumTxs: uint(len(txs)),
LastBlockHash: cs.state.LastBlockHash,
LastBlockParts: cs.state.LastBlockParts,
StateHash: nil, // Will set afterwards.
},
2015-06-24 14:04:40 -07:00
LastValidation: validation,
2015-06-05 14:15:40 -07:00
Data: &types.Data{
Txs: txs,
},
}
// Set the block.Header.StateHash.
err := cs.state.ComputeBlockStateHash(block)
if err != nil {
log.Error("EnterPropose: Error setting state hash", "error", err)
return
}
2015-06-05 14:15:40 -07:00
blockParts = block.MakePartSet()
return block, blockParts
2014-08-10 16:35:08 -07:00
}
2015-06-05 14:15:40 -07:00
// Enter: `timeoutPropose` after start of Propose.
// Enter: proposal block and POL is ready.
// Enter: any +2/3 prevotes for next round.
2014-10-30 03:32:09 -07:00
// Prevote for LockedBlock if we're locked, or ProposealBlock if valid.
// Otherwise vote nil.
2015-06-05 14:15:40 -07:00
func (cs *ConsensusState) EnterPrevote(height uint, round uint) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Round > round || (cs.Round == round && cs.Step >= RoundStepPrevote) {
log.Debug(Fmt("EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
2014-10-21 01:18:46 -07:00
}
2015-06-05 14:15:40 -07:00
defer func() {
2015-06-05 14:15:40 -07:00
// Done EnterPrevote:
cs.Round = round
cs.Step = RoundStepPrevote
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
// Maybe immediately go to EnterPrevoteWait.
if cs.Votes.Prevotes(round).HasTwoThirdsAny() {
go cs.EnterPrevoteWait(height, round)
}
}()
2014-10-21 01:18:46 -07:00
2015-06-05 14:15:40 -07:00
// Sign and broadcast vote as necessary
cs.doPrevote(height, round)
}
func (cs *ConsensusState) doPrevote(height uint, round uint) {
2014-10-21 01:18:46 -07:00
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
2015-06-05 14:15:40 -07:00
log.Debug("EnterPrevote: Block was locked")
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 {
2015-06-05 14:15:40 -07:00
log.Warn("EnterPrevote: 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.
2015-06-05 14:15:40 -07:00
log.Warn("EnterPrevote: 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
}
2015-06-05 14:15:40 -07:00
// Enter: any +2/3 prevotes for next round.
func (cs *ConsensusState) EnterPrevoteWait(height uint, round uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round > round || (cs.Round == round && cs.Step >= RoundStepPrevoteWait) {
log.Debug(Fmt("EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
panic(Fmt("EnterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
}
// Done EnterPrevoteWait:
cs.Round = round
cs.Step = RoundStepPrevoteWait
cs.newStepCh <- cs.getRoundState()
2015-06-24 14:04:40 -07:00
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
2015-06-05 14:15:40 -07:00
go func() {
2015-06-24 14:04:40 -07:00
time.Sleep(timeoutPrevote0 + timeoutPrevote0*time.Duration(round))
2015-06-05 14:15:40 -07:00
cs.EnterPrecommit(height, round)
}()
}
// Enter: +2/3 precomits for block or nil.
// Enter: `timeoutPrevote` after any +2/3 prevotes.
// Enter: any +2/3 precommits for next round.
// Lock & precommit the ProposalBlock if we have enough prevotes for it,
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
// else, precommit locked block or nil otherwise.
func (cs *ConsensusState) EnterPrecommit(height uint, round uint) {
2014-09-14 15:37:32 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Round > round || (cs.Round == round && cs.Step >= RoundStepPrecommit) {
log.Debug(Fmt("EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
2014-09-14 15:37:32 -07:00
}
2015-06-05 14:15:40 -07:00
defer func() {
2015-06-05 14:15:40 -07:00
// Done EnterPrecommit:
cs.Round = round
cs.Step = RoundStepPrecommit
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
// Maybe immediately go to EnterPrecommitWait.
if cs.Votes.Precommits(round).HasTwoThirdsAny() {
go cs.EnterPrecommitWait(height, round)
}
}()
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
hash, partsHeader, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
// If we don't have two thirds of prevotes, just precommit locked block or nil
2014-10-30 03:32:09 -07:00
if !ok {
2015-06-05 14:15:40 -07:00
if cs.LockedBlock != nil {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting lock.")
cs.signAddVote(types.VoteTypePrecommit, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
} else {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting nil.")
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
}
2014-10-30 03:32:09 -07:00
return
2014-09-14 15:37:32 -07:00
}
2014-08-10 16:35:08 -07:00
2015-06-05 14:15:40 -07:00
// +2/3 prevoted nil. Unlock and precommit nil.
2014-10-30 03:32:09 -07:00
if len(hash) == 0 {
2015-05-04 10:15:58 -07:00
if cs.LockedBlock == nil {
2015-06-05 14:15:40 -07:00
log.Info("EnterPrecommit: +2/3 prevoted for nil.")
2015-05-04 10:15:58 -07:00
} else {
2015-06-05 14:15:40 -07:00
log.Info("EnterPrecommit: +2/3 prevoted for nil. Unlocking")
2015-05-04 10:15:58 -07:00
cs.LockedBlock = nil
cs.LockedBlockParts = nil
}
2015-06-05 14:15:40 -07:00
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
2014-10-30 03:32:09 -07:00
return
}
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// At this point, +2/3 prevoted for a particular block.
2014-10-30 03:32:09 -07:00
// If +2/3 prevoted for already locked block, precommit it.
if cs.LockedBlock.HashesTo(hash) {
2015-06-05 14:15:40 -07:00
log.Info("EnterPrecommit: +2/3 prevoted locked block.")
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
2014-10-30 03:32:09 -07:00
return
}
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// If +2/3 prevoted for proposal block, stage and precommit it
if cs.ProposalBlock.HashesTo(hash) {
log.Info("EnterPrecommit: +2/3 prevoted proposal block.")
// Validate the block.
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
panic(Fmt("EnterPrecommit: +2/3 prevoted for an invalid block: %v", err))
}
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
}
2015-06-05 14:15:40 -07:00
// Otherwise, we need to fetch the +2/3 prevoted block.
// We don't have the block yet so we can't lock/precommit it.
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if !cs.ProposalBlockParts.HasHeader(partsHeader) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
}
2015-06-24 14:04:40 -07:00
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
2015-06-05 14:15:40 -07:00
return
}
// Enter: any +2/3 precommits for next round.
func (cs *ConsensusState) EnterPrecommitWait(height uint, round uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || cs.Round > round || (cs.Round == round && cs.Step >= RoundStepPrecommitWait) {
log.Debug(Fmt("EnterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-05-04 10:15:58 -07:00
return
}
2015-06-05 14:15:40 -07:00
if !cs.Votes.Precommits(round).HasTwoThirdsAny() {
panic(Fmt("EnterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
}
2015-05-04 10:15:58 -07:00
2015-06-05 14:15:40 -07:00
// Done EnterPrecommitWait:
cs.Round = round
cs.Step = RoundStepPrecommitWait
cs.newStepCh <- cs.getRoundState()
2015-06-24 14:04:40 -07:00
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
2015-06-05 14:15:40 -07:00
go func() {
2015-06-24 14:04:40 -07:00
time.Sleep(timeoutPrecommit0 + timeoutPrecommitDelta*time.Duration(round))
2015-06-05 14:15:40 -07:00
// If we have +2/3 of precommits for a particular block (or nil),
// we already entered commit (or the next round).
// So just try to transition to the next round,
// which is what we'd do otherwise.
cs.EnterNewRound(height, round+1)
}()
2014-08-10 16:35:08 -07:00
}
2015-06-05 14:15:40 -07:00
// Enter: +2/3 precommits for block
func (cs *ConsensusState) EnterCommit(height uint) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Step >= RoundStepCommit {
log.Debug(Fmt("EnterCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
return
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
defer func() {
2015-06-05 14:15:40 -07:00
// Done Entercommit:
// keep ca.Round the same, it points to the right Precommits set.
cs.Step = RoundStepCommit
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
// Maybe finalize immediately.
2015-06-24 14:04:40 -07:00
cs.tryFinalizeCommit(height)
}()
2015-06-05 14:15:40 -07:00
// SANITY CHECK
2015-06-24 14:04:40 -07:00
hash, partsHeader, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
2014-10-30 03:32:09 -07:00
if !ok {
2015-06-05 14:15:40 -07:00
panic("RunActionCommit() expects +2/3 precommits")
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
// END SANITY CHECK
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
// The Locked* fields no longer matter.
// Move them over to ProposalBlock if they match the commit hash,
// otherwise they can now be cleared.
2014-10-30 03:32:09 -07:00
if cs.LockedBlock.HashesTo(hash) {
cs.ProposalBlock = cs.LockedBlock
cs.ProposalBlockParts = cs.LockedBlockParts
cs.LockedBlock = nil
cs.LockedBlockParts = nil
2015-06-05 14:15:40 -07:00
} else {
cs.LockedBlock = nil
cs.LockedBlockParts = nil
2014-10-30 03:32:09 -07:00
}
// 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.
}
}
2015-06-05 14:15:40 -07:00
}
2015-06-05 14:15:40 -07:00
// If we have the block AND +2/3 commits for it, finalize.
2015-06-24 14:04:40 -07:00
func (cs *ConsensusState) tryFinalizeCommit(height uint) {
// SANITY CHECK
if cs.Height != height {
panic(Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
}
2015-06-24 14:04:40 -07:00
// END SANITY CHECK
hash, _, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
if !ok || len(hash) == 0 {
return
}
if !cs.ProposalBlock.HashesTo(hash) {
return
}
go cs.FinalizeCommit(height)
}
2015-06-05 14:15:40 -07:00
// Increment height and goto RoundStepNewHeight
func (cs *ConsensusState) FinalizeCommit(height uint) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Step != RoundStepCommit {
log.Debug(Fmt("FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
return
2014-10-21 01:18:46 -07:00
}
2015-06-24 14:04:40 -07:00
hash, header, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority()
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
// SANITY CHECK
if !ok {
panic(Fmt("Cannot FinalizeCommit, commit does not have two thirds majority"))
}
if !cs.ProposalBlockParts.HasHeader(header) {
panic(Fmt("Expected ProposalBlockParts header to be commit header"))
}
if !cs.ProposalBlock.HashesTo(hash) {
panic(Fmt("Cannot FinalizeCommit, ProposalBlock does not hash to commit hash"))
}
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
panic(Fmt("+2/3 committed an invalid block: %v", err))
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
// END SANITY CHECK
log.Debug(Fmt("Finalizing commit of block: %v", cs.ProposalBlock))
// We have the block, so stage/save/commit-vote.
2015-06-24 14:04:40 -07:00
cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Votes.Precommits(cs.Round))
2015-06-05 14:15:40 -07:00
// Increment height.
cs.updateToState(cs.stagedState, true)
2015-06-24 14:04:40 -07:00
// cs.StartTime is already set.
// Schedule Round0 to start soon.
go cs.scheduleRound0(height + 1)
2015-06-05 14:15:40 -07:00
// If we're unbonded, broadcast RebondTx.
cs.maybeRebond()
// By here,
// * cs.Height has been increment to height+1
// * cs.Step is now RoundStepNewHeight
2015-06-24 14:04:40 -07:00
// * cs.StartTime is set to when we will start round0.
2015-06-05 14:15:40 -07:00
return
}
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.
2015-06-05 14:15:40 -07:00
if cs.Step >= RoundStepCommit {
2014-10-30 03:32:09 -07:00
return nil
}
2015-06-22 19:04:31 -07:00
// Verify POLRound, which must be -1 or between 0 and proposal.Round exclusive.
if proposal.POLRound != -1 &&
2015-06-24 14:04:40 -07:00
(proposal.POLRound < 0 || proposal.Round <= uint(proposal.POLRound)) {
2015-06-22 19:04:31 -07:00
return ErrInvalidProposalPOLRound
}
2014-10-21 23:30:18 -07:00
// Verify signature
2015-05-29 14:53:57 -07:00
if !cs.Validators.Proposer().PubKey.VerifyBytes(account.SignBytes(cs.state.ChainID, proposal), proposal.Signature) {
2014-10-21 23:30:18 -07:00
return ErrInvalidProposalSignature
}
cs.Proposal = proposal
2015-06-24 14:04:40 -07:00
cs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockPartsHeader)
2014-10-21 23:30:18 -07:00
return nil
}
// NOTE: block is not necessarily valid.
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)
2015-05-04 10:15:58 -07:00
log.Debug("Received complete proposal", "hash", cs.ProposalBlock.Hash())
2015-06-12 20:24:08 -07:00
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
go cs.EnterPrevote(height, round)
2015-06-05 14:15:40 -07:00
} else if cs.Step == RoundStepCommit {
2015-06-24 14:04:40 -07:00
cs.tryFinalizeCommit(height)
}
2014-10-21 23:30:18 -07:00
return true, err
}
return true, nil
}
2015-06-24 14:04:40 -07:00
func (cs *ConsensusState) AddVote(address []byte, vote *types.Vote, peerKey string) (added bool, index uint, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-06-24 14:04:40 -07:00
return cs.addVote(address, vote, peerKey)
}
//-----------------------------------------------------------------------------
2015-06-24 14:04:40 -07:00
func (cs *ConsensusState) addVote(address []byte, vote *types.Vote, peerKey string) (added bool, index uint, err error) {
2015-06-05 14:15:40 -07:00
// A precommit for the previous height?
if vote.Height+1 == cs.Height && vote.Type == types.VoteTypePrecommit {
2015-06-19 15:30:10 -07:00
added, index, err = cs.LastCommit.AddByAddress(address, vote)
2015-05-04 11:18:21 -07:00
if added {
2015-06-19 15:30:10 -07:00
log.Debug(Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
2015-05-04 11:18:21 -07:00
}
return
2015-06-05 14:15:40 -07:00
}
// A prevote/precommit for this height?
if vote.Height == cs.Height {
2015-06-24 14:04:40 -07:00
height := cs.Height
added, index, err = cs.Votes.AddByAddress(address, vote, peerKey)
2015-05-04 11:18:21 -07:00
if added {
2015-06-05 14:15:40 -07:00
switch vote.Type {
case types.VoteTypePrevote:
log.Debug(Fmt("Added to prevotes: %v", cs.Votes.Prevotes(vote.Round).StringShort()))
if cs.Round < vote.Round && cs.Votes.Prevotes(vote.Round).HasTwoThirdsAny() {
// Goto to Prevote vote.Round.
2015-06-05 14:15:40 -07:00
go func() {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrevote(height, vote.Round)
2015-06-12 20:24:08 -07:00
cs.EnterPrevoteWait(height, vote.Round)
2015-06-05 14:15:40 -07:00
}()
2015-06-12 20:24:08 -07:00
} else if cs.Round == vote.Round {
2015-06-05 14:15:40 -07:00
if cs.Votes.Prevotes(cs.Round).HasTwoThirdsMajority() {
// Goto Precommit, whether for block or nil.
2015-06-12 20:24:08 -07:00
go cs.EnterPrecommit(height, cs.Round)
} else if cs.Votes.Prevotes(cs.Round).HasTwoThirdsAny() {
2015-06-05 14:15:40 -07:00
// Goto PrevoteWait
go func() {
2015-06-12 20:24:08 -07:00
cs.EnterPrevote(height, cs.Round)
2015-06-05 14:15:40 -07:00
cs.EnterPrevoteWait(height, cs.Round)
}()
}
2015-06-24 14:04:40 -07:00
} else if cs.Proposal != nil && cs.Proposal.POLRound >= 0 && uint(cs.Proposal.POLRound) == vote.Round {
2015-06-12 20:24:08 -07:00
if cs.isProposalComplete() {
go cs.EnterPrevote(height, cs.Round)
}
2015-06-05 14:15:40 -07:00
}
case types.VoteTypePrecommit:
log.Debug(Fmt("Added to precommit: %v", cs.Votes.Precommits(vote.Round).StringShort()))
if cs.Round < vote.Round {
if hash, _, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority(); ok {
if len(hash) == 0 {
// This is weird, shouldn't happen
log.Warn("This is weird, why did we receive +2/3 of nil precommits?")
// Skip to Precommit of vote.Round
go func() {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
cs.EnterPrecommitWait(height, vote.Round)
}()
} else {
// If hash is block, goto Commit
go func() {
cs.EnterNewRound(height, vote.Round)
2015-06-24 14:04:40 -07:00
cs.EnterCommit(height)
}()
}
} else if cs.Votes.Precommits(vote.Round).HasTwoThirdsAny() {
// Skip to Precommit of vote.Round
go func() {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
cs.EnterPrecommitWait(height, vote.Round)
}()
}
2015-06-12 20:24:08 -07:00
} else if cs.Round == vote.Round {
2015-06-05 14:15:40 -07:00
if hash, _, ok := cs.Votes.Precommits(cs.Round).TwoThirdsMajority(); ok {
if len(hash) == 0 {
// If hash is nil, goto NewRound
2015-06-12 20:24:08 -07:00
go cs.EnterNewRound(height, cs.Round+1)
2015-06-05 14:15:40 -07:00
} else {
// If hash is block, goto Commit
2015-06-24 14:04:40 -07:00
go cs.EnterCommit(height)
2015-06-05 14:15:40 -07:00
}
2015-06-12 20:24:08 -07:00
} else if cs.Votes.Precommits(cs.Round).HasTwoThirdsAny() {
2015-06-05 14:15:40 -07:00
// Goto PrecommitWait
go func() {
2015-06-12 20:24:08 -07:00
cs.EnterPrecommit(height, cs.Round)
2015-06-05 14:15:40 -07:00
cs.EnterPrecommitWait(height, cs.Round)
}()
}
}
default:
panic(Fmt("Unexpected vote type %X", vote.Type)) // Should not happen.
}
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
return
2014-10-21 23:30:18 -07:00
}
2015-06-05 14:15:40 -07:00
// Height mismatch, bad peer? TODO
return
2014-10-21 23:30:18 -07:00
}
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()
2015-04-15 23:40:27 -07:00
// reset the event cache and pass it into the state
cs.evc = events.NewEventCache(cs.evsw)
stateCopy.SetFireable(cs.evc)
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,
}
2015-05-29 14:53:57 -07:00
err := cs.privValidator.SignVote(cs.state.ChainID, vote)
if err == nil {
2015-06-24 14:04:40 -07:00
_, _, err := cs.addVote(cs.privValidator.Address, vote, "")
log.Info("Signed and added vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
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
2015-06-05 14:15:40 -07:00
// Save Block, save the +2/3 Commits we've seen
func (cs *ConsensusState) saveBlock(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 {
2015-06-05 14:15:40 -07:00
panic(Fmt("saveBlock() an invalid block: %v", err))
2014-10-30 03:32:09 -07:00
}
// 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
2015-06-24 14:04:40 -07:00
// Fire off event
go func(block *types.Block) {
cs.evsw.FireEvent(types.EventStringNewBlock(), block)
cs.evc.Flush()
}(block)
2014-10-30 03:32:09 -07:00
}
// implements events.Eventable
2015-04-15 23:40:27 -07:00
func (cs *ConsensusState) SetFireable(evsw events.Fireable) {
cs.evsw = evsw
}