mirror of
https://github.com/fluencelabs/tendermint
synced 2025-04-25 06:42:16 +00:00
extract some of the consensus types into ./types
so they can be used in rpc/core/types/responses.go. ``` So, it seems like we could use the actual structs here, but we don't want to have to import consensus to get them, as then clients are importing too much crap. So probably we should move some types from consensus into consensus/types so we can import. Will these raw messages be identical to: type ResultDumpConsensusState struct { RoundState cstypes.RoundState PeerRoundStates map[string]cstypes.PeerRoundState } ``` https://github.com/tendermint/tendermint/pull/724#discussion_r143598193
This commit is contained in:
parent
d6a87d3c43
commit
dc0e8de9b0
@ -15,6 +15,7 @@ import (
|
||||
abci "github.com/tendermint/abci/types"
|
||||
bc "github.com/tendermint/tendermint/blockchain"
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
mempl "github.com/tendermint/tendermint/mempool"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
@ -456,7 +457,7 @@ func (m *mockTicker) ScheduleTimeout(ti timeoutInfo) {
|
||||
if m.onlyOnce && m.fired {
|
||||
return
|
||||
}
|
||||
if ti.Step == RoundStepNewHeight {
|
||||
if ti.Step == cstypes.RoundStepNewHeight {
|
||||
m.c <- ti
|
||||
m.fired = true
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ import (
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
@ -321,7 +322,7 @@ func (conR *ConsensusReactor) FastSync() bool {
|
||||
func (conR *ConsensusReactor) registerEventCallbacks() {
|
||||
|
||||
types.AddListenerForEvent(conR.evsw, "conR", types.EventStringNewRoundStep(), func(data types.TMEventData) {
|
||||
rs := data.Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := data.Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
conR.broadcastNewRoundStep(rs)
|
||||
})
|
||||
|
||||
@ -344,7 +345,7 @@ func (conR *ConsensusReactor) broadcastProposalHeartbeatMessage(heartbeat types.
|
||||
conR.Switch.Broadcast(StateChannel, struct{ ConsensusMessage }{msg})
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) broadcastNewRoundStep(rs *RoundState) {
|
||||
func (conR *ConsensusReactor) broadcastNewRoundStep(rs *cstypes.RoundState) {
|
||||
|
||||
nrsMsg, csMsg := makeRoundStepMessages(rs)
|
||||
if nrsMsg != nil {
|
||||
@ -381,7 +382,7 @@ func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote) {
|
||||
*/
|
||||
}
|
||||
|
||||
func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
|
||||
func makeRoundStepMessages(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage, csMsg *CommitStepMessage) {
|
||||
nrsMsg = &NewRoundStepMessage{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
@ -389,7 +390,7 @@ func makeRoundStepMessages(rs *RoundState) (nrsMsg *NewRoundStepMessage, csMsg *
|
||||
SecondsSinceStartTime: int(time.Since(rs.StartTime).Seconds()),
|
||||
LastCommitRound: rs.LastCommit.Round(),
|
||||
}
|
||||
if rs.Step == RoundStepCommit {
|
||||
if rs.Step == cstypes.RoundStepCommit {
|
||||
csMsg = &CommitStepMessage{
|
||||
Height: rs.Height,
|
||||
BlockPartsHeader: rs.ProposalBlockParts.Header(),
|
||||
@ -491,8 +492,8 @@ OUTER_LOOP:
|
||||
}
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) gossipDataForCatchup(logger log.Logger, rs *RoundState,
|
||||
prs *PeerRoundState, ps *PeerState, peer p2p.Peer) {
|
||||
func (conR *ConsensusReactor) gossipDataForCatchup(logger log.Logger, rs *cstypes.RoundState,
|
||||
prs *cstypes.PeerRoundState, ps *PeerState, peer p2p.Peer) {
|
||||
|
||||
if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
|
||||
// Ensure that the peer's PartSetHeader is correct
|
||||
@ -606,24 +607,24 @@ OUTER_LOOP:
|
||||
}
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) gossipVotesForHeight(logger log.Logger, rs *RoundState, prs *PeerRoundState, ps *PeerState) bool {
|
||||
func (conR *ConsensusReactor) gossipVotesForHeight(logger log.Logger, rs *cstypes.RoundState, prs *cstypes.PeerRoundState, ps *PeerState) bool {
|
||||
|
||||
// If there are lastCommits to send...
|
||||
if prs.Step == RoundStepNewHeight {
|
||||
if prs.Step == cstypes.RoundStepNewHeight {
|
||||
if ps.PickSendVote(rs.LastCommit) {
|
||||
logger.Debug("Picked rs.LastCommit to send")
|
||||
return true
|
||||
}
|
||||
}
|
||||
// If there are prevotes to send...
|
||||
if prs.Step <= RoundStepPrevote && prs.Round != -1 && prs.Round <= rs.Round {
|
||||
if prs.Step <= cstypes.RoundStepPrevote && prs.Round != -1 && prs.Round <= rs.Round {
|
||||
if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
|
||||
logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
|
||||
return true
|
||||
}
|
||||
}
|
||||
// If there are precommits to send...
|
||||
if prs.Step <= RoundStepPrecommit && prs.Round != -1 && prs.Round <= rs.Round {
|
||||
if prs.Step <= cstypes.RoundStepPrecommit && prs.Round != -1 && prs.Round <= rs.Round {
|
||||
if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
|
||||
logger.Debug("Picked rs.Precommits(prs.Round) to send", "round", prs.Round)
|
||||
return true
|
||||
@ -752,54 +753,6 @@ func (conR *ConsensusReactor) StringIndented(indent string) string {
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// PeerRoundState contains the known state of a peer.
|
||||
// NOTE: Read-only when returned by PeerState.GetRoundState().
|
||||
type PeerRoundState struct {
|
||||
Height int // Height peer is at
|
||||
Round int // Round peer is at, -1 if unknown.
|
||||
Step RoundStepType // Step peer is at
|
||||
StartTime time.Time // Estimated start of round 0 at this height
|
||||
Proposal bool // True if peer has proposal for this round
|
||||
ProposalBlockPartsHeader types.PartSetHeader //
|
||||
ProposalBlockParts *cmn.BitArray //
|
||||
ProposalPOLRound int // Proposal's POL round. -1 if none.
|
||||
ProposalPOL *cmn.BitArray // nil until ProposalPOLMessage received.
|
||||
Prevotes *cmn.BitArray // All votes peer has for this round
|
||||
Precommits *cmn.BitArray // All precommits peer has for this round
|
||||
LastCommitRound int // Round of commit for last height. -1 if none.
|
||||
LastCommit *cmn.BitArray // All commit precommits of commit for last height.
|
||||
CatchupCommitRound int // Round that we have commit for. Not necessarily unique. -1 if none.
|
||||
CatchupCommit *cmn.BitArray // All commit precommits peer has for this height & CatchupCommitRound
|
||||
}
|
||||
|
||||
// String returns a string representation of the PeerRoundState
|
||||
func (prs PeerRoundState) String() string {
|
||||
return prs.StringIndented("")
|
||||
}
|
||||
|
||||
// StringIndented returns a string representation of the PeerRoundState
|
||||
func (prs PeerRoundState) StringIndented(indent string) string {
|
||||
return fmt.Sprintf(`PeerRoundState{
|
||||
%s %v/%v/%v @%v
|
||||
%s Proposal %v -> %v
|
||||
%s POL %v (round %v)
|
||||
%s Prevotes %v
|
||||
%s Precommits %v
|
||||
%s LastCommit %v (round %v)
|
||||
%s Catchup %v (round %v)
|
||||
%s}`,
|
||||
indent, prs.Height, prs.Round, prs.Step, prs.StartTime,
|
||||
indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts,
|
||||
indent, prs.ProposalPOL, prs.ProposalPOLRound,
|
||||
indent, prs.Prevotes,
|
||||
indent, prs.Precommits,
|
||||
indent, prs.LastCommit, prs.LastCommitRound,
|
||||
indent, prs.CatchupCommit, prs.CatchupCommitRound,
|
||||
indent)
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
var (
|
||||
ErrPeerStateHeightRegression = errors.New("Error peer state height regression")
|
||||
ErrPeerStateInvalidStartTime = errors.New("Error peer state invalid startTime")
|
||||
@ -812,7 +765,7 @@ type PeerState struct {
|
||||
logger log.Logger
|
||||
|
||||
mtx sync.Mutex
|
||||
PeerRoundState
|
||||
cstypes.PeerRoundState
|
||||
}
|
||||
|
||||
// NewPeerState returns a new PeerState for the given Peer
|
||||
@ -820,7 +773,7 @@ func NewPeerState(peer p2p.Peer) *PeerState {
|
||||
return &PeerState{
|
||||
Peer: peer,
|
||||
logger: log.NewNopLogger(),
|
||||
PeerRoundState: PeerRoundState{
|
||||
PeerRoundState: cstypes.PeerRoundState{
|
||||
Round: -1,
|
||||
ProposalPOLRound: -1,
|
||||
LastCommitRound: -1,
|
||||
@ -836,7 +789,7 @@ func (ps *PeerState) SetLogger(logger log.Logger) *PeerState {
|
||||
|
||||
// GetRoundState returns an atomic snapshot of the PeerRoundState.
|
||||
// There's no point in mutating it since it won't change PeerState.
|
||||
func (ps *PeerState) GetRoundState() *PeerRoundState {
|
||||
func (ps *PeerState) GetRoundState() *cstypes.PeerRoundState {
|
||||
ps.mtx.Lock()
|
||||
defer ps.mtx.Unlock()
|
||||
|
||||
@ -1227,7 +1180,7 @@ func DecodeMessage(bz []byte) (msgType byte, msg ConsensusMessage, err error) {
|
||||
type NewRoundStepMessage struct {
|
||||
Height int
|
||||
Round int
|
||||
Step RoundStepType
|
||||
Step cstypes.RoundStepType
|
||||
SecondsSinceStartTime int
|
||||
LastCommitRound int
|
||||
}
|
||||
|
@ -17,6 +17,7 @@ import (
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
@ -39,124 +40,6 @@ var (
|
||||
ErrVoteHeightMismatch = errors.New("Error vote height mismatch")
|
||||
)
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
// RoundStepType enum type
|
||||
|
||||
// RoundStepType enumerates the state of the consensus state machine
|
||||
type RoundStepType uint8 // These must be numeric, ordered.
|
||||
|
||||
const (
|
||||
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.
|
||||
)
|
||||
|
||||
// String returns a string
|
||||
func (rs RoundStepType) String() string {
|
||||
switch rs {
|
||||
case RoundStepNewHeight:
|
||||
return "RoundStepNewHeight"
|
||||
case RoundStepNewRound:
|
||||
return "RoundStepNewRound"
|
||||
case RoundStepPropose:
|
||||
return "RoundStepPropose"
|
||||
case RoundStepPrevote:
|
||||
return "RoundStepPrevote"
|
||||
case RoundStepPrevoteWait:
|
||||
return "RoundStepPrevoteWait"
|
||||
case RoundStepPrecommit:
|
||||
return "RoundStepPrecommit"
|
||||
case RoundStepPrecommitWait:
|
||||
return "RoundStepPrecommitWait"
|
||||
case RoundStepCommit:
|
||||
return "RoundStepCommit"
|
||||
default:
|
||||
return "RoundStepUnknown" // Cannot panic.
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// RoundState defines the internal consensus state.
|
||||
// It is Immutable when returned from ConsensusState.GetRoundState()
|
||||
// TODO: Actually, only the top pointer is copied,
|
||||
// so access to field pointers is still racey
|
||||
type RoundState struct {
|
||||
Height int // Height we are working on
|
||||
Round int
|
||||
Step RoundStepType
|
||||
StartTime time.Time
|
||||
CommitTime time.Time // Subjective time when +2/3 precommits for Block at Round were found
|
||||
Validators *types.ValidatorSet
|
||||
Proposal *types.Proposal
|
||||
ProposalBlock *types.Block
|
||||
ProposalBlockParts *types.PartSet
|
||||
LockedRound int
|
||||
LockedBlock *types.Block
|
||||
LockedBlockParts *types.PartSet
|
||||
Votes *HeightVoteSet
|
||||
CommitRound int //
|
||||
LastCommit *types.VoteSet // Last precommits at Height-1
|
||||
LastValidators *types.ValidatorSet
|
||||
}
|
||||
|
||||
// RoundStateEvent returns the H/R/S of the RoundState as an event.
|
||||
func (rs *RoundState) RoundStateEvent() types.EventDataRoundState {
|
||||
edrs := types.EventDataRoundState{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
Step: rs.Step.String(),
|
||||
RoundState: rs,
|
||||
}
|
||||
return edrs
|
||||
}
|
||||
|
||||
// String returns a string
|
||||
func (rs *RoundState) String() string {
|
||||
return rs.StringIndented("")
|
||||
}
|
||||
|
||||
// StringIndented returns a string
|
||||
func (rs *RoundState) StringIndented(indent string) string {
|
||||
return fmt.Sprintf(`RoundState{
|
||||
%s H:%v R:%v S:%v
|
||||
%s StartTime: %v
|
||||
%s CommitTime: %v
|
||||
%s Validators: %v
|
||||
%s Proposal: %v
|
||||
%s ProposalBlock: %v %v
|
||||
%s LockedRound: %v
|
||||
%s LockedBlock: %v %v
|
||||
%s Votes: %v
|
||||
%s LastCommit: %v
|
||||
%s LastValidators: %v
|
||||
%s}`,
|
||||
indent, rs.Height, rs.Round, rs.Step,
|
||||
indent, rs.StartTime,
|
||||
indent, rs.CommitTime,
|
||||
indent, rs.Validators.StringIndented(indent+" "),
|
||||
indent, rs.Proposal,
|
||||
indent, rs.ProposalBlockParts.StringShort(), rs.ProposalBlock.StringShort(),
|
||||
indent, rs.LockedRound,
|
||||
indent, rs.LockedBlockParts.StringShort(), rs.LockedBlock.StringShort(),
|
||||
indent, rs.Votes.StringIndented(indent+" "),
|
||||
indent, rs.LastCommit.StringShort(),
|
||||
indent, rs.LastValidators.StringIndented(indent+" "),
|
||||
indent)
|
||||
}
|
||||
|
||||
// StringShort returns a string
|
||||
func (rs *RoundState) StringShort() string {
|
||||
return fmt.Sprintf(`RoundState{H:%v R:%v S:%v ST:%v}`,
|
||||
rs.Height, rs.Round, rs.Step, rs.StartTime)
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
var (
|
||||
@ -171,10 +54,10 @@ type msgInfo struct {
|
||||
|
||||
// internally generated messages which may update the state
|
||||
type timeoutInfo struct {
|
||||
Duration time.Duration `json:"duration"`
|
||||
Height int `json:"height"`
|
||||
Round int `json:"round"`
|
||||
Step RoundStepType `json:"step"`
|
||||
Duration time.Duration `json:"duration"`
|
||||
Height int `json:"height"`
|
||||
Round int `json:"round"`
|
||||
Step cstypes.RoundStepType `json:"step"`
|
||||
}
|
||||
|
||||
func (ti *timeoutInfo) String() string {
|
||||
@ -199,7 +82,7 @@ type ConsensusState struct {
|
||||
|
||||
// internal state
|
||||
mtx sync.Mutex
|
||||
RoundState
|
||||
cstypes.RoundState
|
||||
state *sm.State // State until height-1.
|
||||
|
||||
// state changes may be triggered by msgs from peers,
|
||||
@ -282,13 +165,13 @@ func (cs *ConsensusState) GetState() *sm.State {
|
||||
}
|
||||
|
||||
// GetRoundState returns a copy of the internal consensus state.
|
||||
func (cs *ConsensusState) GetRoundState() *RoundState {
|
||||
func (cs *ConsensusState) GetRoundState() *cstypes.RoundState {
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
return cs.getRoundState()
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) getRoundState() *RoundState {
|
||||
func (cs *ConsensusState) getRoundState() *cstypes.RoundState {
|
||||
rs := cs.RoundState // copy
|
||||
return &rs
|
||||
}
|
||||
@ -469,20 +352,20 @@ func (cs *ConsensusState) updateHeight(height int) {
|
||||
cs.Height = height
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) updateRoundStep(round int, step RoundStepType) {
|
||||
func (cs *ConsensusState) updateRoundStep(round int, step cstypes.RoundStepType) {
|
||||
cs.Round = round
|
||||
cs.Step = step
|
||||
}
|
||||
|
||||
// enterNewRound(height, 0) at cs.StartTime.
|
||||
func (cs *ConsensusState) scheduleRound0(rs *RoundState) {
|
||||
func (cs *ConsensusState) scheduleRound0(rs *cstypes.RoundState) {
|
||||
//cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
|
||||
sleepDuration := rs.StartTime.Sub(time.Now())
|
||||
cs.scheduleTimeout(sleepDuration, rs.Height, 0, RoundStepNewHeight)
|
||||
cs.scheduleTimeout(sleepDuration, rs.Height, 0, cstypes.RoundStepNewHeight)
|
||||
}
|
||||
|
||||
// Attempt to schedule a timeout (by sending timeoutInfo on the tickChan)
|
||||
func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height, round int, step RoundStepType) {
|
||||
func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height, round int, step cstypes.RoundStepType) {
|
||||
cs.timeoutTicker.ScheduleTimeout(timeoutInfo{duration, height, round, step})
|
||||
}
|
||||
|
||||
@ -524,7 +407,7 @@ func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
|
||||
}
|
||||
|
||||
// Updates ConsensusState and increments height to match that of state.
|
||||
// The round becomes 0 and cs.Step becomes RoundStepNewHeight.
|
||||
// The round becomes 0 and cs.Step becomes cstypes.RoundStepNewHeight.
|
||||
func (cs *ConsensusState) updateToState(state *sm.State) {
|
||||
if cs.CommitRound > -1 && 0 < cs.Height && cs.Height != state.LastBlockHeight {
|
||||
cmn.PanicSanity(cmn.Fmt("updateToState() expected state height of %v but found %v",
|
||||
@ -560,7 +443,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
|
||||
|
||||
// RoundState fields
|
||||
cs.updateHeight(height)
|
||||
cs.updateRoundStep(0, RoundStepNewHeight)
|
||||
cs.updateRoundStep(0, cstypes.RoundStepNewHeight)
|
||||
if cs.CommitTime.IsZero() {
|
||||
// "Now" makes it easier to sync up dev nodes.
|
||||
// We add timeoutCommit to allow transactions
|
||||
@ -578,7 +461,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
|
||||
cs.LockedRound = 0
|
||||
cs.LockedBlock = nil
|
||||
cs.LockedBlockParts = nil
|
||||
cs.Votes = NewHeightVoteSet(state.ChainID, height, validators)
|
||||
cs.Votes = cstypes.NewHeightVoteSet(state.ChainID, height, validators)
|
||||
cs.CommitRound = -1
|
||||
cs.LastCommit = lastPrecommits
|
||||
cs.LastValidators = state.LastValidators
|
||||
@ -699,7 +582,7 @@ func (cs *ConsensusState) handleMsg(mi msgInfo) {
|
||||
}
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
|
||||
func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs cstypes.RoundState) {
|
||||
cs.Logger.Debug("Received tock", "timeout", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
|
||||
|
||||
// timeouts must be for current height, round, step
|
||||
@ -713,19 +596,19 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
|
||||
defer cs.mtx.Unlock()
|
||||
|
||||
switch ti.Step {
|
||||
case RoundStepNewHeight:
|
||||
case cstypes.RoundStepNewHeight:
|
||||
// NewRound event fired from enterNewRound.
|
||||
// XXX: should we fire timeout here (for timeout commit)?
|
||||
cs.enterNewRound(ti.Height, 0)
|
||||
case RoundStepNewRound:
|
||||
case cstypes.RoundStepNewRound:
|
||||
cs.enterPropose(ti.Height, 0)
|
||||
case RoundStepPropose:
|
||||
case cstypes.RoundStepPropose:
|
||||
types.FireEventTimeoutPropose(cs.evsw, cs.RoundStateEvent())
|
||||
cs.enterPrevote(ti.Height, ti.Round)
|
||||
case RoundStepPrevoteWait:
|
||||
case cstypes.RoundStepPrevoteWait:
|
||||
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
|
||||
cs.enterPrecommit(ti.Height, ti.Round)
|
||||
case RoundStepPrecommitWait:
|
||||
case cstypes.RoundStepPrecommitWait:
|
||||
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
|
||||
cs.enterNewRound(ti.Height, ti.Round+1)
|
||||
default:
|
||||
@ -752,7 +635,7 @@ func (cs *ConsensusState) handleTxsAvailable(height int) {
|
||||
// Enter: +2/3 prevotes any or +2/3 precommits for block or any from (height, round)
|
||||
// NOTE: cs.StartTime was already set for height.
|
||||
func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != cstypes.RoundStepNewHeight) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -773,7 +656,7 @@ func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
// Setup new round
|
||||
// we don't fire newStep for this step,
|
||||
// but we fire an event, so update the round step first
|
||||
cs.updateRoundStep(round, RoundStepNewRound)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepNewRound)
|
||||
cs.Validators = validators
|
||||
if round == 0 {
|
||||
// We've already reset these upon new height,
|
||||
@ -794,7 +677,7 @@ func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
waitForTxs := cs.config.WaitForTxs() && round == 0 && !cs.needProofBlock(height)
|
||||
if waitForTxs {
|
||||
if cs.config.CreateEmptyBlocksInterval > 0 {
|
||||
cs.scheduleTimeout(cs.config.EmptyBlocksInterval(), height, round, RoundStepNewRound)
|
||||
cs.scheduleTimeout(cs.config.EmptyBlocksInterval(), height, round, cstypes.RoundStepNewRound)
|
||||
}
|
||||
go cs.proposalHeartbeat(height, round)
|
||||
} else {
|
||||
@ -827,7 +710,7 @@ func (cs *ConsensusState) proposalHeartbeat(height, round int) {
|
||||
for {
|
||||
rs := cs.GetRoundState()
|
||||
// if we've already moved on, no need to send more heartbeats
|
||||
if rs.Step > RoundStepNewRound || rs.Round > round || rs.Height > height {
|
||||
if rs.Step > cstypes.RoundStepNewRound || rs.Round > round || rs.Height > height {
|
||||
return
|
||||
}
|
||||
heartbeat := &types.Heartbeat{
|
||||
@ -849,7 +732,7 @@ func (cs *ConsensusState) proposalHeartbeat(height, round int) {
|
||||
// Enter (CreateEmptyBlocks, CreateEmptyBlocksInterval > 0 ): after enterNewRound(height,round), after timeout of CreateEmptyBlocksInterval
|
||||
// Enter (!CreateEmptyBlocks) : after enterNewRound(height,round), once txs are in the mempool
|
||||
func (cs *ConsensusState) enterPropose(height int, round int) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPropose <= cs.Step) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -857,7 +740,7 @@ func (cs *ConsensusState) enterPropose(height int, round int) {
|
||||
|
||||
defer func() {
|
||||
// Done enterPropose:
|
||||
cs.updateRoundStep(round, RoundStepPropose)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepPropose)
|
||||
cs.newStep()
|
||||
|
||||
// If we have the whole proposal + POL, then goto Prevote now.
|
||||
@ -869,7 +752,7 @@ func (cs *ConsensusState) enterPropose(height int, round int) {
|
||||
}()
|
||||
|
||||
// If we don't get the proposal and all block parts quick enough, enterPrevote
|
||||
cs.scheduleTimeout(cs.config.Propose(round), height, round, RoundStepPropose)
|
||||
cs.scheduleTimeout(cs.config.Propose(round), height, round, cstypes.RoundStepPropose)
|
||||
|
||||
// Nothing more to do if we're not a validator
|
||||
if cs.privValidator == nil {
|
||||
@ -986,14 +869,14 @@ 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) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevote <= cs.Step) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
|
||||
defer func() {
|
||||
// Done enterPrevote:
|
||||
cs.updateRoundStep(round, RoundStepPrevote)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepPrevote)
|
||||
cs.newStep()
|
||||
}()
|
||||
|
||||
@ -1048,7 +931,7 @@ func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
|
||||
|
||||
// Enter: any +2/3 prevotes at next round.
|
||||
func (cs *ConsensusState) enterPrevoteWait(height int, round int) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevoteWait <= cs.Step) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -1059,12 +942,12 @@ func (cs *ConsensusState) enterPrevoteWait(height int, round int) {
|
||||
|
||||
defer func() {
|
||||
// Done enterPrevoteWait:
|
||||
cs.updateRoundStep(round, RoundStepPrevoteWait)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepPrevoteWait)
|
||||
cs.newStep()
|
||||
}()
|
||||
|
||||
// Wait for some more prevotes; enterPrecommit
|
||||
cs.scheduleTimeout(cs.config.Prevote(round), height, round, RoundStepPrevoteWait)
|
||||
cs.scheduleTimeout(cs.config.Prevote(round), height, round, cstypes.RoundStepPrevoteWait)
|
||||
}
|
||||
|
||||
// Enter: `timeoutPrevote` after any +2/3 prevotes.
|
||||
@ -1074,7 +957,7 @@ 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) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrecommit <= cs.Step) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -1083,7 +966,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
|
||||
defer func() {
|
||||
// Done enterPrecommit:
|
||||
cs.updateRoundStep(round, RoundStepPrecommit)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepPrecommit)
|
||||
cs.newStep()
|
||||
}()
|
||||
|
||||
@ -1167,7 +1050,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
|
||||
// Enter: any +2/3 precommits for next round.
|
||||
func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
|
||||
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrecommitWait <= cs.Step) {
|
||||
cs.Logger.Debug(cmn.Fmt("enterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -1178,18 +1061,18 @@ func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
|
||||
|
||||
defer func() {
|
||||
// Done enterPrecommitWait:
|
||||
cs.updateRoundStep(round, RoundStepPrecommitWait)
|
||||
cs.updateRoundStep(round, cstypes.RoundStepPrecommitWait)
|
||||
cs.newStep()
|
||||
}()
|
||||
|
||||
// Wait for some more precommits; enterNewRound
|
||||
cs.scheduleTimeout(cs.config.Precommit(round), height, round, RoundStepPrecommitWait)
|
||||
cs.scheduleTimeout(cs.config.Precommit(round), height, round, cstypes.RoundStepPrecommitWait)
|
||||
|
||||
}
|
||||
|
||||
// Enter: +2/3 precommits for block
|
||||
func (cs *ConsensusState) enterCommit(height int, commitRound int) {
|
||||
if cs.Height != height || RoundStepCommit <= cs.Step {
|
||||
if cs.Height != height || cstypes.RoundStepCommit <= cs.Step {
|
||||
cs.Logger.Debug(cmn.Fmt("enterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -1198,7 +1081,7 @@ func (cs *ConsensusState) enterCommit(height int, commitRound int) {
|
||||
defer func() {
|
||||
// Done enterCommit:
|
||||
// keep cs.Round the same, commitRound points to the right Precommits set.
|
||||
cs.updateRoundStep(cs.Round, RoundStepCommit)
|
||||
cs.updateRoundStep(cs.Round, cstypes.RoundStepCommit)
|
||||
cs.CommitRound = commitRound
|
||||
cs.CommitTime = time.Now()
|
||||
cs.newStep()
|
||||
@ -1255,9 +1138,9 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
|
||||
cs.finalizeCommit(height)
|
||||
}
|
||||
|
||||
// Increment height and goto RoundStepNewHeight
|
||||
// Increment height and goto cstypes.RoundStepNewHeight
|
||||
func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
if cs.Height != height || cs.Step != RoundStepCommit {
|
||||
if cs.Height != height || cs.Step != cstypes.RoundStepCommit {
|
||||
cs.Logger.Debug(cmn.Fmt("finalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
|
||||
return
|
||||
}
|
||||
@ -1351,7 +1234,7 @@ func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
|
||||
// By here,
|
||||
// * cs.Height has been increment to height+1
|
||||
// * cs.Step is now RoundStepNewHeight
|
||||
// * cs.Step is now cstypes.RoundStepNewHeight
|
||||
// * cs.StartTime is set to when we will start round0.
|
||||
}
|
||||
|
||||
@ -1369,8 +1252,8 @@ func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// We don't care about the proposal if we're already in RoundStepCommit.
|
||||
if RoundStepCommit <= cs.Step {
|
||||
// We don't care about the proposal if we're already in cstypes.RoundStepCommit.
|
||||
if cstypes.RoundStepCommit <= cs.Step {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -1415,10 +1298,10 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, ver
|
||||
cs.state.Params().BlockSizeParams.MaxBytes, &n, &err).(*types.Block)
|
||||
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
|
||||
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
|
||||
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
|
||||
if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() {
|
||||
// Move onto the next step
|
||||
cs.enterPrevote(height, cs.Round)
|
||||
} else if cs.Step == RoundStepCommit {
|
||||
} else if cs.Step == cstypes.RoundStepCommit {
|
||||
// If we're waiting on the proposal block...
|
||||
cs.tryFinalizeCommit(height)
|
||||
}
|
||||
@ -1463,7 +1346,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
// A precommit for the previous height?
|
||||
// These come in while we wait timeoutCommit
|
||||
if vote.Height+1 == cs.Height {
|
||||
if !(cs.Step == RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
|
||||
if !(cs.Step == cstypes.RoundStepNewHeight && vote.Type == types.VoteTypePrecommit) {
|
||||
// TODO: give the reason ..
|
||||
// fmt.Errorf("tryAddVote: Wrong height, not a LastCommit straggler commit.")
|
||||
return added, ErrVoteHeightMismatch
|
||||
@ -1476,7 +1359,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
// if we can skip timeoutCommit and have all the votes now,
|
||||
if cs.config.SkipTimeoutCommit && cs.LastCommit.HasAll() {
|
||||
// go straight to new round (skip timeout commit)
|
||||
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, RoundStepNewHeight)
|
||||
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
|
||||
cs.enterNewRound(cs.Height, 0)
|
||||
}
|
||||
}
|
||||
@ -1540,7 +1423,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
if cs.config.SkipTimeoutCommit && precommits.HasAll() {
|
||||
// if we have all the votes now,
|
||||
// go straight to new round (skip timeout commit)
|
||||
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, RoundStepNewHeight)
|
||||
// cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight)
|
||||
cs.enterNewRound(cs.Height, 0)
|
||||
}
|
||||
|
||||
@ -1601,7 +1484,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
|
||||
|
||||
//---------------------------------------------------------
|
||||
|
||||
func CompareHRS(h1, r1 int, s1 RoundStepType, h2, r2 int, s2 RoundStepType) int {
|
||||
func CompareHRS(h1, r1 int, s1 cstypes.RoundStepType, h2, r2 int, s2 cstypes.RoundStepType) int {
|
||||
if h1 < h2 {
|
||||
return -1
|
||||
} else if h1 > h2 {
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
)
|
||||
@ -247,7 +248,7 @@ func TestFullRound1(t *testing.T) {
|
||||
|
||||
// grab proposal
|
||||
re := <-propCh
|
||||
propBlockHash := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState).ProposalBlock.Hash()
|
||||
propBlockHash := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState).ProposalBlock.Hash()
|
||||
|
||||
<-voteCh // wait for prevote
|
||||
// NOTE: voteChan cap of 0 ensures we can complete this
|
||||
@ -344,7 +345,7 @@ func TestLockNoPOL(t *testing.T) {
|
||||
cs1.startRoutines(0)
|
||||
|
||||
re := <-proposalCh
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
theBlockHash := rs.ProposalBlock.Hash()
|
||||
|
||||
<-voteCh // prevote
|
||||
@ -384,7 +385,7 @@ func TestLockNoPOL(t *testing.T) {
|
||||
|
||||
// now we're on a new round and not the proposer, so wait for timeout
|
||||
re = <-timeoutProposeCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
if rs.ProposalBlock != nil {
|
||||
panic("Expected proposal block to be nil")
|
||||
@ -428,7 +429,7 @@ func TestLockNoPOL(t *testing.T) {
|
||||
incrementRound(vs2)
|
||||
|
||||
re = <-proposalCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
// now we're on a new round and are the proposer
|
||||
if !bytes.Equal(rs.ProposalBlock.Hash(), rs.LockedBlock.Hash()) {
|
||||
@ -515,7 +516,7 @@ func TestLockPOLRelock(t *testing.T) {
|
||||
|
||||
<-newRoundCh
|
||||
re := <-proposalCh
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
theBlockHash := rs.ProposalBlock.Hash()
|
||||
|
||||
<-voteCh // prevote
|
||||
@ -591,7 +592,7 @@ func TestLockPOLRelock(t *testing.T) {
|
||||
be := <-newBlockCh
|
||||
b := be.(types.TMEventData).Unwrap().(types.EventDataNewBlockHeader)
|
||||
re = <-newRoundCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
if rs.Height != 2 {
|
||||
panic("Expected height to increment")
|
||||
}
|
||||
@ -627,7 +628,7 @@ func TestLockPOLUnlock(t *testing.T) {
|
||||
startTestRound(cs1, cs1.Height, 0)
|
||||
<-newRoundCh
|
||||
re := <-proposalCh
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
theBlockHash := rs.ProposalBlock.Hash()
|
||||
|
||||
<-voteCh // prevote
|
||||
@ -653,7 +654,7 @@ func TestLockPOLUnlock(t *testing.T) {
|
||||
|
||||
// timeout to new round
|
||||
re = <-timeoutWaitCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
lockedBlockHash := rs.LockedBlock.Hash()
|
||||
|
||||
//XXX: this isnt guaranteed to get there before the timeoutPropose ...
|
||||
@ -713,7 +714,7 @@ func TestLockPOLSafety1(t *testing.T) {
|
||||
startTestRound(cs1, cs1.Height, 0)
|
||||
<-newRoundCh
|
||||
re := <-proposalCh
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
propBlock := rs.ProposalBlock
|
||||
|
||||
<-voteCh // prevote
|
||||
@ -761,7 +762,7 @@ func TestLockPOLSafety1(t *testing.T) {
|
||||
re = <-proposalCh
|
||||
}
|
||||
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
if rs.LockedBlock != nil {
|
||||
panic("we should not be locked!")
|
||||
@ -848,7 +849,7 @@ func TestLockPOLSafety2(t *testing.T) {
|
||||
|
||||
incrementRound(vs2, vs3, vs4)
|
||||
|
||||
cs1.updateRoundStep(0, RoundStepPrecommitWait)
|
||||
cs1.updateRoundStep(0, cstypes.RoundStepPrecommitWait)
|
||||
|
||||
t.Log("### ONTO Round 1")
|
||||
// jump in at round 1
|
||||
@ -929,7 +930,7 @@ func TestSlashingPrevotes(t *testing.T) {
|
||||
re := <-proposalCh
|
||||
<-voteCh // prevote
|
||||
|
||||
rs := re.(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
// we should now be stuck in limbo forever, waiting for more prevotes
|
||||
// add one for a different block should cause us to go into prevote wait
|
||||
@ -1009,7 +1010,7 @@ func TestHalt1(t *testing.T) {
|
||||
startTestRound(cs1, cs1.Height, 0)
|
||||
<-newRoundCh
|
||||
re := <-proposalCh
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
propBlock := rs.ProposalBlock
|
||||
propBlockParts := propBlock.MakePartSet(partSize)
|
||||
|
||||
@ -1032,7 +1033,7 @@ func TestHalt1(t *testing.T) {
|
||||
// timeout to new round
|
||||
<-timeoutWaitCh
|
||||
re = <-newRoundCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
t.Log("### ONTO ROUND 1")
|
||||
/*Round2
|
||||
@ -1050,7 +1051,7 @@ func TestHalt1(t *testing.T) {
|
||||
// receiving that precommit should take us straight to commit
|
||||
<-newBlockCh
|
||||
re = <-newRoundCh
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
|
||||
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
|
||||
if rs.Height != 2 {
|
||||
panic("expected height to increment")
|
||||
|
@ -1,4 +1,4 @@
|
||||
package consensus
|
||||
package types
|
||||
|
||||
import (
|
||||
"strings"
|
@ -1,14 +1,17 @@
|
||||
package consensus
|
||||
package types
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
)
|
||||
|
||||
var config *cfg.Config // NOTE: must be reset for each _test.go file
|
||||
|
||||
func init() {
|
||||
config = ResetConfig("consensus_height_vote_set_test")
|
||||
config = cfg.ResetTestRoot("consensus_height_vote_set_test")
|
||||
}
|
||||
|
||||
func TestPeerCatchupRounds(t *testing.T) {
|
57
consensus/types/reactor.go
Normal file
57
consensus/types/reactor.go
Normal file
@ -0,0 +1,57 @@
|
||||
package types
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
)
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// PeerRoundState contains the known state of a peer.
|
||||
// NOTE: Read-only when returned by PeerState.GetRoundState().
|
||||
type PeerRoundState struct {
|
||||
Height int // Height peer is at
|
||||
Round int // Round peer is at, -1 if unknown.
|
||||
Step RoundStepType // Step peer is at
|
||||
StartTime time.Time // Estimated start of round 0 at this height
|
||||
Proposal bool // True if peer has proposal for this round
|
||||
ProposalBlockPartsHeader types.PartSetHeader //
|
||||
ProposalBlockParts *cmn.BitArray //
|
||||
ProposalPOLRound int // Proposal's POL round. -1 if none.
|
||||
ProposalPOL *cmn.BitArray // nil until ProposalPOLMessage received.
|
||||
Prevotes *cmn.BitArray // All votes peer has for this round
|
||||
Precommits *cmn.BitArray // All precommits peer has for this round
|
||||
LastCommitRound int // Round of commit for last height. -1 if none.
|
||||
LastCommit *cmn.BitArray // All commit precommits of commit for last height.
|
||||
CatchupCommitRound int // Round that we have commit for. Not necessarily unique. -1 if none.
|
||||
CatchupCommit *cmn.BitArray // All commit precommits peer has for this height & CatchupCommitRound
|
||||
}
|
||||
|
||||
// String returns a string representation of the PeerRoundState
|
||||
func (prs PeerRoundState) String() string {
|
||||
return prs.StringIndented("")
|
||||
}
|
||||
|
||||
// StringIndented returns a string representation of the PeerRoundState
|
||||
func (prs PeerRoundState) StringIndented(indent string) string {
|
||||
return fmt.Sprintf(`PeerRoundState{
|
||||
%s %v/%v/%v @%v
|
||||
%s Proposal %v -> %v
|
||||
%s POL %v (round %v)
|
||||
%s Prevotes %v
|
||||
%s Precommits %v
|
||||
%s LastCommit %v (round %v)
|
||||
%s Catchup %v (round %v)
|
||||
%s}`,
|
||||
indent, prs.Height, prs.Round, prs.Step, prs.StartTime,
|
||||
indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts,
|
||||
indent, prs.ProposalPOL, prs.ProposalPOLRound,
|
||||
indent, prs.Prevotes,
|
||||
indent, prs.Precommits,
|
||||
indent, prs.LastCommit, prs.LastCommitRound,
|
||||
indent, prs.CatchupCommit, prs.CatchupCommitRound,
|
||||
indent)
|
||||
}
|
126
consensus/types/state.go
Normal file
126
consensus/types/state.go
Normal file
@ -0,0 +1,126 @@
|
||||
package types
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
// RoundStepType enum type
|
||||
|
||||
// RoundStepType enumerates the state of the consensus state machine
|
||||
type RoundStepType uint8 // These must be numeric, ordered.
|
||||
|
||||
const (
|
||||
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.
|
||||
)
|
||||
|
||||
// String returns a string
|
||||
func (rs RoundStepType) String() string {
|
||||
switch rs {
|
||||
case RoundStepNewHeight:
|
||||
return "RoundStepNewHeight"
|
||||
case RoundStepNewRound:
|
||||
return "RoundStepNewRound"
|
||||
case RoundStepPropose:
|
||||
return "RoundStepPropose"
|
||||
case RoundStepPrevote:
|
||||
return "RoundStepPrevote"
|
||||
case RoundStepPrevoteWait:
|
||||
return "RoundStepPrevoteWait"
|
||||
case RoundStepPrecommit:
|
||||
return "RoundStepPrecommit"
|
||||
case RoundStepPrecommitWait:
|
||||
return "RoundStepPrecommitWait"
|
||||
case RoundStepCommit:
|
||||
return "RoundStepCommit"
|
||||
default:
|
||||
return "RoundStepUnknown" // Cannot panic.
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// RoundState defines the internal consensus state.
|
||||
// It is Immutable when returned from ConsensusState.GetRoundState()
|
||||
// TODO: Actually, only the top pointer is copied,
|
||||
// so access to field pointers is still racey
|
||||
type RoundState struct {
|
||||
Height int // Height we are working on
|
||||
Round int
|
||||
Step RoundStepType
|
||||
StartTime time.Time
|
||||
CommitTime time.Time // Subjective time when +2/3 precommits for Block at Round were found
|
||||
Validators *types.ValidatorSet
|
||||
Proposal *types.Proposal
|
||||
ProposalBlock *types.Block
|
||||
ProposalBlockParts *types.PartSet
|
||||
LockedRound int
|
||||
LockedBlock *types.Block
|
||||
LockedBlockParts *types.PartSet
|
||||
Votes *HeightVoteSet
|
||||
CommitRound int //
|
||||
LastCommit *types.VoteSet // Last precommits at Height-1
|
||||
LastValidators *types.ValidatorSet
|
||||
}
|
||||
|
||||
// RoundStateEvent returns the H/R/S of the RoundState as an event.
|
||||
func (rs *RoundState) RoundStateEvent() types.EventDataRoundState {
|
||||
edrs := types.EventDataRoundState{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
Step: rs.Step.String(),
|
||||
RoundState: rs,
|
||||
}
|
||||
return edrs
|
||||
}
|
||||
|
||||
// String returns a string
|
||||
func (rs *RoundState) String() string {
|
||||
return rs.StringIndented("")
|
||||
}
|
||||
|
||||
// StringIndented returns a string
|
||||
func (rs *RoundState) StringIndented(indent string) string {
|
||||
return fmt.Sprintf(`RoundState{
|
||||
%s H:%v R:%v S:%v
|
||||
%s StartTime: %v
|
||||
%s CommitTime: %v
|
||||
%s Validators: %v
|
||||
%s Proposal: %v
|
||||
%s ProposalBlock: %v %v
|
||||
%s LockedRound: %v
|
||||
%s LockedBlock: %v %v
|
||||
%s Votes: %v
|
||||
%s LastCommit: %v
|
||||
%s LastValidators: %v
|
||||
%s}`,
|
||||
indent, rs.Height, rs.Round, rs.Step,
|
||||
indent, rs.StartTime,
|
||||
indent, rs.CommitTime,
|
||||
indent, rs.Validators.StringIndented(indent+" "),
|
||||
indent, rs.Proposal,
|
||||
indent, rs.ProposalBlockParts.StringShort(), rs.ProposalBlock.StringShort(),
|
||||
indent, rs.LockedRound,
|
||||
indent, rs.LockedBlockParts.StringShort(), rs.LockedBlock.StringShort(),
|
||||
indent, rs.Votes.StringIndented(indent+" "),
|
||||
indent, rs.LastCommit.StringShort(),
|
||||
indent, rs.LastValidators.StringIndented(indent+" "),
|
||||
indent)
|
||||
}
|
||||
|
||||
// StringShort returns a string
|
||||
func (rs *RoundState) StringShort() string {
|
||||
return fmt.Sprintf(`RoundState{H:%v R:%v S:%v ST:%v}`,
|
||||
rs.Height, rs.Round, rs.Step, rs.StartTime)
|
||||
}
|
@ -1,13 +1,10 @@
|
||||
package core
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
|
||||
cm "github.com/tendermint/tendermint/consensus"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
ctypes "github.com/tendermint/tendermint/rpc/core/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// Get the validator set at the given block height.
|
||||
@ -85,22 +82,12 @@ func Validators(heightPtr *int) (*ctypes.ResultValidators, error) {
|
||||
// }
|
||||
// ```
|
||||
func DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) {
|
||||
roundStateBytes, err := json.Marshal(consensusState.GetRoundState())
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to marshal round state")
|
||||
}
|
||||
|
||||
peerRoundStates := make(map[string]json.RawMessage)
|
||||
for i, peer := range p2pSwitch.Peers().List() {
|
||||
peerRoundStates := make(map[string]*cstypes.PeerRoundState)
|
||||
for _, peer := range p2pSwitch.Peers().List() {
|
||||
// TODO: clean this up?
|
||||
peerState := peer.Get(types.PeerStateKey).(*cm.PeerState)
|
||||
peerRoundState := peerState.GetRoundState()
|
||||
peerRoundStateBytes, err := json.Marshal(peerRoundState)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to marshal peer#%d round state", i)
|
||||
}
|
||||
peerRoundStates[peer.Key()] = json.RawMessage(peerRoundStateBytes)
|
||||
peerRoundStates[peer.Key()] = peerRoundState
|
||||
}
|
||||
|
||||
return &ctypes.ResultDumpConsensusState{json.RawMessage(roundStateBytes), peerRoundStates}, nil
|
||||
return &ctypes.ResultDumpConsensusState{consensusState.GetRoundState(), peerRoundStates}, nil
|
||||
}
|
||||
|
@ -3,6 +3,7 @@ package core
|
||||
import (
|
||||
crypto "github.com/tendermint/go-crypto"
|
||||
"github.com/tendermint/tendermint/consensus"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
p2p "github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
@ -17,7 +18,7 @@ import (
|
||||
type Consensus interface {
|
||||
GetState() *sm.State
|
||||
GetValidators() (int, []*types.Validator)
|
||||
GetRoundState() *consensus.RoundState
|
||||
GetRoundState() *cstypes.RoundState
|
||||
}
|
||||
|
||||
type P2P interface {
|
||||
|
@ -1,12 +1,12 @@
|
||||
package core_types
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"strings"
|
||||
|
||||
abci "github.com/tendermint/abci/types"
|
||||
crypto "github.com/tendermint/go-crypto"
|
||||
"github.com/tendermint/go-wire/data"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
@ -76,8 +76,8 @@ type ResultValidators struct {
|
||||
}
|
||||
|
||||
type ResultDumpConsensusState struct {
|
||||
RoundState json.RawMessage `json:"round_state"`
|
||||
PeerRoundStates map[string]json.RawMessage `json:"peer_round_states"`
|
||||
RoundState *cstypes.RoundState `json:"round_state"`
|
||||
PeerRoundStates map[string]*cstypes.PeerRoundState `json:"peer_round_states"`
|
||||
}
|
||||
|
||||
type ResultBroadcastTx struct {
|
||||
|
Loading…
x
Reference in New Issue
Block a user