tendermint/blockchain/pool.go

379 lines
7.8 KiB
Go
Raw Normal View History

package blockchain
2015-03-22 03:30:22 -07:00
import (
2015-03-24 11:02:30 -07:00
"sync"
2015-03-22 03:30:22 -07:00
"sync/atomic"
"time"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/types"
2015-03-22 03:30:22 -07:00
)
const (
maxOutstandingRequestsPerPeer = 10
2015-03-24 11:02:30 -07:00
inputsChannelCapacity = 100
2015-03-22 03:30:22 -07:00
maxTries = 3
requestIntervalMS = 500
requestBatchSize = 50
maxPendingRequests = 50
maxTotalRequests = 100
2015-03-24 11:02:30 -07:00
maxRequestsPerPeer = 20
2015-03-22 03:30:22 -07:00
)
2015-03-22 19:20:54 -07:00
var (
2015-03-24 11:02:30 -07:00
requestTimeoutSeconds = time.Duration(1)
2015-03-22 19:20:54 -07:00
)
2015-03-22 03:30:22 -07:00
type BlockPool struct {
2015-03-24 11:02:30 -07:00
// block requests
requestsMtx sync.Mutex
requests map[uint]*bpRequest
height uint // the lowest key in requests.
numPending int32
numTotal int32
// peers
peersMtx sync.Mutex
peers map[string]*bpPeer
requestsCh chan<- BlockRequest
timeoutsCh chan<- string
repeater *RepeatTimer
running int32 // atomic
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func NewBlockPool(start uint, requestsCh chan<- BlockRequest, timeoutsCh chan<- string) *BlockPool {
2015-03-22 03:30:22 -07:00
return &BlockPool{
2015-03-24 11:02:30 -07:00
peers: make(map[string]*bpPeer),
requests: make(map[uint]*bpRequest),
2015-03-22 03:30:22 -07:00
height: start,
numPending: 0,
numTotal: 0,
requestsCh: requestsCh,
timeoutsCh: timeoutsCh,
repeater: NewRepeatTimer("", requestIntervalMS*time.Millisecond),
2015-03-24 11:02:30 -07:00
running: 0,
2015-03-22 03:30:22 -07:00
}
}
func (bp *BlockPool) Start() {
2015-03-24 11:02:30 -07:00
if atomic.CompareAndSwapInt32(&bp.running, 0, 1) {
2015-03-22 03:30:22 -07:00
log.Info("Starting BlockPool")
go bp.run()
}
}
func (bp *BlockPool) Stop() {
2015-03-24 11:02:30 -07:00
if atomic.CompareAndSwapInt32(&bp.running, 1, 0) {
2015-03-22 03:30:22 -07:00
log.Info("Stopping BlockPool")
bp.repeater.Stop()
}
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) IsRunning() bool {
return atomic.LoadInt32(&bp.running) == 1
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
// Run spawns requests as needed.
2015-03-22 03:30:22 -07:00
func (bp *BlockPool) run() {
2015-03-24 11:02:30 -07:00
RUN_LOOP:
2015-03-22 03:30:22 -07:00
for {
2015-03-24 11:02:30 -07:00
if atomic.LoadInt32(&bp.running) == 0 {
break RUN_LOOP
}
height, numPending, numTotal := bp.GetStatus()
log.Debug("BlockPool.run", "height", height, "numPending", numPending,
"numTotal", numTotal)
if numPending >= maxPendingRequests {
// sleep for a bit.
time.Sleep(requestIntervalMS * time.Millisecond)
} else if numTotal >= maxTotalRequests {
// sleep for a bit.
time.Sleep(requestIntervalMS * time.Millisecond)
} else {
// request for more blocks.
height := bp.nextHeight()
bp.makeRequest(height)
2015-03-22 03:30:22 -07:00
}
}
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) GetStatus() (uint, int32, int32) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
2015-03-22 16:23:24 -07:00
2015-03-24 11:02:30 -07:00
return bp.height, bp.numPending, bp.numTotal
}
2015-03-22 16:23:24 -07:00
2015-03-24 11:02:30 -07:00
// We need to see the second block's Validation to validate the first block.
// So we peek two blocks at a time.
func (bp *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
if r := bp.requests[bp.height]; r != nil {
first = r.block
}
if r := bp.requests[bp.height+1]; r != nil {
second = r.block
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
return
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
// Pop the first block at bp.height
// It must have been validated by 'second'.Validation from PeekTwoBlocks().
func (bp *BlockPool) PopRequest() {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
if r := bp.requests[bp.height]; r == nil || r.block == nil {
panic("PopRequest() requires a valid block")
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
delete(bp.requests, bp.height)
bp.height++
bp.numTotal--
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
// Invalidates the block at bp.height.
// Remove the peer and request from others.
func (bp *BlockPool) RedoRequest(height uint) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
request := bp.requests[height]
if request.block == nil {
panic("Expected block to be non-nil")
2015-03-22 03:30:22 -07:00
}
bp.RemovePeer(request.peerId) // Lock on peersMtx.
2015-03-24 11:02:30 -07:00
request.block = nil
request.peerId = ""
bp.numPending++
go requestRoutine(bp, height)
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) hasBlock(height uint) bool {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
request := bp.requests[height]
return request != nil && request.block != nil
}
func (bp *BlockPool) setPeerForRequest(height uint, peerId string) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
request := bp.requests[height]
if request == nil {
return
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
request.peerId = peerId
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) AddBlock(block *types.Block, peerId string) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
request := bp.requests[block.Height]
if request == nil {
return
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
if request.peerId != peerId {
return
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
if request.block != nil {
return
}
request.block = block
bp.numPending--
}
func (bp *BlockPool) getPeer(peerId string) *bpPeer {
bp.peersMtx.Lock() // Lock
defer bp.peersMtx.Unlock()
peer := bp.peers[peerId]
return peer
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
// Sets the peer's blockchain height.
func (bp *BlockPool) SetPeerHeight(peerId string, height uint) {
bp.peersMtx.Lock() // Lock
defer bp.peersMtx.Unlock()
peer := bp.peers[peerId]
if peer != nil {
peer.height = height
} else {
peer = &bpPeer{
height: height,
id: peerId,
numRequests: 0,
2015-03-22 12:46:53 -07:00
}
2015-03-24 11:02:30 -07:00
bp.peers[peerId] = peer
2015-03-22 12:46:53 -07:00
}
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) RemovePeer(peerId string) {
bp.peersMtx.Lock() // Lock
defer bp.peersMtx.Unlock()
2015-03-22 03:30:22 -07:00
2015-03-24 11:02:30 -07:00
delete(bp.peers, peerId)
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
// Pick an available peer with at least the given minHeight.
// If no peers are available, returns nil.
func (bp *BlockPool) pickIncrAvailablePeer(minHeight uint) *bpPeer {
bp.peersMtx.Lock()
defer bp.peersMtx.Unlock()
for _, peer := range bp.peers {
if peer.numRequests >= maxRequestsPerPeer {
continue
}
if peer.height < minHeight {
continue
}
peer.numRequests++
return peer
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
return nil
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) decrPeer(peerId string) {
bp.peersMtx.Lock()
defer bp.peersMtx.Unlock()
2015-03-22 03:30:22 -07:00
2015-03-24 11:02:30 -07:00
peer := bp.peers[peerId]
if peer == nil {
return
}
peer.numRequests--
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) nextHeight() uint {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
return bp.height + uint(bp.numTotal)
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) makeRequest(height uint) {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
2015-03-22 03:30:22 -07:00
2015-03-24 11:02:30 -07:00
request := &bpRequest{
height: height,
peerId: "",
block: nil,
}
bp.requests[height] = request
nextHeight := bp.height + uint(bp.numTotal)
if nextHeight == height {
bp.numTotal++
bp.numPending++
}
go requestRoutine(bp, height)
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) sendRequest(height uint, peerId string) {
if atomic.LoadInt32(&bp.running) == 0 {
return
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
bp.requestsCh <- BlockRequest{height, peerId}
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
func (bp *BlockPool) sendTimeout(peerId string) {
if atomic.LoadInt32(&bp.running) == 0 {
return
}
bp.timeoutsCh <- peerId
}
func (bp *BlockPool) debug() string {
bp.requestsMtx.Lock() // Lock
defer bp.requestsMtx.Unlock()
str := ""
for h := bp.height; h < bp.height+uint(bp.numTotal); h++ {
if bp.requests[h] == nil {
str += Fmt("H(%v):X ", h)
} else {
str += Fmt("H(%v):", h)
str += Fmt("B?(%v) ", bp.requests[h].block != nil)
}
}
return str
2015-03-22 03:30:22 -07:00
}
//-------------------------------------
2015-03-24 11:02:30 -07:00
type bpPeer struct {
id string
height uint
numRequests int32
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
type bpRequest struct {
height uint
2015-03-22 03:30:22 -07:00
peerId string
2015-03-24 11:02:30 -07:00
block *types.Block
2015-03-22 03:30:22 -07:00
}
2015-03-24 11:02:30 -07:00
//-------------------------------------
// Responsible for making more requests as necessary
// Returns when a block is found (e.g. AddBlock() is called)
func requestRoutine(bp *BlockPool, height uint) {
for {
var peer *bpPeer = nil
PICK_LOOP:
for {
if !bp.IsRunning() {
return
}
peer = bp.pickIncrAvailablePeer(height)
if peer == nil {
time.Sleep(requestIntervalMS * time.Millisecond)
continue PICK_LOOP
}
break PICK_LOOP
}
bp.setPeerForRequest(height, peer.id)
for try := 0; try < maxTries; try++ {
bp.sendRequest(height, peer.id)
time.Sleep(requestTimeoutSeconds * time.Second)
if bp.hasBlock(height) {
bp.decrPeer(peer.id)
return
}
bpHeight, _, _ := bp.GetStatus()
if height < bpHeight {
bp.decrPeer(peer.id)
return
}
}
bp.RemovePeer(peer.id)
bp.sendTimeout(peer.id)
}
}
//-------------------------------------
type BlockRequest struct {
Height uint
PeerId string
2015-03-22 03:30:22 -07:00
}