2014-08-10 16:35:08 -07:00
package consensus
import (
2014-12-09 18:49:04 -08:00
"bytes"
2014-09-14 15:37:32 -07:00
"errors"
2014-10-18 01:42:33 -07:00
"fmt"
2017-09-18 17:49:43 -04:00
"path/filepath"
2015-12-10 11:41:18 -05:00
"reflect"
2017-10-02 15:24:30 -04:00
"runtime/debug"
2014-08-10 16:35:08 -07:00
"sync"
"time"
2017-05-04 20:07:08 +02:00
fail "github.com/ebuchman/fail-test"
2017-07-12 01:02:16 -04:00
2017-05-02 11:53:32 +04:00
wire "github.com/tendermint/go-wire"
2017-07-12 01:02:16 -04:00
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
2017-05-04 20:07:08 +02:00
cfg "github.com/tendermint/tendermint/config"
2017-10-10 12:39:21 +04:00
cstypes "github.com/tendermint/tendermint/consensus/types"
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
)
2016-02-29 16:15:23 -05:00
//-----------------------------------------------------------------------------
2017-05-01 20:09:29 -04:00
// Config
2017-08-04 21:36:11 -04:00
const (
proposalHeartbeatIntervalSeconds = 2
)
2016-02-29 16:15:23 -05:00
//-----------------------------------------------------------------------------
// 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 14:00:23 -04:00
ErrAddingVote = errors . New ( "Error adding vote" )
2015-08-26 18:56:34 -04:00
ErrVoteHeightMismatch = errors . New ( "Error vote height mismatch" )
2014-12-30 17:14:54 -08:00
)
2014-10-30 03:32:09 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
2015-12-05 14:58:12 -05:00
var (
2016-12-23 11:11:22 -05:00
msgQueueSize = 1000
2015-12-05 14:58:12 -05:00
)
2015-12-12 01:28:33 -05:00
// msgs from the reactor which may update the state
2015-12-08 16:00:59 -05:00
type msgInfo struct {
2015-12-22 15:23:22 -05:00
Msg ConsensusMessage ` json:"msg" `
PeerKey string ` json:"peer_key" `
2015-12-08 16:00:59 -05:00
}
2015-12-12 01:28:33 -05:00
// internally generated messages which may update the state
2015-12-08 16:00:59 -05:00
type timeoutInfo struct {
2017-10-10 12:39:21 +04:00
Duration time . Duration ` json:"duration" `
Height int ` json:"height" `
Round int ` json:"round" `
Step cstypes . RoundStepType ` json:"step" `
2015-12-08 16:00:59 -05:00
}
func ( ti * timeoutInfo ) String ( ) string {
2015-12-22 15:23:22 -05:00
return fmt . Sprintf ( "%v ; %d/%d %v" , ti . Duration , ti . Height , ti . Round , ti . Step )
2015-12-08 16:00:59 -05:00
}
2017-07-20 00:05:33 -04: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.
2014-09-03 20:41:57 -07:00
type ConsensusState struct {
2017-04-25 14:50:20 -04:00
cmn . BaseService
2014-10-30 03:32:09 -07:00
2017-05-01 20:09:29 -04:00
// config details
2017-05-04 20:07:08 +02:00
config * cfg . ConsensusConfig
2017-09-18 18:12:31 -04:00
privValidator types . PrivValidator // for signing votes
2017-05-01 20:09:29 -04:00
// services for creating and executing blocks
2016-11-16 20:52:08 -05:00
proxyAppConn proxy . AppConnConsensus
2017-02-20 20:09:15 -05:00
blockStore types . BlockStore
mempool types . Mempool
2016-11-16 20:52:08 -05:00
2017-05-01 20:09:29 -04:00
// internal state
2014-10-07 01:05:54 -07:00
mtx sync . Mutex
2017-10-10 12:39:21 +04:00
cstypes . RoundState
2016-01-06 17:14:20 -08:00
state * sm . State // State until height-1.
2015-04-07 15:24:09 -05:00
2017-05-01 20:09:29 -04:00
// 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 14:58:12 -05:00
2017-05-01 20:09:29 -04:00
// we use PubSub to trigger msg broadcasts in the reactor,
// and to notify external subscribers, eg. through a websocket
2016-10-10 02:58:13 -04:00
evsw types . EventSwitch
2015-12-12 01:28:33 -05:00
2017-05-01 20:09:29 -04:00
// a Write-Ahead Log ensures we can recover from any kind of crash
// and helps us avoid signing conflicting votes
2016-09-08 18:06:25 -04:00
wal * WAL
replayMode bool // so we don't log signing errors during replay
2015-12-22 15:23:22 -05:00
2017-05-01 20:09:29 -04:00
// for tests where we want to limit the number of transitions the state makes
nSteps int
2016-06-26 15:33:11 -04:00
2017-05-01 20:09:29 -04:00
// some functions can be overwritten for testing
2016-06-26 15:33:11 -04:00
decideProposal func ( height , round int )
doPrevote func ( height , round int )
setProposal func ( proposal * types . Proposal ) error
2017-01-12 14:44:42 -05:00
2017-05-01 20:09:29 -04:00
// closed when we finish shutting down
2017-01-12 14:44:42 -05:00
done chan struct { }
2014-09-14 15:37:32 -07:00
}
2017-07-20 00:05:33 -04:00
// NewConsensusState returns a new ConsensusState.
2017-05-04 20:07:08 +02:00
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 ,
2016-01-06 17:14:20 -08:00
proxyAppConn : proxyAppConn ,
2015-12-11 11:57:15 -05:00
blockStore : blockStore ,
mempool : mempool ,
peerMsgQueue : make ( chan msgInfo , msgQueueSize ) ,
internalMsgQueue : make ( chan msgInfo , msgQueueSize ) ,
2016-12-19 10:44:25 -05:00
timeoutTicker : NewTimeoutTicker ( ) ,
2017-01-12 14:44:42 -05:00
done : make ( chan struct { } ) ,
2014-09-14 15:37:32 -07:00
}
2016-06-26 15:33:11 -04: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 16:13:39 -04: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 11:53:32 +04:00
cs . BaseService = * cmn . NewBaseService ( nil , "ConsensusState" , cs )
2014-09-03 20:41:57 -07:00
return cs
2014-08-10 16:35:08 -07:00
}
2015-12-12 01:28:33 -05:00
//----------------------------------------
// Public interface
2017-05-12 23:07:53 +02:00
// SetLogger implements Service.
func ( cs * ConsensusState ) SetLogger ( l log . Logger ) {
cs . BaseService . Logger = l
cs . timeoutTicker . SetLogger ( l )
}
2017-01-19 13:33:58 +04:00
// SetEventSwitch implements events.Eventable
2016-10-10 02:58:13 -04:00
func ( cs * ConsensusState ) SetEventSwitch ( evsw types . EventSwitch ) {
2015-12-12 01:28:33 -05:00
cs . evsw = evsw
}
2017-07-20 00:05:33 -04:00
// String returns a string.
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) String ( ) string {
2016-07-11 23:07:21 -04:00
// better not to access shared variables
2017-04-25 14:50:20 -04:00
return cmn . Fmt ( "ConsensusState" ) //(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step)
2015-06-04 13:36:47 -07:00
}
2017-07-20 00:05:33 -04:00
// GetState returns a copy of the chain state.
2015-01-14 20:34:53 -08:00
func ( cs * ConsensusState ) GetState ( ) * sm . State {
2015-01-11 14:27:46 -08:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
return cs . state . Copy ( )
}
2017-07-20 00:05:33 -04:00
// GetRoundState returns a copy of the internal consensus state.
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) GetRoundState ( ) * cstypes . RoundState {
2014-09-03 20:41:57 -07:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
2014-10-30 03:32:09 -07:00
return cs . getRoundState ( )
}
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) getRoundState ( ) * cstypes . 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-20 00:05:33 -04:00
// GetValidators returns a copy of the current validators.
2016-10-14 21:36:42 -04: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-20 00:05:33 -04:00
// SetPrivValidator sets the private validator account for signing votes.
2017-09-18 18:12:31 -04:00
func ( cs * ConsensusState ) SetPrivValidator ( priv types . PrivValidator ) {
2015-12-12 01:28:33 -05:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
cs . privValidator = priv
2016-11-16 20:52:08 -05:00
}
2017-07-20 00:05:33 -04:00
// SetTimeoutTicker sets the local timer. It may be useful to overwrite for testing.
2016-12-19 10:44:25 -05:00
func ( cs * ConsensusState ) SetTimeoutTicker ( timeoutTicker TimeoutTicker ) {
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
cs . timeoutTicker = timeoutTicker
}
2017-07-20 00:05:33 -04:00
// LoadCommit loads the commit for a given height.
2016-11-16 16:47:31 -05: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-20 00:05:33 -04:00
// OnStart implements cmn.Service.
// It loads the latest state via the WAL, and starts the timeout and receive routines.
2015-08-04 18:44:15 -07:00
func ( cs * ConsensusState ) OnStart ( ) error {
2015-12-10 11:41:18 -05:00
2017-05-04 21:57:58 +02:00
walFile := cs . config . WalFile ( )
2017-03-27 15:41:45 -04:00
if err := cs . OpenWAL ( walFile ) ; err != nil {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error loading ConsensusState wal" , "err" , err . Error ( ) )
2016-08-14 12:31:24 -04:00
return err
}
2016-09-08 18:06:25 -04:00
// 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)
2016-12-19 22:29:32 -05:00
cs . timeoutTicker . Start ( )
2016-01-10 23:31:05 -05:00
// we may have lost some votes if the process crashed
// reload from consensus log to catchup
if err := cs . catchupReplay ( cs . Height ) ; err != nil {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error on catchup replay. Proceeding to start ConsensusState anyway" , "err" , err . Error ( ) )
2017-03-28 12:07:32 -04:00
// NOTE: if we ever do return an error here,
// make sure to stop the timeoutTicker
2016-01-10 23:31:05 -05:00
}
2016-09-09 23:10:23 -04:00
// now start the receiveRoutine
2016-09-08 18:06:25 -04:00
go cs . receiveRoutine ( 0 )
2016-09-09 23:10:23 -04:00
// schedule the first round!
// use GetRoundState so we don't race the receiveRoutine for access
cs . scheduleRound0 ( cs . GetRoundState ( ) )
2015-08-04 18:44:15 -07:00
return nil
2015-07-20 14:40:41 -07:00
}
2015-12-12 17:22:48 -05:00
// timeoutRoutine: receive requests for timeouts on tickChan and fire timeouts on tockChan
// receiveRoutine: serializes processing of proposoals, block parts, votes; coordinates state transitions
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) startRoutines ( maxSteps int ) {
2016-12-19 22:29:32 -05:00
cs . timeoutTicker . Start ( )
2015-12-12 17:22:48 -05:00
go cs . receiveRoutine ( maxSteps )
2015-12-10 11:41:18 -05:00
}
2017-07-20 00:05:33 -04: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 ( )
2016-07-11 20:54:32 -04:00
2016-12-19 22:29:32 -05:00
cs . timeoutTicker . Stop ( )
2016-10-28 15:01:14 -07:00
// Make BaseService.Wait() wait until cs.wal.Wait()
2016-03-03 06:18:32 +00:00
if cs . wal != nil && cs . IsRunning ( ) {
2016-02-29 18:02:22 -05:00
cs . wal . Wait ( )
}
2014-10-30 03:32:09 -07:00
}
2017-07-20 00:05:33 -04:00
// Wait waits for the the main routine to return.
2017-01-12 14:44:42 -05: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-20 00:05:33 -04:00
// OpenWAL opens a file to log all consensus messages and timeouts for deterministic accountability
2017-02-17 19:12:05 -05:00
func ( cs * ConsensusState ) OpenWAL ( walFile string ) ( err error ) {
2017-09-18 17:49:43 -04:00
err = cmn . EnsureDir ( filepath . Dir ( walFile ) , 0700 )
2017-03-27 15:41:45 -04:00
if err != nil {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error ensuring ConsensusState wal dir" , "err" , err . Error ( ) )
2017-03-27 15:41:45 -04:00
return err
}
2016-01-06 18:42:12 -05:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
2017-05-01 20:09:29 -04:00
wal , err := NewWAL ( walFile , cs . config . WalLight )
2016-01-18 14:10:05 -05:00
if err != nil {
return err
2016-01-10 23:31:05 -05:00
}
2017-05-12 23:07:53 +02:00
wal . SetLogger ( cs . Logger . With ( "wal" , walFile ) )
if _ , err := wal . Start ( ) ; err != nil {
return err
}
2016-01-18 14:10:05 -05:00
cs . wal = wal
return nil
2016-01-06 18:42:12 -05:00
}
2015-12-12 17:22:48 -05:00
//------------------------------------------------------------
2017-07-20 00:05:33 -04: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 17:22:48 -05:00
// TODO: should these return anything or let callers just use events?
2015-12-12 01:28:33 -05:00
2017-07-20 00:05:33 -04:00
// AddVote inputs a vote.
2016-07-01 17:47:31 -04:00
func ( cs * ConsensusState ) AddVote ( vote * types . Vote , peerKey string ) ( added bool , err error ) {
2015-12-12 01:28:33 -05:00
if peerKey == "" {
2016-07-01 17:47:31 -04:00
cs . internalMsgQueue <- msgInfo { & VoteMessage { vote } , "" }
2015-12-12 01:28:33 -05:00
} else {
2016-07-01 17:47:31 -04:00
cs . peerMsgQueue <- msgInfo { & VoteMessage { vote } , peerKey }
2015-12-12 01:28:33 -05:00
}
// TODO: wait for event?!
2016-07-01 17:47:31 -04:00
return false , nil
2015-12-12 01:28:33 -05:00
}
2017-07-20 00:05:33 -04:00
// SetProposal inputs a proposal.
2015-12-12 01:28:33 -05:00
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-20 00:05:33 -04:00
// AddProposalBlockPart inputs a part of the proposal block.
2015-12-12 01:28:33 -05:00
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-20 00:05:33 -04:00
// SetProposalAndBlock inputs the proposal and all block parts.
2015-12-12 01:28:33 -05:00
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 14:56:05 -05:00
cs . AddProposalBlockPart ( proposal . Height , proposal . Round , part , peerKey )
2015-12-12 01:28:33 -05:00
}
return nil // TODO errors
}
2015-12-12 17:22:48 -05:00
//------------------------------------------------------------
2015-12-12 01:28:33 -05:00
// internal functions for managing the state
2015-12-08 16:00:59 -05:00
func ( cs * ConsensusState ) updateHeight ( height int ) {
cs . Height = height
}
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) updateRoundStep ( round int , step cstypes . RoundStepType ) {
2015-12-08 16:00:59 -05:00
cs . Round = round
cs . Step = step
}
2015-12-13 19:33:05 -05:00
// enterNewRound(height, 0) at cs.StartTime.
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) scheduleRound0 ( rs * cstypes . RoundState ) {
2017-05-02 11:53:32 +04:00
//cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
2016-09-09 23:10:23 -04:00
sleepDuration := rs . StartTime . Sub ( time . Now ( ) )
2017-10-10 12:39:21 +04:00
cs . scheduleTimeout ( sleepDuration , rs . Height , 0 , cstypes . RoundStepNewHeight )
2015-12-08 16:00:59 -05:00
}
2016-12-19 22:29:32 -05:00
// Attempt to schedule a timeout (by sending timeoutInfo on the tickChan)
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) scheduleTimeout ( duration time . Duration , height , round int , step cstypes . RoundStepType ) {
2016-12-19 22:29:32 -05:00
cs . timeoutTicker . ScheduleTimeout ( timeoutInfo { duration , height , round , step } )
2015-06-05 14:15:40 -07:00
}
2015-12-12 01:28:33 -05: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 :
2015-12-13 19:33:05 -05:00
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 11:53:32 +04:00
cs . Logger . Info ( "Internal msg queue is full. Using a go-routine" )
2015-12-12 01:28:33 -05:00
go func ( ) { cs . internalMsgQueue <- mi } ( )
}
}
2016-04-02 09:10:16 -07:00
// Reconstruct LastCommit from SeenCommit, which we saved along with the block,
2015-12-12 01:28:33 -05:00
// (which happens even before saving the state)
func ( cs * ConsensusState ) reconstructLastCommit ( state * sm . State ) {
if state . LastBlockHeight == 0 {
return
}
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
cmn . PanicCrisis ( cmn . Fmt ( "Failed to retrieve ChainID: %v" , err ) )
}
2016-04-02 09:10:16 -07:00
seenCommit := cs . blockStore . LoadSeenCommit ( state . LastBlockHeight )
2017-09-22 18:17:21 -06:00
lastPrecommits := types . NewVoteSet ( chainID , state . LastBlockHeight , seenCommit . Round ( ) , types . VoteTypePrecommit , state . LastValidators )
2016-07-01 17:47:31 -04:00
for _ , precommit := range seenCommit . Precommits {
2015-12-12 01:28:33 -05:00
if precommit == nil {
continue
}
2016-07-01 17:47:31 -04:00
added , err := lastPrecommits . AddVote ( precommit )
2015-12-12 01:28:33 -05:00
if ! added || err != nil {
2017-04-25 14:50:20 -04:00
cmn . PanicCrisis ( cmn . Fmt ( "Failed to reconstruct LastCommit: %v" , err ) )
2015-12-12 01:28:33 -05:00
}
}
if ! lastPrecommits . HasTwoThirdsMajority ( ) {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( "Failed to reconstruct LastCommit: Does not have +2/3 maj" )
2015-12-12 01:28:33 -05:00
}
cs . LastCommit = lastPrecommits
}
// Updates ConsensusState and increments height to match that of state.
2017-10-10 12:39:21 +04:00
// The round becomes 0 and cs.Step becomes cstypes.RoundStepNewHeight.
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) updateToState ( state * sm . State ) {
if cs . CommitRound > - 1 && 0 < cs . Height && cs . Height != state . LastBlockHeight {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "updateToState() expected state height of %v but found %v" ,
2015-12-12 01:28:33 -05:00
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?!
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "Inconsistent cs.state.LastBlockHeight+1 %v vs cs.Height %v" ,
2015-12-12 01:28:33 -05:00
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 11:53:32 +04:00
cs . Logger . Info ( "Ignoring updateToState()" , "newHeight" , state . LastBlockHeight + 1 , "oldHeight" , cs . state . LastBlockHeight + 1 )
2015-12-12 01:28:33 -05:00
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 ( ) {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( "updateToState(state) called but last Precommit round didn't have +2/3" )
2015-12-12 01:28:33 -05:00
}
lastPrecommits = cs . Votes . Precommits ( cs . CommitRound )
}
2017-02-17 10:51:05 -05:00
// Next desired block height
height := state . LastBlockHeight + 1
2015-12-12 01:28:33 -05:00
// RoundState fields
cs . updateHeight ( height )
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( 0 , cstypes . RoundStepNewHeight )
2015-12-12 01:28:33 -05:00
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)
2017-05-01 20:09:29 -04:00
cs . StartTime = cs . config . Commit ( time . Now ( ) )
2015-12-12 01:28:33 -05:00
} else {
2017-05-01 20:09:29 -04:00
cs . StartTime = cs . config . Commit ( cs . CommitTime )
2015-12-12 01:28:33 -05:00
}
cs . Validators = validators
cs . Proposal = nil
cs . ProposalBlock = nil
cs . ProposalBlockParts = nil
cs . LockedRound = 0
cs . LockedBlock = nil
cs . LockedBlockParts = nil
2017-10-10 12:39:21 +04:00
cs . Votes = cstypes . NewHeightVoteSet ( state . ChainID , height , validators )
2015-12-12 01:28:33 -05:00
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 15:23:22 -05:00
rs := cs . RoundStateEvent ( )
2016-01-18 14:10:05 -05:00
cs . wal . Save ( rs )
2015-12-12 01:28:33 -05:00
cs . nSteps += 1
2015-12-13 19:30:15 -05:00
// newStep is called by updateToStep in NewConsensusState before the evsw is set!
if cs . evsw != nil {
2016-10-10 02:58:13 -04:00
types . FireEventNewRoundStep ( cs . evsw , rs )
2015-12-13 19:30:15 -05:00
}
2015-12-12 01:28:33 -05:00
}
//-----------------------------------------
// the main go routines
2015-12-11 11:57:15 -05:00
// receiveRoutine handles messages which may cause state transitions.
2015-12-12 01:28:33 -05:00
// it's argument (n) is the number of messages to process before exiting - use 0 to run forever
2015-12-11 11:57:15 -05:00
// It keeps the RoundState and is the only thing that updates it.
2017-07-25 10:52:14 -04:00
// Updates (state transitions) happen on timeouts, complete proposals, and 2/3 majorities.
// ConsensusState must be locked before any internal state is updated.
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) receiveRoutine ( maxSteps int ) {
2017-08-16 01:01:09 -04:00
defer func ( ) {
if r := recover ( ) ; r != nil {
2017-10-02 15:24:30 -04:00
cs . Logger . Error ( "CONSENSUS FAILURE!!!" , "err" , r , "stack" , string ( debug . Stack ( ) ) )
2017-08-16 01:01:09 -04:00
}
} ( )
2015-12-10 11:41:18 -05:00
for {
2015-12-12 01:28:33 -05:00
if maxSteps > 0 {
if cs . nSteps >= maxSteps {
2017-05-12 23:07:53 +02:00
cs . Logger . Info ( "reached max steps. exiting receive routine" )
2015-12-12 01:28:33 -05:00
cs . nSteps = 0
return
}
}
2015-12-11 11:57:15 -05:00
rs := cs . RoundState
2015-12-10 11:41:18 -05:00
var mi msgInfo
select {
2017-07-25 13:57:11 -04:00
case height := <- cs . mempool . TxsAvailable ( ) :
cs . handleTxsAvailable ( height )
2015-12-11 11:57:15 -05:00
case mi = <- cs . peerMsgQueue :
2016-01-18 14:10:05 -05:00
cs . wal . Save ( mi )
2015-12-10 11:41:18 -05:00
// handles proposals, block parts, votes
// may generate internal events (votes, complete proposals, 2/3 majorities)
2017-07-25 10:52:14 -04:00
cs . handleMsg ( mi )
2015-12-11 11:57:15 -05:00
case mi = <- cs . internalMsgQueue :
2016-01-18 14:10:05 -05:00
cs . wal . Save ( mi )
2015-12-11 11:57:15 -05:00
// handles proposals, block parts, votes
2017-07-25 10:52:14 -04:00
cs . handleMsg ( mi )
2016-12-19 22:29:32 -05:00
case ti := <- cs . timeoutTicker . Chan ( ) : // tockChan:
2016-01-18 14:10:05 -05:00
cs . wal . Save ( ti )
2015-12-10 11:41:18 -05:00
// if the timeout is relevant to the rs
// go to the next step
cs . handleTimeout ( ti , rs )
case <- cs . Quit :
2016-02-29 18:02:22 -05:00
2016-12-19 10:44:25 -05:00
// 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-02-29 18:02:22 -05:00
2016-01-18 14:10:05 -05: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 14:10:05 -05:00
}
2017-01-12 14:44:42 -05:00
close ( cs . done )
2015-12-10 11:41:18 -05:00
return
}
}
}
2015-12-11 11:57:15 -05:00
// state transitions on complete-proposal, 2/3-any, 2/3-one
2017-07-25 10:52:14 -04:00
func ( cs * ConsensusState ) handleMsg ( mi msgInfo ) {
2015-12-12 01:28:33 -05:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
2015-12-10 11:41:18 -05:00
var err error
2015-12-22 15:23:22 -05:00
msg , peerKey := mi . Msg , mi . PeerKey
2015-12-10 11:41:18 -05:00
switch msg := msg . ( type ) {
case * ProposalMessage :
2015-12-11 11:57:15 -05:00
// will not cause transition.
// once proposal is set, we can receive block parts
2015-12-12 01:28:33 -05:00
err = cs . setProposal ( msg . Proposal )
2015-12-10 11:41:18 -05:00
case * BlockPartMessage :
2015-12-13 19:33:05 -05:00
// if the proposal is complete, we'll enterPrevote or tryFinalizeCommit
2016-03-11 21:38:15 -05:00
_ , err = cs . addProposalBlockPart ( msg . Height , msg . Part , peerKey != "" )
2016-03-02 21:38:05 +00:00
if err != nil && msg . Round != cs . Round {
err = nil
}
2015-12-10 11:41:18 -05:00
case * VoteMessage :
// attempt to add the vote and dupeout the validator if its a duplicate signature
2015-12-11 11:57:15 -05:00
// if the vote gives us a 2/3-any or 2/3-one, we transition
2017-06-23 22:12:45 -04:00
err := cs . tryAddVote ( msg . Vote , peerKey )
2015-12-10 11:41:18 -05:00
if err == ErrAddingVote {
// TODO: punish peer
}
2015-12-13 19:30:15 -05:00
// NOTE: the vote is broadcast to peers by the reactor listening
// for vote events
2015-12-10 11:41:18 -05:00
2015-12-13 19:30:15 -05:00
// 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().
2015-12-10 11:41:18 -05:00
default :
2017-05-02 11:53:32 +04:00
cs . Logger . Error ( "Unknown msg type" , reflect . TypeOf ( msg ) )
2015-12-10 11:41:18 -05:00
}
if err != nil {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error with msg" , "type" , reflect . TypeOf ( msg ) , "peer" , peerKey , "err" , err , "msg" , msg )
2015-12-10 11:41:18 -05:00
}
}
2017-10-10 12:39:21 +04:00
func ( cs * ConsensusState ) handleTimeout ( ti timeoutInfo , rs cstypes . RoundState ) {
2017-05-02 11:53:32 +04:00
cs . Logger . Debug ( "Received tock" , "timeout" , ti . Duration , "height" , ti . Height , "round" , ti . Round , "step" , ti . Step )
2015-12-12 01:28:33 -05:00
2015-12-10 11:41:18 -05:00
// timeouts must be for current height, round, step
2015-12-22 15:23:22 -05:00
if ti . Height != rs . Height || ti . Round < rs . Round || ( ti . Round == rs . Round && ti . Step < rs . Step ) {
2017-05-02 11:53:32 +04:00
cs . Logger . Debug ( "Ignoring tock because we're ahead" , "height" , rs . Height , "round" , rs . Round , "step" , rs . Step )
2015-12-10 11:41:18 -05:00
return
}
2015-12-11 11:57:15 -05:00
// the timeout will now cause a state transition
2015-12-12 01:28:33 -05:00
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
2015-12-11 11:57:15 -05:00
2015-12-22 15:23:22 -05:00
switch ti . Step {
2017-10-10 12:39:21 +04:00
case cstypes . RoundStepNewHeight :
2015-12-13 19:33:05 -05:00
// NewRound event fired from enterNewRound.
2016-12-19 10:44:25 -05:00
// XXX: should we fire timeout here (for timeout commit)?
2015-12-22 15:23:22 -05:00
cs . enterNewRound ( ti . Height , 0 )
2017-10-10 12:39:21 +04:00
case cstypes . RoundStepNewRound :
2017-08-04 21:46:17 -04:00
cs . enterPropose ( ti . Height , 0 )
2017-10-10 12:39:21 +04:00
case cstypes . RoundStepPropose :
2016-10-10 02:58:13 -04:00
types . FireEventTimeoutPropose ( cs . evsw , cs . RoundStateEvent ( ) )
2015-12-22 15:23:22 -05:00
cs . enterPrevote ( ti . Height , ti . Round )
2017-10-10 12:39:21 +04:00
case cstypes . RoundStepPrevoteWait :
2016-10-10 02:58:13 -04:00
types . FireEventTimeoutWait ( cs . evsw , cs . RoundStateEvent ( ) )
2015-12-22 15:23:22 -05:00
cs . enterPrecommit ( ti . Height , ti . Round )
2017-10-10 12:39:21 +04:00
case cstypes . RoundStepPrecommitWait :
2016-10-10 02:58:13 -04:00
types . FireEventTimeoutWait ( cs . evsw , cs . RoundStateEvent ( ) )
2015-12-22 15:23:22 -05:00
cs . enterNewRound ( ti . Height , ti . Round + 1 )
2015-12-10 11:41:18 -05:00
default :
2017-04-25 14:50:20 -04:00
panic ( cmn . Fmt ( "Invalid timeout step: %v" , ti . Step ) )
2015-12-10 11:41:18 -05:00
}
}
2017-07-25 13:57:11 -04:00
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
//-----------------------------------------------------------------------------
2015-12-12 01:28:33 -05:00
// State functions
2015-12-14 00:38:19 -05:00
// Used internally by handleTimeout and handleMsg to make state transitions
2014-08-10 16:35:08 -07:00
2017-07-20 00:05:33 -04:00
// Enter: `timeoutNewHeight` by startTime (commitTime+timeoutCommit),
// or, if SkipTimeout==true, after receiving all precommits from (height,round-1)
2015-06-24 17:05:52 -07:00
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
2017-07-20 00:05:33 -04: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.
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterNewRound ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cs . Step != cstypes . RoundStepNewHeight ) {
2017-05-02 11:53:32 +04: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 16:45:53 -04:00
2015-06-05 14:15:40 -07:00
if now := time . Now ( ) ; cs . StartTime . After ( now ) {
2017-05-02 11:53:32 +04: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
}
2015-12-11 11:57:15 -05:00
2017-05-02 11:53:32 +04: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 16:25:49 -05:00
// we don't fire newStep for this step,
// but we fire an event, so update the round step first
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepNewRound )
2015-06-05 14:15:40 -07:00
cs . Validators = validators
2015-07-05 15:35:26 -07:00
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
}
2015-06-24 17:05:52 -07:00
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-10 02:58:13 -04:00
types . FireEventNewRound ( cs . evsw , cs . RoundStateEvent ( ) )
2015-09-09 16:45:53 -04:00
2017-07-12 01:02:16 -04:00
// Wait for txs to be available in the mempool
2017-07-25 13:57:11 -04:00
// before we enterPropose in round 0. If the last block changed the app hash,
2017-07-20 14:43:16 -04:00
// we may need an empty "proof" block, and enterPropose immediately.
2017-08-04 21:46:17 -04:00
waitForTxs := cs . config . WaitForTxs ( ) && round == 0 && ! cs . needProofBlock ( height )
2017-07-25 13:57:11 -04:00
if waitForTxs {
2017-08-04 21:46:17 -04:00
if cs . config . CreateEmptyBlocksInterval > 0 {
2017-10-10 12:39:21 +04:00
cs . scheduleTimeout ( cs . config . EmptyBlocksInterval ( ) , height , round , cstypes . RoundStepNewRound )
2017-08-04 21:46:17 -04:00
}
2017-08-03 13:25:26 -04:00
go cs . proposalHeartbeat ( height , round )
2017-07-13 13:19:44 -04:00
} else {
cs . enterPropose ( height , round )
}
2017-07-12 01:02:16 -04:00
}
2017-07-20 14:43:16 -04: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 13:25:26 -04:00
func ( cs * ConsensusState ) proposalHeartbeat ( height , round int ) {
2017-07-20 15:09:44 -04:00
counter := 0
2017-09-21 16:32:02 -04:00
addr := cs . privValidator . GetAddress ( )
2017-07-29 14:15:10 -04:00
valIndex , v := cs . Validators . GetByAddress ( addr )
if v == nil {
// not a validator
valIndex = - 1
}
2017-07-12 01:02:16 -04:00
for {
2017-08-03 13:25:26 -04:00
rs := cs . GetRoundState ( )
// if we've already moved on, no need to send more heartbeats
2017-10-10 12:39:21 +04:00
if rs . Step > cstypes . RoundStepNewRound || rs . Round > round || rs . Height > height {
2017-08-03 13:25:26 -04:00
return
}
heartbeat := & types . Heartbeat {
Height : rs . Height ,
Round : rs . Round ,
Sequence : counter ,
ValidatorAddress : addr ,
ValidatorIndex : valIndex ,
2017-07-12 01:02:16 -04:00
}
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
return
}
cs . privValidator . SignHeartbeat ( chainID , heartbeat )
2017-08-03 13:25:26 -04:00
heartbeatEvent := types . EventDataProposalHeartbeat { heartbeat }
types . FireEventProposalHeartbeat ( cs . evsw , heartbeatEvent )
counter += 1
2017-08-04 21:36:11 -04:00
time . Sleep ( proposalHeartbeatIntervalSeconds * time . Second )
2017-07-12 01:02:16 -04:00
}
2014-10-30 03:32:09 -07:00
}
2017-08-04 21:46:17 -04: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
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterPropose ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cstypes . RoundStepPropose <= cs . Step ) {
2017-05-02 11:53:32 +04: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 11:53:32 +04: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 16:25:49 -05:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterPropose:
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepPropose )
2015-12-12 16:25:49 -05:00
cs . newStep ( )
2015-12-14 00:38:19 -05:00
// 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 16:25:49 -05:00
} ( )
2016-02-29 16:15:23 -05:00
// If we don't get the proposal and all block parts quick enough, enterPrevote
2017-10-10 12:39:21 +04:00
cs . scheduleTimeout ( cs . config . Propose ( round ) , height , round , cstypes . RoundStepPropose )
2014-09-14 15:37:32 -07:00
2015-06-24 17:05:52 -07:00
// Nothing more to do if we're not a validator
2015-04-20 20:39:42 -07:00
if cs . privValidator == nil {
2017-05-18 11:26:15 +02:00
cs . Logger . Debug ( "This node is not a validator" )
2015-01-08 22:07:23 -08:00
return
}
2015-06-05 14:15:40 -07:00
2017-07-11 19:18:15 -04:00
if ! cs . isProposer ( ) {
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( "enterPropose: Not our turn to propose" , "proposer" , cs . Validators . GetProposer ( ) . Address , "privValidator" , cs . privValidator )
2017-09-21 16:32:02 -04:00
if cs . Validators . HasAddress ( cs . privValidator . GetAddress ( ) ) {
2017-05-18 11:26:15 +02:00
cs . Logger . Debug ( "This node is a validator" )
2017-05-15 11:02:40 +02:00
} else {
2017-05-18 11:26:15 +02:00
cs . Logger . Debug ( "This node is not a validator" )
2017-05-15 11:02:40 +02:00
}
2015-01-08 22:07:23 -08:00
} else {
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( "enterPropose: Our turn to propose" , "proposer" , cs . Validators . GetProposer ( ) . Address , "privValidator" , cs . privValidator )
2017-05-18 11:26:15 +02:00
cs . Logger . Debug ( "This node is a validator" )
2015-08-26 18:56:34 -04:00
cs . decideProposal ( height , round )
2016-06-26 15:33:11 -04:00
}
2015-06-24 17:05:52 -07:00
}
2014-08-10 16:35:08 -07:00
2017-07-11 19:18:15 -04:00
func ( cs * ConsensusState ) isProposer ( ) bool {
2017-09-21 16:32:02 -04:00
return bytes . Equal ( cs . Validators . GetProposer ( ) . Address , cs . privValidator . GetAddress ( ) )
2017-07-11 19:18:15 -04:00
}
2016-06-26 15:33:11 -04:00
func ( cs * ConsensusState ) defaultDecideProposal ( height , round int ) {
2015-03-22 19:00:08 -07:00
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
2016-08-20 15:08:26 -07:00
polRound , polBlockID := cs . Votes . POLInfo ( )
proposal := types . NewProposal ( height , round , blockParts . Header ( ) , polRound , polBlockID )
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
return
}
if err := cs . privValidator . SignProposal ( chainID , proposal ) ; err == nil {
2014-12-31 16:14:26 -08:00
// Set fields
2015-12-12 01:28:33 -05:00
/ * fields set by setProposal and addBlockPart
2014-12-31 16:14:26 -08:00
cs . Proposal = proposal
2015-01-15 22:43:15 -08:00
cs . ProposalBlock = block
2014-12-31 16:14:26 -08:00
cs . ProposalBlockParts = blockParts
2015-12-12 01:28:33 -05:00
* /
2015-12-05 14:58:12 -05:00
2015-12-12 01:28:33 -05:00
// send proposal and block parts on internal msg queue
2015-12-11 11:57:15 -05:00
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 11:53:32 +04:00
cs . Logger . Info ( "Signed proposal" , "height" , height , "round" , round , "proposal" , proposal )
cs . Logger . Debug ( cmn . Fmt ( "Signed proposal block: %v" , block ) )
2014-12-31 16:14:26 -08:00
} else {
2016-09-08 18:06:25 -04:00
if ! cs . replayMode {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "enterPropose: Error signing proposal" , "height" , height , "round" , round , "err" , err )
2016-09-08 18:06:25 -04:00
}
2015-06-05 14:15:40 -07:00
}
}
2015-06-24 17:05:52 -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 14:00:23 -04: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 14:00:23 -04:00
// if this is false the proposer is lying or we haven't received the POL yet
2015-06-25 20:28:34 -07:00
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 18:56:34 -04: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 11:53:32 +04: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
2016-01-06 17:14:20 -08:00
// Mempool validated transactions
2017-05-01 20:09:29 -04:00
txs := cs . mempool . Reap ( cs . config . MaxBlockSizeTxs )
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
cs . Logger . Error ( "chainID err" , err )
return
}
return types . MakeBlock ( cs . Height , chainID , txs , commit ,
2017-09-11 16:28:00 -04:00
cs . state . LastBlockID , cs . state . Validators . Hash ( ) ,
2017-09-12 15:20:19 -04:00
cs . state . AppHash , cs . state . Params ( ) . BlockPartSizeBytes )
2014-08-10 16:35:08 -07:00
}
2015-06-24 17:05:52 -07:00
// Enter: `timeoutPropose` after entering Propose.
2015-06-05 14:15:40 -07:00
// Enter: proposal block and POL is ready.
2015-06-24 17:05:52 -07:00
// 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-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterPrevote ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cstypes . RoundStepPrevote <= cs . Step ) {
2017-05-02 11:53:32 +04: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 16:45:53 -04:00
2015-12-12 16:25:49 -05:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterPrevote:
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepPrevote )
2015-12-12 16:25:49 -05:00
cs . newStep ( )
} ( )
2015-09-09 16:45:53 -04:00
// fire event for how we got here
2015-12-11 11:57:15 -05:00
if cs . isProposalComplete ( ) {
2016-10-10 02:58:13 -04:00
types . FireEventCompleteProposal ( cs . evsw , cs . RoundStateEvent ( ) )
2015-09-09 16:45:53 -04:00
} else {
// we received +2/3 prevotes for a future round
// TODO: catchup event?
}
2017-05-02 11:53:32 +04: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-06-24 17:05:52 -07:00
2015-09-09 16:45:53 -04: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 15:33:11 -04:00
func ( cs * ConsensusState ) defaultDoPrevote ( height int , round int ) {
2017-07-07 16:58:16 -04: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 16:58:16 -04:00
logger . Info ( "enterPrevote: Block was locked" )
2015-03-22 19:00:08 -07:00
cs . signAddVote ( types . VoteTypePrevote , cs . LockedBlock . Hash ( ) , cs . LockedBlockParts . Header ( ) )
2014-10-30 03:32:09 -07:00
return
2014-10-24 14:37:12 -07:00
}
2014-10-30 03:32:09 -07:00
2014-10-24 14:37:12 -07:00
// If ProposalBlock is nil, prevote nil.
if cs . ProposalBlock == nil {
2017-07-07 16:58:16 -04:00
logger . Info ( "enterPrevote: ProposalBlock is nil" )
2015-03-22 19:00:08 -07:00
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-20 00:05:33 -04:00
// Validate proposal block
2016-01-06 17:14:20 -08:00
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 16:58:16 -04:00
logger . Error ( "enterPrevote: ProposalBlock is invalid" , "err" , err )
2015-03-22 19:00:08 -07:00
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 14:00:23 -04: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 18:01:25 -04:00
logger . Info ( "enterPrevote: ProposalBlock is valid" )
2015-03-22 19:00:08 -07:00
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.
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterPrevoteWait ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cstypes . RoundStepPrevoteWait <= cs . Step ) {
2017-05-02 11:53:32 +04: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 ( ) {
2017-04-25 14:50:20 -04:00
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 11:53:32 +04: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 16:25:49 -05:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterPrevoteWait:
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepPrevoteWait )
2015-12-12 16:25:49 -05:00
cs . newStep ( )
} ( )
2015-06-05 14:15:40 -07:00
2016-02-29 16:15:23 -05:00
// Wait for some more prevotes; enterPrecommit
2017-10-10 12:39:21 +04:00
cs . scheduleTimeout ( cs . config . Prevote ( round ) , height , round , cstypes . RoundStepPrevoteWait )
2015-06-05 14:15:40 -07:00
}
// Enter: `timeoutPrevote` after any +2/3 prevotes.
2017-07-20 00:05:33 -04: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 18:56:34 -04: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 14:00:23 -04:00
// else, precommit nil otherwise.
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterPrecommit ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cstypes . RoundStepPrecommit <= cs . Step ) {
2017-05-02 11:53:32 +04: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 16:45:53 -04:00
2017-05-02 11:53:32 +04: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
2014-11-01 04:04:58 -07:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterPrecommit:
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepPrecommit )
2015-12-12 01:28:33 -05:00
cs . newStep ( )
2014-11-01 04:04:58 -07:00
} ( )
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 16:45:53 -04: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 11:53:32 +04: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 11:53:32 +04:00
cs . Logger . Info ( "enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil." )
2015-06-05 14:15:40 -07:00
}
2015-08-12 14:00:23 -04: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 14:00:23 -04:00
// At this point +2/3 prevoted for a particular block or nil
2016-10-10 02:58:13 -04:00
types . FireEventPolka ( cs . evsw , cs . RoundStateEvent ( ) )
2015-09-09 16:45:53 -04:00
// the latest POLRound should be this round
2016-08-20 15:08:26 -07:00
polRound , _ := cs . Votes . POLInfo ( )
if polRound < round {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "This POLRound should be %v but got %" , round , polRound ) )
2015-09-09 16:45:53 -04:00
}
2015-08-12 14:00:23 -04: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 11:53:32 +04:00
cs . Logger . Info ( "enterPrecommit: +2/3 prevoted for nil." )
2015-05-04 10:15:58 -07:00
} else {
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( "enterPrecommit: +2/3 prevoted for nil. Unlocking" )
2015-08-26 18:56:34 -04:00
cs . LockedRound = 0
2015-05-04 10:15:58 -07:00
cs . LockedBlock = nil
cs . LockedBlockParts = nil
2016-10-10 02:58:13 -04: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 14:00:23 -04: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 11:53:32 +04:00
cs . Logger . Info ( "enterPrecommit: +2/3 prevoted locked block. Relocking" )
2015-08-12 14:00:23 -04:00
cs . LockedRound = round
2016-10-10 02:58:13 -04: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 11:53:32 +04: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.
2016-01-06 17:14:20 -08:00
if err := cs . state . ValidateBlock ( cs . ProposalBlock ) ; err != nil {
2017-04-25 14:50:20 -04:00
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-10 02:58:13 -04: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 16:45:53 -04:00
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
2015-06-24 17:05:52 -07:00
// The +2/3 prevotes for this round is the POL for our unlock.
2015-08-19 16:11:52 -04:00
// 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-10 02:58:13 -04: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.
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterPrecommitWait ( height int , round int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && cstypes . RoundStepPrecommitWait <= cs . Step ) {
2017-05-02 11:53:32 +04: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 ( ) {
2017-04-25 14:50:20 -04:00
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 11:53:32 +04: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 16:25:49 -05:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterPrecommitWait:
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( round , cstypes . RoundStepPrecommitWait )
2015-12-12 16:25:49 -05:00
cs . newStep ( )
} ( )
2015-06-05 14:15:40 -07:00
2016-02-29 16:15:23 -05:00
// Wait for some more precommits; enterNewRound
2017-10-10 12:39:21 +04:00
cs . scheduleTimeout ( cs . config . Precommit ( round ) , height , round , cstypes . RoundStepPrecommitWait )
2015-12-08 16:00:59 -05:00
2014-08-10 16:35:08 -07:00
}
2015-06-05 14:15:40 -07:00
// Enter: +2/3 precommits for block
2015-12-13 19:33:05 -05:00
func ( cs * ConsensusState ) enterCommit ( height int , commitRound int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || cstypes . RoundStepCommit <= cs . Step {
2017-05-02 11:53:32 +04: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 11:53:32 +04: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
2014-11-01 04:04:58 -07:00
defer func ( ) {
2015-12-13 19:33:05 -05:00
// Done enterCommit:
2016-03-05 01:18:14 -05:00
// keep cs.Round the same, commitRound points to the right Precommits set.
2017-10-10 12:39:21 +04:00
cs . updateRoundStep ( cs . Round , cstypes . RoundStepCommit )
2015-09-15 16:13:39 -04:00
cs . CommitRound = commitRound
2016-12-19 10:44:25 -05:00
cs . CommitTime = time . Now ( )
2015-12-12 01:28:33 -05:00
cs . newStep ( )
2015-06-05 14:15:40 -07:00
// Maybe finalize immediately.
2015-09-15 16:13:39 -04:00
cs . tryFinalizeCommit ( height )
2014-11-01 04:04:58 -07:00
} ( )
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 {
2017-04-25 14:50:20 -04:00
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 18:56:34 -04: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.
}
2014-12-09 18:49:04 -08:00
}
2015-06-05 14:15:40 -07:00
}
2014-11-01 04:04:58 -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 16:13:39 -04:00
func ( cs * ConsensusState ) tryFinalizeCommit ( height int ) {
2015-06-24 14:04:40 -07:00
if cs . Height != height {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "tryFinalizeCommit() cs.Height: %v vs height: %v" , cs . Height , height ) )
2014-10-21 18:30:03 -07:00
}
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 11:53:32 +04:00
cs . Logger . Error ( "Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>." , "height" , height )
2015-08-19 16:11:52 -04:00
return
2015-06-24 14:04:40 -07:00
}
2016-08-16 14:59:19 -07:00
if ! cs . ProposalBlock . HashesTo ( blockID . Hash ) {
2016-01-20 13:12:42 -05:00
// TODO: this happens every time if we're not a validator (ugly logs)
2016-10-17 11:29:43 -07:00
// TODO: ^^ wait, why does it matter that we're a validator?
2017-05-02 11:53:32 +04: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 )
2015-08-19 16:11:52 -04:00
return
2015-06-24 14:04:40 -07:00
}
2017-05-15 11:02:40 +02:00
2015-12-10 11:41:18 -05:00
// go
2015-12-14 00:38:19 -05:00
cs . finalizeCommit ( height )
2014-10-21 18:30:03 -07:00
}
2017-10-10 12:39:21 +04:00
// Increment height and goto cstypes.RoundStepNewHeight
2015-12-14 00:38:19 -05:00
func ( cs * ConsensusState ) finalizeCommit ( height int ) {
2017-10-10 12:39:21 +04:00
if cs . Height != height || cs . Step != cstypes . RoundStepCommit {
2017-05-02 11:53:32 +04: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 ( )
2016-01-06 17:14:20 -08:00
block , blockParts := cs . ProposalBlock , cs . ProposalBlockParts
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if ! ok {
2017-04-25 14:50:20 -04:00
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 ) {
2017-04-25 14:50:20 -04:00
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 ) {
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "Cannot finalizeCommit, ProposalBlock does not hash to commit hash" ) )
2015-06-05 14:15:40 -07:00
}
2016-01-06 17:14:20 -08:00
if err := cs . state . ValidateBlock ( block ) ; err != nil {
2017-04-25 14:50:20 -04:00
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 11:53:32 +04:00
cs . Logger . Info ( cmn . Fmt ( "Finalizing commit of block with %d txs" , block . NumTxs ) ,
2016-08-25 01:39:03 -04:00
"height" , block . Height , "hash" , block . Hash ( ) , "root" , block . AppHash )
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( cmn . Fmt ( "%v" , block ) )
2016-01-06 17:14:20 -08:00
2016-09-11 15:32:33 -04:00
fail . Fail ( ) // XXX
2016-09-11 13:16:23 -04:00
// Save to blockStore.
if cs . blockStore . Height ( ) < block . Height {
2016-11-19 19:32:35 -05:00
// NOTE: the seenCommit is local justification to commit this block,
// but may differ from the LastCommit included in the next block
2016-09-11 13:16:23 -04:00
precommits := cs . Votes . Precommits ( cs . CommitRound )
seenCommit := precommits . MakeCommit ( )
cs . blockStore . SaveBlock ( block , blockParts , seenCommit )
} else {
2017-02-17 10:57:09 -05:00
// Happens during replay if we already saved the block but didn't commit
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( "Calling finalizeCommit on already stored block" , "height" , block . Height )
2016-09-11 13:16:23 -04:00
}
2016-01-06 17:14:20 -08:00
2016-09-11 15:32:33 -04:00
fail . Fail ( ) // XXX
2017-04-14 20:30:15 -04:00
// Finish writing to the WAL for this height.
2017-04-15 01:33:30 -04: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)
2017-04-14 20:30:15 -04:00
if cs . wal != nil {
cs . wal . writeEndHeight ( height )
}
fail . Fail ( ) // XXX
2016-01-06 17:14:20 -08:00
// Create a copy of the state for staging
2016-09-11 13:16:23 -04:00
// and an event cache for txs
2016-01-06 17:14:20 -08:00
stateCopy := cs . state . Copy ( )
2016-10-10 02:58:13 -04:00
eventCache := types . NewEventCache ( cs . evsw )
2016-07-05 17:03:09 -04:00
2017-04-14 15:33:19 -04:00
// Execute and commit the block, update and save the state, and update the mempool.
2017-04-14 20:30:15 -04:00
// All calls to the proxyAppConn come here.
2016-09-11 13:16:23 -04:00
// NOTE: the block.AppHash wont reflect these txs until the next block
2016-11-30 17:28:41 -05:00
err := stateCopy . ApplyBlock ( eventCache , cs . proxyAppConn , block , blockParts . Header ( ) , cs . mempool )
if err != nil {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error on ApplyBlock. Did the application crash? Please restart tendermint" , "err" , err )
2017-02-17 10:57:09 -05:00
return
2016-11-30 17:28:41 -05:00
}
2016-01-06 17:14:20 -08:00
2016-09-11 15:32:33 -04:00
fail . Fail ( ) // XXX
2017-04-14 20:30:15 -04:00
// Fire event for new block.
// NOTE: If we fail before firing, these events will never fire
//
2017-04-15 01:33:30 -04:00
// TODO: Either
2017-05-15 11:02:40 +02:00
// * Fire before persisting state, in ApplyBlock
2017-04-14 20:30:15 -04:00
// * Fire on start up if we haven't written any new WAL msgs
2017-04-15 01:33:30 -04:00
// Both options mean we may fire more than once. Is that fine ?
2016-09-11 13:16:23 -04:00
types . FireEventNewBlock ( cs . evsw , types . EventDataNewBlock { block } )
types . FireEventNewBlockHeader ( cs . evsw , types . EventDataNewBlockHeader { block . Header } )
2016-07-05 17:03:09 -04:00
eventCache . Flush ( )
2016-09-11 15:32:33 -04:00
fail . Fail ( ) // XXX
2015-12-12 17:22:48 -05:00
// NewHeightStep!
2016-01-06 17:14:20 -08:00
cs . updateToState ( stateCopy )
2015-12-12 01:28:33 -05:00
2017-04-15 01:33:30 -04: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 23:10:23 -04:00
cs . scheduleRound0 ( & cs . RoundState )
2015-06-05 14:15:40 -07:00
// By here,
// * cs.Height has been increment to height+1
2017-10-10 12:39:21 +04:00
// * cs.Step is now cstypes.RoundStepNewHeight
2015-06-24 14:04:40 -07:00
// * cs.StartTime is set to when we will start round0.
2014-10-20 19:02:10 -07:00
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
2016-06-26 15:33:11 -04:00
func ( cs * ConsensusState ) defaultSetProposal ( proposal * types . Proposal ) error {
2014-10-21 23:30:18 -07:00
// Already have one
2016-06-26 15:33:11 -04: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
}
2017-10-10 12:39:21 +04:00
// We don't care about the proposal if we're already in cstypes.RoundStepCommit.
if cstypes . 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 &&
2015-06-25 20:28:34 -07:00
( proposal . POLRound < 0 || proposal . Round <= proposal . POLRound ) {
2015-06-22 19:04:31 -07:00
return ErrInvalidProposalPOLRound
}
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
return err
}
2014-10-21 23:30:18 -07:00
// Verify signature
2017-09-22 18:17:21 -06:00
if ! cs . Validators . GetProposer ( ) . PubKey . VerifyBytes ( types . SignBytes ( 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.
2016-03-02 21:38:05 +00:00
// Asynchronously triggers either enterPrevote (before we timeout of propose) or tryFinalizeCommit, once we have the full block.
2016-03-11 21:38:15 -05: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 21:38:15 -05: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
2017-09-11 16:28:00 -04:00
cs . ProposalBlock = wire . ReadBinary ( & types . Block { } , cs . ProposalBlockParts . GetReader ( ) ,
2017-09-16 00:16:49 -04:00
cs . state . Params ( ) . BlockSizeParams . MaxBytes , & n , & err ) . ( * types . Block )
2015-12-22 23:24:15 -05:00
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
2017-05-12 23:07:53 +02:00
cs . Logger . Info ( "Received complete proposal block" , "height" , cs . ProposalBlock . Height , "hash" , cs . ProposalBlock . Hash ( ) )
2017-10-10 12:39:21 +04:00
if cs . Step == cstypes . RoundStepPropose && cs . isProposalComplete ( ) {
2015-06-24 18:51:14 -07:00
// Move onto the next step
2015-12-13 19:33:05 -05:00
cs . enterPrevote ( height , cs . Round )
2017-10-10 12:39:21 +04:00
} else if cs . Step == cstypes . RoundStepCommit {
2015-06-24 18:51:14 -07:00
// If we're waiting on the proposal block...
2015-09-15 16:13:39 -04:00
cs . tryFinalizeCommit ( height )
2014-12-09 18:49:04 -08:00
}
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 14:00:23 -04:00
// Attempt to add the vote. if its a duplicate signature, dupeout the validator
2016-07-01 17:47:31 -04:00
func ( cs * ConsensusState ) tryAddVote ( vote * types . Vote , peerKey string ) error {
_ , err := cs . addVote ( vote , peerKey )
2015-08-12 14:00:23 -04:00
if err != nil {
2015-08-26 18:56:34 -04:00
// If the vote height is off, we'll just ignore it,
2015-09-09 16:45:53 -04:00
// But if it's a conflicting sig, broadcast evidence tx for slashing.
// If it's otherwise invalid, punish peer.
2015-08-26 18:56:34 -04:00
if err == ErrVoteHeightMismatch {
2015-12-13 19:30:15 -05:00
return err
2016-07-24 14:32:08 -07:00
} else if _ , ok := err . ( * types . ErrVoteConflictingVotes ) ; ok {
2017-09-21 16:32:02 -04:00
if bytes . Equal ( vote . ValidatorAddress , cs . privValidator . GetAddress ( ) ) {
2017-05-02 11:53:32 +04: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
}
2017-07-07 13:41:50 -04:00
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
2015-12-13 19:30:15 -05:00
return err
2015-08-12 14:00:23 -04:00
} else {
// Probably an invalid signature. Bad peer.
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error attempting to add vote" , "err" , err )
2015-12-13 19:30:15 -05:00
return ErrAddingVote
2015-08-12 14:00:23 -04:00
}
}
2015-12-13 19:30:15 -05:00
return nil
2015-08-12 14:00:23 -04:00
}
2014-11-01 22:42:04 -07:00
//-----------------------------------------------------------------------------
2016-07-01 17:47:31 -04:00
func ( cs * ConsensusState ) addVote ( vote * types . Vote , peerKey string ) ( added bool , err error ) {
2017-06-28 11:12:45 -04:00
cs . Logger . Debug ( "addVote" , "voteHeight" , vote . Height , "voteType" , vote . Type , "valIndex" , vote . ValidatorIndex , "csHeight" , cs . Height )
2015-07-05 21:01:59 -07:00
2015-06-05 14:15:40 -07:00
// A precommit for the previous height?
2016-12-19 10:44:25 -05:00
// These come in while we wait timeoutCommit
2015-08-26 18:56:34 -04:00
if vote . Height + 1 == cs . Height {
2017-10-10 12:39:21 +04:00
if ! ( cs . Step == cstypes . RoundStepNewHeight && vote . Type == types . VoteTypePrecommit ) {
2015-08-26 18:56:34 -04:00
// TODO: give the reason ..
2015-12-11 11:57:15 -05:00
// fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
2016-07-01 17:47:31 -04:00
return added , ErrVoteHeightMismatch
2015-08-26 18:56:34 -04:00
}
2016-07-01 17:47:31 -04:00
added , err = cs . LastCommit . AddVote ( vote )
2015-05-04 11:18:21 -07:00
if added {
2017-05-02 11:53:32 +04:00
cs . Logger . Info ( cmn . Fmt ( "Added to lastPrecommits: %v" , cs . LastCommit . StringShort ( ) ) )
2016-07-01 17:47:31 -04:00
types . FireEventVote ( cs . evsw , types . EventDataVote { vote } )
2016-12-19 10:44:25 -05:00
2017-01-04 01:50:02 +04:00
// if we can skip timeoutCommit and have all the votes now,
2017-05-01 20:09:29 -04:00
if cs . config . SkipTimeoutCommit && cs . LastCommit . HasAll ( ) {
2017-01-11 15:32:03 -05:00
// go straight to new round (skip timeout commit)
2017-10-10 12:39:21 +04:00
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
2016-12-19 20:12:37 -05:00
cs . enterNewRound ( cs . Height , 0 )
2016-12-19 10:44:25 -05:00
}
2015-05-04 11:18:21 -07:00
}
2016-12-19 10:44:25 -05: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
2016-07-01 17:47:31 -04:00
added , err = cs . Votes . AddVote ( vote , peerKey )
2015-05-04 11:18:21 -07:00
if added {
2016-07-01 17:47:31 -04:00
types . FireEventVote ( cs . evsw , types . EventDataVote { vote } )
2015-12-12 01:28:33 -05:00
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 11:53:32 +04: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),
2015-12-13 19:33:05 -05:00
// 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 11:53:32 +04:00
cs . Logger . Info ( "Unlocking because of POL." , "lockedRound" , cs . LockedRound , "POLRound" , vote . Round )
2015-08-26 18:56:34 -04:00
cs . LockedRound = 0
2015-06-24 18:51:14 -07:00
cs . LockedBlock = nil
cs . LockedBlockParts = nil
2016-10-10 02:58:13 -04: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.
2015-12-13 19:33:05 -05:00
cs . enterNewRound ( height , vote . Round ) // if the vote is ahead of us
2015-12-10 11:41:18 -05:00
if prevotes . HasTwoThirdsMajority ( ) {
2015-12-13 19:33:05 -05:00
cs . enterPrecommit ( height , vote . Round )
2015-12-10 11:41:18 -05:00
} else {
2015-12-13 19:33:05 -05:00
cs . enterPrevote ( height , vote . Round ) // if the vote is ahead of us
cs . enterPrevoteWait ( height , vote . Round )
2015-12-10 11:41:18 -05:00
}
2015-06-25 20:28:34 -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-12-13 19:33:05 -05:00
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 11:53:32 +04:00
cs . Logger . Info ( "Added to precommit" , "vote" , vote , "precommits" , precommits . StringShort ( ) )
2016-08-16 14:59:19 -07:00
blockID , ok := precommits . TwoThirdsMajority ( )
2015-08-19 16:11:52 -04:00
if ok {
2016-08-16 14:59:19 -07:00
if len ( blockID . Hash ) == 0 {
2015-12-13 19:33:05 -05:00
cs . enterNewRound ( height , vote . Round + 1 )
2015-12-10 11:41:18 -05:00
} else {
2015-12-13 19:33:05 -05:00
cs . enterNewRound ( height , vote . Round )
cs . enterPrecommit ( height , vote . Round )
cs . enterCommit ( height , vote . Round )
2017-01-11 15:32:03 -05:00
2017-05-01 20:09:29 -04:00
if cs . config . SkipTimeoutCommit && precommits . HasAll ( ) {
2017-01-11 15:32:03 -05:00
// if we have all the votes now,
// go straight to new round (skip timeout commit)
2017-10-10 12:39:21 +04:00
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
2017-01-11 15:32:03 -05:00
cs . enterNewRound ( cs . Height , 0 )
}
2015-12-10 11:41:18 -05:00
}
} else if cs . Round <= vote . Round && precommits . HasTwoThirdsAny ( ) {
2015-12-13 19:33:05 -05:00
cs . enterNewRound ( height , vote . Round )
cs . enterPrecommit ( height , vote . Round )
cs . enterPrecommitWait ( height , vote . Round )
2015-06-05 14:15:40 -07:00
}
default :
2017-04-25 14:50:20 -04:00
cmn . PanicSanity ( cmn . Fmt ( "Unexpected vote type %X" , vote . Type ) ) // Should not happen.
2014-11-01 22:42:04 -07:00
}
2014-10-30 03:32:09 -07:00
}
2015-08-26 18:56:34 -04:00
// Either duplicate, or error upon cs.Votes.AddByIndex()
2015-06-05 14:15:40 -07:00
return
2015-08-26 18:56:34 -04:00
} else {
err = ErrVoteHeightMismatch
2014-10-21 23:30:18 -07:00
}
2015-06-05 14:15:40 -07:00
2015-07-05 21:01:59 -07:00
// Height mismatch, bad peer?
2017-05-02 11:53:32 +04: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 14:00:23 -04:00
func ( cs * ConsensusState ) signVote ( type_ byte , hash [ ] byte , header types . PartSetHeader ) ( * types . Vote , error ) {
2017-09-22 18:17:21 -06:00
chainID , err := cs . state . ChainID ( )
if err != nil {
return nil , err
}
2017-09-21 16:32:02 -04:00
addr := cs . privValidator . GetAddress ( )
2016-12-02 00:12:06 -05:00
valIndex , _ := cs . Validators . GetByAddress ( addr )
2015-03-22 19:00:08 -07:00
vote := & types . Vote {
2016-12-02 00:12:06 -05:00
ValidatorAddress : addr ,
ValidatorIndex : valIndex ,
2015-08-12 22:36:43 -07:00
Height : cs . Height ,
Round : cs . Round ,
Type : type_ ,
2016-08-16 14:59:19 -07:00
BlockID : types . BlockID { hash , header } ,
2014-10-24 14:37:12 -07:00
}
2017-09-22 18:17:21 -06:00
err = cs . privValidator . SignVote ( chainID , vote )
2015-08-12 14:00:23 -04:00
return vote , err
}
2016-09-08 18:06:25 -04:00
// sign the vote and publish on internalMsgQueue
2015-08-12 14:00:23 -04:00
func ( cs * ConsensusState ) signAddVote ( type_ byte , hash [ ] byte , header types . PartSetHeader ) * types . Vote {
2016-11-16 20:58:53 -05:00
// if we don't have a key or we're not in the validator set, do nothing
2017-09-21 16:32:02 -04:00
if cs . privValidator == nil || ! cs . Validators . HasAddress ( cs . privValidator . GetAddress ( ) ) {
2015-08-12 14:00:23 -04:00
return nil
}
vote , err := cs . signVote ( type_ , hash , header )
2014-12-31 16:14:26 -08:00
if err == nil {
2016-07-01 17:47:31 -04:00
cs . sendInternalMessage ( msgInfo { & VoteMessage { vote } , "" } )
2017-06-14 12:50:49 +04:00
cs . Logger . Info ( "Signed and pushed vote" , "height" , cs . Height , "round" , cs . Round , "vote" , vote , "err" , err )
2014-12-31 16:14:26 -08:00
return vote
} else {
2016-06-26 15:33:11 -04:00
//if !cs.replayMode {
2017-06-14 12:50:49 +04:00
cs . Logger . Error ( "Error signing vote" , "height" , cs . Height , "round" , cs . Round , "vote" , vote , "err" , err )
2016-06-26 15:33:11 -04:00
//}
2014-12-31 16:14:26 -08:00
return nil
}
2014-10-24 14:37:12 -07:00
}
2014-10-30 03:32:09 -07:00
2015-12-12 01:28:33 -05:00
//---------------------------------------------------------
2015-12-01 20:12:01 -08:00
2017-10-10 12:39:21 +04:00
func CompareHRS ( h1 , r1 int , s1 cstypes . RoundStepType , h2 , r2 int , s2 cstypes . RoundStepType ) int {
2015-12-01 20:12:01 -08:00
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
}