diff --git a/blockchain/pool.go b/blockchain/pool.go index 8b932531b..e39749dc0 100644 --- a/blockchain/pool.go +++ b/blockchain/pool.go @@ -52,22 +52,22 @@ type BlockPool struct { mtx sync.Mutex // block requests - requesters map[uint64]*bpRequester - height uint64 // the lowest key in requesters. - numPending int32 // number of requests pending assignment or block response + requesters map[int64]*bpRequester + height int64 // the lowest key in requesters. + numPending int32 // number of requests pending assignment or block response // peers peers map[string]*bpPeer - maxPeerHeight uint64 + maxPeerHeight int64 requestsCh chan<- BlockRequest timeoutsCh chan<- string } -func NewBlockPool(start uint64, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool { +func NewBlockPool(start int64, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool { bp := &BlockPool{ peers: make(map[string]*bpPeer), - requesters: make(map[uint64]*bpRequester), + requesters: make(map[int64]*bpRequester), height: start, numPending: 0, @@ -132,7 +132,7 @@ func (pool *BlockPool) removeTimedoutPeers() { } } -func (pool *BlockPool) GetStatus() (height uint64, numPending int32, lenRequesters int) { +func (pool *BlockPool) GetStatus() (height int64, numPending int32, lenRequesters int) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -195,7 +195,7 @@ func (pool *BlockPool) PopRequest() { // Invalidates the block at pool.height, // Remove the peer and redo request from others. -func (pool *BlockPool) RedoRequest(height uint64) { +func (pool *BlockPool) RedoRequest(height int64) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -233,14 +233,14 @@ func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int } // MaxPeerHeight returns the highest height reported by a peer. -func (pool *BlockPool) MaxPeerHeight() uint64 { +func (pool *BlockPool) MaxPeerHeight() int64 { pool.mtx.Lock() defer pool.mtx.Unlock() return pool.maxPeerHeight } // Sets the peer's alleged blockchain height. -func (pool *BlockPool) SetPeerHeight(peerID string, height uint64) { +func (pool *BlockPool) SetPeerHeight(peerID string, height int64) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -279,7 +279,7 @@ func (pool *BlockPool) removePeer(peerID string) { // Pick an available peer with at least the given minHeight. // If no peers are available, returns nil. -func (pool *BlockPool) pickIncrAvailablePeer(minHeight uint64) *bpPeer { +func (pool *BlockPool) pickIncrAvailablePeer(minHeight int64) *bpPeer { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -317,11 +317,11 @@ func (pool *BlockPool) makeNextRequester() { } } -func (pool *BlockPool) requestersLen() uint64 { - return uint64(len(pool.requesters)) +func (pool *BlockPool) requestersLen() int64 { + return int64(len(pool.requesters)) } -func (pool *BlockPool) sendRequest(height uint64, peerID string) { +func (pool *BlockPool) sendRequest(height int64, peerID string) { if !pool.IsRunning() { return } @@ -360,7 +360,7 @@ type bpPeer struct { id string recvMonitor *flow.Monitor - height uint64 + height int64 numPending int32 timeout *time.Timer didTimeout bool @@ -368,7 +368,7 @@ type bpPeer struct { logger log.Logger } -func newBPPeer(pool *BlockPool, peerID string, height uint64) *bpPeer { +func newBPPeer(pool *BlockPool, peerID string, height int64) *bpPeer { peer := &bpPeer{ pool: pool, id: peerID, @@ -429,7 +429,7 @@ func (peer *bpPeer) onTimeout() { type bpRequester struct { cmn.BaseService pool *BlockPool - height uint64 + height int64 gotBlockCh chan struct{} redoCh chan struct{} @@ -438,7 +438,7 @@ type bpRequester struct { block *types.Block } -func newBPRequester(pool *BlockPool, height uint64) *bpRequester { +func newBPRequester(pool *BlockPool, height int64) *bpRequester { bpr := &bpRequester{ pool: pool, height: height, @@ -550,6 +550,6 @@ OUTER_LOOP: //------------------------------------- type BlockRequest struct { - Height uint64 + Height int64 PeerID string } diff --git a/blockchain/pool_test.go b/blockchain/pool_test.go index 6f9a43b18..0856a3714 100644 --- a/blockchain/pool_test.go +++ b/blockchain/pool_test.go @@ -16,21 +16,21 @@ func init() { type testPeer struct { id string - height uint64 + height int64 } -func makePeers(numPeers int, minHeight, maxHeight uint64) map[string]testPeer { +func makePeers(numPeers int, minHeight, maxHeight int64) map[string]testPeer { peers := make(map[string]testPeer, numPeers) for i := 0; i < numPeers; i++ { peerID := cmn.RandStr(12) - height := minHeight + uint64(rand.Intn(int(maxHeight-minHeight))) + height := minHeight + int64(rand.Intn(int(maxHeight-minHeight))) peers[peerID] = testPeer{peerID, height} } return peers } func TestBasic(t *testing.T) { - start := uint64(42) + start := int64(42) peers := makePeers(10, start+1, 1000) timeoutsCh := make(chan string, 100) requestsCh := make(chan BlockRequest, 100) @@ -87,7 +87,7 @@ func TestBasic(t *testing.T) { } func TestTimeout(t *testing.T) { - start := uint64(42) + start := int64(42) peers := makePeers(10, start+1, 1000) timeoutsCh := make(chan string, 100) requestsCh := make(chan BlockRequest, 100) diff --git a/blockchain/reactor.go b/blockchain/reactor.go index 9f3bcb82a..60626b3d8 100644 --- a/blockchain/reactor.go +++ b/blockchain/reactor.go @@ -344,7 +344,7 @@ func DecodeMessage(bz []byte, maxSize int) (msgType byte, msg BlockchainMessage, //------------------------------------- type bcBlockRequestMessage struct { - Height uint64 + Height int64 } func (m *bcBlockRequestMessage) String() string { @@ -352,7 +352,7 @@ func (m *bcBlockRequestMessage) String() string { } type bcNoBlockResponseMessage struct { - Height uint64 + Height int64 } func (brm *bcNoBlockResponseMessage) String() string { @@ -373,7 +373,7 @@ func (m *bcBlockResponseMessage) String() string { //------------------------------------- type bcStatusRequestMessage struct { - Height uint64 + Height int64 } func (m *bcStatusRequestMessage) String() string { @@ -383,7 +383,7 @@ func (m *bcStatusRequestMessage) String() string { //------------------------------------- type bcStatusResponseMessage struct { - Height uint64 + Height int64 } func (m *bcStatusResponseMessage) String() string { diff --git a/blockchain/reactor_test.go b/blockchain/reactor_test.go index d4ada4f7a..7342b72c0 100644 --- a/blockchain/reactor_test.go +++ b/blockchain/reactor_test.go @@ -14,7 +14,7 @@ import ( "github.com/tendermint/tendermint/types" ) -func newBlockchainReactor(maxBlockHeight uint64) *BlockchainReactor { +func newBlockchainReactor(maxBlockHeight int64) *BlockchainReactor { logger := log.TestingLogger() config := cfg.ResetTestRoot("blockchain_reactor_test") @@ -34,7 +34,7 @@ func newBlockchainReactor(maxBlockHeight uint64) *BlockchainReactor { bcReactor.Switch = p2p.NewSwitch(cfg.DefaultP2PConfig()) // Lastly: let's add some blocks in - for blockHeight := uint64(1); blockHeight <= maxBlockHeight; blockHeight++ { + for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ { firstBlock := makeBlock(blockHeight, state) secondBlock := makeBlock(blockHeight+1, state) firstParts := firstBlock.MakePartSet(state.Params.BlockGossipParams.BlockPartSizeBytes) @@ -45,7 +45,7 @@ func newBlockchainReactor(maxBlockHeight uint64) *BlockchainReactor { } func TestNoBlockMessageResponse(t *testing.T) { - maxBlockHeight := uint64(20) + maxBlockHeight := int64(20) bcr := newBlockchainReactor(maxBlockHeight) bcr.Start() @@ -58,7 +58,7 @@ func TestNoBlockMessageResponse(t *testing.T) { chID := byte(0x01) tests := []struct { - height uint64 + height int64 existent bool }{ {maxBlockHeight + 2, false}, @@ -93,14 +93,14 @@ func TestNoBlockMessageResponse(t *testing.T) { //---------------------------------------------- // utility funcs -func makeTxs(height uint64) (txs []types.Tx) { +func makeTxs(height int64) (txs []types.Tx) { for i := 0; i < 10; i++ { txs = append(txs, types.Tx([]byte{byte(height), byte(i)})) } return txs } -func makeBlock(height uint64, state *sm.State) *types.Block { +func makeBlock(height int64, state *sm.State) *types.Block { prevHash := state.LastBlockID.Hash prevParts := types.PartSetHeader{} valHash := state.Validators.Hash() diff --git a/blockchain/store.go b/blockchain/store.go index 8ab16748a..c77f67ed8 100644 --- a/blockchain/store.go +++ b/blockchain/store.go @@ -32,7 +32,7 @@ type BlockStore struct { db dbm.DB mtx sync.RWMutex - height uint64 + height int64 } func NewBlockStore(db dbm.DB) *BlockStore { @@ -44,7 +44,7 @@ func NewBlockStore(db dbm.DB) *BlockStore { } // Height() returns the last known contiguous block height. -func (bs *BlockStore) Height() uint64 { +func (bs *BlockStore) Height() int64 { bs.mtx.RLock() defer bs.mtx.RUnlock() return bs.height @@ -58,7 +58,7 @@ func (bs *BlockStore) GetReader(key []byte) io.Reader { return bytes.NewReader(bytez) } -func (bs *BlockStore) LoadBlock(height uint64) *types.Block { +func (bs *BlockStore) LoadBlock(height int64) *types.Block { var n int var err error r := bs.GetReader(calcBlockMetaKey(height)) @@ -81,7 +81,7 @@ func (bs *BlockStore) LoadBlock(height uint64) *types.Block { return block } -func (bs *BlockStore) LoadBlockPart(height uint64, index int) *types.Part { +func (bs *BlockStore) LoadBlockPart(height int64, index int) *types.Part { var n int var err error r := bs.GetReader(calcBlockPartKey(height, index)) @@ -95,7 +95,7 @@ func (bs *BlockStore) LoadBlockPart(height uint64, index int) *types.Part { return part } -func (bs *BlockStore) LoadBlockMeta(height uint64) *types.BlockMeta { +func (bs *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta { var n int var err error r := bs.GetReader(calcBlockMetaKey(height)) @@ -111,7 +111,7 @@ func (bs *BlockStore) LoadBlockMeta(height uint64) *types.BlockMeta { // The +2/3 and other Precommit-votes for block at `height`. // This Commit comes from block.LastCommit for `height+1`. -func (bs *BlockStore) LoadBlockCommit(height uint64) *types.Commit { +func (bs *BlockStore) LoadBlockCommit(height int64) *types.Commit { var n int var err error r := bs.GetReader(calcBlockCommitKey(height)) @@ -126,7 +126,7 @@ func (bs *BlockStore) LoadBlockCommit(height uint64) *types.Commit { } // NOTE: the Precommit-vote heights are for the block at `height` -func (bs *BlockStore) LoadSeenCommit(height uint64) *types.Commit { +func (bs *BlockStore) LoadSeenCommit(height int64) *types.Commit { var n int var err error r := bs.GetReader(calcSeenCommitKey(height)) @@ -185,7 +185,7 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s bs.db.SetSync(nil, nil) } -func (bs *BlockStore) saveBlockPart(height uint64, index int, part *types.Part) { +func (bs *BlockStore) saveBlockPart(height int64, index int, part *types.Part) { if height != bs.Height()+1 { cmn.PanicSanity(cmn.Fmt("BlockStore can only save contiguous blocks. Wanted %v, got %v", bs.Height()+1, height)) } @@ -195,19 +195,19 @@ func (bs *BlockStore) saveBlockPart(height uint64, index int, part *types.Part) //----------------------------------------------------------------------------- -func calcBlockMetaKey(height uint64) []byte { +func calcBlockMetaKey(height int64) []byte { return []byte(fmt.Sprintf("H:%v", height)) } -func calcBlockPartKey(height uint64, partIndex int) []byte { +func calcBlockPartKey(height int64, partIndex int) []byte { return []byte(fmt.Sprintf("P:%v:%v", height, partIndex)) } -func calcBlockCommitKey(height uint64) []byte { +func calcBlockCommitKey(height int64) []byte { return []byte(fmt.Sprintf("C:%v", height)) } -func calcSeenCommitKey(height uint64) []byte { +func calcSeenCommitKey(height int64) []byte { return []byte(fmt.Sprintf("SC:%v", height)) } @@ -216,7 +216,7 @@ func calcSeenCommitKey(height uint64) []byte { var blockStoreKey = []byte("blockStore") type BlockStoreStateJSON struct { - Height uint64 + Height int64 } func (bsj BlockStoreStateJSON) Save(db dbm.DB) { diff --git a/consensus/byzantine_test.go b/consensus/byzantine_test.go index 6f73fd560..2f5f3f76c 100644 --- a/consensus/byzantine_test.go +++ b/consensus/byzantine_test.go @@ -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(uint64, int) { - return func(height uint64, round int) { + css[i].decideProposal = func(j int) func(int64, int) { + return func(height int64, round int) { byzantineDecideProposalFunc(t, height, round, css[j], switches[j]) } }(i) - css[i].doPrevote = func(height uint64, round int) {} + css[i].doPrevote = func(height int64, round int) {} } eventBus := types.NewEventBus() @@ -162,7 +162,7 @@ func TestByzantine(t *testing.T) { //------------------------------- // byzantine consensus functions -func byzantineDecideProposalFunc(t *testing.T, height uint64, round int, cs *ConsensusState, sw *p2p.Switch) { +func byzantineDecideProposalFunc(t *testing.T, height int64, 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 uint64, round int, cs *Con } } -func sendProposalAndParts(height uint64, round int, cs *ConsensusState, peer p2p.Peer, proposal *types.Proposal, blockHash []byte, parts *types.PartSet) { +func sendProposalAndParts(height int64, 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}) diff --git a/consensus/common_test.go b/consensus/common_test.go index 67a720754..da7c1d8d8 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -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 uint64 + Height int64 Round int types.PrivValidator } @@ -113,13 +113,13 @@ func incrementRound(vss ...*validatorStub) { //------------------------------------------------------------------------------- // Functions for transitioning the consensus state -func startTestRound(cs *ConsensusState, height uint64, round int) { +func startTestRound(cs *ConsensusState, height int64, 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 uint64, round int) (proposal *types.Proposal, block *types.Block) { +func decideProposal(cs1 *ConsensusState, vs *validatorStub, height int64, round int) (proposal *types.Proposal, block *types.Block) { block, blockParts := cs1.createProposalBlock() if block == nil { // on error panic("error creating proposal block") diff --git a/consensus/reactor.go b/consensus/reactor.go index 3502f573d..90dfa3b1c 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -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() uint64 { +func (ps *PeerState) GetHeight() int64 { 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 uint64, round int, index int) { +func (ps *PeerState) SetHasProposalBlockPart(height int64, 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 uint64, round int, type_ byte) *cmn.BitArray { +func (ps *PeerState) getVoteBitArray(height int64, round int, type_ byte) *cmn.BitArray { if !types.IsVoteTypeValid(type_) { return nil } @@ -998,7 +998,7 @@ func (ps *PeerState) getVoteBitArray(height uint64, round int, type_ byte) *cmn. } // 'round': A round for which we have a +2/3 commit. -func (ps *PeerState) ensureCatchupCommitRound(height uint64, round int, numValidators int) { +func (ps *PeerState) ensureCatchupCommitRound(height int64, round int, numValidators int) { if ps.Height != height { return } @@ -1024,13 +1024,13 @@ func (ps *PeerState) ensureCatchupCommitRound(height uint64, round int, numValid // 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 uint64, numValidators int) { +func (ps *PeerState) EnsureVoteBitArrays(height int64, numValidators int) { ps.mtx.Lock() defer ps.mtx.Unlock() ps.ensureVoteBitArrays(height, numValidators) } -func (ps *PeerState) ensureVoteBitArrays(height uint64, numValidators int) { +func (ps *PeerState) ensureVoteBitArrays(height int64, 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 uint64, round int, type_ byte, index int) { +func (ps *PeerState) setHasVote(height int64, 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 uint64 + Height int64 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 uint64 + Height int64 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 uint64 + Height int64 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 uint64 + Height int64 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 uint64 + Height int64 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 uint64 + Height int64 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 uint64 + Height int64 Round int Type byte BlockID types.BlockID diff --git a/consensus/replay.go b/consensus/replay.go index f34911393..e63e9aae0 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -91,7 +91,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 // CONTRACT: csHeight > 0 -func (cs *ConsensusState) catchupReplay(csHeight uint64) error { +func (cs *ConsensusState) catchupReplay(csHeight int64) error { // set replayMode cs.replayMode = true defer func() { cs.replayMode = false }() @@ -152,7 +152,7 @@ func (cs *ConsensusState) catchupReplay(csHeight uint64) error { // Parses marker lines of the form: // #ENDHEIGHT: 12345 /* -func makeHeightSearchFunc(height uint64) auto.SearchFunc { +func makeHeightSearchFunc(height int64) auto.SearchFunc { return func(line string) (int, error) { line = strings.TrimRight(line, "\n") parts := strings.Split(line, " ") @@ -206,7 +206,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { return errors.New(cmn.Fmt("Error calling Info: %v", err)) } - blockHeight := res.LastBlockHeight + blockHeight := int64(res.LastBlockHeight) appHash := res.LastBlockAppHash h.logger.Info("ABCI Handshake", "appHeight", blockHeight, "appHash", fmt.Sprintf("%X", appHash)) @@ -228,7 +228,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 uint64, proxyApp proxy.AppConns) ([]byte, error) { +func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int64, proxyApp proxy.AppConns) ([]byte, error) { storeBlockHeight := h.store.Height() stateBlockHeight := h.state.LastBlockHeight @@ -303,7 +303,7 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight uint64, proxyAp return nil, nil } -func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight uint64, mutateState bool) ([]byte, error) { +func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int64, 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. // @@ -339,7 +339,7 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store } // ApplyBlock on the proxyApp with the last block. -func (h *Handshaker) replayBlock(height uint64, proxyApp proxy.AppConnConsensus) ([]byte, error) { +func (h *Handshaker) replayBlock(height int64, proxyApp proxy.AppConnConsensus) ([]byte, error) { mempool := types.MockMempool{} block := h.store.LoadBlock(height) diff --git a/consensus/replay_test.go b/consensus/replay_test.go index 1588142d3..af0af3e78 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -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 uint64, blockDB dbm.DB, stateDB dbm.DB) { +func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight int64, blockDB dbm.DB, stateDB dbm.DB) { logger := log.TestingLogger() state, _ := sm.GetState(stateDB, consensusReplayConfig.GenesisFile()) state.SetLogger(logger.With("module", "state")) @@ -108,7 +108,7 @@ func TestWALCrash(t *testing.T) { testCases := []struct { name string initFn func(*ConsensusState, context.Context) - heightToStop uint64 + heightToStop int64 }{ {"empty block", func(cs *ConsensusState, ctx context.Context) {}, @@ -134,7 +134,7 @@ func TestWALCrash(t *testing.T) { } } -func crashWALandCheckLiveness(t *testing.T, initFn func(*ConsensusState, context.Context), heightToStop uint64) { +func crashWALandCheckLiveness(t *testing.T, initFn func(*ConsensusState, context.Context), heightToStop int64) { walPaniced := make(chan error) crashingWal := &crashingWAL{panicCh: walPaniced, heightToStop: heightToStop} @@ -203,7 +203,7 @@ LOOP: type crashingWAL struct { next WAL panicCh chan error - heightToStop uint64 + heightToStop int64 msgIndex int // current message index lastPanicedForMsgIndex int // last message for which we panicked @@ -221,7 +221,7 @@ func (e WALWriteError) Error() string { // ReachedHeightToStopError indicates we've reached the required consensus // height and may exit. type ReachedHeightToStopError struct { - height uint64 + height int64 } func (e ReachedHeightToStopError) Error() string { @@ -253,7 +253,7 @@ func (w *crashingWAL) Save(m WALMessage) { } func (w *crashingWAL) Group() *auto.Group { return w.next.Group() } -func (w *crashingWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) { +func (w *crashingWAL) SearchForEndHeight(height int64) (gr *auto.GroupReader, found bool, err error) { return w.next.SearchForEndHeight(height) } @@ -590,21 +590,21 @@ func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBl return &mockBlockStore{config, params, nil, nil} } -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 { +func (bs *mockBlockStore) Height() int64 { return int64(len(bs.chain)) } +func (bs *mockBlockStore) LoadBlock(height int64) *types.Block { return bs.chain[height-1] } +func (bs *mockBlockStore) LoadBlockMeta(height int64) *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 uint64, index int) *types.Part { return nil } +func (bs *mockBlockStore) LoadBlockPart(height int64, index int) *types.Part { return nil } func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { } -func (bs *mockBlockStore) LoadBlockCommit(height uint64) *types.Commit { +func (bs *mockBlockStore) LoadBlockCommit(height int64) *types.Commit { return bs.commits[height-1] } -func (bs *mockBlockStore) LoadSeenCommit(height uint64) *types.Commit { +func (bs *mockBlockStore) LoadSeenCommit(height int64) *types.Commit { return bs.commits[height-1] } diff --git a/consensus/state.go b/consensus/state.go index 3d82d3155..1e85a6cc1 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -54,7 +54,7 @@ type msgInfo struct { // internally generated messages which may update the state type timeoutInfo struct { Duration time.Duration `json:"duration"` - Height uint64 `json:"height"` + Height int64 `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 uint64, round int) - doPrevote func(height uint64, round int) + decideProposal func(height int64, round int) + doPrevote func(height int64, 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() (uint64, []*types.Validator) { +func (cs *ConsensusState) GetValidators() (int64, []*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 uint64) *types.Commit { +func (cs *ConsensusState) LoadCommit(height int64) *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 uint64, round int, part *types.Part, peerKey string) error { +func (cs *ConsensusState) AddProposalBlockPart(height int64, 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 uint64) { +func (cs *ConsensusState) updateHeight(height int64) { 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 uint64, round int, step cstypes.RoundStepType) { +func (cs *ConsensusState) scheduleTimeout(duration time.Duration, height int64, 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 uint64) { +func (cs *ConsensusState) handleTxsAvailable(height int64) { 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 uint64) { // 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 uint64, round int) { +func (cs *ConsensusState) enterNewRound(height int64, 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 @@ -698,7 +698,7 @@ func (cs *ConsensusState) enterNewRound(height uint64, 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 // CONTRACT: height > 0 -func (cs *ConsensusState) needProofBlock(height uint64) bool { +func (cs *ConsensusState) needProofBlock(height int64) bool { if height == 1 { return true } @@ -707,7 +707,7 @@ func (cs *ConsensusState) needProofBlock(height uint64) bool { return !bytes.Equal(cs.state.AppHash, lastBlockMeta.Header.AppHash) } -func (cs *ConsensusState) proposalHeartbeat(height uint64, round int) { +func (cs *ConsensusState) proposalHeartbeat(height int64, round int) { counter := 0 addr := cs.privValidator.GetAddress() valIndex, v := cs.Validators.GetByAddress(addr) @@ -739,7 +739,7 @@ func (cs *ConsensusState) proposalHeartbeat(height uint64, 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 uint64, round int) { +func (cs *ConsensusState) enterPropose(height int64, 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 @@ -786,7 +786,7 @@ func (cs *ConsensusState) isProposer() bool { return bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress()) } -func (cs *ConsensusState) defaultDecideProposal(height uint64, round int) { +func (cs *ConsensusState) defaultDecideProposal(height int64, round int) { var block *types.Block var blockParts *types.PartSet @@ -874,7 +874,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 uint64, round int) { +func (cs *ConsensusState) enterPrevote(height int64, 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 @@ -903,7 +903,7 @@ func (cs *ConsensusState) enterPrevote(height uint64, round int) { // (so we have more time to try and collect +2/3 prevotes for a single block) } -func (cs *ConsensusState) defaultDoPrevote(height uint64, round int) { +func (cs *ConsensusState) defaultDoPrevote(height int64, round int) { logger := cs.Logger.With("height", height, "round", round) // If a block is locked, prevote that. if cs.LockedBlock != nil { @@ -936,7 +936,7 @@ func (cs *ConsensusState) defaultDoPrevote(height uint64, round int) { } // Enter: any +2/3 prevotes at next round. -func (cs *ConsensusState) enterPrevoteWait(height uint64, round int) { +func (cs *ConsensusState) enterPrevoteWait(height int64, 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 @@ -962,7 +962,7 @@ func (cs *ConsensusState) enterPrevoteWait(height uint64, 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 uint64, round int) { +func (cs *ConsensusState) enterPrecommit(height int64, 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 @@ -1055,7 +1055,7 @@ func (cs *ConsensusState) enterPrecommit(height uint64, round int) { } // Enter: any +2/3 precommits for next round. -func (cs *ConsensusState) enterPrecommitWait(height uint64, round int) { +func (cs *ConsensusState) enterPrecommitWait(height int64, 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 @@ -1077,7 +1077,7 @@ func (cs *ConsensusState) enterPrecommitWait(height uint64, round int) { } // Enter: +2/3 precommits for block -func (cs *ConsensusState) enterCommit(height uint64, commitRound int) { +func (cs *ConsensusState) enterCommit(height int64, 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 @@ -1123,7 +1123,7 @@ func (cs *ConsensusState) enterCommit(height uint64, commitRound int) { } // If we have the block AND +2/3 commits for it, finalize. -func (cs *ConsensusState) tryFinalizeCommit(height uint64) { +func (cs *ConsensusState) tryFinalizeCommit(height int64) { if cs.Height != height { cmn.PanicSanity(cmn.Fmt("tryFinalizeCommit() cs.Height: %v vs height: %v", cs.Height, height)) } @@ -1145,7 +1145,7 @@ func (cs *ConsensusState) tryFinalizeCommit(height uint64) { } // Increment height and goto cstypes.RoundStepNewHeight -func (cs *ConsensusState) finalizeCommit(height uint64) { +func (cs *ConsensusState) finalizeCommit(height int64) { 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 @@ -1286,7 +1286,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 uint64, part *types.Part, verify bool) (added bool, err error) { +func (cs *ConsensusState) addProposalBlockPart(height int64, 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 @@ -1495,7 +1495,7 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part //--------------------------------------------------------- -func CompareHRS(h1 uint64, r1 int, s1 cstypes.RoundStepType, h2 uint64, r2 int, s2 cstypes.RoundStepType) int { +func CompareHRS(h1 int64, r1 int, s1 cstypes.RoundStepType, h2 int64, r2 int, s2 cstypes.RoundStepType) int { if h1 < h2 { return -1 } else if h1 > h2 { diff --git a/consensus/types/height_vote_set.go b/consensus/types/height_vote_set.go index 42541861e..0a0a25fea 100644 --- a/consensus/types/height_vote_set.go +++ b/consensus/types/height_vote_set.go @@ -29,7 +29,7 @@ One for their LastCommit round, and another for the official commit round. */ type HeightVoteSet struct { chainID string - height uint64 + height int64 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 uint64, valSet *types.ValidatorSet) *HeightVoteSet { +func NewHeightVoteSet(chainID string, height int64, valSet *types.ValidatorSet) *HeightVoteSet { hvs := &HeightVoteSet{ chainID: chainID, } @@ -46,7 +46,7 @@ func NewHeightVoteSet(chainID string, height uint64, valSet *types.ValidatorSet) return hvs } -func (hvs *HeightVoteSet) Reset(height uint64, valSet *types.ValidatorSet) { +func (hvs *HeightVoteSet) Reset(height int64, valSet *types.ValidatorSet) { hvs.mtx.Lock() defer hvs.mtx.Unlock() @@ -59,7 +59,7 @@ func (hvs *HeightVoteSet) Reset(height uint64, valSet *types.ValidatorSet) { hvs.round = 0 } -func (hvs *HeightVoteSet) Height() uint64 { +func (hvs *HeightVoteSet) Height() int64 { hvs.mtx.Lock() defer hvs.mtx.Unlock() return hvs.height diff --git a/consensus/types/height_vote_set_test.go b/consensus/types/height_vote_set_test.go index 14f66b6ab..e09d1419d 100644 --- a/consensus/types/height_vote_set_test.go +++ b/consensus/types/height_vote_set_test.go @@ -47,7 +47,7 @@ func TestPeerCatchupRounds(t *testing.T) { } -func makeVoteHR(t *testing.T, height uint64, round int, privVals []*types.PrivValidatorFS, valIndex int) *types.Vote { +func makeVoteHR(t *testing.T, height int64, round int, privVals []*types.PrivValidatorFS, valIndex int) *types.Vote { privVal := privVals[valIndex] vote := &types.Vote{ ValidatorAddress: privVal.GetAddress(), diff --git a/consensus/types/reactor.go b/consensus/types/reactor.go index dac2bf4e7..7dfeed923 100644 --- a/consensus/types/reactor.go +++ b/consensus/types/reactor.go @@ -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 uint64 // Height peer is at + Height int64 // 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 diff --git a/consensus/types/state.go b/consensus/types/state.go index c4c91adaa..da4df6a45 100644 --- a/consensus/types/state.go +++ b/consensus/types/state.go @@ -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 uint64 // Height we are working on + Height int64 // Height we are working on Round int Step RoundStepType StartTime time.Time diff --git a/consensus/wal.go b/consensus/wal.go index 7ed951393..69519c165 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -32,7 +32,7 @@ type TimedWALMessage struct { // EndHeightMessage marks the end of the given height inside WAL. // @internal used by scripts/cutWALUntil util. type EndHeightMessage struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` } type WALMessage interface{} @@ -52,7 +52,7 @@ var _ = wire.RegisterInterface( type WAL interface { Save(WALMessage) Group() *auto.Group - SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) + SearchForEndHeight(height int64) (gr *auto.GroupReader, found bool, err error) Start() error Stop() error @@ -142,7 +142,7 @@ func (wal *baseWAL) Save(msg WALMessage) { // Group reader will be nil if found equals false. // // CONTRACT: caller must close group reader. -func (wal *baseWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) { +func (wal *baseWAL) SearchForEndHeight(height int64) (gr *auto.GroupReader, found bool, err error) { var msg *TimedWALMessage // NOTE: starting from the last file in the group because we're usually @@ -304,7 +304,7 @@ type nilWAL struct{} func (nilWAL) Save(m WALMessage) {} func (nilWAL) Group() *auto.Group { return nil } -func (nilWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) { +func (nilWAL) SearchForEndHeight(height int64) (gr *auto.GroupReader, found bool, err error) { return nil, false, nil } func (nilWAL) Start() error { return nil } diff --git a/consensus/wal_test.go b/consensus/wal_test.go index 4a866aa87..38f2ce034 100644 --- a/consensus/wal_test.go +++ b/consensus/wal_test.go @@ -48,7 +48,7 @@ func TestSearchForEndHeight(t *testing.T) { t.Fatal(err) } - h := uint64(3) + h := int64(3) gr, found, err := wal.SearchForEndHeight(h) assert.NoError(t, err, cmn.Fmt("expected not to err on height %d", h)) assert.True(t, found, cmn.Fmt("expected to find end height for %d", h)) diff --git a/lite/client/provider.go b/lite/client/provider.go index 2e54ed554..c98297dec 100644 --- a/lite/client/provider.go +++ b/lite/client/provider.go @@ -24,7 +24,7 @@ type SignStatusClient interface { type provider struct { node SignStatusClient - lastHeight uint64 + lastHeight int64 } // NewProvider can wrap any rpcclient to expose it as @@ -68,7 +68,7 @@ func (p *provider) GetByHash(hash []byte) (lite.FullCommit, error) { } // GetByHeight gets the validator set by height -func (p *provider) GetByHeight(h uint64) (fc lite.FullCommit, err error) { +func (p *provider) GetByHeight(h int64) (fc lite.FullCommit, err error) { commit, err := p.node.Commit(&h) if err != nil { return fc, err @@ -134,7 +134,7 @@ func (p *provider) seedFromCommit(commit *ctypes.ResultCommit) (fc lite.FullComm return fc, nil } -func (p *provider) updateHeight(h uint64) { +func (p *provider) updateHeight(h int64) { if h > p.lastHeight { p.lastHeight = h } diff --git a/lite/commit.go b/lite/commit.go index 2198bbb2d..11ae6d7ff 100644 --- a/lite/commit.go +++ b/lite/commit.go @@ -42,7 +42,7 @@ func NewFullCommit(commit Commit, vals *types.ValidatorSet) FullCommit { } // Height returns the height of the header. -func (c Commit) Height() uint64 { +func (c Commit) Height() int64 { if c.Header == nil { return 0 } diff --git a/lite/dynamic.go b/lite/dynamic.go index a9bea700b..231aed7a4 100644 --- a/lite/dynamic.go +++ b/lite/dynamic.go @@ -19,11 +19,11 @@ var _ Certifier = &Dynamic{} // going forward. type Dynamic struct { cert *Static - lastHeight uint64 + lastHeight int64 } // NewDynamic returns a new dynamic certifier. -func NewDynamic(chainID string, vals *types.ValidatorSet, height uint64) *Dynamic { +func NewDynamic(chainID string, vals *types.ValidatorSet, height int64) *Dynamic { return &Dynamic{ cert: NewStatic(chainID, vals), lastHeight: height, @@ -46,7 +46,7 @@ func (c *Dynamic) Hash() []byte { } // LastHeight returns the last height of this certifier. -func (c *Dynamic) LastHeight() uint64 { +func (c *Dynamic) LastHeight() int64 { return c.lastHeight } diff --git a/lite/dynamic_test.go b/lite/dynamic_test.go index 998a8a212..12db19466 100644 --- a/lite/dynamic_test.go +++ b/lite/dynamic_test.go @@ -28,7 +28,7 @@ func TestDynamicCert(t *testing.T) { cases := []struct { keys lite.ValKeys vals *types.ValidatorSet - height uint64 + height int64 first, last int // who actually signs proper bool // true -> expect no error changed bool // true -> expect validator change error @@ -70,7 +70,7 @@ func TestDynamicUpdate(t *testing.T) { cert := lite.NewDynamic(chainID, vals, 40) // one valid block to give us a sense of time - h := uint64(100) + h := int64(100) good := keys.GenCommit(chainID, h, nil, vals, []byte("foo"), 0, len(keys)) err := cert.Certify(good) require.Nil(err, "%+v", err) @@ -83,7 +83,7 @@ func TestDynamicUpdate(t *testing.T) { cases := []struct { keys lite.ValKeys vals *types.ValidatorSet - height uint64 + height int64 first, last int // who actually signs proper bool // true -> expect no error changed bool // true -> expect too much change error diff --git a/lite/errors/errors.go b/lite/errors/errors.go index 9b1d53342..99e42a0bd 100644 --- a/lite/errors/errors.go +++ b/lite/errors/errors.go @@ -70,7 +70,7 @@ func ErrNoPathFound() error { //-------------------------------------------- type errHeightMismatch struct { - h1, h2 uint64 + h1, h2 int64 } func (e errHeightMismatch) Error() string { @@ -87,6 +87,6 @@ func IsHeightMismatchErr(err error) bool { } // ErrHeightMismatch returns an mismatch error with stack-trace -func ErrHeightMismatch(h1, h2 uint64) error { +func ErrHeightMismatch(h1, h2 int64) error { return errors.WithStack(errHeightMismatch{h1, h2}) } diff --git a/lite/files/commit_test.go b/lite/files/commit_test.go index 586393e2f..c2124379a 100644 --- a/lite/files/commit_test.go +++ b/lite/files/commit_test.go @@ -24,7 +24,7 @@ func TestSerializeFullCommits(t *testing.T) { // some constants appHash := []byte("some crazy thing") chainID := "ser-ial" - h := uint64(25) + h := int64(25) // build a fc keys := lite.GenValKeys(5) diff --git a/lite/files/provider.go b/lite/files/provider.go index 8eb869bab..327b0331a 100644 --- a/lite/files/provider.go +++ b/lite/files/provider.go @@ -60,7 +60,7 @@ func (p *provider) encodeHash(hash []byte) string { return hex.EncodeToString(hash) + Ext } -func (p *provider) encodeHeight(h uint64) string { +func (p *provider) encodeHeight(h int64) string { // pad up to 10^12 for height... return fmt.Sprintf("%012d%s", h, Ext) } @@ -88,7 +88,7 @@ func (p *provider) StoreCommit(fc lite.FullCommit) error { } // GetByHeight returns the closest commit with height <= h. -func (p *provider) GetByHeight(h uint64) (lite.FullCommit, error) { +func (p *provider) GetByHeight(h int64) (lite.FullCommit, error) { // first we look for exact match, then search... path := filepath.Join(p.checkDir, p.encodeHeight(h)) fc, err := LoadFullCommit(path) @@ -109,7 +109,7 @@ func (p *provider) LatestCommit() (fc lite.FullCommit, err error) { // search for height, looks for a file with highest height < h // return certifiers.ErrCommitNotFound() if not there... -func (p *provider) searchForHeight(h uint64) (string, error) { +func (p *provider) searchForHeight(h int64) (string, error) { d, err := os.Open(p.checkDir) if err != nil { return "", errors.WithStack(err) diff --git a/lite/files/provider_test.go b/lite/files/provider_test.go index 7faf7c5ea..b8d8e88bb 100644 --- a/lite/files/provider_test.go +++ b/lite/files/provider_test.go @@ -45,7 +45,7 @@ func TestFileProvider(t *testing.T) { // two seeds for each validator, to check how we handle dups // (10, 0), (10, 1), (10, 1), (10, 2), (10, 2), ... vals := keys.ToValidators(10, int64(count/2)) - h := uint64(20 + 10*i) + h := int64(20 + 10*i) check := keys.GenCommit(chainID, h, nil, vals, appHash, 0, 5) seeds[i] = lite.NewFullCommit(check, vals) } diff --git a/lite/helpers.go b/lite/helpers.go index e12f087f1..9319c4590 100644 --- a/lite/helpers.go +++ b/lite/helpers.go @@ -108,7 +108,7 @@ func makeVote(header *types.Header, vals *types.ValidatorSet, key crypto.PrivKey // Silences warning that vals can also be merkle.Hashable // nolint: interfacer -func genHeader(chainID string, height uint64, txs types.Txs, +func genHeader(chainID string, height int64, txs types.Txs, vals *types.ValidatorSet, appHash []byte) *types.Header { return &types.Header{ @@ -125,7 +125,7 @@ func genHeader(chainID string, height uint64, txs types.Txs, } // GenCommit calls genHeader and signHeader and combines them into a Commit. -func (v ValKeys) GenCommit(chainID string, height uint64, txs types.Txs, +func (v ValKeys) GenCommit(chainID string, height int64, txs types.Txs, vals *types.ValidatorSet, appHash []byte, first, last int) Commit { header := genHeader(chainID, height, txs, vals, appHash) @@ -137,7 +137,7 @@ func (v ValKeys) GenCommit(chainID string, height uint64, txs types.Txs, } // GenFullCommit calls genHeader and signHeader and combines them into a Commit. -func (v ValKeys) GenFullCommit(chainID string, height uint64, txs types.Txs, +func (v ValKeys) GenFullCommit(chainID string, height int64, txs types.Txs, vals *types.ValidatorSet, appHash []byte, first, last int) FullCommit { header := genHeader(chainID, height, txs, vals, appHash) diff --git a/lite/inquirer.go b/lite/inquirer.go index 4c2655f6e..5d6ce60c7 100644 --- a/lite/inquirer.go +++ b/lite/inquirer.go @@ -46,7 +46,7 @@ func (c *Inquiring) Validators() *types.ValidatorSet { } // LastHeight returns the last height. -func (c *Inquiring) LastHeight() uint64 { +func (c *Inquiring) LastHeight() int64 { return c.cert.lastHeight } @@ -95,7 +95,7 @@ func (c *Inquiring) Update(fc FullCommit) error { return err } -func (c *Inquiring) useClosestTrust(h uint64) error { +func (c *Inquiring) useClosestTrust(h int64) error { closest, err := c.trusted.GetByHeight(h) if err != nil { return err @@ -126,7 +126,7 @@ func (c *Inquiring) updateToHash(vhash []byte) error { } // updateToHeight will use divide-and-conquer to find a path to h -func (c *Inquiring) updateToHeight(h uint64) error { +func (c *Inquiring) updateToHeight(h int64) error { // try to update to this height (with checks) fc, err := c.Source.GetByHeight(h) if err != nil { diff --git a/lite/inquirer_test.go b/lite/inquirer_test.go index 4e315e144..c30d82091 100644 --- a/lite/inquirer_test.go +++ b/lite/inquirer_test.go @@ -28,7 +28,7 @@ func TestInquirerValidPath(t *testing.T) { // extend the keys by 1 each time keys = keys.Extend(1) vals := keys.ToValidators(vote, 0) - h := uint64(20 + 10*i) + h := int64(20 + 10*i) appHash := []byte(fmt.Sprintf("h=%d", h)) commits[i] = keys.GenFullCommit(chainID, h, nil, vals, appHash, 0, len(keys)) } @@ -75,7 +75,7 @@ func TestInquirerMinimalPath(t *testing.T) { // extend the validators, so we are just below 2/3 keys = keys.Extend(len(keys)/2 - 1) vals := keys.ToValidators(vote, 0) - h := uint64(5 + 10*i) + h := int64(5 + 10*i) appHash := []byte(fmt.Sprintf("h=%d", h)) commits[i] = keys.GenFullCommit(chainID, h, nil, vals, appHash, 0, len(keys)) } @@ -122,7 +122,7 @@ func TestInquirerVerifyHistorical(t *testing.T) { // extend the keys by 1 each time keys = keys.Extend(1) vals := keys.ToValidators(vote, 0) - h := uint64(20 + 10*i) + h := int64(20 + 10*i) appHash := []byte(fmt.Sprintf("h=%d", h)) commits[i] = keys.GenFullCommit(chainID, h, nil, vals, appHash, 0, len(keys)) } diff --git a/lite/memprovider.go b/lite/memprovider.go index 03c996308..9c454be02 100644 --- a/lite/memprovider.go +++ b/lite/memprovider.go @@ -52,7 +52,7 @@ func (m *memStoreProvider) StoreCommit(fc FullCommit) error { } // GetByHeight returns the FullCommit for height h or an error if the commit is not found. -func (m *memStoreProvider) GetByHeight(h uint64) (FullCommit, error) { +func (m *memStoreProvider) GetByHeight(h int64) (FullCommit, error) { // search from highest to lowest for i := len(m.byHeight) - 1; i >= 0; i-- { fc := m.byHeight[i] diff --git a/lite/performance_test.go b/lite/performance_test.go index da571d0e6..e01b89936 100644 --- a/lite/performance_test.go +++ b/lite/performance_test.go @@ -31,7 +31,7 @@ func benchmarkGenCommit(b *testing.B, keys lite.ValKeys) { chainID := fmt.Sprintf("bench-%d", len(keys)) vals := keys.ToValidators(20, 10) for i := 0; i < b.N; i++ { - h := uint64(1 + i) + h := int64(1 + i) appHash := []byte(fmt.Sprintf("h=%d", h)) keys.GenCommit(chainID, h, nil, vals, appHash, 0, len(keys)) } diff --git a/lite/provider.go b/lite/provider.go index d3364ff19..22dc964a1 100644 --- a/lite/provider.go +++ b/lite/provider.go @@ -9,7 +9,7 @@ type Provider interface { // store of trusted commits. StoreCommit(fc FullCommit) error // GetByHeight returns the closest commit with height <= h. - GetByHeight(h uint64) (FullCommit, error) + GetByHeight(h int64) (FullCommit, error) // GetByHash returns a commit exactly matching this validator hash. GetByHash(hash []byte) (FullCommit, error) // LatestCommit returns the newest commit stored. @@ -55,7 +55,7 @@ func (c cacheProvider) StoreCommit(fc FullCommit) (err error) { // Thus, we query each provider in order until we find an exact match // or we finished querying them all. If at least one returned a non-error, // then this returns the best match (minimum h-h'). -func (c cacheProvider) GetByHeight(h uint64) (fc FullCommit, err error) { +func (c cacheProvider) GetByHeight(h int64) (fc FullCommit, err error) { for _, p := range c.Providers { var tfc FullCommit tfc, err = p.GetByHeight(h) diff --git a/lite/provider_test.go b/lite/provider_test.go index 9b8ac15f3..f1165619f 100644 --- a/lite/provider_test.go +++ b/lite/provider_test.go @@ -21,7 +21,7 @@ func NewMissingProvider() lite.Provider { } func (missingProvider) StoreCommit(lite.FullCommit) error { return nil } -func (missingProvider) GetByHeight(uint64) (lite.FullCommit, error) { +func (missingProvider) GetByHeight(int64) (lite.FullCommit, error) { return lite.FullCommit{}, liteErr.ErrCommitNotFound() } func (missingProvider) GetByHash([]byte) (lite.FullCommit, error) { @@ -57,7 +57,7 @@ func checkProvider(t *testing.T, p lite.Provider, chainID, app string) { // two commits for each validator, to check how we handle dups // (10, 0), (10, 1), (10, 1), (10, 2), (10, 2), ... vals := keys.ToValidators(10, int64(count/2)) - h := uint64(20 + 10*i) + h := int64(20 + 10*i) commits[i] = keys.GenFullCommit(chainID, h, nil, vals, appHash, 0, 5) } @@ -101,7 +101,7 @@ func checkProvider(t *testing.T, p lite.Provider, chainID, app string) { } // this will make a get height, and if it is good, set the data as well -func checkGetHeight(t *testing.T, p lite.Provider, ask, expect uint64) { +func checkGetHeight(t *testing.T, p lite.Provider, ask, expect int64) { fc, err := p.GetByHeight(ask) require.Nil(t, err, "%+v", err) if assert.Equal(t, expect, fc.Height()) { @@ -128,7 +128,7 @@ func TestCacheGetsBestHeight(t *testing.T) { // set a bunch of commits for i := 0; i < count; i++ { vals := keys.ToValidators(10, int64(count/2)) - h := uint64(10 * (i + 1)) + h := int64(10 * (i + 1)) fc := keys.GenFullCommit(chainID, h, nil, vals, appHash, 0, 5) err := p2.StoreCommit(fc) require.NoError(err) diff --git a/lite/static_test.go b/lite/static_test.go index 4ee7cc03d..e4bf435c5 100644 --- a/lite/static_test.go +++ b/lite/static_test.go @@ -26,7 +26,7 @@ func TestStaticCert(t *testing.T) { cases := []struct { keys lite.ValKeys vals *types.ValidatorSet - height uint64 + height int64 first, last int // who actually signs proper bool // true -> expect no error changed bool // true -> expect validator change error diff --git a/mempool/mempool.go b/mempool/mempool.go index 40cea4f12..44a6ab0d6 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -61,12 +61,12 @@ type Mempool struct { proxyAppConn proxy.AppConnMempool txs *clist.CList // concurrent linked-list of good txs counter int64 // simple incrementing counter - height uint64 // the last block Update()'d to + height int64 // the last block Update()'d to rechecking int32 // for re-checking filtered txs on Update() recheckCursor *clist.CElement // next expected response recheckEnd *clist.CElement // re-checking stops here notifiedTxsAvailable bool // true if fired on txsAvailable for this height - txsAvailable chan uint64 // fires the next height once for each height, when the mempool is not empty + txsAvailable chan int64 // fires the next height once for each height, when the mempool is not empty // Keep a cache of already-seen txs. // This reduces the pressure on the proxyApp. @@ -80,7 +80,7 @@ type Mempool struct { // NewMempool returns a new Mempool with the given configuration and connection to an application. // TODO: Extract logger into arguments. -func NewMempool(config *cfg.MempoolConfig, proxyAppConn proxy.AppConnMempool, height uint64) *Mempool { +func NewMempool(config *cfg.MempoolConfig, proxyAppConn proxy.AppConnMempool, height int64) *Mempool { mempool := &Mempool{ config: config, proxyAppConn: proxyAppConn, @@ -102,7 +102,7 @@ func NewMempool(config *cfg.MempoolConfig, proxyAppConn proxy.AppConnMempool, he // ensuring it will trigger once every height when transactions are available. // NOTE: not thread safe - should only be called once, on startup func (mem *Mempool) EnableTxsAvailable() { - mem.txsAvailable = make(chan uint64, 1) + mem.txsAvailable = make(chan int64, 1) } // SetLogger sets the Logger. @@ -310,7 +310,7 @@ func (mem *Mempool) resCbRecheck(req *abci.Request, res *abci.Response) { // TxsAvailable returns a channel which fires once for every height, // and only when transactions are available in the mempool. // NOTE: the returned channel may be nil if EnableTxsAvailable was not called. -func (mem *Mempool) TxsAvailable() <-chan uint64 { +func (mem *Mempool) TxsAvailable() <-chan int64 { return mem.txsAvailable } @@ -357,7 +357,7 @@ func (mem *Mempool) collectTxs(maxTxs int) types.Txs { // Update informs the mempool that the given txs were committed and can be discarded. // NOTE: this should be called *after* block is committed by consensus. // NOTE: unsafe; Lock/Unlock must be managed by caller -func (mem *Mempool) Update(height uint64, txs types.Txs) error { +func (mem *Mempool) Update(height int64, txs types.Txs) error { if err := mem.proxyAppConn.FlushSync(); err != nil { // To flush async resCb calls e.g. from CheckTx return err } @@ -427,13 +427,13 @@ func (mem *Mempool) recheckTxs(goodTxs []types.Tx) { // mempoolTx is a transaction that successfully ran type mempoolTx struct { counter int64 // a simple incrementing counter - height uint64 // height that this tx had been validated in + height int64 // height that this tx had been validated in tx types.Tx // } // Height returns the height for this transaction -func (memTx *mempoolTx) Height() uint64 { - return atomic.LoadUint64(&memTx.height) +func (memTx *mempoolTx) Height() int64 { + return atomic.LoadInt64(&memTx.height) } //-------------------------------------------------------------------------------- diff --git a/mempool/mempool_test.go b/mempool/mempool_test.go index 4db761077..22caee27d 100644 --- a/mempool/mempool_test.go +++ b/mempool/mempool_test.go @@ -37,7 +37,7 @@ func newMempoolWithApp(cc proxy.ClientCreator) *Mempool { return mempool } -func ensureNoFire(t *testing.T, ch <-chan uint64, timeoutMS int) { +func ensureNoFire(t *testing.T, ch <-chan int64, timeoutMS int) { timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond) select { case <-ch: @@ -46,7 +46,7 @@ func ensureNoFire(t *testing.T, ch <-chan uint64, timeoutMS int) { } } -func ensureFire(t *testing.T, ch <-chan uint64, timeoutMS int) { +func ensureFire(t *testing.T, ch <-chan int64, timeoutMS int) { timer := time.NewTimer(time.Duration(timeoutMS) * time.Millisecond) select { case <-ch: diff --git a/mempool/reactor.go b/mempool/reactor.go index d22ffcda9..9aed416f7 100644 --- a/mempool/reactor.go +++ b/mempool/reactor.go @@ -97,7 +97,7 @@ func (memR *MempoolReactor) BroadcastTx(tx types.Tx, cb func(*abci.Response)) er // PeerState describes the state of a peer. type PeerState interface { - GetHeight() uint64 + GetHeight() int64 } // Peer describes a peer. diff --git a/rpc/client/event_test.go b/rpc/client/event_test.go index 44a1410d8..58f43c224 100644 --- a/rpc/client/event_test.go +++ b/rpc/client/event_test.go @@ -53,7 +53,7 @@ func TestBlockEvents(t *testing.T) { } // listen for a new block; ensure height increases by 1 - var firstBlockHeight uint64 + var firstBlockHeight int64 for j := 0; j < 3; j++ { evtTyp := types.EventNewBlock evt, err := client.WaitForOneEvent(c, evtTyp, waitForEventTimeout) @@ -67,7 +67,7 @@ func TestBlockEvents(t *testing.T) { continue } - require.Equal(block.Header.Height, firstBlockHeight+uint64(j)) + require.Equal(block.Header.Height, firstBlockHeight+int64(j)) } } } diff --git a/rpc/client/helpers.go b/rpc/client/helpers.go index c29253938..027964ac9 100644 --- a/rpc/client/helpers.go +++ b/rpc/client/helpers.go @@ -32,7 +32,7 @@ func DefaultWaitStrategy(delta int) (abort error) { // // If waiter is nil, we use DefaultWaitStrategy, but you can also // provide your own implementation -func WaitForHeight(c StatusClient, h uint64, waiter Waiter) error { +func WaitForHeight(c StatusClient, h int64, waiter Waiter) error { if waiter == nil { waiter = DefaultWaitStrategy } diff --git a/rpc/client/helpers_test.go b/rpc/client/helpers_test.go index ca0884e67..13b3b1d03 100644 --- a/rpc/client/helpers_test.go +++ b/rpc/client/helpers_test.go @@ -66,11 +66,11 @@ func TestWaitForHeight(t *testing.T) { require.Nil(pre.Error) prer, ok := pre.Response.(*ctypes.ResultStatus) require.True(ok) - assert.Equal(uint64(10), prer.LatestBlockHeight) + assert.Equal(int64(10), prer.LatestBlockHeight) post := r.Calls[4] require.Nil(post.Error) postr, ok := post.Response.(*ctypes.ResultStatus) require.True(ok) - assert.Equal(uint64(15), postr.LatestBlockHeight) + assert.Equal(int64(15), postr.LatestBlockHeight) } diff --git a/rpc/client/httpclient.go b/rpc/client/httpclient.go index 9fcaec54d..1f49ea4d7 100644 --- a/rpc/client/httpclient.go +++ b/rpc/client/httpclient.go @@ -123,7 +123,7 @@ func (c *HTTP) DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) { return result, nil } -func (c *HTTP) BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, error) { +func (c *HTTP) BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { result := new(ctypes.ResultBlockchainInfo) _, err := c.rpc.Call("blockchain", map[string]interface{}{"minHeight": minHeight, "maxHeight": maxHeight}, @@ -143,7 +143,7 @@ func (c *HTTP) Genesis() (*ctypes.ResultGenesis, error) { return result, nil } -func (c *HTTP) Block(height *uint64) (*ctypes.ResultBlock, error) { +func (c *HTTP) Block(height *int64) (*ctypes.ResultBlock, error) { result := new(ctypes.ResultBlock) _, err := c.rpc.Call("block", map[string]interface{}{"height": height}, result) if err != nil { @@ -152,7 +152,7 @@ func (c *HTTP) Block(height *uint64) (*ctypes.ResultBlock, error) { return result, nil } -func (c *HTTP) Commit(height *uint64) (*ctypes.ResultCommit, error) { +func (c *HTTP) Commit(height *int64) (*ctypes.ResultCommit, error) { result := new(ctypes.ResultCommit) _, err := c.rpc.Call("commit", map[string]interface{}{"height": height}, result) if err != nil { @@ -187,7 +187,7 @@ func (c *HTTP) TxSearch(query string, prove bool) ([]*ctypes.ResultTx, error) { return *results, nil } -func (c *HTTP) Validators(height *uint64) (*ctypes.ResultValidators, error) { +func (c *HTTP) Validators(height *int64) (*ctypes.ResultValidators, error) { result := new(ctypes.ResultValidators) _, err := c.rpc.Call("validators", map[string]interface{}{"height": height}, result) if err != nil { diff --git a/rpc/client/interface.go b/rpc/client/interface.go index b154312cc..c38f188ee 100644 --- a/rpc/client/interface.go +++ b/rpc/client/interface.go @@ -46,9 +46,9 @@ type ABCIClient interface { // SignClient groups together the interfaces need to get valid // signatures and prove anything about the chain type SignClient interface { - Block(height *uint64) (*ctypes.ResultBlock, error) - Commit(height *uint64) (*ctypes.ResultCommit, error) - Validators(height *uint64) (*ctypes.ResultValidators, error) + Block(height *int64) (*ctypes.ResultBlock, error) + Commit(height *int64) (*ctypes.ResultCommit, error) + Validators(height *int64) (*ctypes.ResultValidators, error) Tx(hash []byte, prove bool) (*ctypes.ResultTx, error) TxSearch(query string, prove bool) ([]*ctypes.ResultTx, error) } @@ -56,7 +56,7 @@ type SignClient interface { // HistoryClient shows us data from genesis to now in large chunks. type HistoryClient interface { Genesis() (*ctypes.ResultGenesis, error) - BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, error) + BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) } type StatusClient interface { diff --git a/rpc/client/localclient.go b/rpc/client/localclient.go index 123d82f88..40c249123 100644 --- a/rpc/client/localclient.go +++ b/rpc/client/localclient.go @@ -100,7 +100,7 @@ func (Local) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) { return core.UnsafeDialSeeds(seeds) } -func (Local) BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, error) { +func (Local) BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { return core.BlockchainInfo(minHeight, maxHeight) } @@ -108,15 +108,15 @@ func (Local) Genesis() (*ctypes.ResultGenesis, error) { return core.Genesis() } -func (Local) Block(height *uint64) (*ctypes.ResultBlock, error) { +func (Local) Block(height *int64) (*ctypes.ResultBlock, error) { return core.Block(height) } -func (Local) Commit(height *uint64) (*ctypes.ResultCommit, error) { +func (Local) Commit(height *int64) (*ctypes.ResultCommit, error) { return core.Commit(height) } -func (Local) Validators(height *uint64) (*ctypes.ResultValidators, error) { +func (Local) Validators(height *int64) (*ctypes.ResultValidators, error) { return core.Validators(height) } diff --git a/rpc/client/mock/client.go b/rpc/client/mock/client.go index 9eb0150c2..dc75e04cb 100644 --- a/rpc/client/mock/client.go +++ b/rpc/client/mock/client.go @@ -111,7 +111,7 @@ func (c Client) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) { return core.UnsafeDialSeeds(seeds) } -func (c Client) BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, error) { +func (c Client) BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { return core.BlockchainInfo(minHeight, maxHeight) } @@ -119,14 +119,14 @@ func (c Client) Genesis() (*ctypes.ResultGenesis, error) { return core.Genesis() } -func (c Client) Block(height *uint64) (*ctypes.ResultBlock, error) { +func (c Client) Block(height *int64) (*ctypes.ResultBlock, error) { return core.Block(height) } -func (c Client) Commit(height *uint64) (*ctypes.ResultCommit, error) { +func (c Client) Commit(height *int64) (*ctypes.ResultCommit, error) { return core.Commit(height) } -func (c Client) Validators(height *uint64) (*ctypes.ResultValidators, error) { +func (c Client) Validators(height *int64) (*ctypes.ResultValidators, error) { return core.Validators(height) } diff --git a/rpc/core/blocks.go b/rpc/core/blocks.go index b0f6da0c0..a06ba83ec 100644 --- a/rpc/core/blocks.go +++ b/rpc/core/blocks.go @@ -61,7 +61,7 @@ import ( // ``` // // -func BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, error) { +func BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) { if minHeight == 0 { minHeight = 1 } @@ -69,13 +69,13 @@ func BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, if maxHeight == 0 { maxHeight = blockStore.Height() } else { - maxHeight = cmn.MinUint64(blockStore.Height(), maxHeight) + maxHeight = cmn.MinInt64(blockStore.Height(), maxHeight) } // maximum 20 block metas - const limit uint64 = 20 + const limit int64 = 20 if maxHeight >= limit { // to prevent underflow - minHeight = cmn.MaxUint64(minHeight, maxHeight-limit) + minHeight = cmn.MaxInt64(minHeight, maxHeight-limit) } logger.Debug("BlockchainInfoHandler", "maxHeight", maxHeight, "minHeight", minHeight) @@ -193,7 +193,7 @@ func BlockchainInfo(minHeight, maxHeight uint64) (*ctypes.ResultBlockchainInfo, // "jsonrpc": "2.0" // } // ``` -func Block(heightPtr *uint64) (*ctypes.ResultBlock, error) { +func Block(heightPtr *int64) (*ctypes.ResultBlock, error) { if heightPtr == nil { height := blockStore.Height() blockMeta := blockStore.LoadBlockMeta(height) @@ -284,7 +284,7 @@ func Block(heightPtr *uint64) (*ctypes.ResultBlock, error) { // "jsonrpc": "2.0" // } // ``` -func Commit(heightPtr *uint64) (*ctypes.ResultCommit, error) { +func Commit(heightPtr *int64) (*ctypes.ResultCommit, error) { if heightPtr == nil { height := blockStore.Height() header := blockStore.LoadBlockMeta(height).Header diff --git a/rpc/core/consensus.go b/rpc/core/consensus.go index 11767d5e3..755f15894 100644 --- a/rpc/core/consensus.go +++ b/rpc/core/consensus.go @@ -42,7 +42,7 @@ import ( // "jsonrpc": "2.0" // } // ``` -func Validators(heightPtr *uint64) (*ctypes.ResultValidators, error) { +func Validators(heightPtr *int64) (*ctypes.ResultValidators, error) { if heightPtr == nil { blockHeight, validators := consensusState.GetValidators() return &ctypes.ResultValidators{blockHeight, validators}, nil diff --git a/rpc/core/pipe.go b/rpc/core/pipe.go index ae89da8b8..d0b0f87d3 100644 --- a/rpc/core/pipe.go +++ b/rpc/core/pipe.go @@ -21,7 +21,7 @@ var subscribeTimeout = 5 * time.Second type Consensus interface { GetState() *sm.State - GetValidators() (uint64, []*types.Validator) + GetValidators() (int64, []*types.Validator) GetRoundState() *cstypes.RoundState } diff --git a/rpc/core/types/responses.go b/rpc/core/types/responses.go index 983d1383b..18c9e8e7f 100644 --- a/rpc/core/types/responses.go +++ b/rpc/core/types/responses.go @@ -12,7 +12,7 @@ import ( ) type ResultBlockchainInfo struct { - LastHeight uint64 `json:"last_height"` + LastHeight int64 `json:"last_height"` BlockMetas []*types.BlockMeta `json:"block_metas"` } @@ -51,7 +51,7 @@ type ResultStatus struct { PubKey crypto.PubKey `json:"pub_key"` LatestBlockHash data.Bytes `json:"latest_block_hash"` LatestAppHash data.Bytes `json:"latest_app_hash"` - LatestBlockHeight uint64 `json:"latest_block_height"` + LatestBlockHeight int64 `json:"latest_block_height"` LatestBlockTime int64 `json:"latest_block_time"` // nano Syncing bool `json:"syncing"` } @@ -86,7 +86,7 @@ type Peer struct { } type ResultValidators struct { - BlockHeight uint64 `json:"block_height"` + BlockHeight int64 `json:"block_height"` Validators []*types.Validator `json:"validators"` } @@ -107,11 +107,11 @@ type ResultBroadcastTxCommit struct { CheckTx abci.ResponseCheckTx `json:"check_tx"` DeliverTx abci.ResponseDeliverTx `json:"deliver_tx"` Hash data.Bytes `json:"hash"` - Height uint64 `json:"height"` + Height int64 `json:"height"` } type ResultTx struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` Index uint32 `json:"index"` TxResult abci.ResponseDeliverTx `json:"tx_result"` Tx types.Tx `json:"tx"` diff --git a/scripts/cutWALUntil/main.go b/scripts/cutWALUntil/main.go index a7948a267..843368952 100644 --- a/scripts/cutWALUntil/main.go +++ b/scripts/cutWALUntil/main.go @@ -22,9 +22,9 @@ func main() { os.Exit(1) } - var heightToStop uint64 + var heightToStop int64 var err error - if heightToStop, err = strconv.ParseUint(os.Args[2], 10, 64); err != nil { + if heightToStop, err = strconv.ParseInt(os.Args[2], 10, 64); err != nil { panic(fmt.Errorf("failed to parse height: %v", err)) } diff --git a/state/errors.go b/state/errors.go index 16f1a4e65..f7520cf6c 100644 --- a/state/errors.go +++ b/state/errors.go @@ -9,22 +9,22 @@ type ( ErrProxyAppConn error ErrUnknownBlock struct { - Height uint64 + Height int64 } ErrBlockHashMismatch struct { CoreHash []byte AppHash []byte - Height uint64 + Height int64 } ErrAppBlockHeightTooHigh struct { - CoreHeight uint64 - AppHeight uint64 + CoreHeight int64 + AppHeight int64 } ErrLastStateMismatch struct { - Height uint64 + Height int64 Core []byte App []byte } @@ -35,7 +35,7 @@ type ( } ErrNoValSetForHeight struct { - Height uint64 + Height int64 } ) diff --git a/state/execution.go b/state/execution.go index 3622a663c..c67f90076 100644 --- a/state/execution.go +++ b/state/execution.go @@ -64,7 +64,7 @@ func execBlockOnProxyApp(txEventPublisher types.TxEventPublisher, proxyAppConn p // NOTE: if we count we can access the tx from the block instead of // pulling it from the req txEventPublisher.PublishEventTx(types.EventDataTx{types.TxResult{ - Height: uint64(block.Height), + Height: block.Height, Index: uint32(txIndex), Tx: types.Tx(req.GetDeliverTx().Tx), Result: *txResult, @@ -122,7 +122,7 @@ func updateValidators(validators *types.ValidatorSet, changedValidators []*abci. address := pubkey.Address() power := int64(v.Power) - // mind the overflow from uint64 + // mind the overflow from int64 if power < 0 { return errors.New(cmn.Fmt("Power (%d) overflows int64", v.Power)) } diff --git a/state/execution_test.go b/state/execution_test.go index bb239fe4e..64f17094f 100644 --- a/state/execution_test.go +++ b/state/execution_test.go @@ -43,7 +43,7 @@ func TestApplyBlock(t *testing.T) { //---------------------------------------------------------------------------- // make some bogus txs -func makeTxs(height uint64) (txs []types.Tx) { +func makeTxs(height int64) (txs []types.Tx) { for i := 0; i < nTxsPerBlock; i++ { txs = append(txs, types.Tx([]byte{byte(height), byte(i)})) } @@ -61,7 +61,7 @@ func state() *State { return s } -func makeBlock(height uint64, state *State) *types.Block { +func makeBlock(height int64, state *State) *types.Block { prevHash := state.LastBlockID.Hash prevParts := types.PartSetHeader{} valHash := state.Validators.Hash() diff --git a/state/state.go b/state/state.go index aa2566f0d..47de859e9 100644 --- a/state/state.go +++ b/state/state.go @@ -23,7 +23,7 @@ var ( abciResponsesKey = []byte("abciResponsesKey") ) -func calcValidatorsKey(height uint64) []byte { +func calcValidatorsKey(height int64) []byte { return []byte(cmn.Fmt("validatorsKey:%v", height)) } @@ -45,7 +45,7 @@ type State struct { // These fields are updated by SetBlockAndValidators. // LastBlockHeight=0 at genesis (ie. block(H=0) does not exist) // LastValidators is used to validate block.LastCommit. - LastBlockHeight uint64 + LastBlockHeight int64 LastBlockID types.BlockID LastBlockTime time.Time Validators *types.ValidatorSet @@ -54,7 +54,7 @@ type State struct { // the change only applies to the next block. // So, if s.LastBlockHeight causes a valset change, // we set s.LastHeightValidatorsChanged = s.LastBlockHeight + 1 - LastHeightValidatorsChanged uint64 + LastHeightValidatorsChanged int64 // AppHash is updated after Commit AppHash []byte @@ -163,7 +163,7 @@ func (s *State) LoadABCIResponses() *ABCIResponses { } // LoadValidators loads the ValidatorSet for a given height. -func (s *State) LoadValidators(height uint64) (*types.ValidatorSet, error) { +func (s *State) LoadValidators(height int64) (*types.ValidatorSet, error) { valInfo := s.loadValidators(height) if valInfo == nil { return nil, ErrNoValSetForHeight{height} @@ -180,7 +180,7 @@ func (s *State) LoadValidators(height uint64) (*types.ValidatorSet, error) { return valInfo.ValidatorSet, nil } -func (s *State) loadValidators(height uint64) *ValidatorsInfo { +func (s *State) loadValidators(height int64) *ValidatorsInfo { buf := s.db.Get(calcValidatorsKey(height)) if len(buf) == 0 { return nil @@ -256,7 +256,7 @@ func (s *State) SetBlockAndValidators(header *types.Header, blockPartsHeader typ } -func (s *State) setBlockAndValidators(height uint64, blockID types.BlockID, blockTime time.Time, +func (s *State) setBlockAndValidators(height int64, blockID types.BlockID, blockTime time.Time, prevValSet, nextValSet *types.ValidatorSet) { s.LastBlockHeight = height @@ -276,7 +276,7 @@ func (s *State) GetValidators() (last *types.ValidatorSet, current *types.Valida // ABCIResponses retains the responses of the various ABCI calls during block processing. // It is persisted to disk before calling Commit. type ABCIResponses struct { - Height uint64 + Height int64 DeliverTx []*abci.ResponseDeliverTx EndBlock *abci.ResponseEndBlock @@ -303,7 +303,7 @@ func (a *ABCIResponses) Bytes() []byte { // ValidatorsInfo represents the latest validator set, or the last height it changed type ValidatorsInfo struct { ValidatorSet *types.ValidatorSet - LastHeightChanged uint64 + LastHeightChanged int64 } // Bytes serializes the ValidatorsInfo using go-wire diff --git a/state/state_test.go b/state/state_test.go index cccfc8b6e..9b78b3871 100644 --- a/state/state_test.go +++ b/state/state_test.go @@ -138,7 +138,7 @@ func TestValidatorChangesSaveLoad(t *testing.T) { assert := assert.New(t) // change vals at these heights - changeHeights := []uint64{1, 2, 4, 5, 10, 15, 16, 17, 20} + changeHeights := []int64{1, 2, 4, 5, 10, 15, 16, 17, 20} N := len(changeHeights) // each valset is just one validator. @@ -155,7 +155,7 @@ func TestValidatorChangesSaveLoad(t *testing.T) { highestHeight := changeHeights[N-1] + 5 changeIndex := 0 pubkey := pubkeys[changeIndex] - for i := uint64(1); i < highestHeight; i++ { + for i := int64(1); i < highestHeight; i++ { // when we get to a change height, // use the next pubkey if changeIndex < len(changeHeights) && i == changeHeights[changeIndex] { @@ -171,7 +171,7 @@ func TestValidatorChangesSaveLoad(t *testing.T) { testCases := make([]valChangeTestCase, highestHeight) changeIndex = 0 pubkey = pubkeys[changeIndex] - for i := uint64(1); i < highestHeight+1; i++ { + for i := int64(1); i < highestHeight+1; i++ { // we we get to the height after a change height // use the next pubkey (note our counter starts at 0 this time) if changeIndex < len(changeHeights) && i == changeHeights[changeIndex]+1 { @@ -192,7 +192,7 @@ func TestValidatorChangesSaveLoad(t *testing.T) { } } -func makeHeaderPartsResponses(state *State, height uint64, +func makeHeaderPartsResponses(state *State, height int64, pubkey crypto.PubKey) (*types.Header, types.PartSetHeader, *ABCIResponses) { block := makeBlock(height, state) @@ -216,6 +216,6 @@ func makeHeaderPartsResponses(state *State, height uint64, } type valChangeTestCase struct { - height uint64 + height int64 vals crypto.PubKey } diff --git a/types/block.go b/types/block.go index eb14fc6c6..4c91c5fe1 100644 --- a/types/block.go +++ b/types/block.go @@ -23,7 +23,7 @@ type Block struct { // MakeBlock returns a new block and corresponding partset from the given information. // TODO: Add version information to the Block struct. -func MakeBlock(height uint64, chainID string, txs []Tx, commit *Commit, +func MakeBlock(height int64, chainID string, txs []Tx, commit *Commit, prevBlockID BlockID, valHash, appHash []byte, partSize int) (*Block, *PartSet) { block := &Block{ Header: &Header{ @@ -45,7 +45,7 @@ func MakeBlock(height uint64, chainID string, txs []Tx, commit *Commit, } // ValidateBasic performs basic validation that doesn't involve state data. -func (b *Block) ValidateBasic(chainID string, lastBlockHeight uint64, lastBlockID BlockID, +func (b *Block) ValidateBasic(chainID string, lastBlockHeight int64, lastBlockID BlockID, lastBlockTime time.Time, appHash []byte) error { if b.ChainID != chainID { return errors.New(cmn.Fmt("Wrong Block.Header.ChainID. Expected %v, got %v", chainID, b.ChainID)) @@ -158,7 +158,7 @@ func (b *Block) StringShort() string { // Header defines the structure of a Tendermint block header type Header struct { ChainID string `json:"chain_id"` - Height uint64 `json:"height"` + Height int64 `json:"height"` Time time.Time `json:"time"` NumTxs int `json:"num_txs"` // XXX: Can we get rid of this? LastBlockID BlockID `json:"last_block_id"` @@ -250,7 +250,7 @@ func (commit *Commit) FirstPrecommit() *Vote { } // Height returns the height of the commit -func (commit *Commit) Height() uint64 { +func (commit *Commit) Height() int64 { if len(commit.Precommits) == 0 { return 0 } diff --git a/types/canonical_json.go b/types/canonical_json.go index f50c54611..a2e91164f 100644 --- a/types/canonical_json.go +++ b/types/canonical_json.go @@ -18,7 +18,7 @@ type CanonicalJSONPartSetHeader struct { type CanonicalJSONProposal struct { BlockPartsHeader CanonicalJSONPartSetHeader `json:"block_parts_header"` - Height uint64 `json:"height"` + Height int64 `json:"height"` POLBlockID CanonicalJSONBlockID `json:"pol_block_id"` POLRound int `json:"pol_round"` Round int `json:"round"` @@ -26,13 +26,13 @@ type CanonicalJSONProposal struct { type CanonicalJSONVote struct { BlockID CanonicalJSONBlockID `json:"block_id"` - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` Type byte `json:"type"` } type CanonicalJSONHeartbeat struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` Sequence int `json:"sequence"` ValidatorAddress data.Bytes `json:"validator_address"` diff --git a/types/events.go b/types/events.go index 7d161540b..08ebf46da 100644 --- a/types/events.go +++ b/types/events.go @@ -118,7 +118,7 @@ type EventDataProposalHeartbeat struct { // NOTE: This goes into the replay WAL type EventDataRoundState struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` Step string `json:"step"` diff --git a/types/heartbeat.go b/types/heartbeat.go index 8d8254532..da9b342b4 100644 --- a/types/heartbeat.go +++ b/types/heartbeat.go @@ -18,7 +18,7 @@ import ( type Heartbeat struct { ValidatorAddress data.Bytes `json:"validator_address"` ValidatorIndex int `json:"validator_index"` - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` Sequence int `json:"sequence"` Signature crypto.Signature `json:"signature"` diff --git a/types/priv_validator.go b/types/priv_validator.go index 493efa268..5dfd521f4 100644 --- a/types/priv_validator.go +++ b/types/priv_validator.go @@ -51,7 +51,7 @@ type PrivValidator interface { type PrivValidatorFS struct { Address data.Bytes `json:"address"` PubKey crypto.PubKey `json:"pub_key"` - LastHeight uint64 `json:"last_height"` + LastHeight int64 `json:"last_height"` LastRound int `json:"last_round"` LastStep int8 `json:"last_step"` LastSignature crypto.Signature `json:"last_signature,omitempty"` // so we dont lose signatures @@ -222,7 +222,7 @@ func (privVal *PrivValidatorFS) SignProposal(chainID string, proposal *Proposal) // signBytesHRS signs the given signBytes if the height/round/step (HRS) // are greater than the latest state. If the HRS are equal, // it returns the privValidator.LastSignature. -func (privVal *PrivValidatorFS) signBytesHRS(height uint64, round int, step int8, signBytes []byte) (crypto.Signature, error) { +func (privVal *PrivValidatorFS) signBytesHRS(height int64, round int, step int8, signBytes []byte) (crypto.Signature, error) { sig := crypto.Signature{} // If height regression, err diff --git a/types/priv_validator_test.go b/types/priv_validator_test.go index 4e1636c08..3b13ed909 100644 --- a/types/priv_validator_test.go +++ b/types/priv_validator_test.go @@ -20,7 +20,7 @@ func TestGenLoadValidator(t *testing.T) { _, tempFilePath := cmn.Tempfile("priv_validator_") privVal := GenPrivValidatorFS(tempFilePath) - height := uint64(100) + height := int64(100) privVal.LastHeight = height privVal.Save() addr := privVal.GetAddress() @@ -99,7 +99,7 @@ func TestSignVote(t *testing.T) { block1 := BlockID{[]byte{1, 2, 3}, PartSetHeader{}} block2 := BlockID{[]byte{3, 2, 1}, PartSetHeader{}} - height, round := uint64(10), 1 + height, round := int64(10), 1 voteType := VoteTypePrevote // sign a vote for first time @@ -133,7 +133,7 @@ func TestSignProposal(t *testing.T) { block1 := PartSetHeader{5, []byte{1, 2, 3}} block2 := PartSetHeader{10, []byte{3, 2, 1}} - height, round := uint64(10), 1 + height, round := int64(10), 1 // sign a proposal for first time proposal := newProposal(height, round, block1) @@ -158,7 +158,7 @@ func TestSignProposal(t *testing.T) { } } -func newVote(addr data.Bytes, idx int, height uint64, round int, typ byte, blockID BlockID) *Vote { +func newVote(addr data.Bytes, idx int, height int64, round int, typ byte, blockID BlockID) *Vote { return &Vote{ ValidatorAddress: addr, ValidatorIndex: idx, @@ -169,7 +169,7 @@ func newVote(addr data.Bytes, idx int, height uint64, round int, typ byte, block } } -func newProposal(height uint64, round int, partsHeader PartSetHeader) *Proposal { +func newProposal(height int64, round int, partsHeader PartSetHeader) *Proposal { return &Proposal{ Height: height, Round: round, diff --git a/types/proposal.go b/types/proposal.go index 21e169b54..93e788961 100644 --- a/types/proposal.go +++ b/types/proposal.go @@ -20,7 +20,7 @@ var ( // to be considered valid. It may depend on votes from a previous round, // a so-called Proof-of-Lock (POL) round, as noted in the POLRound and POLBlockID. type Proposal struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` BlockPartsHeader PartSetHeader `json:"block_parts_header"` POLRound int `json:"pol_round"` // -1 if null. @@ -30,7 +30,7 @@ type Proposal struct { // NewProposal returns a new Proposal. // If there is no POLRound, polRound should be -1. -func NewProposal(height uint64, round int, blockPartsHeader PartSetHeader, polRound int, polBlockID BlockID) *Proposal { +func NewProposal(height int64, round int, blockPartsHeader PartSetHeader, polRound int, polBlockID BlockID) *Proposal { return &Proposal{ Height: height, Round: round, diff --git a/types/protobuf.go b/types/protobuf.go index f7c8b5120..c8c9f8434 100644 --- a/types/protobuf.go +++ b/types/protobuf.go @@ -13,7 +13,7 @@ type tm2pb struct{} func (tm2pb) Header(header *Header) *types.Header { return &types.Header{ ChainId: header.ChainID, - Height: header.Height, + Height: uint64(header.Height), Time: uint64(header.Time.Unix()), NumTxs: uint64(header.NumTxs), LastBlockId: TM2PB.BlockID(header.LastBlockID), diff --git a/types/services.go b/types/services.go index a7d39172b..0e0075540 100644 --- a/types/services.go +++ b/types/services.go @@ -25,10 +25,10 @@ type Mempool interface { Size() int CheckTx(Tx, func(*abci.Response)) error Reap(int) Txs - Update(height uint64, txs Txs) error + Update(height int64, txs Txs) error Flush() - TxsAvailable() <-chan uint64 + TxsAvailable() <-chan int64 EnableTxsAvailable() } @@ -42,9 +42,9 @@ func (m MockMempool) Unlock() {} func (m MockMempool) Size() int { return 0 } func (m MockMempool) CheckTx(tx Tx, cb func(*abci.Response)) error { return nil } func (m MockMempool) Reap(n int) Txs { return Txs{} } -func (m MockMempool) Update(height uint64, txs Txs) error { return nil } +func (m MockMempool) Update(height int64, txs Txs) error { return nil } func (m MockMempool) Flush() {} -func (m MockMempool) TxsAvailable() <-chan uint64 { return make(chan uint64) } +func (m MockMempool) TxsAvailable() <-chan int64 { return make(chan int64) } func (m MockMempool) EnableTxsAvailable() {} //------------------------------------------------------ @@ -53,14 +53,14 @@ func (m MockMempool) EnableTxsAvailable() {} // BlockStoreRPC is the block store interface used by the RPC. // UNSTABLE type BlockStoreRPC interface { - Height() uint64 + Height() int64 - LoadBlockMeta(height uint64) *BlockMeta - LoadBlock(height uint64) *Block - LoadBlockPart(height uint64, index int) *Part + LoadBlockMeta(height int64) *BlockMeta + LoadBlock(height int64) *Block + LoadBlockPart(height int64, index int) *Part - LoadBlockCommit(height uint64) *Commit - LoadSeenCommit(height uint64) *Commit + LoadBlockCommit(height int64) *Commit + LoadSeenCommit(height int64) *Commit } // BlockStore defines the BlockStore interface. diff --git a/types/tx.go b/types/tx.go index fbea8ff55..5761b83e5 100644 --- a/types/tx.go +++ b/types/tx.go @@ -116,7 +116,7 @@ func (tp TxProof) Validate(dataHash []byte) error { // // One usage is indexing transaction results. type TxResult struct { - Height uint64 `json:"height"` + Height int64 `json:"height"` Index uint32 `json:"index"` Tx Tx `json:"tx"` Result abci.ResponseDeliverTx `json:"result"` diff --git a/types/validator_set.go b/types/validator_set.go index 97e12ce92..cba9f206f 100644 --- a/types/validator_set.go +++ b/types/validator_set.go @@ -223,7 +223,7 @@ func (valSet *ValidatorSet) Iterate(fn func(index int, val *Validator) bool) { } // Verify that +2/3 of the set had signed the given signBytes -func (valSet *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, height uint64, commit *Commit) error { +func (valSet *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, height int64, commit *Commit) error { if valSet.Size() != len(commit.Precommits) { return fmt.Errorf("Invalid commit -- wrong set size: %v vs %v", valSet.Size(), len(commit.Precommits)) } @@ -283,7 +283,7 @@ func (valSet *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, height // * 10% of the valset can't just declare themselves kings // * If the validator set is 3x old size, we need more proof to trust func (valSet *ValidatorSet) VerifyCommitAny(newSet *ValidatorSet, chainID string, - blockID BlockID, height uint64, commit *Commit) error { + blockID BlockID, height int64, commit *Commit) error { if newSet.Size() != len(commit.Precommits) { return errors.Errorf("Invalid commit -- wrong set size: %v vs %v", newSet.Size(), len(commit.Precommits)) diff --git a/types/vote.go b/types/vote.go index 544cf67ab..bb8679f46 100644 --- a/types/vote.go +++ b/types/vote.go @@ -51,7 +51,7 @@ func IsVoteTypeValid(type_ byte) bool { type Vote struct { ValidatorAddress data.Bytes `json:"validator_address"` ValidatorIndex int `json:"validator_index"` - Height uint64 `json:"height"` + Height int64 `json:"height"` Round int `json:"round"` Type byte `json:"type"` BlockID BlockID `json:"block_id"` // zero if vote is nil. diff --git a/types/vote_set.go b/types/vote_set.go index 579a7e9b0..941852a84 100644 --- a/types/vote_set.go +++ b/types/vote_set.go @@ -45,7 +45,7 @@ import ( */ type VoteSet struct { chainID string - height uint64 + height int64 round int type_ byte @@ -60,7 +60,7 @@ type VoteSet struct { } // Constructs a new VoteSet struct used to accumulate votes for given height/round. -func NewVoteSet(chainID string, height uint64, round int, type_ byte, valSet *ValidatorSet) *VoteSet { +func NewVoteSet(chainID string, height int64, round int, type_ byte, valSet *ValidatorSet) *VoteSet { if height == 0 { cmn.PanicSanity("Cannot make VoteSet for height == 0, doesn't make sense.") } @@ -83,7 +83,7 @@ func (voteSet *VoteSet) ChainID() string { return voteSet.chainID } -func (voteSet *VoteSet) Height() uint64 { +func (voteSet *VoteSet) Height() int64 { if voteSet == nil { return 0 } else { @@ -523,7 +523,7 @@ func (vs *blockVotes) getByIndex(index int) *Vote { // Common interface between *consensus.VoteSet and types.Commit type VoteSetReader interface { - Height() uint64 + Height() int64 Round() int Type() byte Size() int diff --git a/types/vote_set_test.go b/types/vote_set_test.go index 713ebbf93..b093c44fd 100644 --- a/types/vote_set_test.go +++ b/types/vote_set_test.go @@ -10,7 +10,7 @@ import ( ) // NOTE: privValidators are in order -func randVoteSet(height uint64, round int, type_ byte, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []*PrivValidatorFS) { +func randVoteSet(height int64, round int, type_ byte, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []*PrivValidatorFS) { valSet, privValidators := RandValidatorSet(numValidators, votingPower) return NewVoteSet("test_chain_id", height, round, type_, valSet), valSet, privValidators } @@ -24,7 +24,7 @@ func withValidator(vote *Vote, addr []byte, idx int) *Vote { } // Convenience: Return new vote with different height -func withHeight(vote *Vote, height uint64) *Vote { +func withHeight(vote *Vote, height int64) *Vote { vote = vote.Copy() vote.Height = height return vote @@ -69,7 +69,7 @@ func signAddVote(privVal *PrivValidatorFS, vote *Vote, voteSet *VoteSet) (bool, } func TestAddVote(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1) val0 := privValidators[0] @@ -112,7 +112,7 @@ func TestAddVote(t *testing.T) { } func Test2_3Majority(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1) voteProto := &Vote{ @@ -164,7 +164,7 @@ func Test2_3Majority(t *testing.T) { } func Test2_3MajorityRedux(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 100, 1) blockHash := crypto.CRandBytes(32) @@ -262,7 +262,7 @@ func Test2_3MajorityRedux(t *testing.T) { } func TestBadVotes(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1) voteProto := &Vote{ @@ -321,7 +321,7 @@ func TestBadVotes(t *testing.T) { } func TestConflicts(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 4, 1) blockHash1 := cmn.RandBytes(32) blockHash2 := cmn.RandBytes(32) @@ -450,7 +450,7 @@ func TestConflicts(t *testing.T) { } func TestMakeCommit(t *testing.T) { - height, round := uint64(1), 0 + height, round := int64(1), 0 voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrecommit, 10, 1) blockHash, blockPartsHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)}