tendermint/consensus/state.go

1153 lines
38 KiB
Go
Raw Normal View History

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"
"time"
. "github.com/tendermint/go-common"
2015-11-01 11:34:08 -08:00
"github.com/tendermint/go-wire"
bc "github.com/tendermint/tendermint/blockchain"
"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")
2015-08-12 11:00:23 -07:00
ErrAddingVote = errors.New("Error adding vote")
2015-08-26 15:56:34 -07:00
ErrVoteHeightMismatch = errors.New("Error vote height mismatch")
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:
return "RoundStepUnknown" // Cannot panic.
2014-12-30 17:14:54 -08:00
}
}
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 int // Height we are working on
Round int
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 *types.ValidatorSet
Proposal *types.Proposal
ProposalBlock *types.Block
ProposalBlockParts *types.PartSet
LockedRound int
LockedBlock *types.Block
LockedBlockParts *types.PartSet
2015-06-05 14:15:40 -07:00
Votes *HeightVoteSet
2015-09-15 13:13:39 -07:00
CommitRound int //
LastCommit *types.VoteSet // Last precommits at Height-1
LastValidators *types.ValidatorSet
2014-09-14 15:37:32 -07:00
}
2015-09-09 13:45:53 -07:00
func (rs *RoundState) RoundStateEvent() *types.EventDataRoundState {
return &types.EventDataRoundState{
CurrentTime: time.Now(),
Height: rs.Height,
Round: rs.Round,
Step: rs.Step.String(),
StartTime: rs.StartTime,
CommitTime: rs.CommitTime,
Proposal: rs.Proposal,
ProposalBlock: rs.ProposalBlock,
LockedRound: rs.LockedRound,
LockedBlock: rs.LockedBlock,
POLRound: rs.Votes.POLRound(),
}
}
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
2015-06-24 18:51:14 -07:00
%s LockedRound: %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
%s LastValidators: %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(),
2015-06-24 18:51:14 -07:00
indent, rs.LockedRound,
2014-12-23 01:35:54 -08:00
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(),
indent, rs.LastValidators.StringIndented(indent+" "),
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 {
BaseService
2014-10-30 03:32:09 -07:00
blockStore *bc.BlockStore
mempoolReactor *mempl.MempoolReactor
privValidator *types.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{
2015-09-09 13:45:53 -07:00
blockStore: blockStore,
mempoolReactor: mempoolReactor,
newStepCh: make(chan *RoundState, 10),
2014-09-14 15:37:32 -07:00
}
2015-09-15 13:13:39 -07:00
cs.updateToState(state)
2015-06-24 14:04:40 -07:00
// Don't call scheduleRound0 yet.
// We do that upon Start().
2015-06-19 15:30:10 -07:00
cs.reconstructLastCommit(state)
cs.BaseService = *NewBaseService(log, "ConsensusState", cs)
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-11-01 11:34:08 -08:00
lastPrecommits := types.NewVoteSet(state.LastBlockHeight, 0, types.VoteTypePrecommit, state.LastValidators)
seenValidation := cs.blockStore.LoadSeenValidation(state.LastBlockHeight)
2015-06-05 14:15:40 -07:00
for idx, precommit := range seenValidation.Precommits {
2015-07-09 21:46:15 -07:00
if precommit == nil {
continue
}
added, _, err := lastPrecommits.AddByIndex(idx, precommit)
if !added || err != nil {
2015-07-19 16:42:52 -07:00
PanicCrisis(Fmt("Failed to reconstruct LastCommit: %v", err))
}
}
2015-06-05 14:15:40 -07:00
if !lastPrecommits.HasTwoThirdsMajority() {
2015-07-19 16:42:52 -07:00
PanicSanity("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) OnStart() error {
2015-07-21 18:31:01 -07:00
cs.BaseService.OnStart()
cs.scheduleRound0(cs.Height)
return nil
}
2015-07-21 18:31:01 -07:00
func (cs *ConsensusState) OnStop() {
2015-09-09 13:45:53 -07:00
// It's asynchronous so, there's not much to stop.
2015-07-21 18:31:01 -07:00
cs.BaseService.OnStop()
2014-10-30 03:32:09 -07:00
}
// EnterNewRound(height, 0) at cs.StartTime.
func (cs *ConsensusState) scheduleRound0(height int) {
2015-07-19 14:49:13 -07:00
//log.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
2015-06-05 14:15:40 -07:00
sleepDuration := cs.StartTime.Sub(time.Now())
go func() {
if 0 < sleepDuration {
2015-06-05 14:15:40 -07:00
time.Sleep(sleepDuration)
2015-09-09 13:45:53 -07:00
// TODO: event?
2015-06-05 14:15:40 -07:00
}
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, 0, false)
2015-06-05 14:15:40 -07:00
}()
}
2014-10-30 03:32:09 -07:00
// 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.
2015-09-15 13:13:39 -07:00
func (cs *ConsensusState) updateToState(state *sm.State) {
if cs.CommitRound > -1 && 0 < cs.Height && cs.Height != state.LastBlockHeight {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("updateToState() expected state height of %v but found %v",
2014-12-29 18:09:06 -08:00
cs.Height, state.LastBlockHeight))
2014-09-14 15:37:32 -07:00
}
2015-07-09 21:46:15 -07:00
if cs.state != nil && cs.state.LastBlockHeight+1 != cs.Height {
// This might happen when someone else is mutating cs.state.
// Someone forgot to pass in state.Copy() somewhere?!
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("Inconsistent cs.state.LastBlockHeight+1 %v vs cs.Height %v",
2015-07-09 21:46:15 -07:00
cs.state.LastBlockHeight+1, cs.Height))
}
2014-09-14 15:37:32 -07:00
// If state isn't further out than cs.state, just ignore.
// This happens when SwitchToConsensus() is called in the reactor.
// We don't want to reset e.g. the Votes.
if cs.state != nil && (state.LastBlockHeight <= cs.state.LastBlockHeight) {
2015-07-19 14:49:13 -07:00
log.Notice("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
return
}
2014-09-14 15:37:32 -07:00
// Reset fields based on state.
2015-11-01 11:34:08 -08:00
validators := state.Validators
height := state.LastBlockHeight + 1 // next desired block height
lastPrecommits := (*types.VoteSet)(nil)
2015-09-15 13:13:39 -07:00
if cs.CommitRound > -1 && cs.Votes != nil {
if !cs.Votes.Precommits(cs.CommitRound).HasTwoThirdsMajority() {
PanicSanity("updateToState(state) called but last Precommit round didn't have +2/3")
}
2015-09-15 13:13:39 -07:00
lastPrecommits = cs.Votes.Precommits(cs.CommitRound)
2015-06-05 14:15:40 -07:00
}
// 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
2015-06-24 18:51:14 -07:00
cs.LockedRound = 0
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-09-15 13:13:39 -07:00
cs.CommitRound = -1
2015-06-19 15:30:10 -07:00
cs.LastCommit = lastPrecommits
2015-11-01 11:34:08 -08:00
cs.LastValidators = state.LastValidators
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
func (cs *ConsensusState) SetPrivValidator(priv *types.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-24 18:51:14 -07:00
// Enter: +2/3 precommits for nil at (height,round-1)
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
// Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
2015-06-05 14:15:40 -07:00
// NOTE: cs.StartTime was already set for height.
2015-09-09 13:45:53 -07:00
func (cs *ConsensusState) EnterNewRound(height int, round int, timedOut bool) {
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-08-26 15:56:34 -07:00
log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
2014-10-30 03:32:09 -07:00
}
2015-09-09 13:45:53 -07:00
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
}
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-07-19 14:49:13 -07:00
log.Notice(Fmt("EnterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
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
if round == 0 {
// We've already reset these upon new height,
// and meanwhile we might have received a proposal
// for round 0.
} else {
cs.Proposal = nil
cs.ProposalBlock = nil
cs.ProposalBlockParts = nil
}
cs.Votes.SetRound(round + 1) // also track next round (round+1) to allow round-skipping
2015-06-05 14:15:40 -07:00
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringNewRound(), cs.RoundStateEvent())
2015-06-05 14:15:40 -07:00
// Immediately go to EnterPropose.
go cs.EnterPropose(height, round)
2014-10-30 03:32:09 -07:00
}
// Enter: from NewRound(height,round).
func (cs *ConsensusState) EnterPropose(height int, round int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
2015-08-26 15:56:34 -07:00
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-07-19 14:49:13 -07:00
log.Info(Fmt("EnterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
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
2015-08-26 15:56:34 -07:00
cs.newStepCh <- cs.getRoundState()
// If we have the whole proposal + POL, then goto Prevote now.
// else, we'll EnterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
// or else after timeoutPropose
if cs.isProposalComplete() {
2015-09-09 13:45:53 -07:00
go cs.EnterPrevote(height, cs.Round, false)
2015-08-26 15:56:34 -07:00
}
2015-06-05 14:15:40 -07:00
}()
2015-08-26 15:56:34 -07:00
// This step times out after `timeoutPropose`
2015-06-05 14:15:40 -07:00
go func() {
2015-08-26 15:56:34 -07:00
time.Sleep(timeoutPropose)
2015-09-09 13:45:53 -07:00
cs.EnterPrevote(height, round, true)
}()
2014-09-14 15:37:32 -07:00
// Nothing more to do if we're not a validator
if cs.privValidator == nil {
return
}
2015-06-05 14:15:40 -07:00
if !bytes.Equal(cs.Validators.Proposer().Address, cs.privValidator.Address) {
2015-07-19 14:49:13 -07:00
log.Info("EnterPropose: Not our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
} else {
2015-07-19 14:49:13 -07:00
log.Info("EnterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
2015-08-26 15:56:34 -07:00
cs.decideProposal(height, round)
2014-08-10 16:35:08 -07:00
}
}
2014-08-10 16:35:08 -07:00
// Decides on the next proposal and sets them onto cs.Proposal*
func (cs *ConsensusState) decideProposal(height int, round int) {
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
proposal := types.NewProposal(height, 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 {
2015-07-19 14:49:13 -07:00
log.Notice("Signed and set proposal", "height", height, "round", round, "proposal", proposal)
2015-08-12 11:00:23 -07: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 {
log.Warn("EnterPropose: Error signing proposal", "height", height, "round", round, "error", err)
2015-06-05 14:15:40 -07:00
}
2015-06-24 14:04:40 -07:00
2015-06-05 14:15:40 -07:00
}
// Returns true if the proposal block is complete &&
// (if POLRound was proposed, we have +2/3 prevotes from there).
2015-06-05 14:15:40 -07:00
func (cs *ConsensusState) isProposalComplete() bool {
if cs.Proposal == nil || cs.ProposalBlock == nil {
return false
}
2015-08-12 11:00:23 -07:00
// we have the proposal. if there's a POLRound,
// make sure we have the prevotes from it too
2015-06-24 14:04:40 -07:00
if cs.Proposal.POLRound < 0 {
return true
} else {
2015-08-12 11:00:23 -07:00
// if this is false the proposer is lying or we haven't received the POL yet
return cs.Votes.Prevotes(cs.Proposal.POLRound).HasTwoThirdsMajority()
2015-06-24 14:04:40 -07:00
}
2015-06-05 14:15:40 -07:00
}
// Create the next block to propose and return it.
2015-08-26 15:56:34 -07:00
// NOTE: keep 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: len(txs),
2015-06-05 14:15:40 -07:00
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,
},
}
2015-08-07 10:43:24 -07:00
block.FillHeader()
2015-06-05 14:15:40 -07:00
// 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
}
// Enter: `timeoutPropose` after entering Propose.
2015-06-05 14:15:40 -07:00
// Enter: proposal block and POL is ready.
// Enter: any +2/3 prevotes for future round.
// Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
2014-10-30 03:32:09 -07:00
// Otherwise vote nil.
2015-09-09 13:45:53 -07:00
func (cs *ConsensusState) EnterPrevote(height int, round int, timedOut bool) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
2015-08-26 15:56:34 -07:00
log.Debug(Fmt("EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
2014-10-21 01:18:46 -07:00
}
2015-09-09 13:45:53 -07:00
// fire event for how we got here
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutPropose(), cs.RoundStateEvent())
} else if cs.isProposalComplete() {
cs.evsw.FireEvent(types.EventStringCompleteProposal(), cs.RoundStateEvent())
} else {
// we received +2/3 prevotes for a future round
// TODO: catchup event?
}
2015-07-19 14:49:13 -07:00
log.Info(Fmt("EnterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
// Sign and broadcast vote as necessary
cs.doPrevote(height, round)
// Done EnterPrevote:
cs.Round = round
cs.Step = RoundStepPrevote
cs.newStepCh <- cs.getRoundState()
2015-09-09 13:45:53 -07:00
// Once `addVote` hits any +2/3 prevotes, we will go to PrevoteWait
// (so we have more time to try and collect +2/3 prevotes for a single block)
2015-06-05 14:15:40 -07:00
}
func (cs *ConsensusState) doPrevote(height int, round int) {
2014-10-21 01:18:46 -07:00
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
2015-07-19 14:49:13 -07:00
log.Info("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
2015-08-12 11:00:23 -07:00
// NOTE: the proposal signature is validated when it is received,
// and the proposal block parts are validated as they are received (against the merkle hash in the proposal)
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-24 18:51:14 -07:00
// Enter: any +2/3 prevotes at next round.
func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
2015-06-05 14:15:40 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
2015-08-26 15:56:34 -07:00
log.Debug(Fmt("EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
}
if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("EnterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
2015-06-05 14:15:40 -07:00
}
2015-07-19 14:49:13 -07:00
log.Info(Fmt("EnterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
// 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-08-12 11:00:23 -07:00
time.Sleep(timeoutPrevote0 + timeoutPrevoteDelta*time.Duration(round))
2015-09-09 13:45:53 -07:00
cs.EnterPrecommit(height, round, true)
2015-06-05 14:15:40 -07:00
}()
}
// Enter: +2/3 precomits for block or nil.
// Enter: `timeoutPrevote` after any +2/3 prevotes.
// Enter: any +2/3 precommits for next round.
2015-08-26 15:56:34 -07:00
// Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
2015-06-05 14:15:40 -07:00
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
2015-08-12 11:00:23 -07:00
// else, precommit nil otherwise.
2015-09-09 13:45:53 -07:00
func (cs *ConsensusState) EnterPrecommit(height int, round int, timedOut bool) {
2014-09-14 15:37:32 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
2015-08-26 15:56:34 -07:00
log.Debug(Fmt("EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
2014-09-14 15:37:32 -07:00
}
2015-09-09 13:45:53 -07:00
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
}
2015-07-19 14:49:13 -07:00
log.Info(Fmt("EnterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
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()
}()
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
hash, partsHeader, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
2015-09-09 13:45:53 -07:00
// If we don't have a polka, we must precommit nil
2014-10-30 03:32:09 -07:00
if !ok {
2015-06-05 14:15:40 -07:00
if cs.LockedBlock != nil {
2015-08-12 11:00:23 -07:00
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit while we're locked. Precommitting nil")
2015-06-05 14:15:40 -07:00
} else {
2015-07-19 14:49:13 -07:00
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting nil.")
2015-06-05 14:15:40 -07:00
}
2015-08-12 11:00:23 -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
}
2014-08-10 16:35:08 -07:00
2015-08-12 11:00:23 -07:00
// At this point +2/3 prevoted for a particular block or nil
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringPolka(), cs.RoundStateEvent())
// the latest POLRound should be this round
if cs.Votes.POLRound() < round {
PanicSanity(Fmt("This POLRound should be %v but got %", round, cs.Votes.POLRound()))
}
2015-08-12 11:00:23 -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-07-19 14:49:13 -07:00
log.Info("EnterPrecommit: +2/3 prevoted for nil.")
2015-05-04 10:15:58 -07:00
} else {
2015-07-19 14:49:13 -07:00
log.Info("EnterPrecommit: +2/3 prevoted for nil. Unlocking")
2015-08-26 15:56:34 -07:00
cs.LockedRound = 0
2015-05-04 10:15:58 -07:00
cs.LockedBlock = nil
cs.LockedBlockParts = nil
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
2015-05-04 10:15:58 -07:00
}
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.
2015-08-12 11:00:23 -07:00
// If we're already locked on that block, precommit it, and update the LockedRound
2014-10-30 03:32:09 -07:00
if cs.LockedBlock.HashesTo(hash) {
2015-09-09 13:45:53 -07:00
log.Info("EnterPrecommit: +2/3 prevoted locked block. Relocking")
2015-08-12 11:00:23 -07:00
cs.LockedRound = round
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringRelock(), cs.RoundStateEvent())
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) {
2015-09-09 13:45:53 -07:00
log.Info("EnterPrecommit: +2/3 prevoted proposal block. Locking", "hash", hash)
2015-06-05 14:15:40 -07:00
// Validate the block.
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
2015-07-19 16:42:52 -07:00
PanicConsensus(Fmt("EnterPrecommit: +2/3 prevoted for an invalid block: %v", err))
2015-06-05 14:15:40 -07:00
}
2015-06-24 18:51:14 -07:00
cs.LockedRound = round
2015-06-05 14:15:40 -07:00
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringLock(), cs.RoundStateEvent())
2015-06-05 14:15:40 -07:00
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
2014-10-30 03:32:09 -07:00
return
2014-09-14 15:37:32 -07:00
}
2015-09-09 13:45:53 -07:00
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
// The +2/3 prevotes for this round is the POL for our unlock.
// TODO: In the future save the POL prevotes for justification.
cs.LockedRound = 0
2015-06-05 14:15:40 -07:00
cs.LockedBlock = nil
cs.LockedBlockParts = nil
if !cs.ProposalBlockParts.HasHeader(partsHeader) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
}
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
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 int, round int) {
2015-06-05 14:15:40 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
2015-08-26 15:56:34 -07:00
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() {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("EnterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
2015-06-05 14:15:40 -07:00
}
2015-07-19 14:49:13 -07:00
log.Info(Fmt("EnterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
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.
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, round+1, true)
2015-06-05 14:15:40 -07:00
}()
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 int, commitRound int) {
2014-10-21 01:18:46 -07:00
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || RoundStepCommit <= cs.Step {
2015-08-26 15:56:34 -07:00
log.Debug(Fmt("EnterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
2014-10-30 03:32:09 -07:00
}
log.Info(Fmt("EnterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
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
2015-09-15 13:13:39 -07:00
cs.CommitRound = commitRound
cs.newStepCh <- cs.getRoundState()
2015-06-05 14:15:40 -07:00
// Maybe finalize immediately.
2015-09-15 13:13:39 -07:00
cs.tryFinalizeCommit(height)
}()
hash, partsHeader, ok := cs.Votes.Precommits(commitRound).TwoThirdsMajority()
2014-10-30 03:32:09 -07:00
if !ok {
2015-07-19 16:42:52 -07:00
PanicSanity("RunActionCommit() expects +2/3 precommits")
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,
2015-08-26 15:56:34 -07:00
// otherwise they'll be cleared in updateToState.
2014-10-30 03:32:09 -07:00
if cs.LockedBlock.HashesTo(hash) {
cs.ProposalBlock = cs.LockedBlock
cs.ProposalBlockParts = cs.LockedBlockParts
}
// 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-09-15 13:13:39 -07:00
func (cs *ConsensusState) tryFinalizeCommit(height int) {
2015-06-24 14:04:40 -07:00
if cs.Height != height {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
}
2015-06-24 14:04:40 -07:00
2015-09-15 13:13:39 -07:00
hash, _, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
2015-06-24 14:04:40 -07:00
if !ok || len(hash) == 0 {
log.Warn("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.")
return
2015-06-24 14:04:40 -07:00
}
if !cs.ProposalBlock.HashesTo(hash) {
log.Warn("Attempt to finalize failed. We don't have the commit block.")
return
2015-06-24 14:04:40 -07:00
}
2015-09-15 13:13:39 -07:00
go cs.FinalizeCommit(height)
}
2015-06-05 14:15:40 -07:00
// Increment height and goto RoundStepNewHeight
2015-09-15 13:13:39 -07:00
func (cs *ConsensusState) FinalizeCommit(height int) {
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 {
2015-08-26 15:56:34 -07:00
log.Debug(Fmt("FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
return
2014-10-21 01:18:46 -07:00
}
2015-09-15 13:13:39 -07:00
hash, header, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if !ok {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("Cannot FinalizeCommit, commit does not have two thirds majority"))
2015-06-05 14:15:40 -07:00
}
if !cs.ProposalBlockParts.HasHeader(header) {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("Expected ProposalBlockParts header to be commit header"))
2015-06-05 14:15:40 -07:00
}
if !cs.ProposalBlock.HashesTo(hash) {
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("Cannot FinalizeCommit, ProposalBlock does not hash to commit hash"))
2015-06-05 14:15:40 -07:00
}
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
2015-07-19 16:42:52 -07:00
PanicConsensus(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
2015-07-19 14:49:13 -07:00
log.Info(Fmt("Finalizing commit of block: %v", cs.ProposalBlock))
2015-06-05 14:15:40 -07:00
// We have the block, so stage/save/commit-vote.
2015-09-15 13:13:39 -07:00
cs.saveBlock(cs.ProposalBlock, cs.ProposalBlockParts, cs.Votes.Precommits(cs.CommitRound))
2015-06-05 14:15:40 -07:00
// Increment height.
2015-09-15 13:13:39 -07:00
cs.updateToState(cs.stagedState)
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
// 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 *types.Proposal) error {
2014-10-21 23:30:18 -07:00
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 RoundStepCommit <= cs.Step {
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 &&
(proposal.POLRound < 0 || proposal.Round <= proposal.POLRound) {
2015-06-22 19:04:31 -07:00
return ErrInvalidProposalPOLRound
}
2014-10-21 23:30:18 -07:00
// Verify signature
2015-11-01 11:34:08 -08:00
if !cs.Validators.Proposer().PubKey.VerifyBytes(types.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.
2015-09-09 13:45:53 -07:00
// This can trigger us to go into EnterPrevote asynchronously (before we timeout of propose) or to attempt to commit
func (cs *ConsensusState) AddProposalBlockPart(height int, 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() {
2015-06-24 18:51:14 -07:00
// Added and completed!
2015-11-10 13:10:43 -08:00
var n int
2014-10-21 23:30:18 -07:00
var err error
2015-11-10 13:10:43 -08:00
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
2015-07-19 14:49:13 -07:00
log.Info("Received complete proposal", "hash", cs.ProposalBlock.Hash())
2015-06-12 20:24:08 -07:00
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
2015-06-24 18:51:14 -07:00
// Move onto the next step
2015-09-09 13:45:53 -07:00
go cs.EnterPrevote(height, cs.Round, false)
2015-06-05 14:15:40 -07:00
} else if cs.Step == RoundStepCommit {
2015-06-24 18:51:14 -07:00
// If we're waiting on the proposal block...
2015-09-15 13:13:39 -07:00
cs.tryFinalizeCommit(height)
}
2014-10-21 23:30:18 -07:00
return true, err
}
2015-06-24 18:51:14 -07:00
return added, nil
2014-10-21 23:30:18 -07:00
}
2015-08-12 11:00:23 -07:00
// Attempt to add the vote. if its a duplicate signature, dupeout the validator
2015-08-26 15:56:34 -07:00
func (cs *ConsensusState) TryAddVote(valIndex int, vote *types.Vote, peerKey string) (bool, error) {
2015-11-01 11:34:08 -08:00
added, _, err := cs.AddVote(valIndex, vote, peerKey)
2015-08-12 11:00:23 -07:00
if err != nil {
2015-08-26 15:56:34 -07:00
// If the vote height is off, we'll just ignore it,
2015-09-09 13:45:53 -07:00
// But if it's a conflicting sig, broadcast evidence tx for slashing.
// If it's otherwise invalid, punish peer.
2015-08-26 15:56:34 -07:00
if err == ErrVoteHeightMismatch {
return added, err
2015-11-01 11:34:08 -08:00
} else if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
2015-08-12 11:00:23 -07:00
log.Warn("Found conflicting vote. Publish evidence")
2015-11-01 11:34:08 -08:00
/* XXX
2015-08-12 11:00:23 -07:00
evidenceTx := &types.DupeoutTx{
Address: address,
VoteA: *errDupe.VoteA,
VoteB: *errDupe.VoteB,
}
cs.mempoolReactor.BroadcastTx(evidenceTx) // shouldn't need to check returned err
2015-11-01 11:34:08 -08:00
*/
2015-08-12 11:00:23 -07:00
return added, err
} else {
// Probably an invalid signature. Bad peer.
log.Warn("Error attempting to add vote", "error", err)
return added, ErrAddingVote
}
}
return added, nil
}
2015-09-09 13:45:53 -07:00
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.addVote(valIndex, vote, peerKey)
}
//-----------------------------------------------------------------------------
2015-08-26 15:56:34 -07:00
func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
2015-08-12 11:00:23 -07:00
log.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "csHeight", cs.Height)
2015-09-09 13:45:53 -07:00
defer func() {
if added {
cs.evsw.FireEvent(types.EventStringVote(), &types.EventDataVote{valIndex, address, vote})
}
}()
2015-06-05 14:15:40 -07:00
// A precommit for the previous height?
2015-08-26 15:56:34 -07:00
if vote.Height+1 == cs.Height {
if !(cs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
// TODO: give the reason ..
// fmt.Errorf("TryAddVote: Wrong height, not a LastCommit straggler commit.")
return added, nil, ErrVoteHeightMismatch
}
added, address, err = cs.LastCommit.AddByIndex(valIndex, vote)
2015-05-04 11:18:21 -07:00
if added {
2015-07-19 14:49:13 -07:00
log.Info(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
2015-08-26 15:56:34 -07:00
added, address, err = cs.Votes.AddByIndex(valIndex, 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:
2015-06-24 18:51:14 -07:00
prevotes := cs.Votes.Prevotes(vote.Round)
2015-07-19 14:49:13 -07:00
log.Info(Fmt("Added to prevotes: %v", prevotes.StringShort()))
2015-06-24 18:51:14 -07:00
// First, unlock if prevotes is a valid POL.
2015-06-25 12:52:16 -07:00
// >> lockRound < POLRound <= unlockOrChangeLockRound (see spec)
// NOTE: If (lockRound < POLRound) but !(POLRound <= unlockOrChangeLockRound),
// we'll still EnterNewRound(H,vote.R) and EnterPrecommit(H,vote.R) to process it
// there.
if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
2015-06-24 18:51:14 -07:00
hash, _, ok := prevotes.TwoThirdsMajority()
if ok && !cs.LockedBlock.HashesTo(hash) {
2015-07-19 14:49:13 -07:00
log.Notice("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
2015-08-26 15:56:34 -07:00
cs.LockedRound = 0
2015-06-24 18:51:14 -07:00
cs.LockedBlock = nil
cs.LockedBlockParts = nil
2015-09-09 13:45:53 -07:00
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
2015-06-24 18:51:14 -07:00
}
}
if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
// Round-skip over to PrevoteWait or goto Precommit.
2015-06-05 14:15:40 -07:00
go func() {
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, vote.Round, false)
2015-06-24 18:51:14 -07:00
if prevotes.HasTwoThirdsMajority() {
2015-09-09 13:45:53 -07:00
cs.EnterPrecommit(height, vote.Round, false)
2015-06-24 18:51:14 -07:00
} else {
2015-09-09 13:45:53 -07:00
cs.EnterPrevote(height, vote.Round, false)
2015-06-24 18:51:14 -07:00
cs.EnterPrevoteWait(height, vote.Round)
}
2015-06-05 14:15:40 -07:00
}()
} else if cs.Proposal != nil && 0 <= cs.Proposal.POLRound && cs.Proposal.POLRound == vote.Round {
2015-06-24 18:51:14 -07:00
// If the proposal is now complete, enter prevote of cs.Round.
2015-06-12 20:24:08 -07:00
if cs.isProposalComplete() {
2015-09-09 13:45:53 -07:00
go cs.EnterPrevote(height, cs.Round, false)
2015-06-12 20:24:08 -07:00
}
2015-06-05 14:15:40 -07:00
}
case types.VoteTypePrecommit:
2015-06-24 18:51:14 -07:00
precommits := cs.Votes.Precommits(vote.Round)
2015-07-19 14:49:13 -07:00
log.Info(Fmt("Added to precommit: %v", precommits.StringShort()))
hash, _, ok := precommits.TwoThirdsMajority()
if ok {
2015-06-24 18:51:14 -07:00
go func() {
if len(hash) == 0 {
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, vote.Round+1, false)
2015-06-24 18:51:14 -07:00
} else {
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterCommit(height, vote.Round)
2015-06-05 14:15:40 -07:00
}
2015-06-24 18:51:14 -07:00
}()
} else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
go func() {
2015-09-09 13:45:53 -07:00
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterPrecommitWait(height, vote.Round)
}()
2015-06-05 14:15:40 -07:00
}
default:
2015-07-19 16:42:52 -07:00
PanicSanity(Fmt("Unexpected vote type %X", vote.Type)) // Should not happen.
}
2014-10-30 03:32:09 -07:00
}
2015-08-26 15:56:34 -07:00
// Either duplicate, or error upon cs.Votes.AddByIndex()
2015-06-05 14:15:40 -07:00
return
2015-08-26 15:56:34 -07:00
} else {
err = ErrVoteHeightMismatch
2014-10-21 23:30:18 -07:00
}
2015-06-05 14:15:40 -07:00
// Height mismatch, bad peer?
2015-07-19 14:49:13 -07:00
log.Info("Vote ignored and not added", "voteHeight", vote.Height, "csHeight", cs.Height)
2015-06-05 14:15:40 -07:00
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 {
2015-07-19 16:42:52 -07:00
PanicSanity("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
}
}
2015-08-12 11:00:23 -07:00
func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
vote := &types.Vote{
Height: cs.Height,
Round: cs.Round,
Type: type_,
BlockHash: hash,
BlockPartsHeader: header,
}
2015-05-29 14:53:57 -07:00
err := cs.privValidator.SignVote(cs.state.ChainID, vote)
2015-08-12 11:00:23 -07:00
return vote, err
}
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, err := cs.signVote(type_, hash, header)
if err == nil {
2015-08-26 15:56:34 -07:00
// NOTE: store our index in the cs so we don't have to do this every time
valIndex, _ := cs.Validators.GetByAddress(cs.privValidator.Address)
_, _, err := cs.addVote(valIndex, vote, "")
2015-07-19 14:49:13 -07:00
log.Notice("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 *types.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-07-19 16:42:52 -07:00
PanicSanity(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.
cs.mempoolReactor.ResetForBlockAndState(block, cs.stagedState)
2014-10-30 03:32:09 -07:00
2015-06-24 14:04:40 -07:00
// Fire off event
2015-08-12 11:00:23 -07:00
if cs.evsw != nil && cs.evc != nil {
2015-09-22 18:12:34 -07:00
cs.evsw.FireEvent(types.EventStringNewBlock(), types.EventDataNewBlock{block})
go cs.evc.Flush()
2015-08-12 11:00:23 -07:00
}
2015-06-24 14:04:40 -07:00
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
}
2015-07-20 16:55:05 -07:00
func (cs *ConsensusState) String() string {
return Fmt("ConsensusState(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step)
}