Browse Source

consensus events

pull/142/head
Ethan Buchman 9 years ago
parent
commit
aab024f18d
2 changed files with 143 additions and 89 deletions
  1. +97
    -84
      consensus/state.go
  2. +46
    -5
      types/events.go

+ 97
- 84
consensus/state.go View File

@ -244,6 +244,22 @@ type RoundState struct {
LastValidators *types.ValidatorSet
}
func (rs *RoundState) RoundStateEvent() *types.EventDataRoundState {
return &types.EventDataRoundState{
CurrentTime: time.Now(),
Height: rs.Height,
Round: rs.Round,
Step: rs.Step.String(),
StartTime: rs.StartTime,
CommitTime: rs.CommitTime,
Proposal: rs.Proposal,
ProposalBlock: rs.ProposalBlock,
LockedRound: rs.LockedRound,
LockedBlock: rs.LockedBlock,
POLRound: rs.Votes.POLRound(),
}
}
func (rs *RoundState) String() string {
return rs.StringIndented("")
}
@ -300,18 +316,13 @@ type ConsensusState struct {
evsw events.Fireable
evc *events.EventCache // set in stageBlock and passed into state
timeoutChan chan TimeoutEvent // so we can track timeouts
timeoutQuitChan chan struct{}
}
func NewConsensusState(state *sm.State, blockStore *bc.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState {
cs := &ConsensusState{
blockStore: blockStore,
mempoolReactor: mempoolReactor,
newStepCh: make(chan *RoundState, 10),
timeoutChan: make(chan TimeoutEvent), // XXX: blocks!
timeoutQuitChan: make(chan struct{}, 1),
blockStore: blockStore,
mempoolReactor: mempoolReactor,
newStepCh: make(chan *RoundState, 10),
}
cs.updateToState(state, true)
// Don't call scheduleRound0 yet.
@ -368,15 +379,12 @@ func (cs *ConsensusState) NewStepCh() chan *RoundState {
func (cs *ConsensusState) OnStart() error {
cs.BaseService.OnStart()
go cs.logTimeouts(cs.timeoutChan, cs.timeoutQuitChan)
cs.scheduleRound0(cs.Height)
return nil
}
func (cs *ConsensusState) OnStop() {
// It's mostly asynchronous so, there's not much to stop.
// just the timeout tracker
close(cs.timeoutQuitChan)
// It's asynchronous so, there's not much to stop.
cs.BaseService.OnStop()
}
@ -387,9 +395,9 @@ func (cs *ConsensusState) scheduleRound0(height int) {
go func() {
if 0 < sleepDuration {
time.Sleep(sleepDuration)
// TODO: fire on timeoutCh ?
// TODO: event?
}
cs.EnterNewRound(height, 0)
cs.EnterNewRound(height, 0, false)
}()
}
@ -496,13 +504,18 @@ func (cs *ConsensusState) SetPrivValidator(priv *types.PrivValidator) {
// Enter: `timeoutPrecommits` after any +2/3 precommits from (height,round-1)
// Enter: `startTime = commitTime+timeoutCommit` from NewHeight(height)
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) EnterNewRound(height int, round int) {
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
}
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
}
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)
}
@ -530,6 +543,8 @@ func (cs *ConsensusState) EnterNewRound(height int, round int) {
}
cs.Votes.SetRound(round + 1) // also track next round (round+1) to allow round-skipping
cs.evsw.FireEvent(types.EventStringNewRound(), cs.RoundStateEvent())
// Immediately go to EnterPropose.
go cs.EnterPropose(height, round)
}
@ -554,14 +569,14 @@ func (cs *ConsensusState) EnterPropose(height int, round int) {
// 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)
go cs.EnterPrevote(height, cs.Round, false)
}
}()
// This step times out after `timeoutPropose`
go func() {
time.Sleep(timeoutPropose)
cs.EnterPrevote(height, round)
cs.EnterPrevote(height, round, true)
}()
// Nothing more to do if we're not a validator
@ -674,13 +689,24 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Enter: any +2/3 prevotes for future round.
// Prevote for LockedBlock if we're locked, or ProposalBlock if valid.
// Otherwise vote nil.
func (cs *ConsensusState) EnterPrevote(height int, round int) {
func (cs *ConsensusState) EnterPrevote(height int, round int, timedOut bool) {
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
}
// fire event for how we got here
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutPropose(), cs.RoundStateEvent())
} else if cs.isProposalComplete() {
cs.evsw.FireEvent(types.EventStringCompleteProposal(), cs.RoundStateEvent())
} else {
// we received +2/3 prevotes for a future round
// TODO: catchup event?
}
log.Info(Fmt("EnterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
// Sign and broadcast vote as necessary
@ -690,10 +716,9 @@ func (cs *ConsensusState) EnterPrevote(height int, round int) {
cs.Round = round
cs.Step = RoundStepPrevote
cs.newStepCh <- cs.getRoundState()
/* This isn't necessary because addVote() does it for us.
if cs.Votes.Prevotes(round).HasTwoThirdsAny() {
go cs.EnterPrevoteWait(height, round)
}*/
// 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)
}
func (cs *ConsensusState) doPrevote(height int, round int) {
@ -748,8 +773,7 @@ func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
// After `timeoutPrevote0+timeoutPrevoteDelta*round`, EnterPrecommit()
go func() {
time.Sleep(timeoutPrevote0 + timeoutPrevoteDelta*time.Duration(round))
cs.timeoutChan <- TimeoutEvent{RoundStepPrevote, height, round}
cs.EnterPrecommit(height, round)
cs.EnterPrecommit(height, round, true)
}()
}
@ -759,13 +783,18 @@ func (cs *ConsensusState) EnterPrevoteWait(height int, round int) {
// Lock & precommit the ProposalBlock if we have enough prevotes for it (a POL in this round)
// 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) {
func (cs *ConsensusState) EnterPrecommit(height int, round int, timedOut bool) {
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
}
if timedOut {
cs.evsw.FireEvent(types.EventStringTimeoutWait(), cs.RoundStateEvent())
}
log.Info(Fmt("EnterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
@ -773,19 +802,14 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
cs.Round = round
cs.Step = RoundStepPrecommit
cs.newStepCh <- cs.getRoundState()
/* This isn't necessary because addVote() does it for us.
if cs.Votes.Precommits(round).HasTwoThirdsAny() {
go cs.EnterPrecommitWait(height, round)
}*/
}()
hash, partsHeader, ok := cs.Votes.Prevotes(round).TwoThirdsMajority()
// If we don't have two thirds of prevotes, we must precommit nil
// If we don't have a polka, we must precommit nil
if !ok {
if cs.LockedBlock != nil {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit while we're locked. Precommitting nil")
//cs.signAddVote(types.VoteTypePrecommit, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
} else {
log.Info("EnterPrecommit: No +2/3 prevotes during EnterPrecommit. Precommitting nil.")
}
@ -794,6 +818,12 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
}
// At this point +2/3 prevoted for a particular block or nil
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()))
}
// +2/3 prevoted nil. Unlock and precommit nil.
if len(hash) == 0 {
@ -804,6 +834,7 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
@ -813,15 +844,16 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
// If we're already locked on that block, precommit it, and update the LockedRound
if cs.LockedBlock.HashesTo(hash) {
log.Info("EnterPrecommit: +2/3 prevoted locked block.")
log.Info("EnterPrecommit: +2/3 prevoted locked block. Relocking")
cs.LockedRound = round
cs.evsw.FireEvent(types.EventStringRelock(), cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
return
}
// If +2/3 prevoted for proposal block, stage and precommit it
if cs.ProposalBlock.HashesTo(hash) {
log.Info("EnterPrecommit: +2/3 prevoted proposal block.", "hash", hash)
log.Info("EnterPrecommit: +2/3 prevoted proposal block. Locking", "hash", hash)
// Validate the block.
if err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts); err != nil {
PanicConsensus(Fmt("EnterPrecommit: +2/3 prevoted for an invalid block: %v", err))
@ -829,20 +861,15 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
cs.LockedRound = round
cs.LockedBlock = cs.ProposalBlock
cs.LockedBlockParts = cs.ProposalBlockParts
cs.evsw.FireEvent(types.EventStringLock(), cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, hash, partsHeader)
return
}
// Otherwise, we need to fetch the +2/3 prevoted block.
// Unlock and precommit nil.
// There was a polka in this round for a block we don't have.
// Fetch that block, unlock, and precommit nil.
// The +2/3 prevotes for this round is the POL for our unlock.
// TODO: In the future save the POL prevotes for justification.
// NOTE: we could have performed this check sooner above.
if cs.Votes.POLRound() < round {
PanicSanity(Fmt("This POLRound should be %v but got %", round, cs.Votes.POLRound()))
}
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
@ -850,6 +877,7 @@ func (cs *ConsensusState) EnterPrecommit(height int, round int) {
cs.ProposalBlock = nil
cs.ProposalBlockParts = types.NewPartSetFromHeader(partsHeader)
}
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
}
@ -875,12 +903,11 @@ func (cs *ConsensusState) EnterPrecommitWait(height int, round int) {
// After `timeoutPrecommit0+timeoutPrecommitDelta*round`, EnterNewRound()
go func() {
time.Sleep(timeoutPrecommit0 + timeoutPrecommitDelta*time.Duration(round))
cs.timeoutChan <- TimeoutEvent{RoundStepPrecommit, height, round}
// 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(height, round+1)
cs.EnterNewRound(height, round+1, true)
}()
}
@ -1029,6 +1056,7 @@ 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()
@ -1055,7 +1083,7 @@ 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)
go cs.EnterPrevote(height, cs.Round, false)
} else if cs.Step == RoundStepCommit {
// If we're waiting on the proposal block...
cs.tryFinalizeCommit(height, cs.Round)
@ -1065,20 +1093,13 @@ func (cs *ConsensusState) AddProposalBlockPart(height int, part *types.Part) (ad
return added, nil
}
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.addVote(valIndex, vote, peerKey)
}
// Attempt to add the vote. if its a duplicate signature, dupeout the validator
func (cs *ConsensusState) TryAddVote(valIndex int, vote *types.Vote, peerKey string) (bool, error) {
added, address, err := cs.AddVote(valIndex, vote, peerKey)
if err != nil {
// If the vote height is off, we'll just ignore it,
// But if it's a conflicting sig, broadcast evidence tx for slashing
// and otherwise punish peer.
// But if it's a conflicting sig, broadcast evidence tx for slashing.
// If it's otherwise invalid, punish peer.
if err == ErrVoteHeightMismatch {
return added, err
} else if errDupe, ok := err.(*types.ErrVoteConflictingSignature); ok {
@ -1099,11 +1120,24 @@ func (cs *ConsensusState) TryAddVote(valIndex int, vote *types.Vote, peerKey str
return added, nil
}
func (cs *ConsensusState) AddVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
return cs.addVote(valIndex, vote, peerKey)
}
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string) (added bool, address []byte, err error) {
log.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "csHeight", cs.Height)
defer func() {
if added {
cs.evsw.FireEvent(types.EventStringVote(), &types.EventDataVote{valIndex, address, vote})
}
}()
// A precommit for the previous height?
if vote.Height+1 == cs.Height {
if !(cs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
@ -1139,23 +1173,24 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
cs.evsw.FireEvent(types.EventStringUnlock(), cs.RoundStateEvent())
}
}
if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
// Round-skip over to PrevoteWait or goto Precommit.
go func() {
cs.EnterNewRound(height, vote.Round)
cs.EnterNewRound(height, vote.Round, false)
if prevotes.HasTwoThirdsMajority() {
cs.EnterPrecommit(height, vote.Round)
cs.EnterPrecommit(height, vote.Round, false)
} else {
cs.EnterPrevote(height, vote.Round)
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)
go cs.EnterPrevote(height, cs.Round, false)
}
}
case types.VoteTypePrecommit:
@ -1165,17 +1200,17 @@ func (cs *ConsensusState) addVote(valIndex int, vote *types.Vote, peerKey string
if ok {
go func() {
if len(hash) == 0 {
cs.EnterNewRound(height, vote.Round+1)
cs.EnterNewRound(height, vote.Round+1, false)
} else {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
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() {
cs.EnterNewRound(height, vote.Round)
cs.EnterPrecommit(height, vote.Round)
cs.EnterNewRound(height, vote.Round, false)
cs.EnterPrecommit(height, vote.Round, false)
cs.EnterPrecommitWait(height, vote.Round)
}()
}
@ -1290,25 +1325,3 @@ func (cs *ConsensusState) SetFireable(evsw events.Fireable) {
func (cs *ConsensusState) String() string {
return Fmt("ConsensusState(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step)
}
//-----------------------------------------------------------------------------
// timeout tracking
type TimeoutEvent struct {
Type RoundStepType
Height int
Round int
}
func (cs *ConsensusState) logTimeouts(timeoutChan chan TimeoutEvent, quitChan <-chan struct{}) {
for {
select {
case timeout := <-timeoutChan:
log.Info("Timeout in consensus state", "height", timeout.Height, "round", timeout.Round, "step", timeout.Type.String())
case <-quitChan:
return
}
}
}

+ 46
- 5
types/events.go View File

@ -2,6 +2,7 @@ package types
import (
"fmt"
"time"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/wire"
@ -22,6 +23,16 @@ func EventStringDupeout() string { return "Dupeout" }
func EventStringNewBlock() string { return "NewBlock" }
func EventStringFork() string { return "Fork" }
func EventStringNewRound() string { return fmt.Sprintf("NewRound") }
func EventStringTimeoutPropose() string { return fmt.Sprintf("TimeoutPropose") }
func EventStringCompleteProposal() string { return fmt.Sprintf("CompleteProposal") }
func EventStringPolka() string { return fmt.Sprintf("Polka") }
func EventStringUnlock() string { return fmt.Sprintf("Unlock") }
func EventStringLock() string { return fmt.Sprintf("Lock") }
func EventStringRelock() string { return fmt.Sprintf("Relock") }
func EventStringTimeoutWait() string { return fmt.Sprintf("TimeoutWait") }
func EventStringVote() string { return fmt.Sprintf("Vote") }
//----------------------------------------
const (
@ -30,6 +41,9 @@ const (
EventDataTypeTx = byte(0x03)
EventDataTypeCall = byte(0x04)
EventDataTypeLog = byte(0x05)
EventDataTypeRoundState = byte(0x11)
EventDataTypeVote = byte(0x12)
)
type EventData interface {
@ -43,10 +57,12 @@ var _ = wire.RegisterInterface(
wire.ConcreteType{EventDataTx{}, EventDataTypeTx},
wire.ConcreteType{EventDataCall{}, EventDataTypeCall},
wire.ConcreteType{EventDataLog{}, EventDataTypeLog},
wire.ConcreteType{EventDataRoundState{}, EventDataTypeRoundState},
wire.ConcreteType{EventDataVote{}, EventDataTypeVote},
)
// Most event messages are basic types (a block, a transaction)
// but some (an input to a call tx or a receive) are more exotic:
// but some (an input to a call tx or a receive) are more exotic
type EventDataNewBlock struct {
Block *Block `json:"block"`
@ -84,7 +100,32 @@ type EventDataLog struct {
Height int64 `json:"height"`
}
func (_ EventDataNewBlock) AssertIsEventData() {}
func (_ EventDataTx) AssertIsEventData() {}
func (_ EventDataCall) AssertIsEventData() {}
func (_ EventDataLog) AssertIsEventData() {}
// We fire the most recent round state that led to the event
// (ie. NewRound will have the previous rounds state)
type EventDataRoundState struct {
CurrentTime time.Time `json:"current_time"`
Height int `json:"height"`
Round int `json:"round"`
Step string `json:"step"`
StartTime time.Time `json:"start_time"`
CommitTime time.Time `json:"commit_time"`
Proposal *Proposal `json:"proposal"`
ProposalBlock *Block `json:"proposal_block"`
LockedRound int `json:"locked_round"`
LockedBlock *Block `json:"locked_block"`
POLRound int `json:"pol_round"`
}
type EventDataVote struct {
Index int
Address []byte
Vote *Vote
}
func (_ EventDataNewBlock) AssertIsEventData() {}
func (_ EventDataTx) AssertIsEventData() {}
func (_ EventDataCall) AssertIsEventData() {}
func (_ EventDataLog) AssertIsEventData() {}
func (_ EventDataRoundState) AssertIsEventData() {}
func (_ EventDataVote) AssertIsEventData() {}

Loading…
Cancel
Save