Browse Source

int64 height

uint64 is considered dangerous. the details will follow in a blog post.
pull/914/head
Anton Kaliaev 7 years ago
parent
commit
922af7c405
No known key found for this signature in database GPG Key ID: 7B6881D965918214
67 changed files with 274 additions and 274 deletions
  1. +19
    -19
      blockchain/pool.go
  2. +5
    -5
      blockchain/pool_test.go
  3. +4
    -4
      blockchain/reactor.go
  4. +6
    -6
      blockchain/reactor_test.go
  5. +13
    -13
      blockchain/store.go
  6. +5
    -5
      consensus/byzantine_test.go
  7. +3
    -3
      consensus/common_test.go
  8. +14
    -14
      consensus/reactor.go
  9. +6
    -6
      consensus/replay.go
  10. +12
    -12
      consensus/replay_test.go
  11. +24
    -24
      consensus/state.go
  12. +4
    -4
      consensus/types/height_vote_set.go
  13. +1
    -1
      consensus/types/height_vote_set_test.go
  14. +1
    -1
      consensus/types/reactor.go
  15. +1
    -1
      consensus/types/state.go
  16. +4
    -4
      consensus/wal.go
  17. +1
    -1
      consensus/wal_test.go
  18. +3
    -3
      lite/client/provider.go
  19. +1
    -1
      lite/commit.go
  20. +3
    -3
      lite/dynamic.go
  21. +3
    -3
      lite/dynamic_test.go
  22. +2
    -2
      lite/errors/errors.go
  23. +1
    -1
      lite/files/commit_test.go
  24. +3
    -3
      lite/files/provider.go
  25. +1
    -1
      lite/files/provider_test.go
  26. +3
    -3
      lite/helpers.go
  27. +3
    -3
      lite/inquirer.go
  28. +3
    -3
      lite/inquirer_test.go
  29. +1
    -1
      lite/memprovider.go
  30. +1
    -1
      lite/performance_test.go
  31. +2
    -2
      lite/provider.go
  32. +4
    -4
      lite/provider_test.go
  33. +1
    -1
      lite/static_test.go
  34. +9
    -9
      mempool/mempool.go
  35. +2
    -2
      mempool/mempool_test.go
  36. +1
    -1
      mempool/reactor.go
  37. +2
    -2
      rpc/client/event_test.go
  38. +1
    -1
      rpc/client/helpers.go
  39. +2
    -2
      rpc/client/helpers_test.go
  40. +4
    -4
      rpc/client/httpclient.go
  41. +4
    -4
      rpc/client/interface.go
  42. +4
    -4
      rpc/client/localclient.go
  43. +4
    -4
      rpc/client/mock/client.go
  44. +6
    -6
      rpc/core/blocks.go
  45. +1
    -1
      rpc/core/consensus.go
  46. +1
    -1
      rpc/core/pipe.go
  47. +5
    -5
      rpc/core/types/responses.go
  48. +2
    -2
      scripts/cutWALUntil/main.go
  49. +6
    -6
      state/errors.go
  50. +2
    -2
      state/execution.go
  51. +2
    -2
      state/execution_test.go
  52. +8
    -8
      state/state.go
  53. +5
    -5
      state/state_test.go
  54. +4
    -4
      types/block.go
  55. +3
    -3
      types/canonical_json.go
  56. +1
    -1
      types/events.go
  57. +1
    -1
      types/heartbeat.go
  58. +2
    -2
      types/priv_validator.go
  59. +5
    -5
      types/priv_validator_test.go
  60. +2
    -2
      types/proposal.go
  61. +1
    -1
      types/protobuf.go
  62. +10
    -10
      types/services.go
  63. +1
    -1
      types/tx.go
  64. +2
    -2
      types/validator_set.go
  65. +1
    -1
      types/vote.go
  66. +4
    -4
      types/vote_set.go
  67. +8
    -8
      types/vote_set_test.go

+ 19
- 19
blockchain/pool.go View File

@ -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
}

+ 5
- 5
blockchain/pool_test.go View File

@ -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)


+ 4
- 4
blockchain/reactor.go View File

@ -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 {


+ 6
- 6
blockchain/reactor_test.go View File

@ -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()


+ 13
- 13
blockchain/store.go View File

@ -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) {


+ 5
- 5
consensus/byzantine_test.go View File

@ -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})


+ 3
- 3
consensus/common_test.go View File

@ -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")


+ 14
- 14
consensus/reactor.go View File

@ -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


+ 6
- 6
consensus/replay.go View File

@ -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)


+ 12
- 12
consensus/replay_test.go View File

@ -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]
}

+ 24
- 24
consensus/state.go View File

@ -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 {


+ 4
- 4
consensus/types/height_vote_set.go View File

@ -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


+ 1
- 1
consensus/types/height_vote_set_test.go View File

@ -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(),


+ 1
- 1
consensus/types/reactor.go View File

@ -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


+ 1
- 1
consensus/types/state.go View File

@ -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


+ 4
- 4
consensus/wal.go View File

@ -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 }


+ 1
- 1
consensus/wal_test.go View File

@ -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))


+ 3
- 3
lite/client/provider.go View File

@ -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
}


+ 1
- 1
lite/commit.go View File

@ -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
}


+ 3
- 3
lite/dynamic.go View File

@ -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
}


+ 3
- 3
lite/dynamic_test.go View File

@ -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


+ 2
- 2
lite/errors/errors.go View File

@ -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})
}

+ 1
- 1
lite/files/commit_test.go View File

@ -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)


+ 3
- 3
lite/files/provider.go View File

@ -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)


+ 1
- 1
lite/files/provider_test.go View File

@ -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)
}


+ 3
- 3
lite/helpers.go View File

@ -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)


+ 3
- 3
lite/inquirer.go View File

@ -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 {


+ 3
- 3
lite/inquirer_test.go View File

@ -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))
}


+ 1
- 1
lite/memprovider.go View File

@ -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]


+ 1
- 1
lite/performance_test.go View File

@ -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))
}


+ 2
- 2
lite/provider.go View File

@ -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)


+ 4
- 4
lite/provider_test.go View File

@ -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)


+ 1
- 1
lite/static_test.go View File

@ -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


+ 9
- 9
mempool/mempool.go View File

@ -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)
}
//--------------------------------------------------------------------------------


+ 2
- 2
mempool/mempool_test.go View File

@ -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:


+ 1
- 1
mempool/reactor.go View File

@ -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.


+ 2
- 2
rpc/client/event_test.go View File

@ -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))
}
}
}


+ 1
- 1
rpc/client/helpers.go View File

@ -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
}


+ 2
- 2
rpc/client/helpers_test.go View File

@ -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)
}

+ 4
- 4
rpc/client/httpclient.go View File

@ -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 {


+ 4
- 4
rpc/client/interface.go View File

@ -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 {


+ 4
- 4
rpc/client/localclient.go View File

@ -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)
}


+ 4
- 4
rpc/client/mock/client.go View File

@ -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)
}

+ 6
- 6
rpc/core/blocks.go View File

@ -61,7 +61,7 @@ import (
// ```
//
// <aside class="notice">Returns at most 20 items.</aside>
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


+ 1
- 1
rpc/core/consensus.go View File

@ -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


+ 1
- 1
rpc/core/pipe.go View File

@ -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
}


+ 5
- 5
rpc/core/types/responses.go View File

@ -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"`


+ 2
- 2
scripts/cutWALUntil/main.go View File

@ -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))
}


+ 6
- 6
state/errors.go View File

@ -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
}
)


+ 2
- 2
state/execution.go View File

@ -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))
}


+ 2
- 2
state/execution_test.go View File

@ -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()


+ 8
- 8
state/state.go View File

@ -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


+ 5
- 5
state/state_test.go View File

@ -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
}

+ 4
- 4
types/block.go View File

@ -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
}


+ 3
- 3
types/canonical_json.go View File

@ -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"`


+ 1
- 1
types/events.go View File

@ -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"`


+ 1
- 1
types/heartbeat.go View File

@ -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"`


+ 2
- 2
types/priv_validator.go View File

@ -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


+ 5
- 5
types/priv_validator_test.go View File

@ -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,


+ 2
- 2
types/proposal.go View File

@ -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,


+ 1
- 1
types/protobuf.go View File

@ -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),


+ 10
- 10
types/services.go View File

@ -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.


+ 1
- 1
types/tx.go View File

@ -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"`


+ 2
- 2
types/validator_set.go View File

@ -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))


+ 1
- 1
types/vote.go View File

@ -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.


+ 4
- 4
types/vote_set.go View File

@ -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


+ 8
- 8
types/vote_set_test.go View File

@ -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)}


Loading…
Cancel
Save