mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-26 19:21:44 +00:00
uint64 height (Refs #911)
This commit is contained in:
committed by
Ethan Buchman
parent
b2489b4318
commit
b3492356e6
@ -48,12 +48,12 @@ func TestByzantine(t *testing.T) {
|
||||
if i == 0 {
|
||||
css[i].privValidator = NewByzantinePrivValidator(css[i].privValidator)
|
||||
// make byzantine
|
||||
css[i].decideProposal = func(j int) func(int, int) {
|
||||
return func(height, round int) {
|
||||
css[i].decideProposal = func(j int) func(uint64, int) {
|
||||
return func(height uint64, round int) {
|
||||
byzantineDecideProposalFunc(t, height, round, css[j], switches[j])
|
||||
}
|
||||
}(i)
|
||||
css[i].doPrevote = func(height, round int) {}
|
||||
css[i].doPrevote = func(height uint64, round int) {}
|
||||
}
|
||||
|
||||
eventBus := types.NewEventBus()
|
||||
@ -162,7 +162,7 @@ func TestByzantine(t *testing.T) {
|
||||
//-------------------------------
|
||||
// byzantine consensus functions
|
||||
|
||||
func byzantineDecideProposalFunc(t *testing.T, height, round int, cs *ConsensusState, sw *p2p.Switch) {
|
||||
func byzantineDecideProposalFunc(t *testing.T, height uint64, round int, cs *ConsensusState, sw *p2p.Switch) {
|
||||
// byzantine user should create two proposals and try to split the vote.
|
||||
// Avoid sending on internalMsgQueue and running consensus state.
|
||||
|
||||
@ -197,7 +197,7 @@ func byzantineDecideProposalFunc(t *testing.T, height, round int, cs *ConsensusS
|
||||
}
|
||||
}
|
||||
|
||||
func sendProposalAndParts(height, round int, cs *ConsensusState, peer p2p.Peer, proposal *types.Proposal, blockHash []byte, parts *types.PartSet) {
|
||||
func sendProposalAndParts(height uint64, round int, cs *ConsensusState, peer p2p.Peer, proposal *types.Proposal, blockHash []byte, parts *types.PartSet) {
|
||||
// proposal
|
||||
msg := &ProposalMessage{Proposal: proposal}
|
||||
peer.Send(DataChannel, struct{ ConsensusMessage }{msg})
|
||||
|
@ -54,7 +54,7 @@ func ResetConfig(name string) *cfg.Config {
|
||||
|
||||
type validatorStub struct {
|
||||
Index int // Validator index. NOTE: we don't assume validator set changes.
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
types.PrivValidator
|
||||
}
|
||||
@ -113,13 +113,13 @@ func incrementRound(vss ...*validatorStub) {
|
||||
//-------------------------------------------------------------------------------
|
||||
// Functions for transitioning the consensus state
|
||||
|
||||
func startTestRound(cs *ConsensusState, height, round int) {
|
||||
func startTestRound(cs *ConsensusState, height uint64, round int) {
|
||||
cs.enterNewRound(height, round)
|
||||
cs.startRoutines(0)
|
||||
}
|
||||
|
||||
// Create proposal block from cs1 but sign it with vs
|
||||
func decideProposal(cs1 *ConsensusState, vs *validatorStub, height, round int) (proposal *types.Proposal, block *types.Block) {
|
||||
func decideProposal(cs1 *ConsensusState, vs *validatorStub, height uint64, round int) (proposal *types.Proposal, block *types.Block) {
|
||||
block, blockParts := cs1.createProposalBlock()
|
||||
if block == nil { // on error
|
||||
panic("error creating proposal block")
|
||||
|
@ -861,7 +861,7 @@ func (ps *PeerState) GetRoundState() *cstypes.PeerRoundState {
|
||||
|
||||
// GetHeight returns an atomic snapshot of the PeerRoundState's height
|
||||
// used by the mempool to ensure peers are caught up before broadcasting new txs
|
||||
func (ps *PeerState) GetHeight() int {
|
||||
func (ps *PeerState) GetHeight() uint64 {
|
||||
ps.mtx.Lock()
|
||||
defer ps.mtx.Unlock()
|
||||
return ps.PeerRoundState.Height
|
||||
@ -900,7 +900,7 @@ func (ps *PeerState) InitProposalBlockParts(partsHeader types.PartSetHeader) {
|
||||
}
|
||||
|
||||
// SetHasProposalBlockPart sets the given block part index as known for the peer.
|
||||
func (ps *PeerState) SetHasProposalBlockPart(height int, round int, index int) {
|
||||
func (ps *PeerState) SetHasProposalBlockPart(height uint64, round int, index int) {
|
||||
ps.mtx.Lock()
|
||||
defer ps.mtx.Unlock()
|
||||
|
||||
@ -951,7 +951,7 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote
|
||||
return nil, false
|
||||
}
|
||||
|
||||
func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *cmn.BitArray {
|
||||
func (ps *PeerState) getVoteBitArray(height uint64, round int, type_ byte) *cmn.BitArray {
|
||||
if !types.IsVoteTypeValid(type_) {
|
||||
return nil
|
||||
}
|
||||
@ -998,7 +998,7 @@ func (ps *PeerState) getVoteBitArray(height, round int, type_ byte) *cmn.BitArra
|
||||
}
|
||||
|
||||
// 'round': A round for which we have a +2/3 commit.
|
||||
func (ps *PeerState) ensureCatchupCommitRound(height, round int, numValidators int) {
|
||||
func (ps *PeerState) ensureCatchupCommitRound(height uint64, round int, numValidators int) {
|
||||
if ps.Height != height {
|
||||
return
|
||||
}
|
||||
@ -1024,13 +1024,13 @@ func (ps *PeerState) ensureCatchupCommitRound(height, round int, numValidators i
|
||||
// what votes this peer has received.
|
||||
// NOTE: It's important to make sure that numValidators actually matches
|
||||
// what the node sees as the number of validators for height.
|
||||
func (ps *PeerState) EnsureVoteBitArrays(height int, numValidators int) {
|
||||
func (ps *PeerState) EnsureVoteBitArrays(height uint64, numValidators int) {
|
||||
ps.mtx.Lock()
|
||||
defer ps.mtx.Unlock()
|
||||
ps.ensureVoteBitArrays(height, numValidators)
|
||||
}
|
||||
|
||||
func (ps *PeerState) ensureVoteBitArrays(height int, numValidators int) {
|
||||
func (ps *PeerState) ensureVoteBitArrays(height uint64, numValidators int) {
|
||||
if ps.Height == height {
|
||||
if ps.Prevotes == nil {
|
||||
ps.Prevotes = cmn.NewBitArray(numValidators)
|
||||
@ -1059,7 +1059,7 @@ func (ps *PeerState) SetHasVote(vote *types.Vote) {
|
||||
ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
|
||||
}
|
||||
|
||||
func (ps *PeerState) setHasVote(height int, round int, type_ byte, index int) {
|
||||
func (ps *PeerState) setHasVote(height uint64, round int, type_ byte, index int) {
|
||||
logger := ps.logger.With("peerH/R", cmn.Fmt("%d/%d", ps.Height, ps.Round), "H/R", cmn.Fmt("%d/%d", height, round))
|
||||
logger.Debug("setHasVote", "type", type_, "index", index)
|
||||
|
||||
@ -1253,7 +1253,7 @@ func DecodeMessage(bz []byte) (msgType byte, msg ConsensusMessage, err error) {
|
||||
// NewRoundStepMessage is sent for every step taken in the ConsensusState.
|
||||
// For every height/round/step transition
|
||||
type NewRoundStepMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
Step cstypes.RoundStepType
|
||||
SecondsSinceStartTime int
|
||||
@ -1270,7 +1270,7 @@ func (m *NewRoundStepMessage) String() string {
|
||||
|
||||
// CommitStepMessage is sent when a block is committed.
|
||||
type CommitStepMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
BlockPartsHeader types.PartSetHeader
|
||||
BlockParts *cmn.BitArray
|
||||
}
|
||||
@ -1296,7 +1296,7 @@ func (m *ProposalMessage) String() string {
|
||||
|
||||
// ProposalPOLMessage is sent when a previous proposal is re-proposed.
|
||||
type ProposalPOLMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
ProposalPOLRound int
|
||||
ProposalPOL *cmn.BitArray
|
||||
}
|
||||
@ -1310,7 +1310,7 @@ func (m *ProposalPOLMessage) String() string {
|
||||
|
||||
// BlockPartMessage is sent when gossipping a piece of the proposed block.
|
||||
type BlockPartMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
Part *types.Part
|
||||
}
|
||||
@ -1336,7 +1336,7 @@ func (m *VoteMessage) String() string {
|
||||
|
||||
// HasVoteMessage is sent to indicate that a particular vote has been received.
|
||||
type HasVoteMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
Type byte
|
||||
Index int
|
||||
@ -1351,7 +1351,7 @@ func (m *HasVoteMessage) String() string {
|
||||
|
||||
// VoteSetMaj23Message is sent to indicate that a given BlockID has seen +2/3 votes.
|
||||
type VoteSetMaj23Message struct {
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
Type byte
|
||||
BlockID types.BlockID
|
||||
@ -1366,7 +1366,7 @@ func (m *VoteSetMaj23Message) String() string {
|
||||
|
||||
// VoteSetBitsMessage is sent to communicate the bit-array of votes seen for the BlockID.
|
||||
type VoteSetBitsMessage struct {
|
||||
Height int
|
||||
Height uint64
|
||||
Round int
|
||||
Type byte
|
||||
BlockID types.BlockID
|
||||
|
@ -90,7 +90,7 @@ func (cs *ConsensusState) readReplayMessage(msg *TimedWALMessage, newStepCh chan
|
||||
|
||||
// replay only those messages since the last block.
|
||||
// timeoutRoutine should run concurrently to read off tickChan
|
||||
func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
func (cs *ConsensusState) catchupReplay(csHeight uint64) error {
|
||||
// set replayMode
|
||||
cs.replayMode = true
|
||||
defer func() { cs.replayMode = false }()
|
||||
@ -98,7 +98,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
// Ensure that ENDHEIGHT for this height doesn't exist
|
||||
// NOTE: This is just a sanity check. As far as we know things work fine without it,
|
||||
// and Handshake could reuse ConsensusState if it weren't for this check (since we can crash after writing ENDHEIGHT).
|
||||
gr, found, err := cs.wal.SearchForEndHeight(uint64(csHeight))
|
||||
gr, found, err := cs.wal.SearchForEndHeight(csHeight)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -112,7 +112,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
}
|
||||
|
||||
// Search for last height marker
|
||||
gr, found, err = cs.wal.SearchForEndHeight(uint64(csHeight - 1))
|
||||
gr, found, err = cs.wal.SearchForEndHeight(csHeight - 1)
|
||||
if err == io.EOF {
|
||||
cs.Logger.Error("Replay: wal.group.Search returned EOF", "#ENDHEIGHT", csHeight-1)
|
||||
} else if err != nil {
|
||||
@ -151,7 +151,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
// Parses marker lines of the form:
|
||||
// #ENDHEIGHT: 12345
|
||||
/*
|
||||
func makeHeightSearchFunc(height int) auto.SearchFunc {
|
||||
func makeHeightSearchFunc(height uint64) auto.SearchFunc {
|
||||
return func(line string) (int, error) {
|
||||
line = strings.TrimRight(line, "\n")
|
||||
parts := strings.Split(line, " ")
|
||||
@ -205,7 +205,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
|
||||
return errors.New(cmn.Fmt("Error calling Info: %v", err))
|
||||
}
|
||||
|
||||
blockHeight := int(res.LastBlockHeight) // XXX: beware overflow
|
||||
blockHeight := res.LastBlockHeight
|
||||
appHash := res.LastBlockAppHash
|
||||
|
||||
h.logger.Info("ABCI Handshake", "appHeight", blockHeight, "appHash", fmt.Sprintf("%X", appHash))
|
||||
@ -227,7 +227,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
|
||||
|
||||
// Replay all blocks since appBlockHeight and ensure the result matches the current state.
|
||||
// Returns the final AppHash or an error
|
||||
func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) {
|
||||
func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight uint64, proxyApp proxy.AppConns) ([]byte, error) {
|
||||
|
||||
storeBlockHeight := h.store.Height()
|
||||
stateBlockHeight := h.state.LastBlockHeight
|
||||
@ -302,7 +302,7 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, mutateState bool) ([]byte, error) {
|
||||
func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight uint64, mutateState bool) ([]byte, error) {
|
||||
// App is further behind than it should be, so we need to replay blocks.
|
||||
// We replay all blocks from appBlockHeight+1.
|
||||
//
|
||||
@ -338,7 +338,7 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store
|
||||
}
|
||||
|
||||
// ApplyBlock on the proxyApp with the last block.
|
||||
func (h *Handshaker) replayBlock(height int, proxyApp proxy.AppConnConsensus) ([]byte, error) {
|
||||
func (h *Handshaker) replayBlock(height uint64, proxyApp proxy.AppConnConsensus) ([]byte, error) {
|
||||
mempool := types.MockMempool{}
|
||||
|
||||
block := h.store.LoadBlock(height)
|
||||
|
@ -58,7 +58,7 @@ var data_dir = path.Join(cmn.GoPath(), "src/github.com/tendermint/tendermint/con
|
||||
// and which ones we need the wal for - then we'd also be able to only flush the
|
||||
// wal writer when we need to, instead of with every message.
|
||||
|
||||
func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight int, blockDB dbm.DB, stateDB dbm.DB) {
|
||||
func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight uint64, blockDB dbm.DB, stateDB dbm.DB) {
|
||||
logger := log.TestingLogger()
|
||||
state, _ := sm.GetState(stateDB, consensusReplayConfig.GenesisFile())
|
||||
state.SetLogger(logger.With("module", "state"))
|
||||
@ -590,21 +590,21 @@ func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBl
|
||||
return &mockBlockStore{config, params, nil, nil}
|
||||
}
|
||||
|
||||
func (bs *mockBlockStore) Height() int { return len(bs.chain) }
|
||||
func (bs *mockBlockStore) LoadBlock(height int) *types.Block { return bs.chain[height-1] }
|
||||
func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta {
|
||||
func (bs *mockBlockStore) Height() uint64 { return uint64(len(bs.chain)) }
|
||||
func (bs *mockBlockStore) LoadBlock(height uint64) *types.Block { return bs.chain[height-1] }
|
||||
func (bs *mockBlockStore) LoadBlockMeta(height uint64) *types.BlockMeta {
|
||||
block := bs.chain[height-1]
|
||||
return &types.BlockMeta{
|
||||
BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.params.BlockPartSizeBytes).Header()},
|
||||
Header: block.Header,
|
||||
}
|
||||
}
|
||||
func (bs *mockBlockStore) LoadBlockPart(height int, index int) *types.Part { return nil }
|
||||
func (bs *mockBlockStore) LoadBlockPart(height uint64, index int) *types.Part { return nil }
|
||||
func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) {
|
||||
}
|
||||
func (bs *mockBlockStore) LoadBlockCommit(height int) *types.Commit {
|
||||
func (bs *mockBlockStore) LoadBlockCommit(height uint64) *types.Commit {
|
||||
return bs.commits[height-1]
|
||||
}
|
||||
func (bs *mockBlockStore) LoadSeenCommit(height int) *types.Commit {
|
||||
func (bs *mockBlockStore) LoadSeenCommit(height uint64) *types.Commit {
|
||||
return bs.commits[height-1]
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ type msgInfo struct {
|
||||
// internally generated messages which may update the state
|
||||
type timeoutInfo struct {
|
||||
Duration time.Duration `json:"duration"`
|
||||
Height int `json:"height"`
|
||||
Height uint64 `json:"height"`
|
||||
Round int `json:"round"`
|
||||
Step cstypes.RoundStepType `json:"step"`
|
||||
}
|
||||
@ -104,8 +104,8 @@ type ConsensusState struct {
|
||||
nSteps int
|
||||
|
||||
// some functions can be overwritten for testing
|
||||
decideProposal func(height, round int)
|
||||
doPrevote func(height, round int)
|
||||
decideProposal func(height uint64, round int)
|
||||
doPrevote func(height uint64, round int)
|
||||
setProposal func(proposal *types.Proposal) error
|
||||
|
||||
// closed when we finish shutting down
|
||||
@ -179,7 +179,7 @@ func (cs *ConsensusState) getRoundState() *cstypes.RoundState {
|
||||
}
|
||||
|
||||
// GetValidators returns a copy of the current validators.
|
||||
func (cs *ConsensusState) GetValidators() (int, []*types.Validator) {
|
||||
func (cs *ConsensusState) GetValidators() (uint64, []*types.Validator) {
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
return cs.state.LastBlockHeight, cs.state.Validators.Copy().Validators
|
||||
@ -200,7 +200,7 @@ func (cs *ConsensusState) SetTimeoutTicker(timeoutTicker TimeoutTicker) {
|
||||
}
|
||||
|
||||
// LoadCommit loads the commit for a given height.
|
||||
func (cs *ConsensusState) LoadCommit(height int) *types.Commit {
|
||||
func (cs *ConsensusState) LoadCommit(height uint64) *types.Commit {
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
if height == cs.blockStore.Height() {
|
||||
@ -331,7 +331,7 @@ func (cs *ConsensusState) SetProposal(proposal *types.Proposal, peerKey string)
|
||||
}
|
||||
|
||||
// AddProposalBlockPart inputs a part of the proposal block.
|
||||
func (cs *ConsensusState) AddProposalBlockPart(height, round int, part *types.Part, peerKey string) error {
|
||||
func (cs *ConsensusState) AddProposalBlockPart(height uint64, round int, part *types.Part, peerKey string) error {
|
||||
|
||||
if peerKey == "" {
|
||||
cs.internalMsgQueue <- msgInfo{&BlockPartMessage{height, round, part}, ""}
|
||||
@ -360,7 +360,7 @@ func (cs *ConsensusState) SetProposalAndBlock(proposal *types.Proposal, block *t
|
||||
//------------------------------------------------------------
|
||||
// internal functions for managing the state
|
||||
|
||||
func (cs *ConsensusState) updateHeight(height int) {
|
||||
func (cs *ConsensusState) updateHeight(height uint64) {
|
||||
cs.Height = height
|
||||
}
|
||||
|
||||
@ -377,7 +377,7 @@ func (cs *ConsensusState) scheduleRound0(rs *cstypes.RoundState) {
|
||||
}
|
||||
|
||||
// Attempt to schedule a timeout (by sending timeoutInfo on the tickChan)
|
||||
func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height, round int, step cstypes.RoundStepType) {
|
||||
func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height uint64, round int, step cstypes.RoundStepType) {
|
||||
cs.timeoutTicker.ScheduleTimeout(timeoutInfo{duration, height, round, step})
|
||||
}
|
||||
|
||||
@ -627,7 +627,7 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs cstypes.RoundState) {
|
||||
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) handleTxsAvailable(height int) {
|
||||
func (cs *ConsensusState) handleTxsAvailable(height uint64) {
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
// we only need to do this for round 0
|
||||
@ -644,7 +644,7 @@ func (cs *ConsensusState) handleTxsAvailable(height int) {
|
||||
// Enter: +2/3 precommits for nil at (height,round-1)
|
||||
// Enter: +2/3 prevotes any or +2/3 precommits for block or any from (height, round)
|
||||
// NOTE: cs.StartTime was already set for height.
|
||||
func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
func (cs *ConsensusState) enterNewRound(height uint64, round int) {
|
||||
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
|
||||
@ -697,7 +697,7 @@ func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
|
||||
// needProofBlock returns true on the first height (so the genesis app hash is signed right away)
|
||||
// and where the last block (height-1) caused the app hash to change
|
||||
func (cs *ConsensusState) needProofBlock(height int) bool {
|
||||
func (cs *ConsensusState) needProofBlock(height uint64) bool {
|
||||
if height == 1 {
|
||||
return true
|
||||
}
|
||||
@ -706,7 +706,7 @@ func (cs *ConsensusState) needProofBlock(height int) bool {
|
||||
return !bytes.Equal(cs.state.AppHash, lastBlockMeta.Header.AppHash)
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) proposalHeartbeat(height, round int) {
|
||||
func (cs *ConsensusState) proposalHeartbeat(height uint64, round int) {
|
||||
counter := 0
|
||||
addr := cs.privValidator.GetAddress()
|
||||
valIndex, v := cs.Validators.GetByAddress(addr)
|
||||
@ -738,7 +738,7 @@ func (cs *ConsensusState) proposalHeartbeat(height, round int) {
|
||||
// Enter (CreateEmptyBlocks): from enterNewRound(height,round)
|
||||
// Enter (CreateEmptyBlocks, CreateEmptyBlocksInterval > 0 ): after enterNewRound(height,round), after timeout of CreateEmptyBlocksInterval
|
||||
// Enter (!CreateEmptyBlocks) : after enterNewRound(height,round), once txs are in the mempool
|
||||
func (cs *ConsensusState) enterPropose(height int, round int) {
|
||||
func (cs *ConsensusState) enterPropose(height uint64, round int) {
|
||||
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
|
||||
@ -785,7 +785,7 @@ func (cs *ConsensusState) isProposer() bool {
|
||||
return bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress())
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) defaultDecideProposal(height, round int) {
|
||||
func (cs *ConsensusState) defaultDecideProposal(height uint64, round int) {
|
||||
var block *types.Block
|
||||
var blockParts *types.PartSet
|
||||
|
||||
@ -873,7 +873,7 @@ 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 uint64, round int) {
|
||||
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
|
||||
@ -902,7 +902,7 @@ func (cs *ConsensusState) enterPrevote(height int, round int) {
|
||||
// (so we have more time to try and collect +2/3 prevotes for a single block)
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
|
||||
func (cs *ConsensusState) defaultDoPrevote(height uint64, round int) {
|
||||
logger := cs.Logger.With("height", height, "round", round)
|
||||
// If a block is locked, prevote that.
|
||||
if cs.LockedBlock != nil {
|
||||
@ -935,7 +935,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) {
|
||||
func (cs *ConsensusState) enterPrevoteWait(height uint64, round int) {
|
||||
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
|
||||
@ -961,7 +961,7 @@ 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 uint64, round int) {
|
||||
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
|
||||
@ -1054,7 +1054,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) {
|
||||
func (cs *ConsensusState) enterPrecommitWait(height uint64, round int) {
|
||||
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
|
||||
@ -1076,7 +1076,7 @@ func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
|
||||
}
|
||||
|
||||
// Enter: +2/3 precommits for block
|
||||
func (cs *ConsensusState) enterCommit(height int, commitRound int) {
|
||||
func (cs *ConsensusState) enterCommit(height uint64, commitRound int) {
|
||||
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
|
||||
@ -1122,7 +1122,7 @@ func (cs *ConsensusState) enterCommit(height int, commitRound int) {
|
||||
}
|
||||
|
||||
// If we have the block AND +2/3 commits for it, finalize.
|
||||
func (cs *ConsensusState) tryFinalizeCommit(height int) {
|
||||
func (cs *ConsensusState) tryFinalizeCommit(height uint64) {
|
||||
if cs.Height != height {
|
||||
cmn.PanicSanity(cmn.Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height))
|
||||
}
|
||||
@ -1144,7 +1144,7 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
|
||||
}
|
||||
|
||||
// Increment height and goto cstypes.RoundStepNewHeight
|
||||
func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
func (cs *ConsensusState) finalizeCommit(height uint64) {
|
||||
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
|
||||
@ -1193,7 +1193,7 @@ func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
// WAL replay for blocks with an #ENDHEIGHT
|
||||
// As is, ConsensusState should not be started again
|
||||
// until we successfully call ApplyBlock (ie. here or in Handshake after restart)
|
||||
cs.wal.Save(EndHeightMessage{uint64(height)})
|
||||
cs.wal.Save(EndHeightMessage{height})
|
||||
|
||||
fail.Fail() // XXX
|
||||
|
||||
@ -1285,7 +1285,7 @@ func (cs *ConsensusState) defaultSetProposal(proposal *types.Proposal) error {
|
||||
|
||||
// NOTE: block is not necessarily valid.
|
||||
// Asynchronously triggers either enterPrevote (before we timeout of propose) or tryFinalizeCommit, once we have the full block.
|
||||
func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, verify bool) (added bool, err error) {
|
||||
func (cs *ConsensusState) addProposalBlockPart(height uint64, part *types.Part, verify bool) (added bool, err error) {
|
||||
// Blocks might be reused, so round mismatch is OK
|
||||
if cs.Height != height {
|
||||
return false, nil
|
||||
@ -1494,7 +1494,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
|
||||
|
||||
//---------------------------------------------------------
|
||||
|
||||
func CompareHRS(h1, r1 int, s1 cstypes.RoundStepType, h2, r2 int, s2 cstypes.RoundStepType) int {
|
||||
func CompareHRS(h1 uint64, r1 int, s1 cstypes.RoundStepType, h2 uint64, r2 int, s2 cstypes.RoundStepType) int {
|
||||
if h1 < h2 {
|
||||
return -1
|
||||
} else if h1 > h2 {
|
||||
|
Binary file not shown.
@ -29,7 +29,7 @@ One for their LastCommit round, and another for the official commit round.
|
||||
*/
|
||||
type HeightVoteSet struct {
|
||||
chainID string
|
||||
height int
|
||||
height uint64
|
||||
valSet *types.ValidatorSet
|
||||
|
||||
mtx sync.Mutex
|
||||
@ -38,7 +38,7 @@ type HeightVoteSet struct {
|
||||
peerCatchupRounds map[string][]int // keys: peer.Key; values: at most 2 rounds
|
||||
}
|
||||
|
||||
func NewHeightVoteSet(chainID string, height int, valSet *types.ValidatorSet) *HeightVoteSet {
|
||||
func NewHeightVoteSet(chainID string, height uint64, valSet *types.ValidatorSet) *HeightVoteSet {
|
||||
hvs := &HeightVoteSet{
|
||||
chainID: chainID,
|
||||
}
|
||||
@ -46,7 +46,7 @@ func NewHeightVoteSet(chainID string, height int, valSet *types.ValidatorSet) *H
|
||||
return hvs
|
||||
}
|
||||
|
||||
func (hvs *HeightVoteSet) Reset(height int, valSet *types.ValidatorSet) {
|
||||
func (hvs *HeightVoteSet) Reset(height uint64, valSet *types.ValidatorSet) {
|
||||
hvs.mtx.Lock()
|
||||
defer hvs.mtx.Unlock()
|
||||
|
||||
@ -59,7 +59,7 @@ func (hvs *HeightVoteSet) Reset(height int, valSet *types.ValidatorSet) {
|
||||
hvs.round = 0
|
||||
}
|
||||
|
||||
func (hvs *HeightVoteSet) Height() int {
|
||||
func (hvs *HeightVoteSet) Height() uint64 {
|
||||
hvs.mtx.Lock()
|
||||
defer hvs.mtx.Unlock()
|
||||
return hvs.height
|
||||
|
@ -47,7 +47,7 @@ func TestPeerCatchupRounds(t *testing.T) {
|
||||
|
||||
}
|
||||
|
||||
func makeVoteHR(t *testing.T, height, round int, privVals []*types.PrivValidatorFS, valIndex int) *types.Vote {
|
||||
func makeVoteHR(t *testing.T, height uint64, round int, privVals []*types.PrivValidatorFS, valIndex int) *types.Vote {
|
||||
privVal := privVals[valIndex]
|
||||
vote := &types.Vote{
|
||||
ValidatorAddress: privVal.GetAddress(),
|
||||
|
@ -13,7 +13,7 @@ import (
|
||||
// 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
|
||||
Height uint64 // 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
|
||||
|
@ -58,7 +58,7 @@ func (rs RoundStepType) String() string {
|
||||
// NOTE: Not thread safe. Should only be manipulated by functions downstream
|
||||
// of the cs.receiveRoutine
|
||||
type RoundState struct {
|
||||
Height int // Height we are working on
|
||||
Height uint64 // Height we are working on
|
||||
Round int
|
||||
Step RoundStepType
|
||||
StartTime time.Time
|
||||
|
Reference in New Issue
Block a user