|
|
@ -1,24 +1,22 @@ |
|
|
|
package blockchain |
|
|
|
|
|
|
|
import ( |
|
|
|
"math" |
|
|
|
"sync" |
|
|
|
"time" |
|
|
|
|
|
|
|
flow "github.com/tendermint/tendermint/Godeps/_workspace/src/code.google.com/p/mxk/go1/flowcontrol" |
|
|
|
. "github.com/tendermint/tendermint/common" |
|
|
|
"github.com/tendermint/tendermint/types" |
|
|
|
) |
|
|
|
|
|
|
|
const ( |
|
|
|
maxTries = 3 |
|
|
|
inputsChannelCapacity = 200 |
|
|
|
requestIntervalMS = 500 |
|
|
|
maxPendingRequests = 200 |
|
|
|
maxTotalRequests = 300 |
|
|
|
maxRequestsPerPeer = 300 |
|
|
|
) |
|
|
|
|
|
|
|
var ( |
|
|
|
requestTimeoutSeconds = time.Duration(3) |
|
|
|
requestIntervalMS = 500 |
|
|
|
maxTotalRequests = 300 |
|
|
|
maxPendingRequests = maxTotalRequests |
|
|
|
maxPendingRequestsPerPeer = 30 |
|
|
|
peerTimeoutSeconds = 10 |
|
|
|
minRecvRate = 10240 // 10Kb/s
|
|
|
|
) |
|
|
|
|
|
|
|
/* |
|
|
@ -33,14 +31,13 @@ var ( |
|
|
|
*/ |
|
|
|
|
|
|
|
type BlockPool struct { |
|
|
|
BaseService |
|
|
|
QuitService |
|
|
|
|
|
|
|
// block requests
|
|
|
|
requestsMtx sync.Mutex |
|
|
|
requests map[int]*bpRequest |
|
|
|
height int // the lowest key in requests.
|
|
|
|
numUnassigned int32 // number of requests not yet assigned to a peer
|
|
|
|
numPending int32 // number of requests pending assignment or block response
|
|
|
|
mtx sync.Mutex |
|
|
|
requests map[int]*bpRequester |
|
|
|
height int // the lowest key in requests.
|
|
|
|
numPending int32 // number of requests pending assignment or block response
|
|
|
|
|
|
|
|
// peers
|
|
|
|
peersMtx sync.Mutex |
|
|
@ -48,46 +45,40 @@ type BlockPool struct { |
|
|
|
|
|
|
|
requestsCh chan<- BlockRequest |
|
|
|
timeoutsCh chan<- string |
|
|
|
repeater *RepeatTimer |
|
|
|
} |
|
|
|
|
|
|
|
func NewBlockPool(start int, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool { |
|
|
|
bp := &BlockPool{ |
|
|
|
peers: make(map[string]*bpPeer), |
|
|
|
|
|
|
|
requests: make(map[int]*bpRequest), |
|
|
|
height: start, |
|
|
|
numUnassigned: 0, |
|
|
|
numPending: 0, |
|
|
|
requests: make(map[int]*bpRequester), |
|
|
|
height: start, |
|
|
|
numPending: 0, |
|
|
|
|
|
|
|
requestsCh: requestsCh, |
|
|
|
timeoutsCh: timeoutsCh, |
|
|
|
repeater: nil, |
|
|
|
} |
|
|
|
bp.BaseService = *NewBaseService(log, "BlockPool", bp) |
|
|
|
bp.QuitService = *NewQuitService(log, "BlockPool", bp) |
|
|
|
return bp |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) OnStart() error { |
|
|
|
pool.BaseService.OnStart() |
|
|
|
pool.repeater = NewRepeatTimer("", requestIntervalMS*time.Millisecond) |
|
|
|
go pool.run() |
|
|
|
pool.QuitService.OnStart() |
|
|
|
go pool.makeRequestsRoutine() |
|
|
|
return nil |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) OnStop() { |
|
|
|
pool.BaseService.OnStop() |
|
|
|
pool.repeater.Stop() |
|
|
|
pool.QuitService.OnStop() |
|
|
|
} |
|
|
|
|
|
|
|
// Run spawns requests as needed.
|
|
|
|
func (pool *BlockPool) run() { |
|
|
|
RUN_LOOP: |
|
|
|
func (pool *BlockPool) makeRequestsRoutine() { |
|
|
|
for { |
|
|
|
if !pool.IsRunning() { |
|
|
|
break RUN_LOOP |
|
|
|
break |
|
|
|
} |
|
|
|
_, numPending, _ := pool.GetStatus() |
|
|
|
_, numPending := pool.GetStatus() |
|
|
|
if numPending >= maxPendingRequests { |
|
|
|
// sleep for a bit.
|
|
|
|
time.Sleep(requestIntervalMS * time.Millisecond) |
|
|
@ -101,18 +92,35 @@ RUN_LOOP: |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) GetStatus() (height int, numPending int32, numUnssigned int32) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
func (pool *BlockPool) GetStatus() (height int, numPending int32) { |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
return pool.height, pool.numPending |
|
|
|
} |
|
|
|
|
|
|
|
// TODO: relax conditions, prevent abuse.
|
|
|
|
func (pool *BlockPool) IsCaughtUp() bool { |
|
|
|
pool.mtx.Lock() |
|
|
|
height := pool.height |
|
|
|
pool.mtx.Unlock() |
|
|
|
|
|
|
|
pool.peersMtx.Lock() |
|
|
|
numPeers := len(pool.peers) |
|
|
|
maxPeerHeight := 0 |
|
|
|
for _, peer := range pool.peers { |
|
|
|
maxPeerHeight = MaxInt(maxPeerHeight, peer.height) |
|
|
|
} |
|
|
|
pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
return pool.height, pool.numPending, pool.numUnassigned |
|
|
|
return numPeers >= 3 && height > 0 && height == maxPeerHeight |
|
|
|
} |
|
|
|
|
|
|
|
// We need to see the second block's Validation to validate the first block.
|
|
|
|
// So we peek two blocks at a time.
|
|
|
|
func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
if r := pool.requests[pool.height]; r != nil { |
|
|
|
first = r.block |
|
|
@ -126,8 +134,8 @@ func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) |
|
|
|
// Pop the first block at pool.height
|
|
|
|
// It must have been validated by 'second'.Validation from PeekTwoBlocks().
|
|
|
|
func (pool *BlockPool) PopRequest() { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
if r := pool.requests[pool.height]; r == nil || r.block == nil { |
|
|
|
PanicSanity("PopRequest() requires a valid block") |
|
|
@ -137,108 +145,77 @@ func (pool *BlockPool) PopRequest() { |
|
|
|
pool.height++ |
|
|
|
} |
|
|
|
|
|
|
|
// Invalidates the block at pool.height.
|
|
|
|
// Remove the peer and request from others.
|
|
|
|
// Invalidates the block at pool.height,
|
|
|
|
// Remove the peer and redo request from others.
|
|
|
|
func (pool *BlockPool) RedoRequest(height int) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
request := pool.requests[height] |
|
|
|
if request.block == nil { |
|
|
|
PanicSanity("Expected block to be non-nil") |
|
|
|
} |
|
|
|
// RemovePeer will redo all requests associated with this peer.
|
|
|
|
// TODO: record this malfeasance
|
|
|
|
// maybe punish peer on switch (an invalid block!)
|
|
|
|
pool.RemovePeer(request.peerId) // Lock on peersMtx.
|
|
|
|
request.block = nil |
|
|
|
request.peerId = "" |
|
|
|
pool.numPending++ |
|
|
|
pool.numUnassigned++ |
|
|
|
|
|
|
|
go requestRoutine(pool, height) |
|
|
|
pool.RemovePeer(request.peerID) // Lock on peersMtx.
|
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) hasBlock(height int) bool { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
|
|
|
|
request := pool.requests[height] |
|
|
|
return request != nil && request.block != nil |
|
|
|
} |
|
|
|
// TODO: ensure that blocks come in order for each peer.
|
|
|
|
func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int) { |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
func (pool *BlockPool) setPeerForRequest(height int, peerId string) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
|
|
|
|
request := pool.requests[height] |
|
|
|
request := pool.requests[block.Height] |
|
|
|
if request == nil { |
|
|
|
return |
|
|
|
} |
|
|
|
pool.numUnassigned-- |
|
|
|
request.peerId = peerId |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) removePeerForRequest(height int, peerId string) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
|
|
|
|
request := pool.requests[height] |
|
|
|
if request == nil { |
|
|
|
return |
|
|
|
if request.setBlock(block, peerID) { |
|
|
|
pool.numPending-- |
|
|
|
peer := pool.getPeer(peerID) |
|
|
|
peer.decrPending(blockSize) |
|
|
|
} else { |
|
|
|
// Bad peer?
|
|
|
|
} |
|
|
|
pool.numUnassigned++ |
|
|
|
request.peerId = "" |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) AddBlock(block *types.Block, peerId string) { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
// Sets the peer's alleged blockchain height.
|
|
|
|
func (pool *BlockPool) SetPeerHeight(peerID string, height int) { |
|
|
|
pool.peersMtx.Lock() // Lock
|
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
request := pool.requests[block.Height] |
|
|
|
if request == nil { |
|
|
|
return |
|
|
|
} |
|
|
|
if request.peerId != peerId { |
|
|
|
return |
|
|
|
} |
|
|
|
if request.block != nil { |
|
|
|
return |
|
|
|
peer := pool.peers[peerID] |
|
|
|
if peer != nil { |
|
|
|
peer.height = height |
|
|
|
} else { |
|
|
|
peer = newBPPeer(pool, peerID, height) |
|
|
|
pool.peers[peerID] = peer |
|
|
|
} |
|
|
|
request.block = block |
|
|
|
pool.numPending-- |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) getPeer(peerId string) *bpPeer { |
|
|
|
func (pool *BlockPool) RemovePeer(peerID string) { |
|
|
|
pool.peersMtx.Lock() // Lock
|
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
peer := pool.peers[peerId] |
|
|
|
return peer |
|
|
|
pool.removePeer(peerID) |
|
|
|
} |
|
|
|
|
|
|
|
// Sets the peer's alleged blockchain height.
|
|
|
|
func (pool *BlockPool) SetPeerHeight(peerId string, height int) { |
|
|
|
pool.peersMtx.Lock() // Lock
|
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
peer := pool.peers[peerId] |
|
|
|
if peer != nil { |
|
|
|
peer.height = height |
|
|
|
} else { |
|
|
|
peer = &bpPeer{ |
|
|
|
height: height, |
|
|
|
id: peerId, |
|
|
|
numRequests: 0, |
|
|
|
func (pool *BlockPool) removePeer(peerID string) { |
|
|
|
for _, request := range pool.requests { |
|
|
|
if request.getPeerID() == peerID { |
|
|
|
pool.numPending++ |
|
|
|
request.redo() // pick another peer and ...
|
|
|
|
} |
|
|
|
pool.peers[peerId] = peer |
|
|
|
} |
|
|
|
delete(pool.peers, peerID) |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) RemovePeer(peerId string) { |
|
|
|
func (pool *BlockPool) getPeer(peerID string) *bpPeer { |
|
|
|
pool.peersMtx.Lock() // Lock
|
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
delete(pool.peers, peerId) |
|
|
|
peer := pool.peers[peerID] |
|
|
|
return peer |
|
|
|
} |
|
|
|
|
|
|
|
// Pick an available peer with at least the given minHeight.
|
|
|
@ -248,64 +225,52 @@ func (pool *BlockPool) pickIncrAvailablePeer(minHeight int) *bpPeer { |
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
for _, peer := range pool.peers { |
|
|
|
if peer.numRequests >= maxRequestsPerPeer { |
|
|
|
if peer.isBad() { |
|
|
|
pool.removePeer(peer.id) |
|
|
|
continue |
|
|
|
} |
|
|
|
if peer.numPending >= maxPendingRequestsPerPeer { |
|
|
|
continue |
|
|
|
} |
|
|
|
if peer.height < minHeight { |
|
|
|
continue |
|
|
|
} |
|
|
|
peer.numRequests++ |
|
|
|
peer.incrPending() |
|
|
|
return peer |
|
|
|
} |
|
|
|
return nil |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) decrPeer(peerId string) { |
|
|
|
pool.peersMtx.Lock() |
|
|
|
defer pool.peersMtx.Unlock() |
|
|
|
|
|
|
|
peer := pool.peers[peerId] |
|
|
|
if peer == nil { |
|
|
|
return |
|
|
|
} |
|
|
|
peer.numRequests-- |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) makeNextRequest() { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
nextHeight := pool.height + len(pool.requests) |
|
|
|
request := &bpRequest{ |
|
|
|
height: nextHeight, |
|
|
|
peerId: "", |
|
|
|
block: nil, |
|
|
|
} |
|
|
|
request := newBPRequester(pool, nextHeight) |
|
|
|
|
|
|
|
pool.requests[nextHeight] = request |
|
|
|
pool.numUnassigned++ |
|
|
|
pool.numPending++ |
|
|
|
|
|
|
|
go requestRoutine(pool, nextHeight) |
|
|
|
request.Start() |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) sendRequest(height int, peerId string) { |
|
|
|
func (pool *BlockPool) sendRequest(height int, peerID string) { |
|
|
|
if !pool.IsRunning() { |
|
|
|
return |
|
|
|
} |
|
|
|
pool.requestsCh <- BlockRequest{height, peerId} |
|
|
|
pool.requestsCh <- BlockRequest{height, peerID} |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) sendTimeout(peerId string) { |
|
|
|
func (pool *BlockPool) sendTimeout(peerID string) { |
|
|
|
if !pool.IsRunning() { |
|
|
|
return |
|
|
|
} |
|
|
|
pool.timeoutsCh <- peerId |
|
|
|
pool.timeoutsCh <- peerID |
|
|
|
} |
|
|
|
|
|
|
|
func (pool *BlockPool) debug() string { |
|
|
|
pool.requestsMtx.Lock() // Lock
|
|
|
|
defer pool.requestsMtx.Unlock() |
|
|
|
pool.mtx.Lock() // Lock
|
|
|
|
defer pool.mtx.Unlock() |
|
|
|
|
|
|
|
str := "" |
|
|
|
for h := pool.height; h < pool.height+len(pool.requests); h++ { |
|
|
@ -322,64 +287,189 @@ func (pool *BlockPool) debug() string { |
|
|
|
//-------------------------------------
|
|
|
|
|
|
|
|
type bpPeer struct { |
|
|
|
pool *BlockPool |
|
|
|
id string |
|
|
|
height int |
|
|
|
numRequests int32 |
|
|
|
numPending int32 |
|
|
|
recvMonitor *flow.Monitor |
|
|
|
timeout *time.Timer |
|
|
|
didTimeout bool |
|
|
|
} |
|
|
|
|
|
|
|
type bpRequest struct { |
|
|
|
height int |
|
|
|
peerId string |
|
|
|
block *types.Block |
|
|
|
func newBPPeer(pool *BlockPool, peerID string, height int) *bpPeer { |
|
|
|
peer := &bpPeer{ |
|
|
|
pool: pool, |
|
|
|
id: peerID, |
|
|
|
height: height, |
|
|
|
numPending: 0, |
|
|
|
} |
|
|
|
return peer |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) resetMonitor() { |
|
|
|
bpp.recvMonitor = flow.New(time.Second, time.Second*40) |
|
|
|
var initialValue = float64(minRecvRate) * math.E |
|
|
|
bpp.recvMonitor.Update(int(initialValue)) |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) resetTimeout() { |
|
|
|
if bpp.timeout == nil { |
|
|
|
bpp.timeout = time.AfterFunc(time.Second*peerTimeoutSeconds, bpp.onTimeout) |
|
|
|
} else { |
|
|
|
bpp.timeout.Reset(time.Second * peerTimeoutSeconds) |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) incrPending() { |
|
|
|
if bpp.numPending == 0 { |
|
|
|
bpp.resetMonitor() |
|
|
|
bpp.resetTimeout() |
|
|
|
} |
|
|
|
bpp.numPending++ |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) decrPending(recvSize int) { |
|
|
|
bpp.numPending-- |
|
|
|
if bpp.numPending == 0 { |
|
|
|
bpp.timeout.Stop() |
|
|
|
} else { |
|
|
|
bpp.recvMonitor.Update(recvSize) |
|
|
|
bpp.resetTimeout() |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) onTimeout() { |
|
|
|
bpp.didTimeout = true |
|
|
|
} |
|
|
|
|
|
|
|
func (bpp *bpPeer) isBad() bool { |
|
|
|
if bpp.didTimeout { |
|
|
|
bpp.pool.sendTimeout(bpp.id) |
|
|
|
return true |
|
|
|
} |
|
|
|
if bpp.numPending == 0 { |
|
|
|
return false |
|
|
|
} else { |
|
|
|
bpp.pool.sendTimeout(bpp.id) |
|
|
|
return bpp.recvMonitor.Status().CurRate < minRecvRate |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
|
|
|
type bpRequester struct { |
|
|
|
QuitService |
|
|
|
pool *BlockPool |
|
|
|
height int |
|
|
|
gotBlockCh chan struct{} |
|
|
|
redoCh chan struct{} |
|
|
|
|
|
|
|
mtx sync.Mutex |
|
|
|
peerID string |
|
|
|
block *types.Block |
|
|
|
} |
|
|
|
|
|
|
|
func newBPRequester(pool *BlockPool, height int) *bpRequester { |
|
|
|
bpr := &bpRequester{ |
|
|
|
pool: pool, |
|
|
|
height: height, |
|
|
|
gotBlockCh: make(chan struct{}), |
|
|
|
redoCh: make(chan struct{}), |
|
|
|
|
|
|
|
peerID: "", |
|
|
|
block: nil, |
|
|
|
} |
|
|
|
bpr.QuitService = *NewQuitService(nil, "bpRequester", bpr) |
|
|
|
return bpr |
|
|
|
} |
|
|
|
|
|
|
|
func (bpr *bpRequester) OnStart() error { |
|
|
|
bpr.QuitService.OnStart() |
|
|
|
go bpr.requestRoutine() |
|
|
|
return nil |
|
|
|
} |
|
|
|
|
|
|
|
// Returns true if the peer matches
|
|
|
|
func (bpr *bpRequester) setBlock(block *types.Block, peerID string) bool { |
|
|
|
bpr.mtx.Lock() |
|
|
|
if bpr.block != nil || bpr.peerID != peerID { |
|
|
|
bpr.mtx.Unlock() |
|
|
|
return false |
|
|
|
} |
|
|
|
bpr.block = block |
|
|
|
bpr.mtx.Unlock() |
|
|
|
|
|
|
|
bpr.gotBlockCh <- struct{}{} |
|
|
|
return true |
|
|
|
} |
|
|
|
|
|
|
|
func (bpr *bpRequester) getPeerID() string { |
|
|
|
bpr.mtx.Lock() |
|
|
|
defer bpr.mtx.Unlock() |
|
|
|
return bpr.peerID |
|
|
|
} |
|
|
|
|
|
|
|
func (bpr *bpRequester) reset() { |
|
|
|
bpr.mtx.Lock() |
|
|
|
bpr.peerID = "" |
|
|
|
bpr.block = nil |
|
|
|
bpr.mtx.Unlock() |
|
|
|
} |
|
|
|
|
|
|
|
// Tells bpRequester to pick another peer and try again.
|
|
|
|
func (bpr *bpRequester) redo() { |
|
|
|
bpr.redoCh <- struct{}{} |
|
|
|
} |
|
|
|
|
|
|
|
// Responsible for making more requests as necessary
|
|
|
|
// Returns only when a block is found (e.g. AddBlock() is called)
|
|
|
|
func requestRoutine(pool *BlockPool, height int) { |
|
|
|
func (bpr *bpRequester) requestRoutine() { |
|
|
|
OUTER_LOOP: |
|
|
|
for { |
|
|
|
|
|
|
|
// Pick a peer to send request to.
|
|
|
|
var peer *bpPeer = nil |
|
|
|
PICK_LOOP: |
|
|
|
PICK_PEER_LOOP: |
|
|
|
for { |
|
|
|
if !pool.IsRunning() { |
|
|
|
log.Info("BlockPool not running. Stopping requestRoutine", "height", height) |
|
|
|
if !bpr.IsRunning() || !bpr.pool.IsRunning() { |
|
|
|
return |
|
|
|
} |
|
|
|
peer = pool.pickIncrAvailablePeer(height) |
|
|
|
peer = bpr.pool.pickIncrAvailablePeer(bpr.height) |
|
|
|
if peer == nil { |
|
|
|
//log.Info("No peers available", "height", height)
|
|
|
|
time.Sleep(requestIntervalMS * time.Millisecond) |
|
|
|
continue PICK_LOOP |
|
|
|
continue PICK_PEER_LOOP |
|
|
|
} |
|
|
|
break PICK_LOOP |
|
|
|
break PICK_PEER_LOOP |
|
|
|
} |
|
|
|
|
|
|
|
// set the peer, decrement numUnassigned
|
|
|
|
pool.setPeerForRequest(height, peer.id) |
|
|
|
|
|
|
|
for try := 0; try < maxTries; try++ { |
|
|
|
pool.sendRequest(height, peer.id) |
|
|
|
time.Sleep(requestTimeoutSeconds * time.Second) |
|
|
|
// if successful the block is either in the pool,
|
|
|
|
if pool.hasBlock(height) { |
|
|
|
pool.decrPeer(peer.id) |
|
|
|
bpr.mtx.Lock() |
|
|
|
bpr.peerID = peer.id |
|
|
|
bpr.mtx.Unlock() |
|
|
|
|
|
|
|
// Send request and wait.
|
|
|
|
bpr.pool.sendRequest(bpr.height, peer.id) |
|
|
|
select { |
|
|
|
case <-bpr.pool.Quit: |
|
|
|
bpr.Stop() |
|
|
|
return |
|
|
|
case <-bpr.Quit: |
|
|
|
return |
|
|
|
case <-bpr.redoCh: |
|
|
|
bpr.reset() |
|
|
|
continue OUTER_LOOP // When peer is removed
|
|
|
|
case <-bpr.gotBlockCh: |
|
|
|
// We got the block, now see if it's good.
|
|
|
|
select { |
|
|
|
case <-bpr.pool.Quit: |
|
|
|
bpr.Stop() |
|
|
|
return |
|
|
|
} |
|
|
|
// or already processed and we've moved past it
|
|
|
|
bpHeight, _, _ := pool.GetStatus() |
|
|
|
if height < bpHeight { |
|
|
|
pool.decrPeer(peer.id) |
|
|
|
case <-bpr.Quit: |
|
|
|
return |
|
|
|
case <-bpr.redoCh: |
|
|
|
bpr.reset() |
|
|
|
continue OUTER_LOOP |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
// unset the peer, increment numUnassigned
|
|
|
|
pool.removePeerForRequest(height, peer.id) |
|
|
|
|
|
|
|
// this peer failed us, try again
|
|
|
|
pool.RemovePeer(peer.id) |
|
|
|
pool.sendTimeout(peer.id) |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
@ -387,5 +477,5 @@ func requestRoutine(pool *BlockPool, height int) { |
|
|
|
|
|
|
|
type BlockRequest struct { |
|
|
|
Height int |
|
|
|
PeerId string |
|
|
|
PeerID string |
|
|
|
} |