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"
2015-12-10 11:41:18 -05:00
"reflect"
2014-08-10 16:35:08 -07:00
"sync"
"time"
2015-10-22 17:39:06 -07:00
. "github.com/tendermint/go-common"
2015-11-01 11:34:08 -08:00
"github.com/tendermint/go-wire"
bc "github.com/tendermint/tendermint/blockchain"
2015-04-07 15:24:09 -05:00
"github.com/tendermint/tendermint/events"
2015-04-01 17:30:16 -07:00
mempl "github.com/tendermint/tendermint/mempool"
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
)
2015-04-13 21:43:13 -07:00
var (
2015-06-05 14:15:40 -07:00
timeoutPropose = 3000 * time . Millisecond // Maximum duration of RoundStepPropose
timeoutPrevote0 = 1000 * time . Millisecond // After any +2/3 prevotes received, wait this long for stragglers.
timeoutPrevoteDelta = 0500 * time . Millisecond // timeoutPrevoteN is timeoutPrevote0 + timeoutPrevoteDelta*N
timeoutPrecommit0 = 1000 * time . Millisecond // After any +2/3 precommits received, wait this long for stragglers.
timeoutPrecommitDelta = 0500 * time . Millisecond // timeoutPrecommitN is timeoutPrecommit0 + timeoutPrecommitDelta*N
timeoutCommit = 2000 * time . Millisecond // After +2/3 commits received for committed block, wait this long for stragglers in the next height's RoundStepNewHeight.
2015-12-10 11:41:18 -05:00
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
)
//-----------------------------------------------------------------------------
2015-04-20 23:59:52 -07:00
// RoundStepType enum type
2014-12-30 17:14:54 -08:00
2015-04-20 23:59:52 -07:00
type RoundStepType uint8 // These must be numeric, ordered.
2014-10-20 19:02:10 -07:00
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
)
2015-04-20 23:59:52 -07:00
func ( rs RoundStepType ) String ( ) string {
2014-12-30 17:14:54 -08:00
switch rs {
case RoundStepNewHeight :
return "RoundStepNewHeight"
case RoundStepNewRound :
return "RoundStepNewRound"
case RoundStepPropose :
return "RoundStepPropose"
case RoundStepPrevote :
return "RoundStepPrevote"
2015-06-05 14:15:40 -07:00
case RoundStepPrevoteWait :
return "RoundStepPrevoteWait"
2014-12-30 17:14:54 -08:00
case RoundStepPrecommit :
return "RoundStepPrecommit"
2015-06-05 14:15:40 -07:00
case RoundStepPrecommitWait :
return "RoundStepPrecommitWait"
2014-12-30 17:14:54 -08:00
case RoundStepCommit :
return "RoundStepCommit"
default :
2015-07-05 15:42:37 -07:00
return "RoundStepUnknown" // Cannot panic.
2014-12-30 17:14:54 -08:00
}
}
2014-10-30 03:32:09 -07:00
//-----------------------------------------------------------------------------
2014-09-14 15:37:32 -07:00
// Immutable when returned from ConsensusState.GetRoundState()
type RoundState struct {
2015-06-25 20:28:34 -07:00
Height int // Height we are working on
Round int
2015-04-20 23:59:52 -07:00
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
2015-08-10 20:38:45 -07:00
Validators * types . ValidatorSet
Proposal * types . Proposal
2015-03-22 19:00:08 -07:00
ProposalBlock * types . Block
ProposalBlockParts * types . PartSet
2015-06-25 20:28:34 -07:00
LockedRound int
2015-03-22 19:00:08 -07:00
LockedBlock * types . Block
LockedBlockParts * types . PartSet
2015-06-05 14:15:40 -07:00
Votes * HeightVoteSet
2015-09-15 16:13:39 -04:00
CommitRound int //
2015-08-10 20:38:45 -07:00
LastCommit * types . VoteSet // Last precommits at Height-1
LastValidators * types . ValidatorSet
2014-09-14 15:37:32 -07:00
}
2015-09-09 16:45:53 -04:00
func ( rs * RoundState ) RoundStateEvent ( ) * types . EventDataRoundState {
2015-12-13 19:30:15 -05:00
var header types . PartSetHeader
var parts * BitArray
if rs . ProposalBlockParts != nil {
header = rs . ProposalBlockParts . Header ( )
parts = rs . ProposalBlockParts . BitArray ( )
}
2015-09-09 16:45:53 -04:00
return & types . EventDataRoundState {
2015-12-13 19:30:15 -05:00
CurrentTime : time . Now ( ) ,
Height : rs . Height ,
Round : rs . Round ,
Step : int ( rs . Step ) ,
StartTime : rs . StartTime ,
CommitTime : rs . CommitTime ,
Proposal : rs . Proposal ,
ProposalBlock : rs . ProposalBlock ,
LockedRound : rs . LockedRound ,
LockedBlock : rs . LockedBlock ,
POLRound : rs . Votes . POLRound ( ) ,
BlockPartsHeader : header ,
BlockParts : parts ,
2015-09-09 16:45:53 -04:00
}
}
2014-10-18 01:42:33 -07:00
func ( rs * RoundState ) String ( ) string {
2014-12-23 01:35:54 -08:00
return rs . StringIndented ( "" )
2014-10-18 01:42:33 -07:00
}
2014-12-23 01:35:54 -08:00
func ( rs * RoundState ) StringIndented ( indent string ) string {
2014-10-18 01:42:33 -07:00
return fmt . Sprintf ( ` RoundState {
% s H : % v R : % v S : % v
% s StartTime : % v
2014-10-21 18:30:03 -07:00
% 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
2014-10-21 18:30:03 -07:00
% 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
2015-06-26 17:48:24 -07:00
% s LastValidators : % v
2014-10-18 01:42:33 -07:00
% s } ` ,
indent , rs . Height , rs . Round , rs . Step ,
indent , rs . StartTime ,
2014-10-21 18:30:03 -07:00
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 ( ) ,
2015-06-26 17:48:24 -07:00
indent , rs . LastValidators . StringIndented ( indent + " " ) ,
2014-10-18 01:42:33 -07:00
indent )
}
2014-12-23 01:35:54 -08:00
func ( rs * RoundState ) StringShort ( ) string {
2014-12-30 17:14:54 -08:00
return fmt . Sprintf ( ` RoundState { H:%v R:%v S:%v ST:%v} ` ,
2014-10-24 14:37:12 -07:00
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 14:58:12 -05:00
var (
2015-12-12 01:28:33 -05:00
msgQueueSize = 1000
tickTockBufferSize = 10
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 {
msg ConsensusMessage
peerKey string
}
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 {
duration time . Duration
height int
round int
step RoundStepType
}
func ( ti * timeoutInfo ) String ( ) string {
return fmt . Sprintf ( "%v ; %d/%d %v" , ti . duration , ti . height , ti . round , ti . step )
}
2014-09-03 20:41:57 -07:00
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
2015-12-10 11:41:18 -05:00
QuitService
2014-10-30 03:32:09 -07:00
2015-12-05 14:58:12 -05:00
proxyAppCtx proxy . AppContext
2015-12-01 20:12:01 -08:00
blockStore * bc . BlockStore
mempool * mempl . Mempool
privValidator * types . PrivValidator
2014-09-14 15:37:32 -07:00
2014-10-07 01:05:54 -07:00
mtx sync . Mutex
RoundState
2015-06-05 14:15:40 -07:00
state * sm . State // State until height-1.
stagedBlock * types . Block // Cache last staged block.
stagedState * sm . State // Cache result of staged block.
2015-04-07 15:24:09 -05:00
2015-12-11 11:57:15 -05:00
peerMsgQueue chan msgInfo // serializes msgs affecting state (proposals, block parts, votes)
internalMsgQueue chan msgInfo // like peerMsgQueue but for our own proposals, parts, votes
timeoutTicker * time . Ticker // ticker for timeouts
tickChan chan timeoutInfo // start the timeoutTicker in the timeoutRoutine
tockChan chan timeoutInfo // timeouts are relayed on tockChan to the receiveRoutine
2015-12-05 14:58:12 -05:00
2015-04-15 23:40:27 -07:00
evsw events . Fireable
evc * events . EventCache // set in stageBlock and passed into state
2015-12-12 01:28:33 -05:00
nSteps int // used for testing to limit the number of transitions the state makes
2014-09-14 15:37:32 -07:00
}
2015-12-01 20:12:01 -08:00
func NewConsensusState ( state * sm . State , proxyAppCtx proxy . AppContext , blockStore * bc . BlockStore , mempool * mempl . Mempool ) * ConsensusState {
2014-09-14 15:37:32 -07:00
cs := & ConsensusState {
2015-12-11 11:57:15 -05:00
proxyAppCtx : proxyAppCtx ,
blockStore : blockStore ,
mempool : mempool ,
peerMsgQueue : make ( chan msgInfo , msgQueueSize ) ,
internalMsgQueue : make ( chan msgInfo , msgQueueSize ) ,
timeoutTicker : new ( time . Ticker ) ,
2015-12-12 01:28:33 -05:00
tickChan : make ( chan timeoutInfo , tickTockBufferSize ) ,
tockChan : make ( chan timeoutInfo , tickTockBufferSize ) ,
2014-09-14 15:37:32 -07:00
}
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 )
2015-12-10 11:41:18 -05:00
cs . QuitService = * NewQuitService ( log , "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
// implements events.Eventable
func ( cs * ConsensusState ) SetFireable ( evsw events . Fireable ) {
cs . evsw = evsw
}
func ( cs * ConsensusState ) String ( ) string {
return Fmt ( "ConsensusState(H:%v R:%v S:%v" , cs . Height , cs . Round , cs . Step )
2015-06-04 13:36:47 -07:00
}
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 ( )
}
2014-09-14 15:37:32 -07:00
func ( cs * ConsensusState ) GetRoundState ( ) * 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 ( )
}
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
}
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) SetPrivValidator ( priv * types . PrivValidator ) {
cs . mtx . Lock ( )
defer cs . mtx . Unlock ( )
cs . privValidator = priv
}
2015-08-04 18:44:15 -07:00
func ( cs * ConsensusState ) OnStart ( ) error {
2015-07-21 18:31:01 -07:00
cs . BaseService . OnStart ( )
2015-12-10 11:41:18 -05:00
2015-12-12 17:22:48 -05:00
// first we schedule the round (no go routines)
2015-12-12 01:28:33 -05:00
// then we start the timeout and receive routines.
2015-12-12 17:22:48 -05:00
// tickChan is buffered so scheduleRound0 will finish.
// Then all further access to the RoundState is through the receiveRoutine
2015-12-12 01:28:33 -05:00
cs . scheduleRound0 ( cs . Height )
2015-12-12 17:22:48 -05:00
cs . startRoutines ( 0 )
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 ) {
2015-12-12 17:22:48 -05:00
go cs . timeoutRoutine ( )
go cs . receiveRoutine ( maxSteps )
2015-12-10 11:41:18 -05:00
}
2015-07-21 18:31:01 -07:00
func ( cs * ConsensusState ) OnStop ( ) {
2015-12-12 01:28:33 -05:00
cs . QuitService . OnStop ( )
2014-10-30 03:32:09 -07:00
}
2015-12-12 17:22:48 -05:00
//------------------------------------------------------------
// Public interface for passing messages into the consensus state,
// possibly causing a state transition
// TODO: should these return anything or let callers just use events?
2015-12-12 01:28:33 -05:00
// May block on send if queue is full.
func ( cs * ConsensusState ) AddVote ( valIndex int , vote * types . Vote , peerKey string ) ( added bool , address [ ] byte , err error ) {
if peerKey == "" {
cs . internalMsgQueue <- msgInfo { & VoteMessage { valIndex , vote } , "" }
} else {
cs . peerMsgQueue <- msgInfo { & VoteMessage { valIndex , vote } , peerKey }
}
// TODO: wait for event?!
return false , nil , nil
}
2015-12-12 16:25:49 -05:00
// May block on send if queue is full.
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
}
2015-12-12 16:25:49 -05:00
// May block on send if queue is full.
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
}
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
}
func ( cs * ConsensusState ) updateRoundStep ( round int , step RoundStepType ) {
cs . Round = round
cs . Step = step
}
2015-06-24 17:05:52 -07:00
// EnterNewRound(height, 0) at cs.StartTime.
2015-06-25 20:28:34 -07:00
func ( cs * ConsensusState ) scheduleRound0 ( height int ) {
2015-07-19 21:49:13 +00:00
//log.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
2015-06-05 14:15:40 -07:00
sleepDuration := cs . StartTime . Sub ( time . Now ( ) )
2015-12-12 01:28:33 -05:00
cs . scheduleTimeout ( sleepDuration , height , 0 , 1 )
2015-12-08 16:00:59 -05:00
}
2015-12-10 11:41:18 -05:00
// Attempt to schedule a timeout by sending timeoutInfo on the tickChan.
// The timeoutRoutine is alwaya available to read from tickChan (it won't block).
// The scheduling may fail if the timeoutRoutine has already scheduled a timeout for a later height/round/step.
func ( cs * ConsensusState ) scheduleTimeout ( duration time . Duration , height , round int , step RoundStepType ) {
2015-12-08 16:00:59 -05:00
cs . tickChan <- 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 ) {
timeout := time . After ( 10 * time . Millisecond )
select {
case cs . internalMsgQueue <- mi :
case <- timeout :
log . Debug ( "Timed out trying to send an internal messge. Launching go-routine" )
go func ( ) { cs . internalMsgQueue <- mi } ( )
}
}
// Reconstruct LastCommit from SeenValidation, 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
}
lastPrecommits := types . NewVoteSet ( state . LastBlockHeight , 0 , types . VoteTypePrecommit , state . LastValidators )
seenValidation := cs . blockStore . LoadSeenValidation ( state . LastBlockHeight )
for idx , precommit := range seenValidation . Precommits {
if precommit == nil {
continue
}
added , _ , err := lastPrecommits . AddByIndex ( idx , precommit )
if ! added || err != nil {
PanicCrisis ( Fmt ( "Failed to reconstruct LastCommit: %v" , err ) )
}
}
if ! lastPrecommits . HasTwoThirdsMajority ( ) {
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 {
PanicSanity ( 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?!
PanicSanity ( 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 ) {
log . Notice ( "Ignoring updateToState()" , "newHeight" , state . LastBlockHeight + 1 , "oldHeight" , cs . state . LastBlockHeight + 1 )
return
}
// Reset fields based on state.
validators := state . Validators
height := state . LastBlockHeight + 1 // next desired block height
lastPrecommits := ( * types . VoteSet ) ( nil )
if cs . CommitRound > - 1 && cs . Votes != nil {
if ! cs . Votes . Precommits ( cs . CommitRound ) . HasTwoThirdsMajority ( ) {
PanicSanity ( "updateToState(state) called but last Precommit round didn't have +2/3" )
}
lastPrecommits = cs . Votes . Precommits ( cs . CommitRound )
}
// 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 = time . Now ( ) . Add ( timeoutCommit )
} else {
cs . StartTime = cs . CommitTime . Add ( timeoutCommit )
}
cs . CommitTime = time . Time { }
cs . Validators = validators
cs . Proposal = nil
cs . ProposalBlock = nil
cs . ProposalBlockParts = nil
cs . LockedRound = 0
cs . LockedBlock = nil
cs . LockedBlockParts = nil
cs . Votes = NewHeightVoteSet ( height , validators )
cs . CommitRound = - 1
cs . LastCommit = lastPrecommits
cs . LastValidators = state . LastValidators
cs . state = state
cs . stagedBlock = nil
cs . stagedState = nil
// Finally, broadcast RoundState
cs . newStep ( )
}
func ( cs * ConsensusState ) newStep ( ) {
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 {
cs . evsw . FireEvent ( types . EventStringNewRoundStep ( ) , cs . RoundStateEvent ( ) )
}
2015-12-12 01:28:33 -05:00
}
//-----------------------------------------
// the main go routines
2015-12-10 11:41:18 -05:00
// the state machine sends on tickChan to start a new timer.
// timers are interupted and replaced by new ticks from later steps
// timeouts of 0 on the tickChan will be immediately relayed to the tockChan
func ( cs * ConsensusState ) timeoutRoutine ( ) {
log . Debug ( "Starting timeout routine" )
var ti timeoutInfo
for {
select {
case newti := <- cs . tickChan :
log . Debug ( "Received tick" , "old_ti" , ti , "new_ti" , newti )
// ignore tickers for old height/round/step
if newti . height < ti . height {
continue
} else if newti . height == ti . height {
if newti . round < ti . round {
continue
} else if newti . round == ti . round {
if ti . step > 0 && newti . step <= ti . step {
continue
}
}
}
ti = newti
2015-12-12 01:28:33 -05:00
// if the newti has duration == 0, we relay to the tockChan immediately (no timeout)
if ti . duration == time . Duration ( 0 ) {
go func ( t timeoutInfo ) { cs . tockChan <- t } ( ti )
continue
}
2015-12-10 11:41:18 -05:00
log . Info ( "Scheduling timeout" , "dur" , ti . duration , "height" , ti . height , "round" , ti . round , "step" , ti . step )
2015-12-11 11:57:15 -05:00
cs . timeoutTicker . Stop ( )
cs . timeoutTicker = time . NewTicker ( ti . duration )
case <- cs . timeoutTicker . C :
2015-12-10 11:41:18 -05:00
log . Info ( "Timed out" , "dur" , ti . duration , "height" , ti . height , "round" , ti . round , "step" , ti . step )
2015-12-11 11:57:15 -05:00
cs . timeoutTicker . Stop ( )
// go routine here gaurantees timeoutRoutine doesn't block.
// Determinism comes from playback in the receiveRoutine.
// We can eliminate it by merging the timeoutRoutine into receiveRoutine
// and managing the timeouts ourselves with a millisecond ticker
2015-12-12 01:28:33 -05:00
go func ( t timeoutInfo ) { cs . tockChan <- t } ( ti )
2015-12-10 11:41:18 -05:00
case <- cs . Quit :
return
}
}
}
2015-12-12 01:28:33 -05:00
// a nice idea but probably more trouble than its worth
2015-12-11 11:57:15 -05:00
func ( cs * ConsensusState ) stopTimer ( ) {
cs . timeoutTicker . Stop ( )
}
// 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.
2015-12-13 19:30:15 -05:00
// Updates (state transitions) happen on timeouts, complete proposals, and 2/3 majorities
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) receiveRoutine ( maxSteps int ) {
2015-12-10 11:41:18 -05:00
for {
2015-12-12 01:28:33 -05:00
if maxSteps > 0 {
if cs . nSteps >= maxSteps {
log . Warn ( "reached max steps. exiting receive routine" )
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 {
2015-12-11 11:57:15 -05:00
case mi = <- cs . peerMsgQueue :
2015-12-10 11:41:18 -05:00
// handles proposals, block parts, votes
// may generate internal events (votes, complete proposals, 2/3 majorities)
cs . handleMsg ( mi , rs )
2015-12-11 11:57:15 -05:00
case mi = <- cs . internalMsgQueue :
// handles proposals, block parts, votes
cs . handleMsg ( mi , rs )
2015-12-10 11:41:18 -05:00
case ti := <- cs . tockChan :
// if the timeout is relevant to the rs
// go to the next step
cs . handleTimeout ( ti , rs )
case <- cs . Quit :
return
}
}
}
2015-12-11 11:57:15 -05:00
// state transitions on complete-proposal, 2/3-any, 2/3-one
2015-12-10 11:41:18 -05:00
func ( cs * ConsensusState ) handleMsg ( mi msgInfo , rs RoundState ) {
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
msg , peerKey := mi . msg , mi . peerKey
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-11 11:57:15 -05:00
// if the proposal is complete, we'll EnterPrevote or tryFinalizeCommit
// if we're the only validator, the EnterPrevote may take us through to the next round
2015-12-12 01:28:33 -05:00
_ , err = cs . addProposalBlockPart ( msg . Height , msg . Part )
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
2015-12-13 19:30:15 -05:00
err := cs . tryAddVote ( msg . ValidatorIndex , 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 :
log . Warn ( "Unknown msg type" , reflect . TypeOf ( msg ) )
}
if err != nil {
log . Error ( "error with msg" , "error" , err )
}
}
func ( cs * ConsensusState ) handleTimeout ( ti timeoutInfo , rs RoundState ) {
log . 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
if ti . height != rs . Height || ti . round < rs . Round || ( ti . round == rs . Round && ti . step < rs . Step ) {
log . Debug ( "Ignoring tock because we're ahead" , "height" , rs . Height , "round" , rs . Round , "step" , rs . Step )
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-10 11:41:18 -05:00
switch ti . step {
2015-12-12 17:22:48 -05:00
case RoundStepNewHeight :
// NewRound event fired from EnterNewRound.
// Do we want a timeout event too?
cs . EnterNewRound ( ti . height , 0 )
2015-12-10 11:41:18 -05:00
case RoundStepPropose :
2015-12-11 11:57:15 -05:00
cs . evsw . FireEvent ( types . EventStringTimeoutPropose ( ) , cs . RoundStateEvent ( ) )
cs . EnterPrevote ( ti . height , ti . round )
2015-12-10 11:41:18 -05:00
case RoundStepPrevoteWait :
2015-12-11 11:57:15 -05:00
cs . evsw . FireEvent ( types . EventStringTimeoutWait ( ) , cs . RoundStateEvent ( ) )
cs . EnterPrecommit ( ti . height , ti . round )
2015-12-10 11:41:18 -05:00
case RoundStepPrecommitWait :
2015-12-11 11:57:15 -05:00
cs . evsw . FireEvent ( types . EventStringTimeoutWait ( ) , cs . RoundStateEvent ( ) )
cs . EnterNewRound ( ti . height , ti . round + 1 )
2015-12-10 11:41:18 -05:00
default :
panic ( Fmt ( "Invalid timeout step: %v" , ti . step ) )
}
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
2015-12-12 01:28:33 -05:00
// State functions
// Many of these functions are capitalized but are not really meant to be used
// by external code as it will cause race conditions with running timeout/receiveRoutine.
// Use AddVote, SetProposal, AddProposalBlockPart instead
2014-08-10 16:35:08 -07:00
2015-06-24 18:51:14 -07:00
// Enter: +2/3 precommits for nil at (height,round-1)
2015-06-24 17:05:52 -07:00
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
// Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
2015-06-05 14:15:40 -07:00
// NOTE: cs.StartTime was already set for height.
2015-12-11 11:57:15 -05:00
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 ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
2014-10-30 03:32:09 -07:00
}
2015-09-09 16:45:53 -04:00
2015-06-05 14:15:40 -07:00
if now := time . Now ( ) ; cs . StartTime . After ( now ) {
log . Warn ( "Need to set a buffer and log.Warn() here for sanity." , "startTime" , cs . StartTime , "now" , now )
2014-10-30 03:32:09 -07:00
}
2015-12-12 01:28:33 -05:00
// cs.stopTimer()
2015-12-11 11:57:15 -05:00
2015-07-19 21:49:13 +00:00
log . Notice ( Fmt ( "EnterNewRound(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
// Increment validators if necessary
2015-06-24 14:04:40 -07:00
validators := cs . Validators
2015-06-05 14:15:40 -07:00
if cs . Round < round {
2015-06-24 14:04:40 -07:00
validators = validators . Copy ( )
2015-06-05 14:15:40 -07:00
validators . IncrementAccum ( round - cs . Round )
}
// Setup new round
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
2015-12-08 16:00:59 -05:00
cs . updateRoundStep ( round , 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
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringNewRound ( ) , cs . RoundStateEvent ( ) )
2015-06-05 14:15:40 -07:00
// Immediately go to EnterPropose.
2015-12-10 11:41:18 -05:00
cs . EnterPropose ( height , round )
2014-10-30 03:32:09 -07:00
}
2015-06-24 17:05:52 -07:00
// Enter: from NewRound(height,round).
2015-06-25 20:28:34 -07:00
func ( cs * ConsensusState ) EnterPropose ( height int , round int ) {
2015-12-10 11:41:18 -05:00
// cs.mtx.Lock()
// cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && RoundStepPropose <= cs . Step ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterPropose(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2014-10-21 01:18:46 -07:00
return
}
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "EnterPropose(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
2015-12-12 16:25:49 -05:00
defer func ( ) {
// Done EnterPropose:
cs . updateRoundStep ( round , RoundStepPropose )
cs . newStep ( )
} ( )
2015-08-26 18:56:34 -04:00
// This step times out after `timeoutPropose`
2015-12-10 11:41:18 -05:00
cs . scheduleTimeout ( timeoutPropose , height , round , 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 {
2015-01-08 22:07:23 -08:00
return
}
2015-06-05 14:15:40 -07:00
2015-04-20 20:39:42 -07:00
if ! bytes . Equal ( cs . Validators . Proposer ( ) . Address , cs . privValidator . Address ) {
2015-07-19 21:49:13 +00:00
log . Info ( "EnterPropose: Not our turn to propose" , "proposer" , cs . Validators . Proposer ( ) . Address , "privValidator" , cs . privValidator )
2015-01-08 22:07:23 -08:00
} else {
2015-07-19 21:49:13 +00:00
log . Info ( "EnterPropose: Our turn to propose" , "proposer" , cs . Validators . Proposer ( ) . Address , "privValidator" , cs . privValidator )
2015-08-26 18:56:34 -04:00
cs . decideProposal ( height , round )
2014-08-10 16:35:08 -07:00
}
2015-12-10 11:41:18 -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 ( ) {
2015-12-11 11:57:15 -05:00
cs . EnterPrevote ( height , cs . Round )
2015-12-10 11:41:18 -05:00
}
2015-06-24 17:05:52 -07:00
}
2014-08-10 16:35:08 -07:00
2015-10-28 19:49:35 +02:00
func ( cs * ConsensusState ) decideProposal ( 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
2015-08-10 20:38:45 -07:00
proposal := types . NewProposal ( height , round , blockParts . Header ( ) , cs . Votes . POLRound ( ) )
2015-05-29 17:53:57 -04:00
err := cs . privValidator . SignProposal ( cs . state . ChainID , proposal )
2014-12-31 16:14:26 -08:00
if err == nil {
// 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 } , "" } )
}
2015-12-12 01:28:33 -05:00
log . Notice ( "Signed and sent proposal" , "height" , height , "round" , round , "proposal" , proposal )
log . Debug ( Fmt ( "Signed and sent proposal block: %v" , block ) )
2014-12-31 16:14:26 -08:00
} else {
2015-06-24 17:05:52 -07:00
log . Warn ( "EnterPropose: Error signing proposal" , "height" , height , "round" , round , "error" , err )
2015-06-05 14:15:40 -07:00
}
2015-06-24 14:04:40 -07:00
2015-06-05 14:15:40 -07:00
}
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 ) {
2015-06-05 14:15:40 -07:00
var validation * types . Validation
if cs . Height == 1 {
// We're creating a proposal for the first block.
2015-06-19 15:30:10 -07:00
// The validation is empty, but not nil.
2015-06-05 14:15:40 -07:00
validation = & types . Validation { }
2015-06-19 15:30:10 -07:00
} else if cs . LastCommit . HasTwoThirdsMajority ( ) {
// Make the validation from LastCommit
validation = cs . LastCommit . MakeValidation ( )
2015-06-05 14:15:40 -07:00
} else {
// This shouldn't happen.
log . Error ( "EnterPropose: Cannot propose anything: No validation for the previous block." )
return
}
2015-12-01 20:12:01 -08:00
// Mempool run transactions and the resulting hash
txs , hash , err := cs . mempool . Reap ( )
if err != nil {
log . Warn ( "createProposalBlock: Error getting proposal txs" , "error" , err )
return nil , nil
}
2015-06-05 14:15:40 -07:00
block = & types . Block {
Header : & types . Header {
ChainID : cs . state . ChainID ,
Height : cs . Height ,
Time : time . Now ( ) ,
Fees : 0 , // TODO fees
2015-06-25 20:28:34 -07:00
NumTxs : len ( txs ) ,
2015-06-05 14:15:40 -07:00
LastBlockHash : cs . state . LastBlockHash ,
LastBlockParts : cs . state . LastBlockParts ,
2015-12-01 20:12:01 -08:00
ValidatorsHash : cs . state . Validators . Hash ( ) ,
AppHash : hash ,
2015-06-05 14:15:40 -07:00
} ,
2015-06-24 14:04:40 -07:00
LastValidation : validation ,
2015-06-05 14:15:40 -07:00
Data : & types . Data {
Txs : txs ,
} ,
}
2015-08-07 10:43:24 -07:00
block . FillHeader ( )
2015-06-05 14:15:40 -07:00
blockParts = block . MakePartSet ( )
2015-12-01 20:12:01 -08:00
2015-06-05 14:15:40 -07:00
return block , blockParts
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-11 11:57:15 -05:00
func ( cs * ConsensusState ) EnterPrevote ( height int , round int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && RoundStepPrevote <= cs . Step ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
2014-10-21 01:18:46 -07:00
}
2015-09-09 16:45:53 -04:00
2015-12-12 16:25:49 -05:00
defer func ( ) {
// Done EnterPrevote:
cs . updateRoundStep ( round , RoundStepPrevote )
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 ( ) {
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringCompleteProposal ( ) , cs . RoundStateEvent ( ) )
} else {
// we received +2/3 prevotes for a future round
// TODO: catchup event?
}
2015-12-12 01:28:33 -05:00
// cs.stopTimer()
2015-12-11 11:57:15 -05:00
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "EnterPrevote(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
// Sign and broadcast vote as necessary
cs . doPrevote ( height , round )
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
}
2015-06-25 20:28:34 -07:00
func ( cs * ConsensusState ) doPrevote ( height int , round int ) {
2014-10-21 01:18:46 -07:00
// If a block is locked, prevote that.
if cs . LockedBlock != nil {
2015-07-19 21:49:13 +00:00
log . 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 {
2015-06-05 14:15:40 -07:00
log . Warn ( "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
// Try staging cs.ProposalBlock
2014-10-31 21:30:52 -07:00
err := cs . stageBlock ( cs . ProposalBlock , cs . ProposalBlockParts )
2014-10-21 01:18:46 -07:00
if err != nil {
2014-10-30 03:32:09 -07:00
// ProposalBlock is invalid, prevote nil.
2015-06-05 14:15:40 -07:00
log . Warn ( "EnterPrevote: ProposalBlock is invalid" , "error" , err )
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)
2015-03-22 19:00:08 -07:00
cs . signAddVote ( types . VoteTypePrevote , cs . ProposalBlock . Hash ( ) , cs . ProposalBlockParts . Header ( ) )
2014-10-30 03:32:09 -07:00
return
2014-10-21 01:18:46 -07:00
}
2015-06-24 18:51:14 -07:00
// Enter: any +2/3 prevotes at next round.
2015-06-25 20:28:34 -07:00
func ( cs * ConsensusState ) EnterPrevoteWait ( height int , round int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && RoundStepPrevoteWait <= cs . Step ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
}
if ! cs . Votes . Prevotes ( round ) . HasTwoThirdsAny ( ) {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "EnterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes" , height , round ) )
2015-06-05 14:15:40 -07:00
}
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "EnterPrevoteWait(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
2015-12-12 16:25:49 -05:00
defer func ( ) {
// Done EnterPrevoteWait:
cs . updateRoundStep ( round , RoundStepPrevoteWait )
cs . newStep ( )
} ( )
2015-06-05 14:15:40 -07:00
2015-06-24 14:04:40 -07:00
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
2015-12-10 11:41:18 -05:00
cs . scheduleTimeout ( timeoutPrevote0 + timeoutPrevoteDelta * time . Duration ( round ) , height , round , RoundStepPrevoteWait )
2015-06-05 14:15:40 -07:00
}
// Enter: +2/3 precomits for block or nil.
// Enter: `timeoutPrevote` after any +2/3 prevotes.
// Enter: any +2/3 precommits for next round.
2015-08-26 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-11 11:57:15 -05:00
func ( cs * ConsensusState ) EnterPrecommit ( height int , round int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
// defer cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && RoundStepPrecommit <= cs . Step ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
2014-09-14 15:37:32 -07:00
}
2015-09-09 16:45:53 -04:00
2015-12-12 01:28:33 -05:00
// cs.stopTimer()
2015-09-09 16:45:53 -04:00
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "EnterPrecommit(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
2014-11-01 04:04:58 -07:00
defer func ( ) {
2015-06-05 14:15:40 -07:00
// Done EnterPrecommit:
2015-12-12 16:25:49 -05:00
cs . updateRoundStep ( round , 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
2015-06-05 14:15:40 -07:00
hash , partsHeader , ok := cs . Votes . Prevotes ( round ) . TwoThirdsMajority ( )
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 {
2015-08-12 14:00:23 -04:00
log . Info ( "EnterPrecommit: No +2/3 prevotes during EnterPrecommit while we're locked. Precommitting nil" )
2015-06-05 14:15:40 -07:00
} else {
2015-07-19 21:49:13 +00:00
log . 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
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringPolka ( ) , cs . RoundStateEvent ( ) )
// the latest POLRound should be this round
if cs . Votes . POLRound ( ) < round {
PanicSanity ( Fmt ( "This POLRound should be %v but got %" , round , cs . Votes . POLRound ( ) ) )
}
2015-08-12 14:00:23 -04:00
2015-06-05 14:15:40 -07:00
// +2/3 prevoted nil. Unlock and precommit nil.
2014-10-30 03:32:09 -07:00
if len ( hash ) == 0 {
2015-05-04 10:15:58 -07:00
if cs . LockedBlock == nil {
2015-07-19 21:49:13 +00:00
log . Info ( "EnterPrecommit: +2/3 prevoted for nil." )
2015-05-04 10:15:58 -07:00
} else {
2015-07-19 21:49:13 +00:00
log . 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
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringUnlock ( ) , cs . RoundStateEvent ( ) )
2015-05-04 10:15:58 -07:00
}
2015-06-05 14:15:40 -07:00
cs . signAddVote ( types . VoteTypePrecommit , nil , types . PartSetHeader { } )
2014-10-30 03:32:09 -07:00
return
}
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// At this point, +2/3 prevoted for a particular block.
2015-08-12 14:00:23 -04:00
// If we're already locked on that block, precommit it, and update the LockedRound
2014-10-30 03:32:09 -07:00
if cs . LockedBlock . HashesTo ( hash ) {
2015-09-09 16:45:53 -04:00
log . Info ( "EnterPrecommit: +2/3 prevoted locked block. Relocking" )
2015-08-12 14:00:23 -04:00
cs . LockedRound = round
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringRelock ( ) , cs . RoundStateEvent ( ) )
2015-03-22 19:00:08 -07:00
cs . signAddVote ( types . VoteTypePrecommit , hash , partsHeader )
2014-10-30 03:32:09 -07:00
return
}
2014-09-14 15:37:32 -07:00
2015-06-05 14:15:40 -07:00
// If +2/3 prevoted for proposal block, stage and precommit it
if cs . ProposalBlock . HashesTo ( hash ) {
2015-09-09 16:45:53 -04:00
log . Info ( "EnterPrecommit: +2/3 prevoted proposal block. Locking" , "hash" , hash )
2015-06-05 14:15:40 -07:00
// Validate the block.
if err := cs . stageBlock ( cs . ProposalBlock , cs . ProposalBlockParts ) ; err != nil {
2015-07-19 23:42:52 +00:00
PanicConsensus ( Fmt ( "EnterPrecommit: +2/3 prevoted for an invalid block: %v" , err ) )
2015-06-05 14:15:40 -07:00
}
2015-06-24 18:51:14 -07:00
cs . LockedRound = round
2015-06-05 14:15:40 -07:00
cs . LockedBlock = cs . ProposalBlock
cs . LockedBlockParts = cs . ProposalBlockParts
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringLock ( ) , cs . RoundStateEvent ( ) )
2015-06-05 14:15:40 -07:00
cs . signAddVote ( types . VoteTypePrecommit , hash , partsHeader )
2014-10-30 03:32:09 -07:00
return
2014-09-14 15:37:32 -07:00
}
2015-09-09 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
if ! cs . ProposalBlockParts . HasHeader ( partsHeader ) {
cs . ProposalBlock = nil
cs . ProposalBlockParts = types . NewPartSetFromHeader ( partsHeader )
}
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringUnlock ( ) , cs . RoundStateEvent ( ) )
2015-06-24 14:04:40 -07:00
cs . signAddVote ( types . VoteTypePrecommit , nil , types . PartSetHeader { } )
2015-06-05 14:15:40 -07:00
return
}
// Enter: any +2/3 precommits for next round.
2015-06-25 20:28:34 -07:00
func ( cs * ConsensusState ) EnterPrecommitWait ( height int , round int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || round < cs . Round || ( cs . Round == round && RoundStepPrecommitWait <= cs . Step ) {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-05-04 10:15:58 -07:00
return
}
2015-06-05 14:15:40 -07:00
if ! cs . Votes . Precommits ( round ) . HasTwoThirdsAny ( ) {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "EnterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes" , height , round ) )
2015-06-05 14:15:40 -07:00
}
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "EnterPrecommitWait(%v/%v). Current: %v/%v/%v" , height , round , cs . Height , cs . Round , cs . Step ) )
2015-05-04 10:15:58 -07:00
2015-12-12 16:25:49 -05:00
defer func ( ) {
// Done EnterPrecommitWait:
cs . updateRoundStep ( round , RoundStepPrecommitWait )
cs . newStep ( )
} ( )
2015-06-05 14:15:40 -07:00
2015-06-24 14:04:40 -07:00
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
2015-12-10 11:41:18 -05:00
cs . scheduleTimeout ( timeoutPrecommit0 + timeoutPrecommitDelta * time . Duration ( round ) , height , round , 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-08-19 16:11:52 -04:00
func ( cs * ConsensusState ) EnterCommit ( height int , commitRound int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2015-06-24 17:05:52 -07:00
if cs . Height != height || RoundStepCommit <= cs . Step {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "EnterCommit(%v/%v): Invalid args. Current step: %v/%v/%v" , height , commitRound , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
2014-10-30 03:32:09 -07:00
}
2015-08-19 16:11:52 -04:00
log . Info ( Fmt ( "EnterCommit(%v/%v). Current: %v/%v/%v" , height , commitRound , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
2014-11-01 04:04:58 -07:00
defer func ( ) {
2015-06-05 14:15:40 -07:00
// Done Entercommit:
// keep ca.Round the same, it points to the right Precommits set.
2015-12-12 16:25:49 -05:00
cs . updateRoundStep ( cs . Round , RoundStepCommit )
2015-09-15 16:13:39 -04:00
cs . CommitRound = commitRound
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
} ( )
2015-08-19 16:11:52 -04:00
hash , partsHeader , ok := cs . Votes . Precommits ( commitRound ) . TwoThirdsMajority ( )
2014-10-30 03:32:09 -07:00
if ! ok {
2015-07-19 23:42:52 +00:00
PanicSanity ( "RunActionCommit() expects +2/3 precommits" )
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
// The Locked* fields no longer matter.
// Move them over to ProposalBlock if they match the commit hash,
2015-08-26 18:56:34 -04:00
// otherwise they'll be cleared in updateToState.
2014-10-30 03:32:09 -07:00
if cs . LockedBlock . HashesTo ( hash ) {
cs . ProposalBlock = cs . LockedBlock
cs . ProposalBlockParts = cs . LockedBlockParts
}
// If we don't have the block being committed, set up to get it.
if ! cs . ProposalBlock . HashesTo ( hash ) {
2014-11-03 15:50:23 -08:00
if ! cs . ProposalBlockParts . HasHeader ( partsHeader ) {
2014-10-30 03:32:09 -07:00
// We're getting the wrong block.
// Set up ProposalBlockParts and keep waiting.
cs . ProposalBlock = nil
2015-03-22 19:00:08 -07:00
cs . ProposalBlockParts = types . NewPartSetFromHeader ( 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 {
2015-07-19 23:42:52 +00:00
PanicSanity ( 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
2015-09-15 16:13:39 -04:00
hash , _ , ok := cs . Votes . Precommits ( cs . CommitRound ) . TwoThirdsMajority ( )
2015-06-24 14:04:40 -07:00
if ! ok || len ( hash ) == 0 {
2015-08-19 16:11:52 -04:00
log . Warn ( "Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>." )
return
2015-06-24 14:04:40 -07:00
}
if ! cs . ProposalBlock . HashesTo ( hash ) {
2015-08-19 16:11:52 -04:00
log . Warn ( "Attempt to finalize failed. We don't have the commit block." )
return
2015-06-24 14:04:40 -07:00
}
2015-12-10 11:41:18 -05:00
// go
cs . FinalizeCommit ( height )
2014-10-21 18:30:03 -07:00
}
2015-06-05 14:15:40 -07:00
// Increment height and goto RoundStepNewHeight
2015-09-15 16:13:39 -04:00
func ( cs * ConsensusState ) FinalizeCommit ( height int ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if cs . Height != height || cs . Step != RoundStepCommit {
2015-08-26 18:56:34 -04:00
log . Debug ( Fmt ( "FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v" , height , cs . Height , cs . Round , cs . Step ) )
2015-06-05 14:15:40 -07:00
return
2014-10-21 01:18:46 -07:00
}
2015-09-15 16:13:39 -04:00
hash , header , ok := cs . Votes . Precommits ( cs . CommitRound ) . TwoThirdsMajority ( )
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
if ! ok {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "Cannot FinalizeCommit, commit does not have two thirds majority" ) )
2015-06-05 14:15:40 -07:00
}
if ! cs . ProposalBlockParts . HasHeader ( header ) {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "Expected ProposalBlockParts header to be commit header" ) )
2015-06-05 14:15:40 -07:00
}
if ! cs . ProposalBlock . HashesTo ( hash ) {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "Cannot FinalizeCommit, ProposalBlock does not hash to commit hash" ) )
2015-06-05 14:15:40 -07:00
}
if err := cs . stageBlock ( cs . ProposalBlock , cs . ProposalBlockParts ) ; err != nil {
2015-07-19 23:42:52 +00:00
PanicConsensus ( Fmt ( "+2/3 committed an invalid block: %v" , err ) )
2014-10-30 03:32:09 -07:00
}
2015-06-05 14:15:40 -07:00
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "Finalizing commit of block: %v" , cs . ProposalBlock ) )
2015-06-05 14:15:40 -07:00
// We have the block, so stage/save/commit-vote.
2015-09-15 16:13:39 -04:00
cs . saveBlock ( cs . ProposalBlock , cs . ProposalBlockParts , cs . Votes . Precommits ( cs . CommitRound ) )
2015-12-12 01:28:33 -05:00
2015-12-12 17:22:48 -05:00
// NewHeightStep!
cs . updateToState ( cs . stagedState )
2015-12-12 01:28:33 -05:00
2015-06-24 14:04:40 -07:00
// cs.StartTime is already set.
// Schedule Round0 to start soon.
2015-12-10 11:41:18 -05:00
cs . scheduleRound0 ( height + 1 )
2015-06-05 14:15:40 -07:00
// By here,
// * cs.Height has been increment to height+1
// * cs.Step is now RoundStepNewHeight
2015-06-24 14:04:40 -07:00
// * cs.StartTime is set to when we will start round0.
2015-06-05 14:15:40 -07:00
return
2014-10-20 19:02:10 -07:00
}
2014-10-21 23:30:18 -07:00
//-----------------------------------------------------------------------------
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) setProposal ( proposal * types . Proposal ) error {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
2014-10-21 23:30:18 -07:00
// Already have one
if cs . Proposal != nil {
return nil
}
2014-10-30 03:32:09 -07:00
// Does not apply
2014-10-21 23:30:18 -07:00
if proposal . Height != cs . Height || proposal . Round != cs . Round {
return nil
}
2014-10-30 03:32:09 -07:00
// We don't care about the proposal if we're already in RoundStepCommit.
2015-06-24 17:05:52 -07:00
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 &&
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
}
2014-10-21 23:30:18 -07:00
// Verify signature
2015-11-01 11:34:08 -08:00
if ! cs . Validators . Proposer ( ) . PubKey . VerifyBytes ( types . SignBytes ( cs . state . ChainID , proposal ) , proposal . Signature ) {
2014-10-21 23:30:18 -07:00
return ErrInvalidProposalSignature
}
cs . Proposal = proposal
2015-06-24 14:04:40 -07:00
cs . ProposalBlockParts = types . NewPartSetFromHeader ( proposal . BlockPartsHeader )
2014-10-21 23:30:18 -07:00
return nil
}
// NOTE: block is not necessarily valid.
2015-09-09 16:45:53 -04:00
// This can trigger us to go into EnterPrevote asynchronously (before we timeout of propose) or to attempt to commit
2015-12-12 01:28:33 -05:00
func ( cs * ConsensusState ) addProposalBlockPart ( height int , part * types . Part ) ( added bool , err error ) {
2015-12-10 11:41:18 -05:00
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
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?
}
2014-10-26 13:26:27 -07:00
added , err = cs . ProposalBlockParts . AddPart ( part )
2014-10-21 23:30:18 -07:00
if err != nil {
return added , err
}
2014-10-26 13:26:27 -07:00
if added && cs . ProposalBlockParts . IsComplete ( ) {
2015-06-24 18:51:14 -07:00
// Added and completed!
2015-11-10 13:10:43 -08:00
var n int
2014-10-21 23:30:18 -07:00
var err error
2015-11-10 13:10:43 -08:00
cs . ProposalBlock = wire . ReadBinary ( & types . Block { } , cs . ProposalBlockParts . GetReader ( ) , types . MaxBlockSize , & n , & err ) . ( * types . Block )
2015-12-13 14:56:05 -05:00
log . Info ( "Received complete proposal" , "hash" , cs . ProposalBlock . Hash ( ) , "round" , cs . Proposal . Round )
2015-06-12 20:24:08 -07:00
if cs . Step == RoundStepPropose && cs . isProposalComplete ( ) {
2015-06-24 18:51:14 -07:00
// Move onto the next step
2015-12-11 11:57:15 -05:00
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 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
2015-12-13 19:30:15 -05:00
func ( cs * ConsensusState ) tryAddVote ( valIndex int , vote * types . Vote , peerKey string ) error {
_ , _ , err := cs . addVote ( valIndex , 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
2015-11-01 11:34:08 -08:00
} else if _ , ok := err . ( * types . ErrVoteConflictingSignature ) ; ok {
2015-08-12 14:00:23 -04:00
log . Warn ( "Found conflicting vote. Publish evidence" )
2015-12-01 20:12:01 -08:00
/ * TODO
2015-08-12 14:00:23 -04:00
evidenceTx := & types . DupeoutTx {
Address : address ,
VoteA : * errDupe . VoteA ,
VoteB : * errDupe . VoteB ,
}
2015-12-01 20:12:01 -08:00
cs . mempool . BroadcastTx ( evidenceTx ) // shouldn't need to check returned err
2015-11-01 11:34:08 -08:00
* /
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.
log . Warn ( "Error attempting to add vote" , "error" , 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
//-----------------------------------------------------------------------------
2015-08-26 18:56:34 -04:00
func ( cs * ConsensusState ) addVote ( valIndex int , vote * types . Vote , peerKey string ) ( added bool , address [ ] byte , err error ) {
2015-08-12 14:00:23 -04:00
log . Debug ( "addVote" , "voteHeight" , vote . Height , "voteType" , vote . Type , "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?
2015-08-26 18:56:34 -04:00
if vote . Height + 1 == cs . Height {
if ! ( cs . Step == RoundStepNewHeight && vote . Type == types . VoteTypePrecommit ) {
// TODO: give the reason ..
2015-12-11 11:57:15 -05:00
// fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
2015-08-26 18:56:34 -04:00
return added , nil , ErrVoteHeightMismatch
}
added , address , err = cs . LastCommit . AddByIndex ( valIndex , vote )
2015-05-04 11:18:21 -07:00
if added {
2015-07-19 21:49:13 +00:00
log . Info ( Fmt ( "Added to lastPrecommits: %v" , cs . LastCommit . StringShort ( ) ) )
2015-12-12 01:28:33 -05:00
cs . evsw . FireEvent ( types . EventStringVote ( ) , & types . EventDataVote { valIndex , address , vote } )
2015-05-04 11:18:21 -07:00
}
return
2015-06-05 14:15:40 -07:00
}
// A prevote/precommit for this height?
if vote . Height == cs . Height {
2015-06-24 14:04:40 -07:00
height := cs . Height
2015-08-26 18:56:34 -04:00
added , address , err = cs . Votes . AddByIndex ( valIndex , vote , peerKey )
2015-05-04 11:18:21 -07:00
if added {
2015-12-12 01:28:33 -05:00
cs . evsw . FireEvent ( types . EventStringVote ( ) , & types . EventDataVote { valIndex , address , 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 )
2015-12-01 20:12:01 -08:00
log . 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
// there.
if ( cs . LockedBlock != nil ) && ( cs . LockedRound < vote . Round ) && ( vote . Round <= cs . Round ) {
2015-06-24 18:51:14 -07:00
hash , _ , ok := prevotes . TwoThirdsMajority ( )
if ok && ! cs . LockedBlock . HashesTo ( hash ) {
2015-07-19 21:49:13 +00:00
log . Notice ( "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
2015-09-09 16:45:53 -04:00
cs . evsw . FireEvent ( types . EventStringUnlock ( ) , cs . RoundStateEvent ( ) )
2015-06-24 18:51:14 -07:00
}
}
if cs . Round <= vote . Round && prevotes . HasTwoThirdsAny ( ) {
// Round-skip over to PrevoteWait or goto Precommit.
2015-12-11 11:57:15 -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-11 11:57:15 -05:00
cs . EnterPrecommit ( height , vote . Round )
2015-12-10 11:41:18 -05:00
} else {
2015-12-11 11:57:15 -05:00
cs . EnterPrevote ( height , vote . Round ) // if the vote is ahead of us
2015-12-10 11:41:18 -05:00
cs . EnterPrevoteWait ( height , vote . Round )
}
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-11 11:57:15 -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 )
2015-12-01 20:12:01 -08:00
log . Info ( "Added to precommit" , "vote" , vote , "precommits" , precommits . StringShort ( ) )
2015-08-19 16:11:52 -04:00
hash , _ , ok := precommits . TwoThirdsMajority ( )
if ok {
2015-12-10 11:41:18 -05:00
if len ( hash ) == 0 {
2015-12-11 11:57:15 -05:00
cs . EnterNewRound ( height , vote . Round + 1 )
2015-12-10 11:41:18 -05:00
} else {
2015-12-11 11:57:15 -05:00
cs . EnterNewRound ( height , vote . Round )
cs . EnterPrecommit ( height , vote . Round )
2015-12-10 11:41:18 -05:00
cs . EnterCommit ( height , vote . Round )
}
} else if cs . Round <= vote . Round && precommits . HasTwoThirdsAny ( ) {
2015-12-11 11:57:15 -05:00
cs . EnterNewRound ( height , vote . Round )
cs . EnterPrecommit ( height , vote . Round )
2015-12-10 11:41:18 -05:00
cs . EnterPrecommitWait ( height , vote . Round )
//}()
2015-06-05 14:15:40 -07:00
}
default :
2015-07-19 23:42:52 +00:00
PanicSanity ( 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?
2015-07-19 21:49:13 +00:00
log . Info ( "Vote ignored and not added" , "voteHeight" , vote . Height , "csHeight" , cs . Height )
2015-06-05 14:15:40 -07:00
return
2014-10-21 23:30:18 -07:00
}
2015-03-22 19:00:08 -07:00
func ( cs * ConsensusState ) stageBlock ( block * types . Block , blockParts * types . PartSet ) error {
2015-01-15 22:43:15 -08:00
if block == nil {
2015-07-19 23:42:52 +00:00
PanicSanity ( "Cannot stage nil block" )
2014-10-24 14:37:12 -07:00
}
2014-09-14 15:37:32 -07:00
// Already staged?
2015-03-26 10:58:20 -07:00
blockHash := block . Hash ( )
if cs . stagedBlock != nil && len ( blockHash ) != 0 && bytes . Equal ( cs . stagedBlock . Hash ( ) , blockHash ) {
2014-09-14 15:37:32 -07:00
return nil
}
2015-12-01 20:12:01 -08:00
// Create a new event cache to cache all events.
cs . evc = events . NewEventCache ( cs . evsw )
2014-09-14 15:37:32 -07:00
// Create a copy of the state for staging
2014-10-14 13:11:54 -07:00
stateCopy := cs . state . Copy ( )
2015-04-15 23:40:27 -07:00
stateCopy . SetFireable ( cs . evc )
2014-09-14 15:37:32 -07:00
2015-12-01 20:12:01 -08:00
// Run the block on the State:
// + update validator sets
// + first rolls back proxyAppCtx
// + run txs on the proxyAppCtx or rollback
err := stateCopy . ExecBlock ( cs . proxyAppCtx , block , blockParts . Header ( ) )
2014-09-14 15:37:32 -07:00
if err != nil {
return err
2014-08-10 16:35:08 -07:00
}
2015-12-01 20:12:01 -08:00
// Everything looks good!
cs . stagedBlock = block
cs . stagedState = stateCopy
return nil
2014-08-10 16:35:08 -07:00
}
2014-10-24 14:37:12 -07:00
2015-08-12 14:00:23 -04:00
func ( cs * ConsensusState ) signVote ( type_ byte , hash [ ] byte , header types . PartSetHeader ) ( * types . Vote , error ) {
2015-03-22 19:00:08 -07:00
vote := & types . Vote {
2015-08-12 22:36:43 -07:00
Height : cs . Height ,
Round : cs . Round ,
Type : type_ ,
BlockHash : hash ,
BlockPartsHeader : header ,
2014-10-24 14:37:12 -07:00
}
2015-05-29 17:53:57 -04:00
err := cs . privValidator . SignVote ( cs . state . ChainID , vote )
2015-08-12 14:00:23 -04:00
return vote , err
}
2015-12-12 01:28:33 -05:00
// signs the vote, publishes on internalMsgQueue
2015-08-12 14:00:23 -04:00
func ( cs * ConsensusState ) signAddVote ( type_ byte , hash [ ] byte , header types . PartSetHeader ) * types . Vote {
if cs . privValidator == nil || ! cs . Validators . HasAddress ( cs . privValidator . Address ) {
return nil
}
vote , err := cs . signVote ( type_ , hash , header )
2014-12-31 16:14:26 -08:00
if err == nil {
2015-12-13 19:30:15 -05:00
// TODO: store our index in the cs so we don't have to do this every time
2015-08-26 18:56:34 -04:00
valIndex , _ := cs . Validators . GetByAddress ( cs . privValidator . Address )
2015-12-11 11:57:15 -05:00
cs . sendInternalMessage ( msgInfo { & VoteMessage { valIndex , vote } , "" } )
log . Notice ( "Signed and pushed vote" , "height" , cs . Height , "round" , cs . Round , "vote" , vote , "error" , err )
2014-12-31 16:14:26 -08:00
return vote
} else {
log . Warn ( "Error signing vote" , "height" , cs . Height , "round" , cs . Round , "vote" , vote , "error" , err )
return nil
}
2014-10-24 14:37:12 -07:00
}
2014-10-30 03:32:09 -07:00
2015-06-05 14:15:40 -07:00
// Save Block, save the +2/3 Commits we've seen
2015-08-10 20:38:45 -07:00
func ( cs * ConsensusState ) saveBlock ( block * types . Block , blockParts * types . PartSet , commits * types . VoteSet ) {
2014-12-31 16:46:49 -08:00
2014-10-30 03:32:09 -07:00
// The proposal must be valid.
2015-01-15 22:43:15 -08:00
if err := cs . stageBlock ( block , blockParts ) ; err != nil {
2015-07-19 23:42:52 +00:00
PanicSanity ( Fmt ( "saveBlock() an invalid block: %v" , err ) )
2014-10-30 03:32:09 -07:00
}
2015-01-08 22:07:23 -08:00
// Save to blockStore.
2015-01-15 22:43:15 -08:00
if cs . blockStore . Height ( ) < block . Height {
2015-01-08 22:07:23 -08:00
seenValidation := commits . MakeValidation ( )
2015-01-15 22:43:15 -08:00
cs . blockStore . SaveBlock ( block , blockParts , seenValidation )
2015-01-08 22:07:23 -08:00
}
2014-10-30 03:32:09 -07:00
2015-12-01 20:12:01 -08:00
// Commit to proxyAppCtx
err := cs . stagedState . Commit ( cs . proxyAppCtx )
if err != nil {
// TODO: handle this gracefully.
PanicQ ( Fmt ( "Commit failed for applicaiton" ) )
}
2015-01-08 22:07:23 -08:00
// Save the state.
2014-10-30 03:32:09 -07:00
cs . stagedState . Save ( )
// Update mempool.
2015-12-01 20:12:01 -08:00
cs . mempool . Update ( block )
2014-10-30 03:32:09 -07:00
2015-06-24 14:04:40 -07:00
// Fire off event
2015-08-12 14:00:23 -04:00
if cs . evsw != nil && cs . evc != nil {
2015-09-22 21:12:34 -04:00
cs . evsw . FireEvent ( types . EventStringNewBlock ( ) , types . EventDataNewBlock { block } )
go cs . evc . Flush ( )
2015-08-12 14:00:23 -04:00
}
2015-06-24 14:04:40 -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
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
}