mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-26 03:01:42 +00:00
uint64 height (Refs #911)
This commit is contained in:
committed by
Ethan Buchman
parent
b2489b4318
commit
b3492356e6
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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) {
|
||||
|
Reference in New Issue
Block a user