Browse Source

move routines to consensus state. drop locks and other go routines

pull/169/head
Ethan Buchman 9 years ago
parent
commit
d9b55101e5
4 changed files with 295 additions and 262 deletions
  1. +22
    -19
      consensus/common_test.go
  2. +4
    -128
      consensus/reactor.go
  3. +217
    -78
      consensus/state.go
  4. +52
    -37
      consensus/state_test.go

+ 22
- 19
consensus/common_test.go View File

@ -21,6 +21,8 @@ import (
var chainID string
var ensureTimeout = time.Duration(2)
func init() {
chainID = config.GetString("chain_id")
}
@ -76,8 +78,10 @@ func decideProposal(cs1 *ConsensusState, cs2 *validatorStub, height, round int)
//-------------------------------------------------------------------------------
// utils
func nilRound(t *testing.T, startRound int, cs1 *ConsensusState, vss ...*validatorStub) {
func nilRound(t *testing.T, cs1 *ConsensusState, vss ...*validatorStub) {
cs1.mtx.Lock()
height, round := cs1.Height, cs1.Round
cs1.mtx.Unlock()
waitFor(t, cs1, height, round, RoundStepPrevote)
@ -132,15 +136,18 @@ func addVoteToFromMany(to *ConsensusState, votes []*types.Vote, froms ...*valida
func addVoteToFrom(to *ConsensusState, from *validatorStub, vote *types.Vote) {
valIndex, _ := to.Validators.GetByAddress(from.PrivValidator.Address)
added, err := to.TryAddVote(valIndex, vote, "")
if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
// let it fly
} else if !added {
fmt.Println("to, from, vote:", to.Height, from.Height, vote.Height)
panic(fmt.Sprintln("Failed to add vote. Err:", err))
} else if err != nil {
panic(fmt.Sprintln("Failed to add vote:", err))
}
to.msgQueue <- msgInfo{msg: &VoteMessage{valIndex, vote}}
// added, err := to.TryAddVote(valIndex, vote, "")
/*
if _, ok := err.(*types.ErrVoteConflictingSignature); ok {
// let it fly
} else if !added {
fmt.Println("to, from, vote:", to.Height, from.Height, vote.Height)
panic(fmt.Sprintln("Failed to add vote. Err:", err))
} else if err != nil {
panic(fmt.Sprintln("Failed to add vote:", err))
}*/
}
func signVoteMany(voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
@ -166,7 +173,7 @@ func signAddVoteToFrom(voteType byte, to *ConsensusState, from *validatorStub, h
}
func ensureNoNewStep(t *testing.T, cs *ConsensusState) {
timeout := time.NewTicker(2 * time.Second)
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
case <-timeout.C:
break
@ -176,7 +183,7 @@ func ensureNoNewStep(t *testing.T, cs *ConsensusState) {
}
func ensureNewStep(t *testing.T, cs *ConsensusState) *RoundState {
timeout := time.NewTicker(2 * time.Second)
timeout := time.NewTicker(ensureTimeout * time.Second)
select {
case <-timeout.C:
panic("We should have gone to the next step, not be stuck waiting")
@ -294,15 +301,11 @@ func simpleConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
evsw := events.NewEventSwitch()
cs.SetFireable(evsw)
// read off the NewHeightStep
// read off the NewHeightStep from updateToState
<-cs.NewStepCh()
// start the reactor routines
// (we should move these to state but the receive routine needs to be able to "broadcast votes"
// --> add good votes to some buffered chan and have a go routine do the broadcast ...
conR := NewConsensusReactor(cs, nil, false)
go conR.receiveRoutine() // serializes processing of proposoals, block parts, votes
go conR.timeoutRoutine() // fires timeouts into the receive routine
// start the transition routines
cs.startRoutines()
for i := 0; i < nValidators; i++ {
vss[i] = NewValidatorStub(privVals[i])


+ 4
- 128
consensus/reactor.go View File

@ -29,7 +29,7 @@ const (
//-----------------------------------------------------------------------------
type ConsensusReactor struct {
p2p.BaseReactor
p2p.BaseReactor // QuitService + p2p.Switch
blockStore *bc.BlockStore
conS *ConsensusState
@ -56,8 +56,6 @@ func (conR *ConsensusReactor) OnStart() error {
return err
}
}
go conR.receiveRoutine() // serializes processing of proposoals, block parts, votes
go conR.timeoutRoutine() // fires timeouts into the receive routine
go conR.broadcastNewRoundStepRoutine()
return nil
}
@ -132,6 +130,9 @@ func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
// Implements Reactor
// NOTE: We process these messages even when we're fast_syncing.
// Messages affect either a peer state or the consensus state.
// Peer state updates can happen in parallel, but processing of
// proposals, block parts, and votes are ordered.
func (conR *ConsensusReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte) {
if !conR.IsRunning() {
log.Debug("Receive", "channel", chID, "peer", peer, "bytes", msgBytes)
@ -210,131 +211,6 @@ func (conR *ConsensusReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte
}
}
// 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 (conR *ConsensusReactor) timeoutRoutine() {
ticker := new(time.Ticker)
var ti timeoutInfo
log.Debug("starting timeout routine!")
for {
select {
case newti := <-conR.conS.tickChan:
log.Debug("Received tick", "new_it", newti.String(), "old_ti", ti.String())
// 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
}
}
}
ticker.Stop()
ti = newti
if ti.duration == time.Duration(0) {
// for new rounds with no sleep
conR.conS.tockChan <- ti
} else {
ticker = time.NewTicker(ti.duration)
}
case <-ticker.C:
log.Debug("timed out! firing on tock")
ticker.Stop()
conR.conS.tockChan <- ti
case <-conR.Quit:
return
}
}
}
// receiveRoutine handles messages which affect the state.
// it should keep the RoundState and be the only thing that updates it
// XXX: for incremental dev, we store this RoundState unprotected cs)
func (conR *ConsensusReactor) receiveRoutine() {
cs := conR.conS
for {
rs := cs.roundState
var mi msgInfo
select {
case mi = <-cs.msgQueue:
// handles proposals, block parts, votes
// may fire on tickChan
conR.handleMessage(mi)
case ti := <-cs.tockChan:
log.Debug("Received tock", "timeout", ti.duration, "height", ti.height, "round", ti.round, "step", ti.step)
// timeouts must be for current height, round, step
if ti.height == rs.Height+1 && ti.round == 0 && ti.step == RoundStepNewHeight {
// legit
log.Debug("received tock for next height")
} else 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)
continue
}
switch ti.step {
case RoundStepPropose:
cs.EnterPrevote(ti.height, ti.round, true)
case RoundStepPrevote:
cs.EnterPrecommit(ti.height, ti.round, true)
case RoundStepPrecommit:
// If we have +2/3 of precommits for a particular block (or nil),
// we already entered commit (or the next round).
// So just try to transition to the next round,
// which is what we'd do otherwise.
cs.EnterNewRound(ti.height, ti.round+1, true)
case RoundStepNewRound:
// ?
case RoundStepNewHeight:
/*if ti.round == rs.Round && rs.Step != RoundStepNewHeight {
continue
}*/
cs.EnterNewRound(ti.height, 0, false)
}
case <-conR.Quit:
return
}
}
}
func (conR *ConsensusReactor) handleMsg(mi messageInfo) {
cs := conR.conS
var err error
msg, peerKey := mi.msg, mi.peerKey
switch msg := msg.(type) {
case *ProposalMessage:
err = cs.SetProposal(msg.Proposal)
case *BlockPartMessage:
_, err = cs.AddProposalBlockPart(msg.Height, msg.Part)
case *VoteMessage:
// attempt to add the vote and dupeout the validator if its a duplicate signature
added, err := cs.TryAddVote(msg.ValidatorIndex, msg.Vote, peerKey)
if err == ErrAddingVote {
// TODO: punish peer
}
if added {
// 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().
conR.broadcastHasVoteMessage(msg.Vote, msg.ValidatorIndex)
}
}
if err != nil {
log.Debug("error with msg", "error", err)
}
}
// Broadcasts HasVoteMessage to peers that care.
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote, index int) {
msg := &HasVoteMessage{


+ 217
- 78
consensus/state.go View File

@ -4,6 +4,7 @@ import (
"bytes"
"errors"
"fmt"
"reflect"
"sync"
"time"
@ -24,6 +25,7 @@ var (
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.
)
var (
@ -176,7 +178,7 @@ func (ti *timeoutInfo) String() string {
// Tracks consensus state across block heights and rounds.
type ConsensusState struct {
BaseService
QuitService
proxyAppCtx proxy.AppContext
blockStore *bc.BlockStore
@ -206,14 +208,14 @@ func NewConsensusState(state *sm.State, proxyAppCtx proxy.AppContext, blockStore
mempool: mempool,
newStepCh: make(chan *RoundState, 10),
msgQueue: make(chan msgInfo, msgQueueSize),
tickChan: make(chan timeoutInfo, tickBufferSize),
tockChan: make(chan timeoutInfo, tockBufferSize),
tickChan: make(chan timeoutInfo),
tockChan: make(chan timeoutInfo),
}
cs.updateToState(state)
// Don't call scheduleRound0 yet.
// We do that upon Start().
cs.reconstructLastCommit(state)
cs.BaseService = *NewBaseService(log, "ConsensusState", cs)
cs.QuitService = *NewQuitService(log, "ConsensusState", cs)
return cs
}
@ -263,10 +265,17 @@ func (cs *ConsensusState) NewStepCh() chan *RoundState {
func (cs *ConsensusState) OnStart() error {
cs.BaseService.OnStart()
cs.startRoutines()
go cs.scheduleRound0(cs.Height)
return nil
}
func (cs *ConsensusState) startRoutines() {
go cs.timeoutRoutine() // receive requests for timeouts on tickChan and fire timeouts on tockChan
go cs.receiveRoutine() // serializes processing of proposoals, block parts, votes and coordinates state transitions
}
func (cs *ConsensusState) OnStop() {
// It's asynchronous so, there's not much to stop.
cs.BaseService.OnStop()
@ -279,6 +288,9 @@ func (cs *ConsensusState) updateHeight(height int) {
}
func (cs *ConsensusState) updateRoundStep(round int, step RoundStepType) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
cs.Round = round
cs.Step = step
@ -291,15 +303,140 @@ func (cs *ConsensusState) scheduleRound0(height int) {
//log.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
sleepDuration := cs.StartTime.Sub(time.Now())
log.Debug("scheduleRound0 by firing on tick", "height", height)
// if sleepDuration is 0 it will just relay it to tockChan right away
cs.scheduleHeightRoundStep(sleepDuration, height, 0, RoundStepNewHeight)
// TODO: this go-routine ...
go func() {
if 0 < sleepDuration {
time.Sleep(sleepDuration)
// TODO: event?
}
cs.EnterNewRound(height, 0, false)
}()
}
func (cs *ConsensusState) scheduleHeightRoundStep(duration time.Duration, height, round int, step RoundStepType) {
// 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) {
cs.tickChan <- timeoutInfo{duration, height, round, step}
}
// 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 timer <-chan time.Time
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
log.Info("Scheduling timeout", "dur", ti.duration, "height", ti.height, "round", ti.round, "step", ti.step)
timer = time.After(ti.duration)
case <-timer:
// these need to not timeout!
// (we can collapse this into receive routine by managing the timer manually)
log.Info("Timed out", "dur", ti.duration, "height", ti.height, "round", ti.round, "step", ti.step)
go func() { cs.tockChan <- ti }()
case <-cs.Quit:
return
}
}
}
// receiveRoutine handles messages which may affect the state.
// It should keep the RoundState and be the only thing that updates it.
// TODO: this round state is still confounded with cs.RoundState
func (cs *ConsensusState) receiveRoutine() {
for {
rs := cs.roundState
var mi msgInfo
select {
case mi = <-cs.msgQueue:
// handles proposals, block parts, votes
// may generate internal events (votes, complete proposals, 2/3 majorities)
cs.handleMsg(mi, rs)
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
}
}
}
func (cs *ConsensusState) handleMsg(mi msgInfo, rs RoundState) {
var err error
msg, peerKey := mi.msg, mi.peerKey
switch msg := msg.(type) {
case *ProposalMessage:
err = cs.SetProposal(msg.Proposal)
case *BlockPartMessage:
_, err = cs.AddProposalBlockPart(msg.Height, msg.Part)
case *VoteMessage:
// attempt to add the vote and dupeout the validator if its a duplicate signature
added, err := cs.TryAddVote(msg.ValidatorIndex, msg.Vote, peerKey)
if err == ErrAddingVote {
// TODO: punish peer
}
if added {
// 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().
// XXX TODO: we want this routine to run in the cnsensus state so how do we call the reactor?!
// conR.broadcastHasVoteMessage(msg.Vote, msg.ValidatorIndex)
}
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)
// 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
}
switch ti.step {
case RoundStepPropose:
log.Debug("ENTERING PREVOTE")
cs.EnterPrevote(ti.height, ti.round, true)
case RoundStepPrevoteWait:
cs.EnterPrecommit(ti.height, ti.round, true)
case RoundStepPrecommitWait:
cs.EnterNewRound(ti.height, ti.round+1, true)
default:
panic(Fmt("Invalid timeout step: %v", ti.step))
}
}
// 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) {
@ -380,8 +517,6 @@ func (cs *ConsensusState) SetPrivValidator(priv *types.PrivValidator) {
// Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) EnterNewRound(height int, round int, timedOut bool) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
log.Debug(Fmt("EnterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
@ -420,35 +555,22 @@ func (cs *ConsensusState) EnterNewRound(height int, round int, timedOut bool) {
cs.evsw.FireEvent(types.EventStringNewRound(), cs.RoundStateEvent())
// Immediately go to EnterPropose.
go cs.EnterPropose(height, round)
cs.EnterPropose(height, round)
}
// Enter: from NewRound(height,round).
func (cs *ConsensusState) EnterPropose(height int, round int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
// cs.mtx.Lock()
// cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
log.Debug(Fmt("EnterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
log.Info(Fmt("EnterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done EnterPropose:
cs.updateRoundStep(round, RoundStepPropose)
cs.newStepCh <- cs.getRoundState()
// If we have the whole proposal + POL, then goto Prevote now.
// else, we'll EnterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
// or else after timeoutPropose
if cs.isProposalComplete() {
go cs.EnterPrevote(height, cs.Round, false)
}
}()
// This step times out after `timeoutPropose`
cs.tickChan <- timeoutInfo{timeoutPropose, height, round, RoundStepPropose}
log.Debug("started timer")
cs.scheduleTimeout(timeoutPropose, height, round, RoundStepPropose)
// Nothing more to do if we're not a validator
if cs.privValidator == nil {
@ -461,6 +583,17 @@ func (cs *ConsensusState) EnterPropose(height int, round int) {
log.Info("EnterPropose: Our turn to propose", "proposer", cs.Validators.Proposer().Address, "privValidator", cs.privValidator)
cs.decideProposal(height, round)
}
// Done EnterPropose:
cs.updateRoundStep(round, RoundStepPropose)
cs.newStepCh <- cs.getRoundState()
// If we have the whole proposal + POL, then goto Prevote now.
// else, we'll EnterPrevote when the rest of the proposal is received (in AddProposalBlockPart),
// or else after timeoutPropose
if cs.isProposalComplete() {
cs.EnterPrevote(height, cs.Round, false)
}
}
func (cs *ConsensusState) decideProposal(height, round int) {
@ -491,13 +624,13 @@ func (cs *ConsensusState) decideProposal(height, round int) {
cs.ProposalBlockParts = blockParts
// TODO: can we do better than just launching a go routine?
go func() {
/*go func() {
cs.msgQueue <- msgInfo{&ProposalMessage{proposal}, ""}
for i := 0; i < blockParts.Total(); i++ {
part := blockParts.GetPart(i)
cs.msgQueue <- msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""}
}
}()
}()*/
} else {
log.Warn("EnterPropose: Error signing proposal", "height", height, "round", round, "error", err)
}
@ -574,8 +707,8 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
// Otherwise vote nil.
func (cs *ConsensusState) EnterPrevote(height int, round int, timedOut bool) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
log.Debug(Fmt("EnterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
@ -598,7 +731,9 @@ func (cs *ConsensusState) EnterPrevote(height int, round int, timedOut bool) {
// Done EnterPrevote:
cs.updateRoundStep(round, RoundStepPrevote)
log.Debug("wait on new step")
cs.newStepCh <- cs.getRoundState()
log.Debug("done new step")
// 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)
@ -637,8 +772,8 @@ func (cs *ConsensusState) doPrevote(height int, round int) {
// Enter: any +2/3 prevotes at next round.
func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
log.Debug(Fmt("EnterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
@ -653,7 +788,7 @@ func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
cs.newStepCh <- cs.getRoundState()
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
cs.tickChan <- timeoutInfo{timeoutPrevote0 + timeoutPrevoteDelta*time.Duration(round), height, round, RoundStepPrevote}
cs.scheduleTimeout(timeoutPrevote0+timeoutPrevoteDelta*time.Duration(round), height, round, RoundStepPrevoteWait)
}
// Enter: +2/3 precomits for block or nil.
@ -663,8 +798,8 @@ func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
// else, unlock an existing lock and precommit nil if +2/3 of prevotes were nil,
// else, precommit nil otherwise.
func (cs *ConsensusState) EnterPrecommit(height int, round int, timedOut bool) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
// defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
log.Debug(Fmt("EnterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
@ -762,8 +897,8 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int, timedOut bool) {
// Enter: any +2/3 precommits for next round.
func (cs *ConsensusState) EnterPrecommitWait(height int, round int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
log.Debug(Fmt("EnterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
@ -778,14 +913,14 @@ func (cs *ConsensusState) EnterPrecommitWait(height int, round int) {
cs.newStepCh <- cs.getRoundState()
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
cs.tickChan <- timeoutInfo{timeoutPrecommit0 + timeoutPrecommitDelta*time.Duration(round), height, round, RoundStepPrecommit}
cs.scheduleTimeout(timeoutPrecommit0+timeoutPrecommitDelta*time.Duration(round), height, round, RoundStepPrecommitWait)
}
// Enter: +2/3 precommits for block
func (cs *ConsensusState) EnterCommit(height int, commitRound int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || RoundStepCommit <= cs.Step {
log.Debug(Fmt("EnterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
return
@ -844,13 +979,14 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
log.Warn("Attempt to finalize failed. We don't have the commit block.")
return
}
go cs.FinalizeCommit(height)
// go
cs.FinalizeCommit(height)
}
// Increment height and goto RoundStepNewHeight
func (cs *ConsensusState) FinalizeCommit(height int) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
if cs.Height != height || cs.Step != RoundStepCommit {
log.Debug(Fmt("FinalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
@ -879,7 +1015,8 @@ func (cs *ConsensusState) FinalizeCommit(height int) {
cs.updateToState(cs.stagedState)
// cs.StartTime is already set.
// Schedule Round0 to start soon.
go cs.scheduleRound0(height + 1)
// go
cs.scheduleRound0(height + 1)
// By here,
// * cs.Height has been increment to height+1
@ -891,8 +1028,8 @@ func (cs *ConsensusState) FinalizeCommit(height int) {
//-----------------------------------------------------------------------------
func (cs *ConsensusState) SetProposal(proposal *types.Proposal) error {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
// Already have one
if cs.Proposal != nil {
@ -928,8 +1065,8 @@ func (cs *ConsensusState) SetProposal(proposal *types.Proposal) error {
// NOTE: block is not necessarily valid.
// This can trigger us to go into EnterPrevote asynchronously (before we timeout of propose) or to attempt to commit
func (cs *ConsensusState) AddProposalBlockPart(height int, part *types.Part) (added bool, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
//cs.mtx.Lock()
//defer cs.mtx.Unlock()
// Blocks might be reused, so round mismatch is OK
if cs.Height != height {
@ -953,7 +1090,8 @@ func (cs *ConsensusState) AddProposalBlockPart(height int, part *types.Part) (ad
log.Info("Received complete proposal", "hash", cs.ProposalBlock.Hash())
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
// Move onto the next step
go cs.EnterPrevote(height, cs.Round, false)
//go cs.EnterPrevote(height, cs.Round, false)
cs.EnterPrevote(height, cs.Round, false)
} else if cs.Step == RoundStepCommit {
// If we're waiting on the proposal block...
cs.tryFinalizeCommit(height)
@ -993,8 +1131,8 @@ func (cs *ConsensusState) TryAddVote(valIndex int, vote *types.Vote, peerKey str
}
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
// cs.mtx.Lock()
// defer cs.mtx.Unlock()
return cs.addVote(valIndex, vote, peerKey)
}
@ -1050,19 +1188,20 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
}
if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
// Round-skip over to PrevoteWait or goto Precommit.
go func() {
cs.EnterNewRound(height, vote.Round, false)
if prevotes.HasTwoThirdsMajority() {
cs.EnterPrecommit(height, vote.Round, false)
} else {
cs.EnterPrevote(height, vote.Round, false)
cs.EnterPrevoteWait(height, vote.Round)
}
}()
//go func() {
cs.EnterNewRound(height, vote.Round, false)
if prevotes.HasTwoThirdsMajority() {
cs.EnterPrecommit(height, vote.Round, false)
} else {
cs.EnterPrevote(height, vote.Round, false)
cs.EnterPrevoteWait(height, vote.Round)
}
//}()
} else if cs.Proposal != nil && 0 <= cs.Proposal.POLRound && cs.Proposal.POLRound == vote.Round {
// If the proposal is now complete, enter prevote of cs.Round.
if cs.isProposalComplete() {
go cs.EnterPrevote(height, cs.Round, false)
// go
cs.EnterPrevote(height, cs.Round, false)
}
}
case types.VoteTypePrecommit:
@ -1070,21 +1209,21 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
log.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
hash, _, ok := precommits.TwoThirdsMajority()
if ok {
go func() {
if len(hash) == 0 {
cs.EnterNewRound(height, vote.Round+1, false)
} else {
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterCommit(height, vote.Round)
}
}()
} else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
go func() {
//go func() {
if len(hash) == 0 {
cs.EnterNewRound(height, vote.Round+1, false)
} else {
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterPrecommitWait(height, vote.Round)
}()
cs.EnterCommit(height, vote.Round)
}
//}()
} else if cs.Round <= vote.Round && precommits.HasTwoThirdsAny() {
//go func() {
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterPrecommitWait(height, vote.Round)
//}()
}
default:
PanicSanity(Fmt("Unexpected vote type %X", vote.Type)) // Should not happen.
@ -1161,9 +1300,9 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
// XXX: maybe we can use a "personal" channel in the receiveRoutine select
// and fire these there so we don't possibly block on a full msgQueue.
// though if things are really backed up, we could block on the personal channel too
go func() {
/*go func() {
cs.msgQueue <- msgInfo{&VoteMessage{valIndex, vote}, ""}
}()
}()*/
return vote
} else {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)


+ 52
- 37
consensus/state_test.go View File

@ -55,7 +55,7 @@ func TestProposerSelection0(t *testing.T) {
cs1.newStepCh = make(chan *RoundState) // so it blocks
height, round := cs1.Height, cs1.Round
cs1.EnterNewRound(height, round, false)
go cs1.EnterNewRound(height, round, false)
// lets commit a block and ensure proposer for the next height is correct
prop := cs1.Validators.Proposer()
@ -85,10 +85,21 @@ func TestProposerSelection2(t *testing.T) {
cs1, vss := simpleConsensusState(3) // test needs more work for more than 3 validators
cs1.newStepCh = make(chan *RoundState) // so it blocks
// listen for new round
ch := make(chan struct{})
evsw := events.NewEventSwitch()
evsw.OnStart()
evsw.AddListenerForEvent("tester", types.EventStringNewRound(), func(data types.EventData) {
ch <- struct{}{}
})
cs1.SetFireable(evsw)
// this time we jump in at round 2
incrementRound(vss[1:]...)
incrementRound(vss[1:]...)
cs1.EnterNewRound(cs1.Height, 2, false)
go cs1.EnterNewRound(cs1.Height, 2, false)
<-ch // wait for the new round
// everyone just votes nil. we get a new proposer each round
for i := 0; i < len(vss); i++ {
@ -96,7 +107,9 @@ func TestProposerSelection2(t *testing.T) {
if !bytes.Equal(prop.Address, vss[(i+2)%len(vss)].Address) {
t.Fatalf("expected proposer to be validator %d. Got %X", (i+2)%len(vss), prop.Address)
}
nilRound(t, 2, cs1, vss[1:]...)
go nilRound(t, cs1, vss[1:]...)
<-ch // wait for the new round event each round
incrementRound(vss[1:]...)
}
@ -118,7 +131,7 @@ func TestEnterProposeNoPrivValidator(t *testing.T) {
cs.SetFireable(evsw)
// starts a go routine for EnterPropose
cs.EnterNewRound(height, round, false)
go cs.EnterNewRound(height, round, false)
// Wait until the prevote step
waitFor(t, cs, height, round, RoundStepPrevote)
@ -133,7 +146,7 @@ func TestEnterProposeNoPrivValidator(t *testing.T) {
}
// a validator should not timeout of the prevote round (TODO: unless the block is really big!)
func TestEnterPropose(t *testing.T) {
func TestEnterProposeYesPrivValidator(t *testing.T) {
cs, _ := simpleConsensusState(1)
height, round := cs.Height, cs.Round
@ -146,8 +159,8 @@ func TestEnterPropose(t *testing.T) {
})
cs.SetFireable(evsw)
// starts a go routine for EnterPropose
cs.EnterNewRound(height, round, false)
// starts a go routine for the round
go cs.EnterNewRound(height, round, false)
// Wait until the prevote step
waitFor(t, cs, height, round, RoundStepPrevote)
@ -164,7 +177,7 @@ func TestEnterPropose(t *testing.T) {
t.Error("rs.ProposalBlockParts should be set")
}
// if we're not a validator, EnterPropose should timeout
// if we're a validator, EnterPropose should not timeout
if timeoutEventReceived == true {
t.Fatal("Expected EnterPropose not to timeout")
}
@ -204,7 +217,7 @@ func TestBadProposal(t *testing.T) {
}
// start round
cs1.EnterNewRound(height, round, false)
go cs1.EnterNewRound(height, round, false)
// now we're on a new round and not the proposer
waitFor(t, cs1, height, round, RoundStepPropose)
@ -241,7 +254,7 @@ func TestFullRound1(t *testing.T) {
height, round := cs.Height, cs.Round
// starts a go routine for EnterPropose
cs.EnterNewRound(height, round, false)
go cs.EnterNewRound(height, round, false)
// wait to finish propose and prevote
waitFor(t, cs, height, round, RoundStepPrevote)
@ -259,11 +272,15 @@ func TestFullRound1(t *testing.T) {
validatePrevoteAndPrecommit(t, cs, round, round, vss[0], propBlockHash, propBlockHash)
}
/*
// nil is proposed, so prevote and precommit nil
func TestFullRoundNil(t *testing.T) {
cs, vss := simpleConsensusState(1)
height, round := cs.Height, cs.Round
// TODO: This is not easy to test now because we need receiveRoutine to start things off
// and we want to not be the proposer but still vote ....
// Skip the propose step
cs.EnterPrevote(height, round, true)
@ -273,6 +290,7 @@ func TestFullRoundNil(t *testing.T) {
// should prevote and precommit nil
validatePrevoteAndPrecommit(t, cs, round, 0, vss[0], nil, nil)
}
*/
// run through propose, prevote, precommit commit with two validators
// where the first validator has to wait for votes from the second
@ -283,7 +301,7 @@ func TestFullRound2(t *testing.T) {
height, round := cs1.Height, cs1.Round
// start round and wait for propose and prevote
cs1.EnterNewRound(height, round, false)
go cs1.EnterNewRound(height, round, false)
waitFor(t, cs1, height, round, RoundStepPrevote)
// we should now be stuck in limbo forever, waiting for more prevotes
@ -337,8 +355,8 @@ func TestLockNoPOL(t *testing.T) {
Round1 (cs1, B) // B B // B B2
*/
// start round and wait for propose and prevote
cs1.EnterNewRound(height, 0, false)
// start round and wait for prevote
go cs1.EnterNewRound(height, 0, false)
waitFor(t, cs1, height, 0, RoundStepPrevote)
// we should now be stuck in limbo forever, waiting for more prevotes
@ -457,15 +475,19 @@ func TestLockNoPOL(t *testing.T) {
*/
// now we're on a new round and not the proposer
waitFor(t, cs1, height, 3, RoundStepPropose)
// so set the proposal block
cs1.mtx.Lock()
cs1.Proposal, cs1.ProposalBlock = prop, propBlock
cs1.mtx.Unlock()
// wait for the proposal go ahead
waitFor(t, cs1, height, 3, RoundStepPropose)
//log.Debug("waiting for timeout")
// and wait for timeout
<-timeoutChan
// go func() { <-timeoutChan }()
log.Debug("waiting for prevote")
// go to prevote, prevote for locked block (not proposal)
waitFor(t, cs1, height, 3, RoundStepPrevote)
@ -515,7 +537,7 @@ func TestLockPOLRelock(t *testing.T) {
*/
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _, _ = <-cs1.NewStepCh(), <-voteChan, <-cs1.NewStepCh()
theBlockHash := cs1.ProposalBlock.Hash()
@ -575,17 +597,13 @@ func TestLockPOLRelock(t *testing.T) {
*/
// now we're on a new round and not the proposer
<-cs1.NewStepCh()
cs1.mtx.Lock()
// so set the proposal block
cs1.mtx.Lock()
propBlockHash, propBlockParts := propBlock.Hash(), propBlock.MakePartSet()
cs1.Proposal, cs1.ProposalBlock, cs1.ProposalBlockParts = prop, propBlock, propBlockParts
cs1.mtx.Unlock()
// and wait for timeout
te = <-timeoutChan
if te.Step != RoundStepPropose.String() {
t.Fatalf("expected to timeout of propose. got %v", te.Step)
}
<-cs1.NewStepCh()
// go to prevote, prevote for locked block (not proposal), move on
_, _ = <-voteChan, <-cs1.NewStepCh()
validatePrevote(t, cs1, 0, vss[0], theBlockHash)
@ -663,7 +681,7 @@ func TestLockPOLUnlock(t *testing.T) {
*/
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _, _ = <-cs1.NewStepCh(), <-voteChan, <-cs1.NewStepCh()
theBlockHash := cs1.ProposalBlock.Hash()
@ -713,14 +731,12 @@ func TestLockPOLUnlock(t *testing.T) {
*/
// now we're on a new round and not the proposer,
<-cs1.NewStepCh()
cs1.mtx.Lock()
// so set the proposal block
cs1.mtx.Lock()
cs1.Proposal, cs1.ProposalBlock, cs1.ProposalBlockParts = prop, propBlock, propBlock.MakePartSet()
lockedBlockHash := cs1.LockedBlock.Hash()
cs1.mtx.Unlock()
// and wait for timeout
<-timeoutChan
<-cs1.NewStepCh()
// go to prevote, prevote for locked block (not proposal)
_, _ = <-voteChan, <-cs1.NewStepCh()
@ -783,7 +799,7 @@ func TestLockPOLSafety1(t *testing.T) {
cs1.SetFireable(evsw)
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _, _ = <-cs1.NewStepCh(), <-voteChan, <-cs1.NewStepCh()
propBlock := cs1.ProposalBlock
@ -817,14 +833,13 @@ func TestLockPOLSafety1(t *testing.T) {
*/
// now we're on a new round and not the proposer,
<-cs1.NewStepCh()
// so set proposal
cs1.mtx.Lock()
propBlockHash, propBlockParts := propBlock.Hash(), propBlock.MakePartSet()
cs1.Proposal, cs1.ProposalBlock, cs1.ProposalBlockParts = prop, propBlock, propBlockParts
cs1.mtx.Unlock()
// and wait for timeout
<-timeoutChan
<-cs1.NewStepCh()
if cs1.LockedBlock != nil {
t.Fatal("we should not be locked!")
}
@ -916,7 +931,7 @@ func TestLockPOLSafety2(t *testing.T) {
cs1.SetFireable(evsw)
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _, _ = <-cs1.NewStepCh(), <-voteChan, <-cs1.NewStepCh()
theBlockHash := cs1.ProposalBlock.Hash()
@ -1051,7 +1066,7 @@ func TestSlashingPrevotes(t *testing.T) {
cs1.newStepCh = make(chan *RoundState) // so it blocks
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _ = <-cs1.NewStepCh(), <-cs1.NewStepCh()
// we should now be stuck in limbo forever, waiting for more prevotes
@ -1078,7 +1093,7 @@ func TestSlashingPrecommits(t *testing.T) {
cs1.newStepCh = make(chan *RoundState) // so it blocks
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _ = <-cs1.NewStepCh(), <-cs1.NewStepCh()
// add prevote from cs2
@ -1127,7 +1142,7 @@ func TestHalt1(t *testing.T) {
cs1.SetFireable(evsw)
// start round and wait for propose and prevote
cs1.EnterNewRound(cs1.Height, 0, false)
go cs1.EnterNewRound(cs1.Height, 0, false)
_, _ = <-cs1.NewStepCh(), <-cs1.NewStepCh()
theBlockHash := cs1.ProposalBlock.Hash()


Loading…
Cancel
Save