tendermint/consensus/state.go

1622 lines
55 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"
"path/filepath"
"reflect"
2017-10-02 12:24:30 -07:00
"runtime/debug"
2014-08-10 16:35:08 -07:00
"sync"
"time"
fail "github.com/ebuchman/fail-test"
2017-07-11 22:02:16 -07:00
2017-05-02 00:53:32 -07:00
wire "github.com/tendermint/go-wire"
2017-07-11 22:02:16 -07:00
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
cfg "github.com/tendermint/tendermint/config"
2015-12-01 20:12:01 -08:00
"github.com/tendermint/tendermint/proxy"
2015-04-01 17:30:16 -07:00
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
2014-08-10 16:35:08 -07:00
)
//-----------------------------------------------------------------------------
// Config
2017-08-04 18:36:11 -07:00
const (
proposalHeartbeatIntervalSeconds = 2
)
//-----------------------------------------------------------------------------
// Errors
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
2017-07-19 21:05:33 -07:00
// RoundStepType enumerates the state of the consensus state machine
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
)
2017-07-19 21:05:33 -07:00
// String returns a string
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
//-----------------------------------------------------------------------------
2017-07-19 21:05:33 -07:00
// RoundState defines the internal consensus state.
// It is Immutable when returned from ConsensusState.GetRoundState()
2017-03-02 17:47:07 -08:00
// TODO: Actually, only the top pointer is copied,
// so access to field pointers is still racey
2014-09-14 15:37:32 -07:00
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
}
2017-07-19 21:05:33 -07:00
// RoundStateEvent returns the H/R/S of the RoundState as an event.
func (rs *RoundState) RoundStateEvent() types.EventDataRoundState {
edrs := types.EventDataRoundState{
Height: rs.Height,
Round: rs.Round,
Step: rs.Step.String(),
RoundState: rs,
2015-09-09 13:45:53 -07:00
}
return edrs
2015-09-09 13:45:53 -07:00
}
2017-07-19 21:05:33 -07:00
// String returns a string
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
}
2017-07-19 21:05:33 -07:00
// StringIndented returns a string
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)
}
2017-07-19 21:05:33 -07:00
// StringShort returns a string
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
2015-12-05 11:58:12 -08:00
var (
2016-12-23 08:11:22 -08:00
msgQueueSize = 1000
2015-12-05 11:58:12 -08:00
)
// msgs from the reactor which may update the state
2015-12-08 13:00:59 -08:00
type msgInfo struct {
2015-12-22 12:23:22 -08:00
Msg ConsensusMessage `json:"msg"`
PeerKey string `json:"peer_key"`
2015-12-08 13:00:59 -08:00
}
// internally generated messages which may update the state
2015-12-08 13:00:59 -08:00
type timeoutInfo struct {
2015-12-22 12:23:22 -08:00
Duration time.Duration `json:"duration"`
Height int `json:"height"`
Round int `json:"round"`
Step RoundStepType `json:"step"`
2015-12-08 13:00:59 -08:00
}
func (ti *timeoutInfo) String() string {
2015-12-22 12:23:22 -08:00
return fmt.Sprintf("%v ; %d/%d %v", ti.Duration, ti.Height, ti.Round, ti.Step)
2015-12-08 13:00:59 -08:00
}
2017-07-19 21:05:33 -07:00
// ConsensusState handles execution of the consensus algorithm.
// It processes votes and proposals, and upon reaching agreement,
// commits blocks to the chain and executes them against the application.
// The internal state machine receives input from peers, the internal validator, and from a timer.
type ConsensusState struct {
cmn.BaseService
2014-10-30 03:32:09 -07:00
// config details
config *cfg.ConsensusConfig
2017-09-18 15:12:31 -07:00
privValidator types.PrivValidator // for signing votes
// services for creating and executing blocks
2016-11-16 17:52:08 -08:00
proxyAppConn proxy.AppConnConsensus
blockStore types.BlockStore
mempool types.Mempool
2016-11-16 17:52:08 -08:00
// internal state
2014-10-07 01:05:54 -07:00
mtx sync.Mutex
RoundState
state *sm.State // State until height-1.
// state changes may be triggered by msgs from peers,
// msgs from ourself, or by timeouts
peerMsgQueue chan msgInfo
internalMsgQueue chan msgInfo
timeoutTicker TimeoutTicker
2015-12-05 11:58:12 -08:00
// we use PubSub to trigger msg broadcasts in the reactor,
// and to notify external subscribers, eg. through a websocket
2016-10-09 23:58:13 -07:00
evsw types.EventSwitch
// a Write-Ahead Log ensures we can recover from any kind of crash
// and helps us avoid signing conflicting votes
wal *WAL
replayMode bool // so we don't log signing errors during replay
2015-12-22 12:23:22 -08:00
// for tests where we want to limit the number of transitions the state makes
nSteps int
2016-06-26 12:33:11 -07:00
// some functions can be overwritten for testing
2016-06-26 12:33:11 -07:00
decideProposal func(height, round int)
doPrevote func(height, round int)
setProposal func(proposal *types.Proposal) error
2017-01-12 11:44:42 -08:00
// closed when we finish shutting down
2017-01-12 11:44:42 -08:00
done chan struct{}
2014-09-14 15:37:32 -07:00
}
2017-07-19 21:05:33 -07:00
// NewConsensusState returns a new ConsensusState.
func NewConsensusState(config *cfg.ConsensusConfig, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore types.BlockStore, mempool types.Mempool) *ConsensusState {
2014-09-14 15:37:32 -07:00
cs := &ConsensusState{
2016-05-08 15:00:58 -07:00
config: config,
proxyAppConn: proxyAppConn,
blockStore: blockStore,
mempool: mempool,
peerMsgQueue: make(chan msgInfo, msgQueueSize),
internalMsgQueue: make(chan msgInfo, msgQueueSize),
timeoutTicker: NewTimeoutTicker(),
2017-01-12 11:44:42 -08:00
done: make(chan struct{}),
2014-09-14 15:37:32 -07:00
}
2016-06-26 12:33:11 -07:00
// set function defaults (may be overwritten before calling Start)
cs.decideProposal = cs.defaultDecideProposal
cs.doPrevote = cs.defaultDoPrevote
cs.setProposal = cs.defaultSetProposal
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)
2017-05-02 00:53:32 -07:00
cs.BaseService = *cmn.NewBaseService(nil, "ConsensusState", cs)
return cs
2014-08-10 16:35:08 -07:00
}
//----------------------------------------
// Public interface
2017-05-12 14:07:53 -07:00
// SetLogger implements Service.
func (cs *ConsensusState) SetLogger(l log.Logger) {
cs.BaseService.Logger = l
cs.timeoutTicker.SetLogger(l)
}
// SetEventSwitch implements events.Eventable
2016-10-09 23:58:13 -07:00
func (cs *ConsensusState) SetEventSwitch(evsw types.EventSwitch) {
cs.evsw = evsw
}
2017-07-19 21:05:33 -07:00
// String returns a string.
func (cs *ConsensusState) String() string {
// better not to access shared variables
return cmn.Fmt("ConsensusState") //(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step)
}
2017-07-19 21:05:33 -07:00
// GetState returns a copy of the chain state.
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()
}
2017-07-19 21:05:33 -07:00
// GetRoundState returns a copy of the internal consensus state.
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
}
2017-07-19 21:05:33 -07:00
// GetValidators returns a copy of the current validators.
2016-10-14 18:36:42 -07:00
func (cs *ConsensusState) GetValidators() (int, []*types.Validator) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.state.LastBlockHeight, cs.state.Validators.Copy().Validators
}
2017-07-19 21:05:33 -07:00
// SetPrivValidator sets the private validator account for signing votes.
2017-09-18 15:12:31 -07:00
func (cs *ConsensusState) SetPrivValidator(priv types.PrivValidator) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
cs.privValidator = priv
2016-11-16 17:52:08 -08:00
}
2017-07-19 21:05:33 -07:00
// SetTimeoutTicker sets the local timer. It may be useful to overwrite for testing.
func (cs *ConsensusState) SetTimeoutTicker(timeoutTicker TimeoutTicker) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
cs.timeoutTicker = timeoutTicker
}
2017-07-19 21:05:33 -07:00
// LoadCommit loads the commit for a given height.
2016-11-16 13:47:31 -08:00
func (cs *ConsensusState) LoadCommit(height int) *types.Commit {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if height == cs.blockStore.Height() {
return cs.blockStore.LoadSeenCommit(height)
}
return cs.blockStore.LoadBlockCommit(height)
}
2017-07-19 21:05:33 -07:00
// OnStart implements cmn.Service.
// It loads the latest state via the WAL, and starts the timeout and receive routines.
func (cs *ConsensusState) OnStart() error {
walFile := cs.config.WalFile()
2017-03-27 12:41:45 -07:00
if err := cs.OpenWAL(walFile); err != nil {
cs.Logger.Error("Error loading ConsensusState wal", "err", err.Error())
return err
}
// we need the timeoutRoutine for replay so
// we don't block on the tick chan.
// NOTE: we will get a build up of garbage go routines
// firing on the tockChan until the receiveRoutine is started
// to deal with them (by that point, at most one will be valid)
cs.timeoutTicker.Start()
// we may have lost some votes if the process crashed
// reload from consensus log to catchup
if err := cs.catchupReplay(cs.Height); err != nil {
cs.Logger.Error("Error on catchup replay. Proceeding to start ConsensusState anyway", "err", err.Error())
2017-03-28 09:07:32 -07:00
// NOTE: if we ever do return an error here,
// make sure to stop the timeoutTicker
}
2016-09-09 20:10:23 -07:00
// now start the receiveRoutine
go cs.receiveRoutine(0)
2016-09-09 20:10:23 -07:00
// schedule the first round!
// use GetRoundState so we don't race the receiveRoutine for access
cs.scheduleRound0(cs.GetRoundState())
return nil
}
2015-12-12 14:22:48 -08:00
// timeoutRoutine: receive requests for timeouts on tickChan and fire timeouts on tockChan
// receiveRoutine: serializes processing of proposoals, block parts, votes; coordinates state transitions
func (cs *ConsensusState) startRoutines(maxSteps int) {
cs.timeoutTicker.Start()
2015-12-12 14:22:48 -08:00
go cs.receiveRoutine(maxSteps)
}
2017-07-19 21:05:33 -07:00
// OnStop implements cmn.Service. It stops all routines and waits for the WAL to finish.
2015-07-21 18:31:01 -07:00
func (cs *ConsensusState) OnStop() {
2016-10-28 12:14:24 -07:00
cs.BaseService.OnStop()
cs.timeoutTicker.Stop()
2016-10-28 15:01:14 -07:00
// Make BaseService.Wait() wait until cs.wal.Wait()
if cs.wal != nil && cs.IsRunning() {
cs.wal.Wait()
}
2014-10-30 03:32:09 -07:00
}
2017-07-19 21:05:33 -07:00
// Wait waits for the the main routine to return.
2017-01-12 11:44:42 -08:00
// NOTE: be sure to Stop() the event switch and drain
// any event channels or this may deadlock
func (cs *ConsensusState) Wait() {
<-cs.done
}
2017-07-19 21:05:33 -07:00
// OpenWAL opens a file to log all consensus messages and timeouts for deterministic accountability
func (cs *ConsensusState) OpenWAL(walFile string) (err error) {
err = cmn.EnsureDir(filepath.Dir(walFile), 0700)
2017-03-27 12:41:45 -07:00
if err != nil {
cs.Logger.Error("Error ensuring ConsensusState wal dir", "err", err.Error())
2017-03-27 12:41:45 -07:00
return err
}
cs.mtx.Lock()
defer cs.mtx.Unlock()
wal, err := NewWAL(walFile, cs.config.WalLight)
2016-01-18 11:10:05 -08:00
if err != nil {
return err
}
2017-05-12 14:07:53 -07:00
wal.SetLogger(cs.Logger.With("wal", walFile))
if _, err := wal.Start(); err != nil {
return err
}
2016-01-18 11:10:05 -08:00
cs.wal = wal
return nil
}
2015-12-12 14:22:48 -08:00
//------------------------------------------------------------
2017-07-19 21:05:33 -07:00
// Public interface for passing messages into the consensus state, possibly causing a state transition.
// If peerKey == "", the msg is considered internal.
// Messages are added to the appropriate queue (peer or internal).
// If the queue is full, the function may block.
2015-12-12 14:22:48 -08:00
// TODO: should these return anything or let callers just use events?
2017-07-19 21:05:33 -07:00
// AddVote inputs a vote.
func (cs *ConsensusState) AddVote(vote *types.Vote, peerKey string) (added bool, err error) {
if peerKey == "" {
cs.internalMsgQueue <- msgInfo{&VoteMessage{vote}, ""}
} else {
cs.peerMsgQueue <- msgInfo{&VoteMessage{vote}, peerKey}
}
// TODO: wait for event?!
return false, nil
}
2017-07-19 21:05:33 -07:00
// SetProposal inputs a proposal.
func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerKey string) error {
if peerKey == "" {
cs.internalMsgQueue <- msgInfo{&ProposalMessage{proposal}, ""}
} else {
cs.peerMsgQueue <- msgInfo{&ProposalMessage{proposal}, peerKey}
}
// TODO: wait for event?!
return nil
}
2017-07-19 21:05:33 -07:00
// AddProposalBlockPart inputs a part of the proposal block.
func (cs *ConsensusState) AddProposalBlockPart(height, round int, part *types.Part, peerKey string) error {
if peerKey == "" {
cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""}
} else {
cs.peerMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, peerKey}
}
// TODO: wait for event?!
return nil
}
2017-07-19 21:05:33 -07:00
// SetProposalAndBlock inputs the proposal and all block parts.
func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *types.Block, parts *types.PartSet, peerKey string) error {
cs.SetProposal(proposal, peerKey)
for i := 0; i < parts.Total(); i++ {
part := parts.GetPart(i)
2015-12-13 11:56:05 -08:00
cs.AddProposalBlockPart(proposal.Height, proposal.Round, part, peerKey)
}
return nil // TODO errors
}
2015-12-12 14:22:48 -08:00
//------------------------------------------------------------
// internal functions for managing the state
2015-12-08 13:00:59 -08:00
func (cs *ConsensusState) updateHeight(height int) {
cs.Height = height
}
func (cs *ConsensusState) updateRoundStep(round int, step RoundStepType) {
cs.Round = round
cs.Step = step
}
// enterNewRound(height, 0) at cs.StartTime.
2016-09-09 20:10:23 -07:00
func (cs *ConsensusState) scheduleRound0(rs *RoundState) {
2017-05-02 00:53:32 -07:00
//cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
2016-09-09 20:10:23 -07:00
sleepDuration := rs.StartTime.Sub(time.Now())
cs.scheduleTimeout(sleepDuration, rs.Height, 0, RoundStepNewHeight)
2015-12-08 13:00:59 -08:00
}
// Attempt to schedule a timeout (by sending timeoutInfo on the tickChan)
func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height, round int, step RoundStepType) {
cs.timeoutTicker.ScheduleTimeout(timeoutInfo{duration, height, round, step})
2015-06-05 14:15:40 -07:00
}
// send a msg into the receiveRoutine regarding our own proposal, block part, or vote
func (cs *ConsensusState) sendInternalMessage(mi msgInfo) {
select {
case cs.internalMsgQueue <- mi:
default:
// NOTE: using the go-routine means our votes can
// be processed out of order.
// TODO: use CList here for strict determinism and
// attempt push to internalMsgQueue in receiveRoutine
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Internal msg queue is full. Using a go-routine")
go func() { cs.internalMsgQueue <- mi }()
}
}
2016-04-02 09:10:16 -07:00
// Reconstruct LastCommit from SeenCommit, which we saved along with the block,
// (which happens even before saving the state)
func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
if state.LastBlockHeight == 0 {
return
}
2016-04-02 09:10:16 -07:00
seenCommit := cs.blockStore.LoadSeenCommit(state.LastBlockHeight)
2017-05-01 21:43:49 -07:00
lastPrecommits := types.NewVoteSet(cs.state.ChainID, state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
for _, precommit := range seenCommit.Precommits {
if precommit == nil {
continue
}
added, err := lastPrecommits.AddVote(precommit)
if !added || err != nil {
cmn.PanicCrisis(cmn.Fmt("Failed to reconstruct LastCommit: %v", err))
}
}
if !lastPrecommits.HasTwoThirdsMajority() {
cmn.PanicSanity("Failed to reconstruct LastCommit: Does not have +2/3 maj")
}
cs.LastCommit = lastPrecommits
}
// Updates ConsensusState and increments height to match that of state.
// The round becomes 0 and cs.Step becomes RoundStepNewHeight.
func (cs *ConsensusState) updateToState(state *sm.State) {
if cs.CommitRound > -1 && 0 < cs.Height && cs.Height != state.LastBlockHeight {
cmn.PanicSanity(cmn.Fmt("updateToState() expected state height of %v but found %v",
cs.Height, state.LastBlockHeight))
}
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?!
cmn.PanicSanity(cmn.Fmt("Inconsistent cs.state.LastBlockHeight+1 %v vs cs.Height %v",
cs.state.LastBlockHeight+1, cs.Height))
}
// 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) {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
return
}
// Reset fields based on state.
validators := state.Validators
lastPrecommits := (*types.VoteSet)(nil)
if cs.CommitRound > -1 && cs.Votes != nil {
if !cs.Votes.Precommits(cs.CommitRound).HasTwoThirdsMajority() {
cmn.PanicSanity("updateToState(state) called but last Precommit round didn't have +2/3")
}
lastPrecommits = cs.Votes.Precommits(cs.CommitRound)
}
// Next desired block height
height := state.LastBlockHeight + 1
// RoundState fields
cs.updateHeight(height)
cs.updateRoundStep(0, RoundStepNewHeight)
if cs.CommitTime.IsZero() {
// "Now" makes it easier to sync up dev nodes.
// We add timeoutCommit to allow transactions
// to be gathered for the first block.
// And alternative solution that relies on clocks:
// cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
cs.StartTime = cs.config.Commit(time.Now())
} else {
cs.StartTime = cs.config.Commit(cs.CommitTime)
}
cs.Validators = validators
cs.Proposal = nil
cs.ProposalBlock = nil
cs.ProposalBlockParts = nil
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
2017-05-01 21:43:49 -07:00
cs.Votes = NewHeightVoteSet(state.ChainID, height, validators)
cs.CommitRound = -1
cs.LastCommit = lastPrecommits
cs.LastValidators = state.LastValidators
cs.state = state
// Finally, broadcast RoundState
cs.newStep()
}
func (cs *ConsensusState) newStep() {
2015-12-22 12:23:22 -08:00
rs := cs.RoundStateEvent()
2016-01-18 11:10:05 -08:00
cs.wal.Save(rs)
cs.nSteps += 1
// newStep is called by updateToStep in NewConsensusState before the evsw is set!
if cs.evsw != nil {
2016-10-09 23:58:13 -07:00
types.FireEventNewRoundStep(cs.evsw, rs)
}
}
//-----------------------------------------
// the main go routines
// receiveRoutine handles messages which may cause state transitions.
// it's argument (n) is the number of messages to process before exiting - use 0 to run forever
// It keeps the RoundState and is the only thing that updates it.
2017-07-25 07:52:14 -07:00
// Updates (state transitions) happen on timeouts, complete proposals, and 2/3 majorities.
// ConsensusState must be locked before any internal state is updated.
func (cs *ConsensusState) receiveRoutine(maxSteps int) {
defer func() {
if r := recover(); r != nil {
2017-10-02 12:24:30 -07:00
cs.Logger.Error("CONSENSUS FAILURE!!!", "err", r, "stack", string(debug.Stack()))
}
}()
for {
if maxSteps > 0 {
if cs.nSteps >= maxSteps {
2017-05-12 14:07:53 -07:00
cs.Logger.Info("reached max steps. exiting receive routine")
cs.nSteps = 0
return
}
}
rs := cs.RoundState
var mi msgInfo
select {
case height := <-cs.mempool.TxsAvailable():
cs.handleTxsAvailable(height)
case mi = <-cs.peerMsgQueue:
2016-01-18 11:10:05 -08:00
cs.wal.Save(mi)
// handles proposals, block parts, votes
// may generate internal events (votes, complete proposals, 2/3 majorities)
2017-07-25 07:52:14 -07:00
cs.handleMsg(mi)
case mi = <-cs.internalMsgQueue:
2016-01-18 11:10:05 -08:00
cs.wal.Save(mi)
// handles proposals, block parts, votes
2017-07-25 07:52:14 -07:00
cs.handleMsg(mi)
case ti := <-cs.timeoutTicker.Chan(): // tockChan:
2016-01-18 11:10:05 -08:00
cs.wal.Save(ti)
// if the timeout is relevant to the rs
// go to the next step
cs.handleTimeout(ti, rs)
case <-cs.Quit:
// NOTE: the internalMsgQueue may have signed messages from our
// priv_val that haven't hit the WAL, but its ok because
// priv_val tracks LastSig
2016-01-18 11:10:05 -08:00
// close wal now that we're done writing to it
if cs.wal != nil {
2016-10-28 15:01:14 -07:00
cs.wal.Stop()
2016-01-18 11:10:05 -08:00
}
2017-01-12 11:44:42 -08:00
close(cs.done)
return
}
}
}
// state transitions on complete-proposal, 2/3-any, 2/3-one
2017-07-25 07:52:14 -07:00
func (cs *ConsensusState) handleMsg(mi msgInfo) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
var err error
2015-12-22 12:23:22 -08:00
msg, peerKey := mi.Msg, mi.PeerKey
switch msg := msg.(type) {
case *ProposalMessage:
// will not cause transition.
// once proposal is set, we can receive block parts
err = cs.setProposal(msg.Proposal)
case *BlockPartMessage:
// if the proposal is complete, we'll enterPrevote or tryFinalizeCommit
2016-03-11 18:38:15 -08:00
_, err = cs.addProposalBlockPart(msg.Height, msg.Part, peerKey != "")
if err != nil && msg.Round != cs.Round {
err = nil
}
case *VoteMessage:
// attempt to add the vote and dupeout the validator if its a duplicate signature
// if the vote gives us a 2/3-any or 2/3-one, we transition
2017-06-23 19:12:45 -07:00
err := cs.tryAddVote(msg.Vote, peerKey)
if err == ErrAddingVote {
// TODO: punish peer
}
// NOTE: the vote is broadcast to peers by the reactor listening
// for vote events
// TODO: If rs.Height == vote.Height && rs.Round < vote.Round,
// the peer is sending us CatchupCommit precommits.
// We could make note of this and help filter in broadcastHasVoteMessage().
default:
2017-05-02 00:53:32 -07:00
cs.Logger.Error("Unknown msg type", reflect.TypeOf(msg))
}
if err != nil {
cs.Logger.Error("Error with msg", "type", reflect.TypeOf(msg), "peer", peerKey, "err", err, "msg", msg)
}
}
func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug("Received tock", "timeout", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
// timeouts must be for current height, round, step
2015-12-22 12:23:22 -08:00
if ti.Height != rs.Height || ti.Round < rs.Round || (ti.Round == rs.Round && ti.Step < rs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug("Ignoring tock because we're ahead", "height", rs.Height, "round", rs.Round, "step", rs.Step)
return
}
// the timeout will now cause a state transition
cs.mtx.Lock()
defer cs.mtx.Unlock()
2015-12-22 12:23:22 -08:00
switch ti.Step {
2015-12-12 14:22:48 -08:00
case RoundStepNewHeight:
// NewRound event fired from enterNewRound.
// XXX: should we fire timeout here (for timeout commit)?
2015-12-22 12:23:22 -08:00
cs.enterNewRound(ti.Height, 0)
case RoundStepNewRound:
cs.enterPropose(ti.Height, 0)
case RoundStepPropose:
2016-10-09 23:58:13 -07:00
types.FireEventTimeoutPropose(cs.evsw, cs.RoundStateEvent())
2015-12-22 12:23:22 -08:00
cs.enterPrevote(ti.Height, ti.Round)
case RoundStepPrevoteWait:
2016-10-09 23:58:13 -07:00
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
2015-12-22 12:23:22 -08:00
cs.enterPrecommit(ti.Height, ti.Round)
case RoundStepPrecommitWait:
2016-10-09 23:58:13 -07:00
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
2015-12-22 12:23:22 -08:00
cs.enterNewRound(ti.Height, ti.Round+1)
default:
panic(cmn.Fmt("Invalid timeout step: %v", ti.Step))
}
}
func (cs *ConsensusState) handleTxsAvailable(height int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
// we only need to do this for round 0
cs.enterPropose(height, 0)
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
// State functions
// Used internally by handleTimeout and handleMsg to make state transitions
2014-08-10 16:35:08 -07:00
2017-07-19 21:05:33 -07:00
// Enter: `timeoutNewHeight` by startTime (commitTime+timeoutCommit),
// or, if SkipTimeout==true, after receiving all precommits from (height,round-1)
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
2017-07-19 21:05:33 -07:00
// Enter: +2/3 precommits for nil at (height,round-1)
// Enter: +2/3 prevotes any or +2/3 precommits for block or any from (height, round)
2015-06-05 14:15:40 -07:00
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) enterNewRound(height int, round int) {
2015-06-24 14:04:40 -07:00
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
2015-06-05 14:15:40 -07:00
if now := time.Now(); cs.StartTime.After(now) {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Need to set a buffer and log message here for sanity.", "startTime", cs.StartTime, "now", now)
2014-10-30 03:32:09 -07:00
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.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
2015-12-12 13:25:49 -08:00
// we don't fire newStep for this step,
// but we fire an event, so update the round step first
2015-12-08 13:00:59 -08:00
cs.updateRoundStep(round, RoundStepNewRound)
2015-06-05 14:15:40 -07:00
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
2016-10-09 23:58:13 -07:00
types.FireEventNewRound(cs.evsw, cs.RoundStateEvent())
2015-09-09 13:45:53 -07:00
2017-07-11 22:02:16 -07:00
// Wait for txs to be available in the mempool
// before we enterPropose in round 0. If the last block changed the app hash,
// we may need an empty "proof" block, and enterPropose immediately.
waitForTxs := cs.config.WaitForTxs() && round == 0 && !cs.needProofBlock(height)
if waitForTxs {
if cs.config.CreateEmptyBlocksInterval > 0 {
2017-08-08 14:09:04 -07:00
cs.scheduleTimeout(cs.config.EmptyBlocksInterval(), height, round, RoundStepNewRound)
}
2017-08-03 10:25:26 -07:00
go cs.proposalHeartbeat(height, round)
2017-07-13 10:19:44 -07:00
} else {
cs.enterPropose(height, round)
}
2017-07-11 22:02:16 -07:00
}
// needProofBlock returns true on the first height (so the genesis app hash is signed right away)
// and where the last block (height-1) caused the app hash to change
func (cs *ConsensusState) needProofBlock(height int) bool {
if height == 1 {
return true
}
lastBlockMeta := cs.blockStore.LoadBlockMeta(height - 1)
if !bytes.Equal(cs.state.AppHash, lastBlockMeta.Header.AppHash) {
return true
}
return false
}
2017-08-03 10:25:26 -07:00
func (cs *ConsensusState) proposalHeartbeat(height, round int) {
2017-07-20 12:09:44 -07:00
counter := 0
addr := cs.privValidator.GetAddress()
2017-07-29 11:15:10 -07:00
valIndex, v := cs.Validators.GetByAddress(addr)
if v == nil {
// not a validator
valIndex = -1
}
2017-07-11 22:02:16 -07:00
for {
2017-08-03 10:25:26 -07:00
rs := cs.GetRoundState()
// if we've already moved on, no need to send more heartbeats
if rs.Step > RoundStepNewRound || rs.Round > round || rs.Height > height {
return
}
heartbeat := &types.Heartbeat{
Height: rs.Height,
Round: rs.Round,
Sequence: counter,
ValidatorAddress: addr,
ValidatorIndex: valIndex,
2017-07-11 22:02:16 -07:00
}
2017-08-03 10:25:26 -07:00
cs.privValidator.SignHeartbeat(cs.state.ChainID, heartbeat)
heartbeatEvent := types.EventDataProposalHeartbeat{heartbeat}
types.FireEventProposalHeartbeat(cs.evsw, heartbeatEvent)
counter += 1
2017-08-04 18:36:11 -07:00
time.Sleep(proposalHeartbeatIntervalSeconds * time.Second)
2017-07-11 22:02:16 -07:00
}
2014-10-30 03:32:09 -07:00
}
// Enter (CreateEmptyBlocks): from enterNewRound(height,round)
// Enter (CreateEmptyBlocks, CreateEmptyBlocksInterval > 0 ): after enterNewRound(height,round), after timeout of CreateEmptyBlocksInterval
// Enter (!CreateEmptyBlocks) : after enterNewRound(height,round), once txs are in the mempool
func (cs *ConsensusState) enterPropose(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.Fmt("enterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
2015-12-12 13:25:49 -08:00
defer func() {
// Done enterPropose:
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(round, RoundStepPropose)
cs.newStep()
// 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() {
cs.enterPrevote(height, cs.Round)
}
2015-12-12 13:25:49 -08:00
}()
// If we don't get the proposal and all block parts quick enough, enterPrevote
cs.scheduleTimeout(cs.config.Propose(round), height, round, RoundStepPropose)
2014-09-14 15:37:32 -07:00
// Nothing more to do if we're not a validator
if cs.privValidator == nil {
cs.Logger.Debug("This node is not a validator")
return
}
2015-06-05 14:15:40 -07:00
2017-07-11 16:18:15 -07:00
if !cs.isProposer() {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
if cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
cs.Logger.Debug("This node is a validator")
} else {
cs.Logger.Debug("This node is not a validator")
}
} else {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
cs.Logger.Debug("This node is a validator")
2015-08-26 15:56:34 -07:00
cs.decideProposal(height, round)
2016-06-26 12:33:11 -07:00
}
}
2014-08-10 16:35:08 -07:00
2017-07-11 16:18:15 -07:00
func (cs *ConsensusState) isProposer() bool {
return bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress())
2017-07-11 16:18:15 -07:00
}
2016-06-26 12:33:11 -07:00
func (cs *ConsensusState) defaultDecideProposal(height, 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()
2015-12-01 20:12:01 -08:00
if block == nil { // on error
return
}
2014-08-10 16:35:08 -07:00
}
2014-09-14 15:37:32 -07:00
// Make proposal
polRound, polBlockID := cs.Votes.POLInfo()
proposal := types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
2015-05-29 14:53:57 -07:00
err := cs.privValidator.SignProposal(cs.state.ChainID, proposal)
if err == nil {
// Set fields
/* fields set by setProposal and addBlockPart
cs.Proposal = proposal
2015-01-15 22:43:15 -08:00
cs.ProposalBlock = block
cs.ProposalBlockParts = blockParts
*/
2015-12-05 11:58:12 -08:00
// send proposal and block parts on internal msg queue
cs.sendInternalMessage(msgInfo{&ProposalMessage{proposal}, ""})
for i := 0; i < blockParts.Total(); i++ {
part := blockParts.GetPart(i)
cs.sendInternalMessage(msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""})
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Signed proposal", "height", height, "round", round, "proposal", proposal)
cs.Logger.Debug(cmn.Fmt("Signed proposal block: %v", block))
} else {
if !cs.replayMode {
cs.Logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "err", err)
}
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-12-01 20:12:01 -08:00
// Returns nil block upon error.
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) {
2016-04-02 09:10:16 -07:00
var commit *types.Commit
2015-06-05 14:15:40 -07:00
if cs.Height == 1 {
// We're creating a proposal for the first block.
2016-04-02 09:10:16 -07:00
// The commit is empty, but not nil.
commit = &types.Commit{}
2015-06-19 15:30:10 -07:00
} else if cs.LastCommit.HasTwoThirdsMajority() {
2016-04-02 09:10:16 -07:00
// Make the commit from LastCommit
commit = cs.LastCommit.MakeCommit()
2015-06-05 14:15:40 -07:00
} else {
// This shouldn't happen.
2017-05-02 00:53:32 -07:00
cs.Logger.Error("enterPropose: Cannot propose anything: No commit for the previous block.")
2015-06-05 14:15:40 -07:00
return
}
2015-12-01 20:12:01 -08:00
// Mempool validated transactions
txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
return types.MakeBlock(cs.Height, cs.state.ChainID, txs, commit,
cs.state.LastBlockID, cs.state.Validators.Hash(),
cs.state.AppHash, cs.state.Params().BlockPartSizeBytes)
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.
func (cs *ConsensusState) enterPrevote(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
2015-12-12 13:25:49 -08:00
defer func() {
// Done enterPrevote:
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(round, RoundStepPrevote)
cs.newStep()
}()
2015-09-09 13:45:53 -07:00
// fire event for how we got here
if cs.isProposalComplete() {
2016-10-09 23:58:13 -07:00
types.FireEventCompleteProposal(cs.evsw, cs.RoundStateEvent())
2015-09-09 13:45:53 -07:00
} else {
// we received +2/3 prevotes for a future round
// TODO: catchup event?
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.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)
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
}
2016-06-26 12:33:11 -07:00
func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
2017-07-07 13:58:16 -07:00
logger := cs.Logger.With("height", height, "round", round)
2014-10-21 01:18:46 -07:00
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
2017-07-07 13:58:16 -07:00
logger.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 {
2017-07-07 13:58:16 -07:00
logger.Info("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
2017-07-19 21:05:33 -07:00
// Validate proposal block
err := cs.state.ValidateBlock(cs.ProposalBlock)
2014-10-21 01:18:46 -07:00
if err != nil {
2014-10-30 03:32:09 -07:00
// ProposalBlock is invalid, prevote nil.
2017-07-07 13:58:16 -07:00
logger.Error("enterPrevote: ProposalBlock is invalid", "err", 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)
2017-07-09 15:01:25 -07:00
logger.Info("enterPrevote: ProposalBlock is valid")
cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
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) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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() {
cmn.PanicSanity(cmn.Fmt("enterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
2015-06-05 14:15:40 -07:00
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.Fmt("enterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
2015-06-05 14:15:40 -07:00
2015-12-12 13:25:49 -08:00
defer func() {
// Done enterPrevoteWait:
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(round, RoundStepPrevoteWait)
cs.newStep()
}()
2015-06-05 14:15:40 -07:00
// Wait for some more prevotes; enterPrecommit
cs.scheduleTimeout(cs.config.Prevote(round), height, round, RoundStepPrevoteWait)
2015-06-05 14:15:40 -07:00
}
// Enter: `timeoutPrevote` after any +2/3 prevotes.
2017-07-19 21:05:33 -07:00
// Enter: +2/3 precomits for block or nil.
2015-06-05 14:15:40 -07:00
// 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.
func (cs *ConsensusState) enterPrecommit(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.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() {
// Done enterPrecommit:
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(round, RoundStepPrecommit)
cs.newStep()
}()
2014-09-14 15:37:32 -07:00
2016-08-16 14:59:19 -07:00
blockID, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
2015-06-05 14:15:40 -07:00
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 {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
2015-06-05 14:15:40 -07:00
} else {
2017-05-02 00:53:32 -07:00
cs.Logger.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
2016-10-09 23:58:13 -07:00
types.FireEventPolka(cs.evsw, cs.RoundStateEvent())
2015-09-09 13:45:53 -07:00
// the latest POLRound should be this round
polRound, _ := cs.Votes.POLInfo()
if polRound < round {
cmn.PanicSanity(cmn.Fmt("This POLRound should be %v but got %", round, polRound))
2015-09-09 13:45:53 -07:00
}
2015-08-12 11:00:23 -07:00
2015-06-05 14:15:40 -07:00
// +2/3 prevoted nil. Unlock and precommit nil.
2016-08-16 14:59:19 -07:00
if len(blockID.Hash) == 0 {
2015-05-04 10:15:58 -07:00
if cs.LockedBlock == nil {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPrecommit: +2/3 prevoted for nil.")
2015-05-04 10:15:58 -07:00
} else {
2017-05-02 00:53:32 -07:00
cs.Logger.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
2016-10-09 23:58:13 -07:00
types.FireEventUnlock(cs.evsw, 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
2016-08-16 14:59:19 -07:00
if cs.LockedBlock.HashesTo(blockID.Hash) {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
2015-08-12 11:00:23 -07:00
cs.LockedRound = round
2016-10-09 23:58:13 -07:00
types.FireEventRelock(cs.evsw, cs.RoundStateEvent())
2016-08-16 14:59:19 -07:00
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.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
2016-08-16 14:59:19 -07:00
if cs.ProposalBlock.HashesTo(blockID.Hash) {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", blockID.Hash)
2015-06-05 14:15:40 -07:00
// Validate the block.
if err := cs.state.ValidateBlock(cs.ProposalBlock); err != nil {
cmn.PanicConsensus(cmn.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
2016-10-09 23:58:13 -07:00
types.FireEventLock(cs.evsw, cs.RoundStateEvent())
2016-08-16 14:59:19 -07:00
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.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
2016-08-16 14:59:19 -07:00
if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
2015-06-05 14:15:40 -07:00
cs.ProposalBlock = nil
2016-08-16 14:59:19 -07:00
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
2015-06-05 14:15:40 -07:00
}
2016-10-09 23:58:13 -07:00
types.FireEventUnlock(cs.evsw, 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
}
// Enter: any +2/3 precommits for next round.
func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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() {
cmn.PanicSanity(cmn.Fmt("enterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
2015-06-05 14:15:40 -07:00
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.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-12-12 13:25:49 -08:00
defer func() {
// Done enterPrecommitWait:
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(round, RoundStepPrecommitWait)
cs.newStep()
}()
2015-06-05 14:15:40 -07:00
// Wait for some more precommits; enterNewRound
cs.scheduleTimeout(cs.config.Precommit(round), height, round, RoundStepPrecommitWait)
2015-12-08 13:00:59 -08: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) {
if cs.Height != height || RoundStepCommit <= cs.Step {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
}
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.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() {
// Done enterCommit:
// keep cs.Round the same, commitRound points to the right Precommits set.
2015-12-12 13:25:49 -08:00
cs.updateRoundStep(cs.Round, RoundStepCommit)
2015-09-15 13:13:39 -07:00
cs.CommitRound = commitRound
cs.CommitTime = time.Now()
cs.newStep()
2015-06-05 14:15:40 -07:00
// Maybe finalize immediately.
2015-09-15 13:13:39 -07:00
cs.tryFinalizeCommit(height)
}()
2016-08-16 14:59:19 -07:00
blockID, ok := cs.Votes.Precommits(commitRound).TwoThirdsMajority()
2014-10-30 03:32:09 -07:00
if !ok {
cmn.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.
2016-08-16 14:59:19 -07:00
if cs.LockedBlock.HashesTo(blockID.Hash) {
2014-10-30 03:32:09 -07:00
cs.ProposalBlock = cs.LockedBlock
cs.ProposalBlockParts = cs.LockedBlockParts
}
// If we don't have the block being committed, set up to get it.
2016-08-16 14:59:19 -07:00
if !cs.ProposalBlock.HashesTo(blockID.Hash) {
if !cs.ProposalBlockParts.HasHeader(blockID.PartsHeader) {
2014-10-30 03:32:09 -07:00
// We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting.
cs.ProposalBlock = nil
2016-08-16 14:59:19 -07:00
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.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 {
cmn.PanicSanity(cmn.Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
}
2015-06-24 14:04:40 -07:00
2016-08-16 14:59:19 -07:00
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(blockID.Hash) == 0 {
2017-05-02 00:53:32 -07:00
cs.Logger.Error("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.", "height", height)
return
2015-06-24 14:04:40 -07:00
}
2016-08-16 14:59:19 -07:00
if !cs.ProposalBlock.HashesTo(blockID.Hash) {
// TODO: this happens every time if we're not a validator (ugly logs)
// TODO: ^^ wait, why does it matter that we're a validator?
2017-05-02 00:53:32 -07:00
cs.Logger.Error("Attempt to finalize failed. We don't have the commit block.", "height", height, "proposal-block", cs.ProposalBlock.Hash(), "commit-block", blockID.Hash)
return
2015-06-24 14:04:40 -07:00
}
// go
cs.finalizeCommit(height)
}
2015-06-05 14:15:40 -07:00
// Increment height and goto RoundStepNewHeight
func (cs *ConsensusState) finalizeCommit(height int) {
2015-06-05 14:15:40 -07:00
if cs.Height != height || cs.Step != RoundStepCommit {
2017-05-02 00:53:32 -07:00
cs.Logger.Debug(cmn.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
}
2016-08-16 14:59:19 -07:00
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
block, blockParts := cs.ProposalBlock, cs.ProposalBlockParts
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if !ok {
cmn.PanicSanity(cmn.Fmt("Cannot finalizeCommit, commit does not have two thirds majority"))
2015-06-05 14:15:40 -07:00
}
2016-08-16 14:59:19 -07:00
if !blockParts.HasHeader(blockID.PartsHeader) {
cmn.PanicSanity(cmn.Fmt("Expected ProposalBlockParts header to be commit header"))
2015-06-05 14:15:40 -07:00
}
2016-08-16 14:59:19 -07:00
if !block.HashesTo(blockID.Hash) {
cmn.PanicSanity(cmn.Fmt("Cannot finalizeCommit, ProposalBlock does not hash to commit hash"))
2015-06-05 14:15:40 -07:00
}
if err := cs.state.ValidateBlock(block); err != nil {
cmn.PanicConsensus(cmn.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
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.Fmt("Finalizing commit of block with %d txs", block.NumTxs),
2016-08-24 22:39:03 -07:00
"height", block.Height, "hash", block.Hash(), "root", block.AppHash)
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.Fmt("%v", block))
2016-09-11 12:32:33 -07:00
fail.Fail() // XXX
// Save to blockStore.
if cs.blockStore.Height() < block.Height {
2016-11-19 16:32:35 -08:00
// NOTE: the seenCommit is local justification to commit this block,
// but may differ from the LastCommit included in the next block
precommits := cs.Votes.Precommits(cs.CommitRound)
seenCommit := precommits.MakeCommit()
cs.blockStore.SaveBlock(block, blockParts, seenCommit)
} else {
// Happens during replay if we already saved the block but didn't commit
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Calling finalizeCommit on already stored block", "height", block.Height)
}
2016-09-11 12:32:33 -07:00
fail.Fail() // XXX
// Finish writing to the WAL for this height.
2017-04-14 22:33:30 -07:00
// NOTE: If we fail before writing this, we'll never write it,
// and just recover by running ApplyBlock in the Handshake.
// If we moved it before persisting the block, we'd have to allow
// WAL replay for blocks with an #ENDHEIGHT
// As is, ConsensusState should not be started again
// until we successfully call ApplyBlock (ie. here or in Handshake after restart)
if cs.wal != nil {
cs.wal.writeEndHeight(height)
}
fail.Fail() // XXX
// Create a copy of the state for staging
// and an event cache for txs
stateCopy := cs.state.Copy()
2016-10-09 23:58:13 -07:00
eventCache := types.NewEventCache(cs.evsw)
// Execute and commit the block, update and save the state, and update the mempool.
// All calls to the proxyAppConn come here.
// NOTE: the block.AppHash wont reflect these txs until the next block
2016-11-30 14:28:41 -08:00
err := stateCopy.ApplyBlock(eventCache, cs.proxyAppConn, block, blockParts.Header(), cs.mempool)
if err != nil {
cs.Logger.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "err", err)
return
2016-11-30 14:28:41 -08:00
}
2016-09-11 12:32:33 -07:00
fail.Fail() // XXX
// Fire event for new block.
// NOTE: If we fail before firing, these events will never fire
//
2017-04-14 22:33:30 -07:00
// TODO: Either
// * Fire before persisting state, in ApplyBlock
// * Fire on start up if we haven't written any new WAL msgs
2017-04-14 22:33:30 -07:00
// Both options mean we may fire more than once. Is that fine ?
types.FireEventNewBlock(cs.evsw, types.EventDataNewBlock{block})
types.FireEventNewBlockHeader(cs.evsw, types.EventDataNewBlockHeader{block.Header})
eventCache.Flush()
2016-09-11 12:32:33 -07:00
fail.Fail() // XXX
2015-12-12 14:22:48 -08:00
// NewHeightStep!
cs.updateToState(stateCopy)
2017-04-14 22:33:30 -07:00
fail.Fail() // XXX
2015-06-24 14:04:40 -07:00
// cs.StartTime is already set.
// Schedule Round0 to start soon.
2016-09-09 20:10:23 -07:00
cs.scheduleRound0(&cs.RoundState)
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.
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
2016-06-26 12:33:11 -07:00
func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
2014-10-21 23:30:18 -07:00
// Already have one
2016-06-26 12:33:11 -07:00
// TODO: possibly catch double proposals
2014-10-21 23:30:18 -07:00
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
if !cs.Validators.GetProposer().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.
// Asynchronously triggers either enterPrevote (before we timeout of propose) or tryFinalizeCommit, once we have the full block.
2016-03-11 18:38:15 -08:00
func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, verify bool) (added bool, err error) {
2014-10-21 23:30:18 -07:00
// 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?
}
2016-03-11 18:38:15 -08:00
added, err = cs.ProposalBlockParts.AddPart(part, verify)
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
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(),
2017-09-15 21:16:49 -07:00
cs.state.Params().BlockSizeParams.MaxBytes, &n, &err).(*types.Block)
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
2017-05-12 14:07:53 -07:00
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "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
cs.enterPrevote(height, cs.Round)
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
func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error {
_, err := cs.addVote(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 err
} else if _, ok := err.(*types.ErrVoteConflictingVotes); ok {
if bytes.Equal(vote.ValidatorAddress, cs.privValidator.GetAddress()) {
2017-05-02 00:53:32 -07:00
cs.Logger.Error("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
2016-02-07 16:56:59 -08:00
return err
}
cs.Logger.Error("Found conflicting vote. Publish evidence (TODO)", "height", vote.Height, "round", vote.Round, "type", vote.Type, "valAddr", vote.ValidatorAddress, "valIndex", vote.ValidatorIndex)
// TODO: track evidence for inclusion in a block
return err
2015-08-12 11:00:23 -07:00
} else {
// Probably an invalid signature. Bad peer.
cs.Logger.Error("Error attempting to add vote", "err", err)
return ErrAddingVote
2015-08-12 11:00:23 -07:00
}
}
return nil
2015-08-12 11:00:23 -07:00
}
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool, err error) {
2017-06-28 08:12:45 -07:00
cs.Logger.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "valIndex", vote.ValidatorIndex, "csHeight", cs.Height)
2015-06-05 14:15:40 -07:00
// A precommit for the previous height?
// These come in while we wait timeoutCommit
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, ErrVoteHeightMismatch
2015-08-26 15:56:34 -07:00
}
added, err = cs.LastCommit.AddVote(vote)
2015-05-04 11:18:21 -07:00
if added {
2017-05-02 00:53:32 -07:00
cs.Logger.Info(cmn.Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
// if we can skip timeoutCommit and have all the votes now,
if cs.config.SkipTimeoutCommit && cs.LastCommit.HasAll() {
// go straight to new round (skip timeout commit)
2016-12-19 17:12:37 -08:00
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, RoundStepNewHeight)
cs.enterNewRound(cs.Height, 0)
}
2015-05-04 11:18:21 -07:00
}
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, err = cs.Votes.AddVote(vote, peerKey)
2015-05-04 11:18:21 -07:00
if added {
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
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)
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Added to prevote", "vote", vote, "prevotes", 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
2015-06-25 12:52:16 -07:00
// there.
if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
2016-08-16 14:59:19 -07:00
blockID, ok := prevotes.TwoThirdsMajority()
if ok && !cs.LockedBlock.HashesTo(blockID.Hash) {
2017-05-02 00:53:32 -07:00
cs.Logger.Info("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
2016-10-09 23:58:13 -07:00
types.FireEventUnlock(cs.evsw, 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.
cs.enterNewRound(height, vote.Round) // if the vote is ahead of us
if prevotes.HasTwoThirdsMajority() {
cs.enterPrecommit(height, vote.Round)
} else {
cs.enterPrevote(height, vote.Round) // if the vote is ahead of us
cs.enterPrevoteWait(height, vote.Round)
}
} 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() {
cs.enterPrevote(height, cs.Round)
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)
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
2016-08-16 14:59:19 -07:00
blockID, ok := precommits.TwoThirdsMajority()
if ok {
2016-08-16 14:59:19 -07:00
if len(blockID.Hash) == 0 {
cs.enterNewRound(height, vote.Round+1)
} else {
cs.enterNewRound(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
cs.enterCommit(height, vote.Round)
if cs.config.SkipTimeoutCommit && precommits.HasAll() {
// if we have all the votes now,
// go straight to new round (skip timeout commit)
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, RoundStepNewHeight)
cs.enterNewRound(cs.Height, 0)
}
}
} else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
cs.enterNewRound(height, vote.Round)
cs.enterPrecommit(height, vote.Round)
cs.enterPrecommitWait(height, vote.Round)
2015-06-05 14:15:40 -07:00
}
default:
cmn.PanicSanity(cmn.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?
2017-05-02 00:53:32 -07:00
cs.Logger.Info("Vote ignored and not added", "voteHeight", vote.Height, "csHeight", cs.Height, "err", err)
2015-06-05 14:15:40 -07:00
return
2014-10-21 23:30:18 -07:00
}
2015-08-12 11:00:23 -07:00
func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
addr := cs.privValidator.GetAddress()
valIndex, _ := cs.Validators.GetByAddress(addr)
vote := &types.Vote{
ValidatorAddress: addr,
ValidatorIndex: valIndex,
Height: cs.Height,
Round: cs.Round,
Type: type_,
2016-08-16 14:59:19 -07:00
BlockID: types.BlockID{hash, 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
}
// sign the vote and publish on internalMsgQueue
2015-08-12 11:00:23 -07:00
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
2016-11-16 17:58:53 -08:00
// if we don't have a key or we're not in the validator set, do nothing
if cs.privValidator == nil || !cs.Validators.HasAddress(cs.privValidator.GetAddress()) {
2015-08-12 11:00:23 -07:00
return nil
}
vote, err := cs.signVote(type_, hash, header)
if err == nil {
cs.sendInternalMessage(msgInfo{&VoteMessage{vote}, ""})
cs.Logger.Info("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "err", err)
return vote
} else {
2016-06-26 12:33:11 -07:00
//if !cs.replayMode {
cs.Logger.Error("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "err", err)
2016-06-26 12:33:11 -07:00
//}
return nil
}
}
2014-10-30 03:32:09 -07:00
//---------------------------------------------------------
2015-12-01 20:12:01 -08:00
func CompareHRS(h1, r1 int, s1 RoundStepType, h2, r2 int, s2 RoundStepType) int {
if h1 < h2 {
return -1
} else if h1 > h2 {
return 1
}
if r1 < r2 {
return -1
} else if r1 > r2 {
return 1
}
if s1 < s2 {
return -1
} else if s1 > s2 {
return 1
}
return 0
}