uint64 height (Refs #911)

This commit is contained in:
Anton Kaliaev
2017-11-30 13:08:38 -06:00
committed by Ethan Buchman
parent b2489b4318
commit b3492356e6
64 changed files with 296 additions and 270 deletions

View File

@ -52,22 +52,22 @@ type BlockPool struct {
mtx sync.Mutex
// block requests
requesters map[int]*bpRequester
height int // the lowest key in requesters.
numPending int32 // number of requests pending assignment or block response
requesters map[uint64]*bpRequester
height uint64 // the lowest key in requesters.
numPending int32 // number of requests pending assignment or block response
// peers
peers map[string]*bpPeer
maxPeerHeight int
maxPeerHeight uint64
requestsCh chan<- BlockRequest
timeoutsCh chan<- string
}
func NewBlockPool(start int, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool {
func NewBlockPool(start uint64, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool {
bp := &BlockPool{
peers: make(map[string]*bpPeer),
requesters: make(map[int]*bpRequester),
requesters: make(map[uint64]*bpRequester),
height: start,
numPending: 0,
@ -132,7 +132,7 @@ func (pool *BlockPool) removeTimedoutPeers() {
}
}
func (pool *BlockPool) GetStatus() (height int, numPending int32, lenRequesters int) {
func (pool *BlockPool) GetStatus() (height uint64, 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 int) {
func (pool *BlockPool) RedoRequest(height uint64) {
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() int {
func (pool *BlockPool) MaxPeerHeight() uint64 {
pool.mtx.Lock()
defer pool.mtx.Unlock()
return pool.maxPeerHeight
}
// Sets the peer's alleged blockchain height.
func (pool *BlockPool) SetPeerHeight(peerID string, height int) {
func (pool *BlockPool) SetPeerHeight(peerID string, height uint64) {
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 int) *bpPeer {
func (pool *BlockPool) pickIncrAvailablePeer(minHeight uint64) *bpPeer {
pool.mtx.Lock()
defer pool.mtx.Unlock()
@ -304,7 +304,7 @@ func (pool *BlockPool) makeNextRequester() {
pool.mtx.Lock()
defer pool.mtx.Unlock()
nextHeight := pool.height + len(pool.requesters)
nextHeight := pool.height + pool.requestersLen()
request := newBPRequester(pool, nextHeight)
// request.SetLogger(pool.Logger.With("height", nextHeight))
@ -317,7 +317,11 @@ func (pool *BlockPool) makeNextRequester() {
}
}
func (pool *BlockPool) sendRequest(height int, peerID string) {
func (pool *BlockPool) requestersLen() uint64 {
return uint64(len(pool.requesters))
}
func (pool *BlockPool) sendRequest(height uint64, peerID string) {
if !pool.IsRunning() {
return
}
@ -337,7 +341,8 @@ func (pool *BlockPool) debug() string {
defer pool.mtx.Unlock()
str := ""
for h := pool.height; h < pool.height+len(pool.requesters); h++ {
nextHeight := pool.height + pool.requestersLen()
for h := pool.height; h < nextHeight; h++ {
if pool.requesters[h] == nil {
str += cmn.Fmt("H(%v):X ", h)
} else {
@ -355,7 +360,7 @@ type bpPeer struct {
id string
recvMonitor *flow.Monitor
height int
height uint64
numPending int32
timeout *time.Timer
didTimeout bool
@ -363,7 +368,7 @@ type bpPeer struct {
logger log.Logger
}
func newBPPeer(pool *BlockPool, peerID string, height int) *bpPeer {
func newBPPeer(pool *BlockPool, peerID string, height uint64) *bpPeer {
peer := &bpPeer{
pool: pool,
id: peerID,
@ -424,7 +429,7 @@ func (peer *bpPeer) onTimeout() {
type bpRequester struct {
cmn.BaseService
pool *BlockPool
height int
height uint64
gotBlockCh chan struct{}
redoCh chan struct{}
@ -433,7 +438,7 @@ type bpRequester struct {
block *types.Block
}
func newBPRequester(pool *BlockPool, height int) *bpRequester {
func newBPRequester(pool *BlockPool, height uint64) *bpRequester {
bpr := &bpRequester{
pool: pool,
height: height,
@ -545,6 +550,6 @@ OUTER_LOOP:
//-------------------------------------
type BlockRequest struct {
Height int
Height uint64
PeerID string
}

View File

@ -16,21 +16,21 @@ func init() {
type testPeer struct {
id string
height int
height uint64
}
func makePeers(numPeers int, minHeight, maxHeight int) map[string]testPeer {
func makePeers(numPeers int, minHeight, maxHeight uint64) map[string]testPeer {
peers := make(map[string]testPeer, numPeers)
for i := 0; i < numPeers; i++ {
peerID := cmn.RandStr(12)
height := minHeight + rand.Intn(maxHeight-minHeight)
height := minHeight + uint64(rand.Intn(int(maxHeight-minHeight)))
peers[peerID] = testPeer{peerID, height}
}
return peers
}
func TestBasic(t *testing.T) {
start := 42
start := uint64(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 := 42
start := uint64(42)
peers := makePeers(10, start+1, 1000)
timeoutsCh := make(chan string, 100)
requestsCh := make(chan BlockRequest, 100)

View File

@ -347,7 +347,7 @@ func DecodeMessage(bz []byte, maxSize int) (msgType byte, msg BlockchainMessage,
//-------------------------------------
type bcBlockRequestMessage struct {
Height int
Height uint64
}
func (m *bcBlockRequestMessage) String() string {
@ -355,7 +355,7 @@ func (m *bcBlockRequestMessage) String() string {
}
type bcNoBlockResponseMessage struct {
Height int
Height uint64
}
func (brm *bcNoBlockResponseMessage) String() string {
@ -376,7 +376,7 @@ func (m *bcBlockResponseMessage) String() string {
//-------------------------------------
type bcStatusRequestMessage struct {
Height int
Height uint64
}
func (m *bcStatusRequestMessage) String() string {
@ -386,7 +386,7 @@ func (m *bcStatusRequestMessage) String() string {
//-------------------------------------
type bcStatusResponseMessage struct {
Height int
Height uint64
}
func (m *bcStatusResponseMessage) String() string {

View File

@ -14,7 +14,7 @@ import (
"github.com/tendermint/tendermint/types"
)
func newBlockchainReactor(maxBlockHeight int) *BlockchainReactor {
func newBlockchainReactor(maxBlockHeight uint64) *BlockchainReactor {
logger := log.TestingLogger()
config := cfg.ResetTestRoot("blockchain_reactor_test")
@ -34,7 +34,7 @@ func newBlockchainReactor(maxBlockHeight int) *BlockchainReactor {
bcReactor.Switch = p2p.NewSwitch(cfg.DefaultP2PConfig())
// Lastly: let's add some blocks in
for blockHeight := 1; blockHeight <= maxBlockHeight; blockHeight++ {
for blockHeight := uint64(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 int) *BlockchainReactor {
}
func TestNoBlockMessageResponse(t *testing.T) {
maxBlockHeight := 20
maxBlockHeight := uint64(20)
bcr := newBlockchainReactor(maxBlockHeight)
bcr.Start()
@ -58,7 +58,7 @@ func TestNoBlockMessageResponse(t *testing.T) {
chID := byte(0x01)
tests := []struct {
height int
height uint64
existent bool
}{
{maxBlockHeight + 2, false},
@ -93,19 +93,19 @@ func TestNoBlockMessageResponse(t *testing.T) {
//----------------------------------------------
// utility funcs
func makeTxs(blockNumber int) (txs []types.Tx) {
func makeTxs(height uint64) (txs []types.Tx) {
for i := 0; i < 10; i++ {
txs = append(txs, types.Tx([]byte{byte(blockNumber), byte(i)}))
txs = append(txs, types.Tx([]byte{byte(height), byte(i)}))
}
return txs
}
func makeBlock(blockNumber int, state *sm.State) *types.Block {
func makeBlock(height uint64, state *sm.State) *types.Block {
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
valHash := state.Validators.Hash()
prevBlockID := types.BlockID{prevHash, prevParts}
block, _ := types.MakeBlock(blockNumber, "test_chain", makeTxs(blockNumber),
block, _ := types.MakeBlock(height, "test_chain", makeTxs(height),
new(types.Commit), prevBlockID, valHash, state.AppHash, state.Params.BlockGossipParams.BlockPartSizeBytes)
return block
}

View File

@ -7,7 +7,7 @@ import (
"io"
"sync"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/types"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
@ -32,7 +32,7 @@ type BlockStore struct {
db dbm.DB
mtx sync.RWMutex
height int
height uint64
}
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() int {
func (bs *BlockStore) Height() uint64 {
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 int) *types.Block {
func (bs *BlockStore) LoadBlock(height uint64) *types.Block {
var n int
var err error
r := bs.GetReader(calcBlockMetaKey(height))
@ -81,7 +81,7 @@ func (bs *BlockStore) LoadBlock(height int) *types.Block {
return block
}
func (bs *BlockStore) LoadBlockPart(height int, index int) *types.Part {
func (bs *BlockStore) LoadBlockPart(height uint64, 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 int, index int) *types.Part {
return part
}
func (bs *BlockStore) LoadBlockMeta(height int) *types.BlockMeta {
func (bs *BlockStore) LoadBlockMeta(height uint64) *types.BlockMeta {
var n int
var err error
r := bs.GetReader(calcBlockMetaKey(height))
@ -111,7 +111,7 @@ func (bs *BlockStore) LoadBlockMeta(height int) *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 int) *types.Commit {
func (bs *BlockStore) LoadBlockCommit(height uint64) *types.Commit {
var n int
var err error
r := bs.GetReader(calcBlockCommitKey(height))
@ -126,7 +126,7 @@ func (bs *BlockStore) LoadBlockCommit(height int) *types.Commit {
}
// NOTE: the Precommit-vote heights are for the block at `height`
func (bs *BlockStore) LoadSeenCommit(height int) *types.Commit {
func (bs *BlockStore) LoadSeenCommit(height uint64) *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 int, index int, part *types.Part) {
func (bs *BlockStore) saveBlockPart(height uint64, 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 int, index int, part *types.Part) {
//-----------------------------------------------------------------------------
func calcBlockMetaKey(height int) []byte {
func calcBlockMetaKey(height uint64) []byte {
return []byte(fmt.Sprintf("H:%v", height))
}
func calcBlockPartKey(height int, partIndex int) []byte {
func calcBlockPartKey(height uint64, partIndex int) []byte {
return []byte(fmt.Sprintf("P:%v:%v", height, partIndex))
}
func calcBlockCommitKey(height int) []byte {
func calcBlockCommitKey(height uint64) []byte {
return []byte(fmt.Sprintf("C:%v", height))
}
func calcSeenCommitKey(height int) []byte {
func calcSeenCommitKey(height uint64) []byte {
return []byte(fmt.Sprintf("SC:%v", height))
}
@ -216,7 +216,7 @@ func calcSeenCommitKey(height int) []byte {
var blockStoreKey = []byte("blockStore")
type BlockStoreStateJSON struct {
Height int
Height uint64
}
func (bsj BlockStoreStateJSON) Save(db dbm.DB) {