mirror of
https://github.com/fluencelabs/tendermint
synced 2025-05-19 18:01:18 +00:00
switch fast sync to new implementation
This commit is contained in:
parent
4d54cced43
commit
0c84d780d7
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"fmt"
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"sync"
|
@ -1,331 +1,94 @@
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
"sort"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
flow "github.com/tendermint/tendermint/libs/flowrate"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
/*
|
||||
eg, L = latency = 0.1s
|
||||
P = num peers = 10
|
||||
FN = num full nodes
|
||||
BS = 1kB block size
|
||||
CB = 1 Mbit/s = 128 kB/s
|
||||
CB/P = 12.8 kB
|
||||
B/S = CB/P/BS = 12.8 blocks/s
|
||||
|
||||
12.8 * 0.1 = 1.28 blocks on conn
|
||||
*/
|
||||
|
||||
const (
|
||||
requestIntervalMS = 2
|
||||
maxTotalRequesters = 600
|
||||
maxPendingRequests = maxTotalRequesters
|
||||
maxPendingRequestsPerPeer = 20
|
||||
|
||||
// Minimum recv rate to ensure we're receiving blocks from a peer fast
|
||||
// enough. If a peer is not sending us data at at least that rate, we
|
||||
// consider them to have timedout and we disconnect.
|
||||
//
|
||||
// Assuming a DSL connection (not a good choice) 128 Kbps (upload) ~ 15 KB/s,
|
||||
// sending data across atlantic ~ 7.5 KB/s.
|
||||
minRecvRate = 7680
|
||||
|
||||
// Maximum difference between current and new block's height.
|
||||
maxDiffBetweenCurrentAndReceivedBlockHeight = 100
|
||||
)
|
||||
|
||||
var peerTimeout = 15 * time.Second // not const so we can override with tests
|
||||
|
||||
/*
|
||||
Peers self report their heights when we join the block pool.
|
||||
Starting from our latest pool.height, we request blocks
|
||||
in sequence from peers that reported higher heights than ours.
|
||||
Every so often we ask peers what height they're on so we can keep going.
|
||||
|
||||
Requests are continuously made for blocks of higher heights until
|
||||
the limit is reached. If most of the requests have no available peers, and we
|
||||
are not at peer limits, we can probably switch to consensus reactor
|
||||
*/
|
||||
|
||||
type BlockPool struct {
|
||||
cmn.BaseService
|
||||
startTime time.Time
|
||||
|
||||
mtx sync.Mutex
|
||||
// block requests
|
||||
requesters map[int64]*bpRequester
|
||||
height int64 // the lowest key in requesters.
|
||||
// peers
|
||||
peers map[p2p.ID]*bpPeer
|
||||
maxPeerHeight int64 // the biggest reported height
|
||||
|
||||
// atomic
|
||||
numPending int32 // number of requests pending assignment or block response
|
||||
|
||||
requestsCh chan<- BlockRequest
|
||||
errorsCh chan<- peerError
|
||||
type blockData struct {
|
||||
block *types.Block
|
||||
peer *bpPeer
|
||||
}
|
||||
|
||||
// NewBlockPool returns a new BlockPool with the height equal to start. Block
|
||||
// requests and errors will be sent to requestsCh and errorsCh accordingly.
|
||||
func NewBlockPool(start int64, requestsCh chan<- BlockRequest, errorsCh chan<- peerError) *BlockPool {
|
||||
bp := &BlockPool{
|
||||
peers: make(map[p2p.ID]*bpPeer),
|
||||
|
||||
requesters: make(map[int64]*bpRequester),
|
||||
height: start,
|
||||
numPending: 0,
|
||||
|
||||
requestsCh: requestsCh,
|
||||
errorsCh: errorsCh,
|
||||
func (bd *blockData) String() string {
|
||||
if bd == nil {
|
||||
return fmt.Sprintf("blockData nil")
|
||||
}
|
||||
bp.BaseService = *cmn.NewBaseService(nil, "BlockPool", bp)
|
||||
return bp
|
||||
if bd.block == nil {
|
||||
if bd.peer == nil {
|
||||
return fmt.Sprintf("block: nil peer: nil")
|
||||
}
|
||||
return fmt.Sprintf("block: nil peer: %v", bd.peer.id)
|
||||
}
|
||||
return fmt.Sprintf("block: %v peer: %v", bd.block.Height, bd.peer.id)
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service by spawning requesters routine and recording
|
||||
// pool's start time.
|
||||
func (pool *BlockPool) OnStart() error {
|
||||
go pool.makeRequestersRoutine()
|
||||
pool.startTime = time.Now()
|
||||
return nil
|
||||
type blockPool struct {
|
||||
logger log.Logger
|
||||
peers map[p2p.ID]*bpPeer
|
||||
blocks map[int64]p2p.ID
|
||||
|
||||
requests map[int64]bool // list of blocks to be assigned peers for blockRequest
|
||||
nextRequestHeight int64 // next request to be added to requests
|
||||
|
||||
height int64 // processing height
|
||||
maxPeerHeight int64 // maximum height of all peers
|
||||
numPending int32 // total numPending across peers
|
||||
toBcR bcRMessageInterface
|
||||
}
|
||||
|
||||
// spawns requesters as needed
|
||||
func (pool *BlockPool) makeRequestersRoutine() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
break
|
||||
}
|
||||
|
||||
_, numPending, lenRequesters := pool.GetStatus()
|
||||
if numPending >= maxPendingRequests {
|
||||
// sleep for a bit.
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
// check for timed out peers
|
||||
pool.removeTimedoutPeers()
|
||||
} else if lenRequesters >= maxTotalRequesters {
|
||||
// sleep for a bit.
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
// check for timed out peers
|
||||
pool.removeTimedoutPeers()
|
||||
} else {
|
||||
// request for more blocks.
|
||||
pool.makeNextRequester()
|
||||
}
|
||||
func newBlockPool(height int64, toBcR bcRMessageInterface) *blockPool {
|
||||
return &blockPool{
|
||||
peers: make(map[p2p.ID]*bpPeer),
|
||||
maxPeerHeight: 0,
|
||||
blocks: make(map[int64]p2p.ID),
|
||||
requests: make(map[int64]bool),
|
||||
nextRequestHeight: height,
|
||||
height: height,
|
||||
toBcR: toBcR,
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) removeTimedoutPeers() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
for _, peer := range pool.peers {
|
||||
if !peer.didTimeout && peer.numPending > 0 {
|
||||
curRate := peer.recvMonitor.Status().CurRate
|
||||
// curRate can be 0 on start
|
||||
if curRate != 0 && curRate < minRecvRate {
|
||||
err := errors.New("peer is not sending us data fast enough")
|
||||
pool.sendError(err, peer.id)
|
||||
pool.Logger.Error("SendTimeout", "peer", peer.id,
|
||||
"reason", err,
|
||||
"curRate", fmt.Sprintf("%d KB/s", curRate/1024),
|
||||
"minRate", fmt.Sprintf("%d KB/s", minRecvRate/1024))
|
||||
peer.didTimeout = true
|
||||
}
|
||||
}
|
||||
if peer.didTimeout {
|
||||
pool.removePeer(peer.id)
|
||||
}
|
||||
func (pool *blockPool) String() string {
|
||||
peerStr := fmt.Sprintf("Pool Peers:")
|
||||
for _, p := range pool.peers {
|
||||
peerStr += fmt.Sprintf("%v,", p)
|
||||
}
|
||||
return peerStr
|
||||
}
|
||||
|
||||
func (pool *blockPool) setLogger(l log.Logger) {
|
||||
pool.logger = l
|
||||
}
|
||||
|
||||
// GetStatus returns pool's height, numPending requests and the number of
|
||||
// requesters.
|
||||
func (pool *BlockPool) GetStatus() (height int64, numPending int32, lenRequesters int) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
return pool.height, atomic.LoadInt32(&pool.numPending), len(pool.requesters)
|
||||
// requests ready to be send in the future.
|
||||
func (pool *blockPool) getStatus() (height int64, numPending int32, maxPeerHeight int64) {
|
||||
return pool.height, pool.numPending, pool.maxPeerHeight
|
||||
}
|
||||
|
||||
// IsCaughtUp returns true if this node is caught up, false - otherwise.
|
||||
// TODO: relax conditions, prevent abuse.
|
||||
func (pool *BlockPool) IsCaughtUp() bool {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
// Need at least 1 peer to be considered caught up.
|
||||
if len(pool.peers) == 0 {
|
||||
pool.Logger.Debug("Blockpool has no peers")
|
||||
return false
|
||||
}
|
||||
|
||||
// Some conditions to determine if we're caught up.
|
||||
// Ensures we've either received a block or waited some amount of time,
|
||||
// and that we're synced to the highest known height.
|
||||
// Note we use maxPeerHeight - 1 because to sync block H requires block H+1
|
||||
// to verify the LastCommit.
|
||||
receivedBlockOrTimedOut := pool.height > 0 || time.Since(pool.startTime) > 5*time.Second
|
||||
ourChainIsLongestAmongPeers := pool.maxPeerHeight == 0 || pool.height >= (pool.maxPeerHeight-1)
|
||||
isCaughtUp := receivedBlockOrTimedOut && ourChainIsLongestAmongPeers
|
||||
return isCaughtUp
|
||||
}
|
||||
|
||||
// We need to see the second block's Commit to validate the first block.
|
||||
// So we peek two blocks at a time.
|
||||
// The caller will verify the commit.
|
||||
func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
if r := pool.requesters[pool.height]; r != nil {
|
||||
first = r.getBlock()
|
||||
}
|
||||
if r := pool.requesters[pool.height+1]; r != nil {
|
||||
second = r.getBlock()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Pop the first block at pool.height
|
||||
// It must have been validated by 'second'.Commit from PeekTwoBlocks().
|
||||
func (pool *BlockPool) PopRequest() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
if r := pool.requesters[pool.height]; r != nil {
|
||||
/* The block can disappear at any time, due to removePeer().
|
||||
if r := pool.requesters[pool.height]; r == nil || r.block == nil {
|
||||
PanicSanity("PopRequest() requires a valid block")
|
||||
}
|
||||
*/
|
||||
r.Stop()
|
||||
delete(pool.requesters, pool.height)
|
||||
pool.height++
|
||||
} else {
|
||||
panic(fmt.Sprintf("Expected requester to pop, got nothing at height %v", pool.height))
|
||||
}
|
||||
}
|
||||
|
||||
// Invalidates the block at pool.height,
|
||||
// Remove the peer and redo request from others.
|
||||
// Returns the ID of the removed peer.
|
||||
func (pool *BlockPool) RedoRequest(height int64) p2p.ID {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
request := pool.requesters[height]
|
||||
peerID := request.getPeerID()
|
||||
if peerID != p2p.ID("") {
|
||||
// RemovePeer will redo all requesters associated with this peer.
|
||||
pool.removePeer(peerID)
|
||||
}
|
||||
return peerID
|
||||
}
|
||||
|
||||
// TODO: ensure that blocks come in order for each peer.
|
||||
func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
requester := pool.requesters[block.Height]
|
||||
if requester == nil {
|
||||
pool.Logger.Info("peer sent us a block we didn't expect", "peer", peerID, "curHeight", pool.height, "blockHeight", block.Height)
|
||||
diff := pool.height - block.Height
|
||||
if diff < 0 {
|
||||
diff *= -1
|
||||
}
|
||||
if diff > maxDiffBetweenCurrentAndReceivedBlockHeight {
|
||||
pool.sendError(errors.New("peer sent us a block we didn't expect with a height too far ahead/behind"), peerID)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if requester.setBlock(block, peerID) {
|
||||
atomic.AddInt32(&pool.numPending, -1)
|
||||
peer := pool.peers[peerID]
|
||||
if peer != nil {
|
||||
peer.decrPending(blockSize)
|
||||
}
|
||||
} else {
|
||||
pool.Logger.Info("invalid peer", "peer", peerID, "blockHeight", block.Height)
|
||||
pool.sendError(errors.New("invalid peer"), peerID)
|
||||
}
|
||||
}
|
||||
|
||||
// MaxPeerHeight returns the highest reported height.
|
||||
func (pool *BlockPool) MaxPeerHeight() int64 {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
func (pool blockPool) getMaxPeerHeight() int64 {
|
||||
return pool.maxPeerHeight
|
||||
}
|
||||
|
||||
// SetPeerHeight sets the peer's alleged blockchain height.
|
||||
func (pool *BlockPool) SetPeerHeight(peerID p2p.ID, height int64) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
peer := pool.peers[peerID]
|
||||
if peer != nil {
|
||||
peer.height = height
|
||||
} else {
|
||||
peer = newBPPeer(pool, peerID, height)
|
||||
peer.setLogger(pool.Logger.With("peer", peerID))
|
||||
pool.peers[peerID] = peer
|
||||
}
|
||||
|
||||
if height > pool.maxPeerHeight {
|
||||
pool.maxPeerHeight = height
|
||||
}
|
||||
func (pool *blockPool) reachedMaxHeight() bool {
|
||||
return pool.maxPeerHeight == 0 || pool.height >= pool.maxPeerHeight
|
||||
}
|
||||
|
||||
// RemovePeer removes the peer with peerID from the pool. If there's no peer
|
||||
// with peerID, function is a no-op.
|
||||
func (pool *BlockPool) RemovePeer(peerID p2p.ID) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
pool.removePeer(peerID)
|
||||
func (pool *blockPool) rescheduleRequest(peerID p2p.ID, height int64) {
|
||||
pool.logger.Debug("reschedule requests made to peer for height ", "peerID", peerID, "height", height)
|
||||
pool.requests[height] = true
|
||||
delete(pool.blocks, height)
|
||||
delete(pool.peers[peerID].blocks, height)
|
||||
}
|
||||
|
||||
func (pool *BlockPool) removePeer(peerID p2p.ID) {
|
||||
for _, requester := range pool.requesters {
|
||||
if requester.getPeerID() == peerID {
|
||||
requester.redo(peerID)
|
||||
}
|
||||
}
|
||||
|
||||
peer, ok := pool.peers[peerID]
|
||||
if ok {
|
||||
if peer.timeout != nil {
|
||||
peer.timeout.Stop()
|
||||
}
|
||||
|
||||
delete(pool.peers, peerID)
|
||||
|
||||
// Find a new peer with the biggest height and update maxPeerHeight if the
|
||||
// peer's height was the biggest.
|
||||
if peer.height == pool.maxPeerHeight {
|
||||
pool.updateMaxPeerHeight()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If no peers are left, maxPeerHeight is set to 0.
|
||||
func (pool *BlockPool) updateMaxPeerHeight() {
|
||||
// Updates the pool's max height. If no peers are left maxPeerHeight is set to 0.
|
||||
func (pool *blockPool) updateMaxPeerHeight() {
|
||||
var max int64
|
||||
for _, peer := range pool.peers {
|
||||
if peer.height > max {
|
||||
@ -335,296 +98,243 @@ func (pool *BlockPool) updateMaxPeerHeight() {
|
||||
pool.maxPeerHeight = max
|
||||
}
|
||||
|
||||
// Pick an available peer with at least the given minHeight.
|
||||
// If no peers are available, returns nil.
|
||||
func (pool *BlockPool) pickIncrAvailablePeer(minHeight int64) *bpPeer {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
// Adds a new peer or updates an existing peer with a new height.
|
||||
// If the peer is too short it is removed.
|
||||
func (pool *blockPool) updatePeer(peerID p2p.ID, height int64) error {
|
||||
pool.logger.Debug("updatePeer", "peerID", peerID, "height", height)
|
||||
peer := pool.peers[peerID]
|
||||
|
||||
if height < pool.height {
|
||||
pool.logger.Info("Peer height too small", "peer", peerID, "height", height, "fsm_height", pool.height)
|
||||
|
||||
// Don't add or update a peer that is not useful.
|
||||
if peer != nil {
|
||||
pool.logger.Info("remove short peer", "peer", peerID, "height", height, "fsm_height", pool.height)
|
||||
pool.removePeer(peerID, errPeerTooShort)
|
||||
}
|
||||
return errPeerTooShort
|
||||
}
|
||||
|
||||
if peer == nil {
|
||||
// Add new peer.
|
||||
peer = newBPPeer(peerID, height, pool.toBcR.sendPeerError)
|
||||
peer.setLogger(pool.logger.With("peer", peerID))
|
||||
pool.peers[peerID] = peer
|
||||
} else {
|
||||
// Update existing peer.
|
||||
// Remove any requests made for heights in (height, peer.height].
|
||||
for h, block := range pool.peers[peerID].blocks {
|
||||
if h <= height {
|
||||
continue
|
||||
}
|
||||
// Reschedule the requests for all blocks waiting for the peer, or received and not processed yet.
|
||||
if block == nil {
|
||||
// Since block was not yet received it is counted in numPending, decrement.
|
||||
pool.numPending--
|
||||
pool.peers[peerID].numPending--
|
||||
}
|
||||
pool.rescheduleRequest(peerID, h)
|
||||
}
|
||||
peer.height = height
|
||||
}
|
||||
|
||||
pool.updateMaxPeerHeight()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stops the peer timer and deletes the peer. Recomputes the max peer height.
|
||||
func (pool *blockPool) deletePeer(peerID p2p.ID) {
|
||||
if p, ok := pool.peers[peerID]; ok {
|
||||
if p.timeout != nil {
|
||||
p.timeout.Stop()
|
||||
}
|
||||
delete(pool.peers, peerID)
|
||||
|
||||
if p.height == pool.maxPeerHeight {
|
||||
pool.updateMaxPeerHeight()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Removes any blocks and requests associated with the peer and deletes the peer.
|
||||
// Also triggers new requests if blocks have been removed.
|
||||
func (pool *blockPool) removePeer(peerID p2p.ID, err error) {
|
||||
pool.logger.Debug("removing peer", "peerID", peerID)
|
||||
|
||||
peer := pool.peers[peerID]
|
||||
if peer == nil {
|
||||
return
|
||||
}
|
||||
// Reschedule the requests for all blocks waiting for the peer, or received and not processed yet.
|
||||
for h, block := range pool.peers[peerID].blocks {
|
||||
if block == nil {
|
||||
pool.numPending--
|
||||
}
|
||||
pool.rescheduleRequest(peerID, h)
|
||||
}
|
||||
pool.deletePeer(peerID)
|
||||
|
||||
}
|
||||
|
||||
// Called every time FSM advances its height.
|
||||
func (pool *blockPool) removeShortPeers() {
|
||||
for _, peer := range pool.peers {
|
||||
if peer.didTimeout {
|
||||
pool.removePeer(peer.id)
|
||||
if peer.height < pool.height {
|
||||
pool.removePeer(peer.id, nil)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Validates that the block comes from the peer it was expected from and stores it in the 'blocks' map.
|
||||
func (pool *blockPool) addBlock(peerID p2p.ID, block *types.Block, blockSize int) error {
|
||||
if _, ok := pool.peers[peerID]; !ok {
|
||||
pool.logger.Error("peer doesn't exist", "peer", peerID, "block_receieved", block.Height)
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
b, ok := pool.peers[peerID].blocks[block.Height]
|
||||
if !ok {
|
||||
pool.logger.Error("peer sent us a block we didn't expect", "peer", peerID, "blockHeight", block.Height)
|
||||
if expPeerID, pok := pool.blocks[block.Height]; pok {
|
||||
pool.logger.Error("expected this block from peer", "peer", expPeerID)
|
||||
}
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
if b != nil {
|
||||
pool.logger.Error("already have a block for height", "height", block.Height)
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
|
||||
pool.peers[peerID].blocks[block.Height] = block
|
||||
pool.blocks[block.Height] = peerID
|
||||
pool.numPending--
|
||||
pool.peers[peerID].decrPending(blockSize)
|
||||
pool.logger.Debug("added new block", "height", block.Height, "from_peer", peerID, "total", len(pool.blocks))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pool *blockPool) getBlockAndPeerAtHeight(height int64) (bData *blockData, err error) {
|
||||
peerID := pool.blocks[height]
|
||||
peer := pool.peers[peerID]
|
||||
if peer == nil {
|
||||
return &blockData{}, errMissingBlocks
|
||||
}
|
||||
|
||||
block, ok := peer.blocks[height]
|
||||
if !ok || block == nil {
|
||||
return &blockData{}, errMissingBlocks
|
||||
}
|
||||
|
||||
return &blockData{peer: peer, block: block}, nil
|
||||
|
||||
}
|
||||
|
||||
func (pool *blockPool) getNextTwoBlocks() (first, second *blockData, err error) {
|
||||
first, err = pool.getBlockAndPeerAtHeight(pool.height)
|
||||
second, err2 := pool.getBlockAndPeerAtHeight(pool.height + 1)
|
||||
if err == nil {
|
||||
err = err2
|
||||
}
|
||||
|
||||
if err == errMissingBlocks {
|
||||
// We need both to sync the first block.
|
||||
pool.logger.Error("missing blocks at height and/ or height+1", "height", pool.height)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Remove peers that sent us the first two blocks, blocks will also be removed by removePeer().
|
||||
func (pool *blockPool) invalidateFirstTwoBlocks(err error) {
|
||||
first, err1 := pool.getBlockAndPeerAtHeight(pool.height)
|
||||
second, err2 := pool.getBlockAndPeerAtHeight(pool.height + 1)
|
||||
|
||||
if err1 == nil {
|
||||
pool.removePeer(first.peer.id, err)
|
||||
}
|
||||
if err2 == nil {
|
||||
pool.removePeer(second.peer.id, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) processedCurrentHeightBlock() {
|
||||
peerID, peerOk := pool.blocks[pool.height]
|
||||
if peerOk {
|
||||
delete(pool.peers[peerID].blocks, pool.height)
|
||||
}
|
||||
delete(pool.blocks, pool.height)
|
||||
pool.logger.Debug("processed and removed block at height", "height", pool.height)
|
||||
pool.height++
|
||||
pool.removeShortPeers()
|
||||
}
|
||||
|
||||
func (pool *blockPool) removeBadPeers() {
|
||||
pool.removeShortPeers()
|
||||
for _, peer := range pool.peers {
|
||||
if err := peer.isGood(); err != nil {
|
||||
pool.removePeer(peer.id, err)
|
||||
if err == errSlowPeer {
|
||||
peer.errFunc(errSlowPeer, peer.id)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) makeRequestBatch(maxNumPendingRequests int32) []int {
|
||||
pool.removeBadPeers()
|
||||
// If running low on planned requests, make more.
|
||||
numNeeded := int32(cmn.MinInt(int(maxNumPendingRequests), len(pool.peers)*int(maxRequestsPerPeer))) - pool.numPending
|
||||
for int32(len(pool.requests)) < numNeeded {
|
||||
if pool.nextRequestHeight > pool.maxPeerHeight {
|
||||
break
|
||||
}
|
||||
pool.requests[pool.nextRequestHeight] = true
|
||||
pool.nextRequestHeight++
|
||||
}
|
||||
|
||||
heights := make([]int, 0, len(pool.requests))
|
||||
for k := range pool.requests {
|
||||
heights = append(heights, int(k))
|
||||
}
|
||||
sort.Ints(heights)
|
||||
return heights
|
||||
}
|
||||
|
||||
func (pool *blockPool) makeNextRequests(maxNumPendingRequests int32) {
|
||||
heights := pool.makeRequestBatch(maxNumPendingRequests)
|
||||
pool.logger.Debug("makeNextRequests will make following requests", "number", len(heights), "heights", heights)
|
||||
|
||||
for _, height := range heights {
|
||||
h := int64(height)
|
||||
if err := pool.sendRequest(h); err != nil {
|
||||
// Errors from sendRequest() are handled by this function
|
||||
return
|
||||
}
|
||||
delete(pool.requests, h)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) sendRequest(height int64) error {
|
||||
for _, peer := range pool.peers {
|
||||
if peer.numPending >= int32(maxRequestsPerPeer) {
|
||||
continue
|
||||
}
|
||||
if peer.numPending >= maxPendingRequestsPerPeer {
|
||||
if peer.height < height {
|
||||
continue
|
||||
}
|
||||
if peer.height < minHeight {
|
||||
continue
|
||||
pool.logger.Debug("assign request to peer", "peer", peer.id, "height", height)
|
||||
if err := pool.toBcR.sendBlockRequest(peer.id, height); err == errNilPeerForBlockRequest {
|
||||
pool.removePeer(peer.id, err)
|
||||
pool.toBcR.sendPeerError(err, peer.id)
|
||||
}
|
||||
|
||||
pool.blocks[height] = peer.id
|
||||
pool.numPending++
|
||||
|
||||
peer.blocks[height] = nil
|
||||
peer.incrPending()
|
||||
return peer
|
||||
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pool *BlockPool) makeNextRequester() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
nextHeight := pool.height + pool.requestersLen()
|
||||
if nextHeight > pool.maxPeerHeight {
|
||||
return
|
||||
}
|
||||
|
||||
request := newBPRequester(pool, nextHeight)
|
||||
|
||||
pool.requesters[nextHeight] = request
|
||||
atomic.AddInt32(&pool.numPending, 1)
|
||||
|
||||
err := request.Start()
|
||||
if err != nil {
|
||||
request.Logger.Error("Error starting request", "err", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) requestersLen() int64 {
|
||||
return int64(len(pool.requesters))
|
||||
}
|
||||
|
||||
func (pool *BlockPool) sendRequest(height int64, peerID p2p.ID) {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
pool.requestsCh <- BlockRequest{height, peerID}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) sendError(err error, peerID p2p.ID) {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
pool.errorsCh <- peerError{err, peerID}
|
||||
}
|
||||
|
||||
// for debugging purposes
|
||||
//nolint:unused
|
||||
func (pool *BlockPool) debug() string {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
str := ""
|
||||
nextHeight := pool.height + pool.requestersLen()
|
||||
for h := pool.height; h < nextHeight; h++ {
|
||||
if pool.requesters[h] == nil {
|
||||
str += fmt.Sprintf("H(%v):X ", h)
|
||||
} else {
|
||||
str += fmt.Sprintf("H(%v):", h)
|
||||
str += fmt.Sprintf("B?(%v) ", pool.requesters[h].block != nil)
|
||||
}
|
||||
}
|
||||
return str
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bpPeer struct {
|
||||
pool *BlockPool
|
||||
id p2p.ID
|
||||
recvMonitor *flow.Monitor
|
||||
|
||||
height int64
|
||||
numPending int32
|
||||
timeout *time.Timer
|
||||
didTimeout bool
|
||||
|
||||
logger log.Logger
|
||||
}
|
||||
|
||||
func newBPPeer(pool *BlockPool, peerID p2p.ID, height int64) *bpPeer {
|
||||
peer := &bpPeer{
|
||||
pool: pool,
|
||||
id: peerID,
|
||||
height: height,
|
||||
numPending: 0,
|
||||
logger: log.NewNopLogger(),
|
||||
}
|
||||
return peer
|
||||
}
|
||||
|
||||
func (peer *bpPeer) setLogger(l log.Logger) {
|
||||
peer.logger = l
|
||||
}
|
||||
|
||||
func (peer *bpPeer) resetMonitor() {
|
||||
peer.recvMonitor = flow.New(time.Second, time.Second*40)
|
||||
initialValue := float64(minRecvRate) * math.E
|
||||
peer.recvMonitor.SetREMA(initialValue)
|
||||
}
|
||||
|
||||
func (peer *bpPeer) resetTimeout() {
|
||||
if peer.timeout == nil {
|
||||
peer.timeout = time.AfterFunc(peerTimeout, peer.onTimeout)
|
||||
} else {
|
||||
peer.timeout.Reset(peerTimeout)
|
||||
}
|
||||
}
|
||||
|
||||
func (peer *bpPeer) incrPending() {
|
||||
if peer.numPending == 0 {
|
||||
peer.resetMonitor()
|
||||
peer.resetTimeout()
|
||||
}
|
||||
peer.numPending++
|
||||
}
|
||||
|
||||
func (peer *bpPeer) decrPending(recvSize int) {
|
||||
peer.numPending--
|
||||
if peer.numPending == 0 {
|
||||
peer.timeout.Stop()
|
||||
} else {
|
||||
peer.recvMonitor.Update(recvSize)
|
||||
peer.resetTimeout()
|
||||
}
|
||||
}
|
||||
|
||||
func (peer *bpPeer) onTimeout() {
|
||||
peer.pool.mtx.Lock()
|
||||
defer peer.pool.mtx.Unlock()
|
||||
|
||||
err := errors.New("peer did not send us anything")
|
||||
peer.pool.sendError(err, peer.id)
|
||||
peer.logger.Error("SendTimeout", "reason", err, "timeout", peerTimeout)
|
||||
peer.didTimeout = true
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bpRequester struct {
|
||||
cmn.BaseService
|
||||
pool *BlockPool
|
||||
height int64
|
||||
gotBlockCh chan struct{}
|
||||
redoCh chan p2p.ID //redo may send multitime, add peerId to identify repeat
|
||||
|
||||
mtx sync.Mutex
|
||||
peerID p2p.ID
|
||||
block *types.Block
|
||||
}
|
||||
|
||||
func newBPRequester(pool *BlockPool, height int64) *bpRequester {
|
||||
bpr := &bpRequester{
|
||||
pool: pool,
|
||||
height: height,
|
||||
gotBlockCh: make(chan struct{}, 1),
|
||||
redoCh: make(chan p2p.ID, 1),
|
||||
|
||||
peerID: "",
|
||||
block: nil,
|
||||
}
|
||||
bpr.BaseService = *cmn.NewBaseService(nil, "bpRequester", bpr)
|
||||
return bpr
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) OnStart() error {
|
||||
go bpr.requestRoutine()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Returns true if the peer matches and block doesn't already exist.
|
||||
func (bpr *bpRequester) setBlock(block *types.Block, peerID p2p.ID) bool {
|
||||
bpr.mtx.Lock()
|
||||
if bpr.block != nil || bpr.peerID != peerID {
|
||||
bpr.mtx.Unlock()
|
||||
return false
|
||||
}
|
||||
bpr.block = block
|
||||
bpr.mtx.Unlock()
|
||||
|
||||
select {
|
||||
case bpr.gotBlockCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) getBlock() *types.Block {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
return bpr.block
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) getPeerID() p2p.ID {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
return bpr.peerID
|
||||
}
|
||||
|
||||
// This is called from the requestRoutine, upon redo().
|
||||
func (bpr *bpRequester) reset() {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
|
||||
if bpr.block != nil {
|
||||
atomic.AddInt32(&bpr.pool.numPending, 1)
|
||||
}
|
||||
|
||||
bpr.peerID = ""
|
||||
bpr.block = nil
|
||||
}
|
||||
|
||||
// Tells bpRequester to pick another peer and try again.
|
||||
// NOTE: Nonblocking, and does nothing if another redo
|
||||
// was already requested.
|
||||
func (bpr *bpRequester) redo(peerId p2p.ID) {
|
||||
select {
|
||||
case bpr.redoCh <- peerId:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// Responsible for making more requests as necessary
|
||||
// Returns only when a block is found (e.g. AddBlock() is called)
|
||||
func (bpr *bpRequester) requestRoutine() {
|
||||
OUTER_LOOP:
|
||||
for {
|
||||
// Pick a peer to send request to.
|
||||
var peer *bpPeer
|
||||
PICK_PEER_LOOP:
|
||||
for {
|
||||
if !bpr.IsRunning() || !bpr.pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
peer = bpr.pool.pickIncrAvailablePeer(bpr.height)
|
||||
if peer == nil {
|
||||
//log.Info("No peers available", "height", height)
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
continue PICK_PEER_LOOP
|
||||
}
|
||||
break PICK_PEER_LOOP
|
||||
}
|
||||
bpr.mtx.Lock()
|
||||
bpr.peerID = peer.id
|
||||
bpr.mtx.Unlock()
|
||||
|
||||
// Send request and wait.
|
||||
bpr.pool.sendRequest(bpr.height, peer.id)
|
||||
WAIT_LOOP:
|
||||
for {
|
||||
select {
|
||||
case <-bpr.pool.Quit():
|
||||
bpr.Stop()
|
||||
return
|
||||
case <-bpr.Quit():
|
||||
return
|
||||
case peerID := <-bpr.redoCh:
|
||||
if peerID == bpr.peerID {
|
||||
bpr.reset()
|
||||
continue OUTER_LOOP
|
||||
} else {
|
||||
continue WAIT_LOOP
|
||||
}
|
||||
case <-bpr.gotBlockCh:
|
||||
// We got a block!
|
||||
// Continue the for-loop and wait til Quit.
|
||||
continue WAIT_LOOP
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type BlockRequest struct {
|
||||
Height int64
|
||||
PeerID p2p.ID
|
||||
pool.logger.Error("could not find peer to send request for block at height", "height", height)
|
||||
return errNoPeerFoundForHeight
|
||||
}
|
||||
|
@ -1,222 +1,575 @@
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func init() {
|
||||
peerTimeout = 2 * time.Second
|
||||
}
|
||||
|
||||
type testPeer struct {
|
||||
id p2p.ID
|
||||
height int64
|
||||
inputChan chan inputData //make sure each peer's data is sequential
|
||||
id p2p.ID
|
||||
height int64
|
||||
}
|
||||
|
||||
type inputData struct {
|
||||
t *testing.T
|
||||
pool *BlockPool
|
||||
request BlockRequest
|
||||
type testPeerResult struct {
|
||||
id p2p.ID
|
||||
height int64
|
||||
numPending int32
|
||||
blocks map[int64]*types.Block
|
||||
}
|
||||
|
||||
func (p testPeer) runInputRoutine() {
|
||||
go func() {
|
||||
for input := range p.inputChan {
|
||||
p.simulateInput(input)
|
||||
type testBcR struct {
|
||||
logger log.Logger
|
||||
}
|
||||
|
||||
type testValues struct {
|
||||
numRequestsSent int32
|
||||
}
|
||||
|
||||
var testResults testValues
|
||||
|
||||
func resetPoolTestResults() {
|
||||
testResults.numRequestsSent = 0
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendPeerError(err error, peerID p2p.ID) {
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendStatusRequest() {
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendBlockRequest(peerID p2p.ID, height int64) error {
|
||||
testResults.numRequestsSent++
|
||||
return nil
|
||||
}
|
||||
|
||||
func (testR *testBcR) resetStateTimer(name string, timer *time.Timer, timeout time.Duration, f func()) {
|
||||
}
|
||||
|
||||
func (testR *testBcR) switchToConsensus() {
|
||||
}
|
||||
|
||||
func newTestBcR() *testBcR {
|
||||
testBcR := &testBcR{logger: log.TestingLogger()}
|
||||
return testBcR
|
||||
}
|
||||
|
||||
type tPBlocks struct {
|
||||
id p2p.ID
|
||||
create bool
|
||||
}
|
||||
|
||||
func makeBlockPool(bcr *testBcR, height int64, peers []bpPeer, blocks map[int64]tPBlocks) *blockPool {
|
||||
bPool := newBlockPool(height, bcr)
|
||||
txs := []types.Tx{types.Tx("foo"), types.Tx("bar")}
|
||||
|
||||
var maxH int64
|
||||
for _, p := range peers {
|
||||
if p.height > maxH {
|
||||
maxH = p.height
|
||||
}
|
||||
}()
|
||||
}
|
||||
bPool.peers[p.id] = newBPPeer(p.id, p.height, bcr.sendPeerError)
|
||||
bPool.peers[p.id].setLogger(bcr.logger)
|
||||
|
||||
// Request desired, pretend like we got the block immediately.
|
||||
func (p testPeer) simulateInput(input inputData) {
|
||||
block := &types.Block{Header: types.Header{Height: input.request.Height}}
|
||||
input.pool.AddBlock(input.request.PeerID, block, 123)
|
||||
input.t.Logf("Added block from peer %v (height: %v)", input.request.PeerID, input.request.Height)
|
||||
}
|
||||
|
||||
type testPeers map[p2p.ID]testPeer
|
||||
|
||||
func (ps testPeers) start() {
|
||||
for _, v := range ps {
|
||||
v.runInputRoutine()
|
||||
}
|
||||
}
|
||||
|
||||
func (ps testPeers) stop() {
|
||||
for _, v := range ps {
|
||||
close(v.inputChan)
|
||||
}
|
||||
}
|
||||
|
||||
func makePeers(numPeers int, minHeight, maxHeight int64) testPeers {
|
||||
peers := make(testPeers, numPeers)
|
||||
for i := 0; i < numPeers; i++ {
|
||||
peerID := p2p.ID(cmn.RandStr(12))
|
||||
height := minHeight + cmn.RandInt63n(maxHeight-minHeight)
|
||||
peers[peerID] = testPeer{peerID, height, make(chan inputData, 10)}
|
||||
}
|
||||
return peers
|
||||
}
|
||||
|
||||
func TestBlockPoolBasic(t *testing.T) {
|
||||
start := int64(42)
|
||||
peers := makePeers(10, start+1, 1000)
|
||||
errorsCh := make(chan peerError, 1000)
|
||||
requestsCh := make(chan BlockRequest, 1000)
|
||||
pool := NewBlockPool(start, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
|
||||
err := pool.Start()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
defer pool.Stop()
|
||||
|
||||
peers.start()
|
||||
defer peers.stop()
|
||||
|
||||
// Introduce each peer.
|
||||
go func() {
|
||||
for _, peer := range peers {
|
||||
pool.SetPeerHeight(peer.id, peer.height)
|
||||
bPool.maxPeerHeight = maxH
|
||||
for h, p := range blocks {
|
||||
bPool.blocks[h] = p.id
|
||||
bPool.peers[p.id].blocks[h] = nil
|
||||
if p.create {
|
||||
bPool.peers[p.id].blocks[h] = types.MakeBlock(int64(h), txs, nil, nil)
|
||||
} else {
|
||||
bPool.peers[p.id].incrPending()
|
||||
}
|
||||
}()
|
||||
}
|
||||
bPool.setLogger(bcr.logger)
|
||||
return bPool
|
||||
}
|
||||
|
||||
// Start a goroutine to pull blocks
|
||||
go func() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
func poolCopy(pool *blockPool) *blockPool {
|
||||
return &blockPool{
|
||||
peers: peersCopy(pool.peers),
|
||||
logger: pool.logger,
|
||||
blocks: pool.blocks,
|
||||
requests: pool.requests,
|
||||
height: pool.height,
|
||||
nextRequestHeight: pool.height,
|
||||
maxPeerHeight: pool.maxPeerHeight,
|
||||
toBcR: pool.toBcR,
|
||||
}
|
||||
}
|
||||
|
||||
func peersCopy(peers map[p2p.ID]*bpPeer) map[p2p.ID]*bpPeer {
|
||||
peerCopy := make(map[p2p.ID]*bpPeer)
|
||||
for _, p := range peers {
|
||||
peerCopy[p.id] = newBPPeer(p.id, p.height, p.errFunc)
|
||||
}
|
||||
return peerCopy
|
||||
}
|
||||
|
||||
func TestBlockPoolUpdatePeerNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args testPeer
|
||||
errWanted error
|
||||
addWanted bool
|
||||
delWanted bool
|
||||
maxHeightWanted int64
|
||||
}{
|
||||
|
||||
{
|
||||
name: "add a first short peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 50},
|
||||
errWanted: errPeerTooShort,
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
{
|
||||
name: "add a first good peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 101},
|
||||
addWanted: true,
|
||||
maxHeightWanted: int64(101),
|
||||
},
|
||||
{
|
||||
name: "increase the height of P1 from 120 to 123",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 123},
|
||||
maxHeightWanted: int64(123),
|
||||
},
|
||||
{
|
||||
name: "decrease the height of P1 from 120 to 110",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 110},
|
||||
maxHeightWanted: int64(110),
|
||||
},
|
||||
{
|
||||
name: "decrease the height of P1 from 120 to 90",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 90},
|
||||
delWanted: true,
|
||||
errWanted: errPeerTooShort,
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
|
||||
beforePool := poolCopy(pool)
|
||||
err := pool.updatePeer(tt.args.id, tt.args.height)
|
||||
if err != tt.errWanted {
|
||||
t.Errorf("blockPool.updatePeer() error = %v, wantErr %v", err, tt.errWanted)
|
||||
}
|
||||
|
||||
if tt.errWanted != nil {
|
||||
// error case
|
||||
if tt.delWanted {
|
||||
assert.Equal(t, len(beforePool.peers)-1, len(pool.peers))
|
||||
return
|
||||
}
|
||||
assert.Equal(t, beforePool, pool)
|
||||
return
|
||||
}
|
||||
first, second := pool.PeekTwoBlocks()
|
||||
if first != nil && second != nil {
|
||||
pool.PopRequest()
|
||||
|
||||
if tt.addWanted {
|
||||
// add case only
|
||||
assert.Equal(t, len(beforePool.peers)+1, len(pool.peers))
|
||||
} else {
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Pull from channels
|
||||
for {
|
||||
select {
|
||||
case err := <-errorsCh:
|
||||
t.Error(err)
|
||||
case request := <-requestsCh:
|
||||
t.Logf("Pulled new BlockRequest %v", request)
|
||||
if request.Height == 300 {
|
||||
return // Done!
|
||||
// update case only
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
}
|
||||
|
||||
peers[request.PeerID].inputChan <- inputData{t, pool, request}
|
||||
}
|
||||
// both add and update
|
||||
assert.Equal(t, pool.peers[tt.args.id].height, tt.args.height)
|
||||
assert.Equal(t, tt.maxHeightWanted, pool.maxPeerHeight)
|
||||
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolTimeout(t *testing.T) {
|
||||
start := int64(42)
|
||||
peers := makePeers(10, start+1, 1000)
|
||||
errorsCh := make(chan peerError, 1000)
|
||||
requestsCh := make(chan BlockRequest, 1000)
|
||||
pool := NewBlockPool(start, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
err := pool.Start()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
defer pool.Stop()
|
||||
func TestBlockPoolRemovePeerNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
for _, peer := range peers {
|
||||
t.Logf("Peer %v", peer.id)
|
||||
type args struct {
|
||||
peerID p2p.ID
|
||||
err error
|
||||
}
|
||||
|
||||
// Introduce each peer.
|
||||
go func() {
|
||||
for _, peer := range peers {
|
||||
pool.SetPeerHeight(peer.id, peer.height)
|
||||
}
|
||||
}()
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args args
|
||||
maxHeightWanted int64
|
||||
}{
|
||||
{
|
||||
name: "attempt to delete non-existing peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P99", nil},
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "delete the only peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P1", nil},
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
{
|
||||
name: "delete the shortest of two peers",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P1", nil},
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "delete the tallest of two peers",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P2", nil},
|
||||
maxHeightWanted: int64(100),
|
||||
},
|
||||
}
|
||||
|
||||
// Start a goroutine to pull blocks
|
||||
go func() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
tt.pool.removePeer(tt.args.peerID, tt.args.err)
|
||||
assert.Equal(t, tt.maxHeightWanted, tt.pool.maxPeerHeight)
|
||||
_, ok := tt.pool.peers[tt.args.peerID]
|
||||
assert.False(t, ok)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolRemoveShortPeersNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
maxHeightWanted int64
|
||||
noChange bool
|
||||
}{
|
||||
{
|
||||
name: "no short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 110}, {id: "P3", height: 120}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(120),
|
||||
noChange: true,
|
||||
},
|
||||
{
|
||||
name: "one short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 90}, {id: "P3", height: 120}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "all short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 90}, {id: "P2", height: 91}, {id: "P3", height: 92}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
beforePool := poolCopy(pool)
|
||||
|
||||
pool.removeShortPeers()
|
||||
assert.Equal(t, tt.maxHeightWanted, pool.maxPeerHeight)
|
||||
if tt.noChange {
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
return
|
||||
}
|
||||
first, second := pool.PeekTwoBlocks()
|
||||
if first != nil && second != nil {
|
||||
pool.PopRequest()
|
||||
} else {
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Pull from channels
|
||||
counter := 0
|
||||
timedOut := map[p2p.ID]struct{}{}
|
||||
for {
|
||||
select {
|
||||
case err := <-errorsCh:
|
||||
t.Log(err)
|
||||
// consider error to be always timeout here
|
||||
if _, ok := timedOut[err.peerID]; !ok {
|
||||
counter++
|
||||
if counter == len(peers) {
|
||||
return // Done!
|
||||
for _, peer := range tt.pool.peers {
|
||||
bPeer, bok := beforePool.peers[peer.id]
|
||||
if bok && bPeer.height < beforePool.height {
|
||||
_, ok := pool.peers[peer.id]
|
||||
assert.False(t, ok)
|
||||
}
|
||||
}
|
||||
case request := <-requestsCh:
|
||||
t.Logf("Pulled new BlockRequest %+v", request)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolRemovePeer(t *testing.T) {
|
||||
peers := make(testPeers, 10)
|
||||
for i := 0; i < 10; i++ {
|
||||
peerID := p2p.ID(fmt.Sprintf("%d", i+1))
|
||||
height := int64(i + 1)
|
||||
peers[peerID] = testPeer{peerID, height, make(chan inputData)}
|
||||
}
|
||||
requestsCh := make(chan BlockRequest)
|
||||
errorsCh := make(chan peerError)
|
||||
|
||||
pool := NewBlockPool(1, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
err := pool.Start()
|
||||
require.NoError(t, err)
|
||||
defer pool.Stop()
|
||||
|
||||
// add peers
|
||||
for peerID, peer := range peers {
|
||||
pool.SetPeerHeight(peerID, peer.height)
|
||||
}
|
||||
assert.EqualValues(t, 10, pool.MaxPeerHeight())
|
||||
|
||||
// remove not-existing peer
|
||||
assert.NotPanics(t, func() { pool.RemovePeer(p2p.ID("Superman")) })
|
||||
|
||||
// remove peer with biggest height
|
||||
pool.RemovePeer(p2p.ID("10"))
|
||||
assert.EqualValues(t, 9, pool.MaxPeerHeight())
|
||||
|
||||
// remove all peers
|
||||
for peerID := range peers {
|
||||
pool.RemovePeer(peerID)
|
||||
func TestBlockPoolSendRequestBatch(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
maxRequestsPerPeer int32
|
||||
expRequests map[int64]bool
|
||||
expPeerResults []testPeerResult
|
||||
expNumPending int32
|
||||
}{
|
||||
{
|
||||
name: "one peer - send up to maxRequestsPerPeer block requests",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}}, map[int64]tPBlocks{}),
|
||||
maxRequestsPerPeer: 2,
|
||||
expRequests: map[int64]bool{10: true, 11: true},
|
||||
expPeerResults: []testPeerResult{{id: "P1", height: 100, numPending: 2, blocks: map[int64]*types.Block{10: nil, 11: nil}}},
|
||||
expNumPending: 2,
|
||||
},
|
||||
{
|
||||
name: "n peers - send n*maxRequestsPerPeer block requests",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}}, map[int64]tPBlocks{}),
|
||||
maxRequestsPerPeer: 2,
|
||||
expRequests: map[int64]bool{10: true, 11: true},
|
||||
expPeerResults: []testPeerResult{
|
||||
{id: "P1", height: 100, numPending: 2, blocks: map[int64]*types.Block{10: nil, 11: nil}},
|
||||
{id: "P2", height: 100, numPending: 2, blocks: map[int64]*types.Block{12: nil, 13: nil}}},
|
||||
expNumPending: 4,
|
||||
},
|
||||
}
|
||||
|
||||
assert.EqualValues(t, 0, pool.MaxPeerHeight())
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
resetPoolTestResults()
|
||||
|
||||
pool := tt.pool
|
||||
|
||||
maxRequestsPerPeer = int32(tt.maxRequestsPerPeer)
|
||||
pool.makeNextRequests(10)
|
||||
assert.Equal(t, tt.expNumPending, pool.numPending)
|
||||
assert.Equal(t, testResults.numRequestsSent, maxRequestsPerPeer*int32(len(pool.peers)))
|
||||
|
||||
for _, tPeer := range tt.expPeerResults {
|
||||
peer := pool.peers[tPeer.id]
|
||||
assert.NotNil(t, peer)
|
||||
assert.Equal(t, tPeer.numPending, peer.numPending)
|
||||
}
|
||||
assert.Equal(t, testResults.numRequestsSent, maxRequestsPerPeer*int32(len(pool.peers)))
|
||||
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolAddBlock(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
txs := []types.Tx{types.Tx("foo"), types.Tx("bar")}
|
||||
|
||||
type args struct {
|
||||
peerID p2p.ID
|
||||
block *types.Block
|
||||
blockSize int
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args args
|
||||
wantErr bool
|
||||
}{
|
||||
{name: "block from unknown peer",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}}, map[int64]tPBlocks{}),
|
||||
args: args{
|
||||
peerID: "P2",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "unexpected block 11 from known peer - waiting for 10",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", false}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(11), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "unexpected block 10 from known peer - already have 10",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "expected block from known peer",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", false}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
if err := pool.addBlock(tt.args.peerID, tt.args.block, tt.args.blockSize); (err != nil) != tt.wantErr {
|
||||
t.Errorf("blockPool.addBlock() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolGetNextTwoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
firstWanted int64
|
||||
secondWanted int64
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "both blocks missing",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 0,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "second block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 18: {"P2", true}}),
|
||||
firstWanted: 15,
|
||||
secondWanted: 0,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "first block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{18: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 16,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "both blocks present",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}, 11: {"P2", true}}),
|
||||
firstWanted: 10,
|
||||
secondWanted: 11,
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
gotFirst, gotSecond, err := pool.getNextTwoBlocks()
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.firstWanted != 0 {
|
||||
peer := pool.blocks[tt.firstWanted]
|
||||
block := pool.peers[peer].blocks[tt.firstWanted]
|
||||
if !reflect.DeepEqual(gotFirst.block, block) {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() gotFirst = %v, want %v", gotFirst.block.Height, tt.firstWanted)
|
||||
}
|
||||
}
|
||||
if tt.secondWanted != 0 {
|
||||
peer := pool.blocks[tt.secondWanted]
|
||||
block := pool.peers[peer].blocks[tt.secondWanted]
|
||||
if !reflect.DeepEqual(gotSecond.block, block) {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() gotFirst = %v, want %v", gotSecond.block.Height, tt.secondWanted)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolInvalidateFirstTwoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
firstWanted int64
|
||||
secondWanted int64
|
||||
wantChange bool
|
||||
}{
|
||||
{
|
||||
name: "both blocks missing",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 0,
|
||||
wantChange: false,
|
||||
},
|
||||
{
|
||||
name: "second block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 18: {"P2", true}}),
|
||||
firstWanted: 15,
|
||||
secondWanted: 0,
|
||||
wantChange: true,
|
||||
},
|
||||
{
|
||||
name: "first block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{18: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 16,
|
||||
wantChange: true,
|
||||
},
|
||||
{
|
||||
name: "both blocks present",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}, 11: {"P2", true}}),
|
||||
firstWanted: 10,
|
||||
secondWanted: 11,
|
||||
wantChange: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
gotFirst, gotSecond, _ := pool.getNextTwoBlocks()
|
||||
|
||||
beforePool := poolCopy(pool)
|
||||
pool.invalidateFirstTwoBlocks(errNoPeerResponse)
|
||||
if !tt.wantChange {
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
return
|
||||
}
|
||||
if tt.firstWanted != 0 {
|
||||
_, ok := pool.peers[gotFirst.peer.id]
|
||||
assert.False(t, ok)
|
||||
_, ok = pool.blocks[tt.firstWanted]
|
||||
assert.False(t, ok)
|
||||
assert.True(t, pool.requests[tt.firstWanted])
|
||||
}
|
||||
if tt.secondWanted != 0 {
|
||||
_, ok := pool.peers[gotSecond.peer.id]
|
||||
assert.False(t, ok)
|
||||
_, ok = pool.blocks[tt.secondWanted]
|
||||
assert.False(t, ok)
|
||||
assert.True(t, pool.requests[tt.secondWanted])
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -11,14 +11,15 @@ import (
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
const (
|
||||
// BlockchainChannel is a channel for blocks and status updates (`BlockStore` height)
|
||||
BlockchainChannel = byte(0x40)
|
||||
|
||||
trySyncIntervalMS = 10
|
||||
trySendIntervalMS = 10
|
||||
|
||||
// stop syncing when last block's time is
|
||||
// within this much of the system time.
|
||||
@ -37,6 +38,11 @@ const (
|
||||
bcBlockResponseMessageFieldKeySize
|
||||
)
|
||||
|
||||
var (
|
||||
maxRequestsPerPeer int32 = 20
|
||||
maxNumPendingRequests int32 = 600
|
||||
)
|
||||
|
||||
type consensusReactor interface {
|
||||
// for when we switch from blockchain reactor and fast sync to
|
||||
// the consensus machine
|
||||
@ -58,14 +64,37 @@ type BlockchainReactor struct {
|
||||
|
||||
// immutable
|
||||
initialState sm.State
|
||||
state sm.State
|
||||
|
||||
blockExec *sm.BlockExecutor
|
||||
store *BlockStore
|
||||
pool *BlockPool
|
||||
fastSync bool
|
||||
|
||||
requestsCh <-chan BlockRequest
|
||||
errorsCh <-chan peerError
|
||||
fastSync bool
|
||||
|
||||
fsm *bReactorFSM
|
||||
blocksSynced int
|
||||
|
||||
// Receive goroutine forwards messages to this channel to be processed in the context of the poolRoutine.
|
||||
messagesForFSMCh chan bReactorMessageData
|
||||
|
||||
// Switch goroutine may send RemovePeer to the blockchain reactor. This is an error message that is relayed
|
||||
// to this channel to be processed in the context of the poolRoutine.
|
||||
errorsForFSMCh chan bReactorMessageData
|
||||
|
||||
// This channel is used by the FSM and indirectly the block pool to report errors to the blockchain reactor and
|
||||
// the switch.
|
||||
errorsFromFSMCh chan peerError
|
||||
}
|
||||
|
||||
type BlockRequest struct {
|
||||
Height int64
|
||||
PeerID p2p.ID
|
||||
}
|
||||
|
||||
// bReactorMessageData structure is used by the reactor when sending messages to the FSM.
|
||||
type bReactorMessageData struct {
|
||||
event bReactorEvent
|
||||
data bReactorEventData
|
||||
}
|
||||
|
||||
// NewBlockchainReactor returns new reactor instance.
|
||||
@ -77,26 +106,23 @@ func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *Bl
|
||||
store.Height()))
|
||||
}
|
||||
|
||||
requestsCh := make(chan BlockRequest, maxTotalRequesters)
|
||||
|
||||
const capacity = 1000 // must be bigger than peers count
|
||||
errorsCh := make(chan peerError, capacity) // so we don't block in #Receive#pool.AddBlock
|
||||
|
||||
pool := NewBlockPool(
|
||||
store.Height()+1,
|
||||
requestsCh,
|
||||
errorsCh,
|
||||
)
|
||||
const capacity = 1000
|
||||
errorsFromFSMCh := make(chan peerError, capacity)
|
||||
messagesForFSMCh := make(chan bReactorMessageData, capacity)
|
||||
errorsForFSMCh := make(chan bReactorMessageData, capacity)
|
||||
|
||||
bcR := &BlockchainReactor{
|
||||
initialState: state,
|
||||
blockExec: blockExec,
|
||||
store: store,
|
||||
pool: pool,
|
||||
fastSync: fastSync,
|
||||
requestsCh: requestsCh,
|
||||
errorsCh: errorsCh,
|
||||
initialState: state,
|
||||
state: state,
|
||||
blockExec: blockExec,
|
||||
fastSync: fastSync,
|
||||
store: store,
|
||||
messagesForFSMCh: messagesForFSMCh,
|
||||
errorsFromFSMCh: errorsFromFSMCh,
|
||||
errorsForFSMCh: errorsForFSMCh,
|
||||
}
|
||||
fsm := NewFSM(store.Height()+1, bcR)
|
||||
bcR.fsm = fsm
|
||||
bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
|
||||
return bcR
|
||||
}
|
||||
@ -104,16 +130,12 @@ func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *Bl
|
||||
// SetLogger implements cmn.Service by setting the logger on reactor and pool.
|
||||
func (bcR *BlockchainReactor) SetLogger(l log.Logger) {
|
||||
bcR.BaseService.Logger = l
|
||||
bcR.pool.Logger = l
|
||||
bcR.fsm.setLogger(l)
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStart() error {
|
||||
if bcR.fastSync {
|
||||
err := bcR.pool.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go bcR.poolRoutine()
|
||||
}
|
||||
return nil
|
||||
@ -121,7 +143,7 @@ func (bcR *BlockchainReactor) OnStart() error {
|
||||
|
||||
// OnStop implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStop() {
|
||||
bcR.pool.Stop()
|
||||
_ = bcR.Stop()
|
||||
}
|
||||
|
||||
// GetChannels implements Reactor
|
||||
@ -147,16 +169,11 @@ func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
|
||||
// bcStatusResponseMessage from the peer and call pool.SetPeerHeight
|
||||
}
|
||||
|
||||
// RemovePeer implements Reactor by removing peer from the pool.
|
||||
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
bcR.pool.RemovePeer(peer.ID())
|
||||
}
|
||||
|
||||
// respondToPeer loads a block and sends it to the requesting peer,
|
||||
// if we have it. Otherwise, we'll respond saying we don't have it.
|
||||
// According to the Tendermint spec, if all nodes are honest,
|
||||
// no node should be requesting for a block that's non-existent.
|
||||
func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
|
||||
func (bcR *BlockchainReactor) sendBlockToPeer(msg *bcBlockRequestMessage,
|
||||
src p2p.Peer) (queued bool) {
|
||||
|
||||
block := bcR.store.LoadBlock(msg.Height)
|
||||
@ -165,23 +182,49 @@ func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
|
||||
bcR.Logger.Info("peer asking for a block we don't have", "src", src, "height", msg.Height)
|
||||
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendStatusResponseToPeer(msg *bcStatusRequestMessage, src p2p.Peer) (queued bool) {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendMessageToFSMAsync(msg bReactorMessageData) {
|
||||
bcR.Logger.Error("send message to FSM for processing", "msg", msg.String())
|
||||
bcR.messagesForFSMCh <- msg
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendRemovePeerToFSM(peerID p2p.ID) {
|
||||
msgData := bReactorMessageData{
|
||||
event: peerRemoveEv,
|
||||
data: bReactorEventData{
|
||||
peerId: peerID,
|
||||
err: errSwitchRemovesPeer,
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
}
|
||||
|
||||
// RemovePeer implements Reactor by removing peer from the pool.
|
||||
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
bcR.sendRemovePeerToFSM(peer.ID())
|
||||
}
|
||||
|
||||
// Receive implements Reactor by handling 4 types of messages (look below).
|
||||
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
||||
msg, err := decodeMsg(msgBytes)
|
||||
if err != nil {
|
||||
bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
||||
bcR.Logger.Error("error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
if err = msg.ValidateBasic(); err != nil {
|
||||
bcR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
||||
bcR.Logger.Error("peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
@ -190,23 +233,44 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
|
||||
|
||||
switch msg := msg.(type) {
|
||||
case *bcBlockRequestMessage:
|
||||
if queued := bcR.respondToPeer(msg, src); !queued {
|
||||
if queued := bcR.sendBlockToPeer(msg, src); !queued {
|
||||
// Unfortunately not queued since the queue is full.
|
||||
bcR.Logger.Error("Could not send block message to peer", "src", src, "height", msg.Height)
|
||||
}
|
||||
case *bcBlockResponseMessage:
|
||||
bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes))
|
||||
|
||||
case *bcStatusRequestMessage:
|
||||
// Send peer our state.
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
queued := src.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
// sorry
|
||||
if queued := bcR.sendStatusResponseToPeer(msg, src); !queued {
|
||||
// Unfortunately not queued since the queue is full.
|
||||
bcR.Logger.Error("Could not send status message to peer", "src", src)
|
||||
}
|
||||
|
||||
case *bcBlockResponseMessage:
|
||||
msgData := bReactorMessageData{
|
||||
event: blockResponseEv,
|
||||
data: bReactorEventData{
|
||||
peerId: src.ID(),
|
||||
height: msg.Block.Height,
|
||||
block: msg.Block,
|
||||
length: len(msgBytes),
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
|
||||
case *bcStatusResponseMessage:
|
||||
// Got a peer status. Unverified.
|
||||
bcR.pool.SetPeerHeight(src.ID(), msg.Height)
|
||||
msgData := bReactorMessageData{
|
||||
event: statusResponseEv,
|
||||
data: bReactorEventData{
|
||||
peerId: src.ID(),
|
||||
height: msg.Height,
|
||||
length: len(msgBytes),
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
|
||||
default:
|
||||
bcR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
||||
bcR.Logger.Error(fmt.Sprintf("unknown message type %v", reflect.TypeOf(msg)))
|
||||
}
|
||||
}
|
||||
|
||||
@ -214,155 +278,212 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
|
||||
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
||||
func (bcR *BlockchainReactor) poolRoutine() {
|
||||
|
||||
bcR.fsm.start()
|
||||
|
||||
trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
|
||||
trySendTicker := time.NewTicker(trySendIntervalMS * time.Millisecond)
|
||||
|
||||
statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
|
||||
switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
|
||||
|
||||
blocksSynced := 0
|
||||
|
||||
chainID := bcR.initialState.ChainID
|
||||
state := bcR.initialState
|
||||
|
||||
lastHundred := time.Now()
|
||||
lastRate := 0.0
|
||||
|
||||
didProcessCh := make(chan struct{}, 1)
|
||||
doProcessCh := make(chan struct{}, 1)
|
||||
doSendCh := make(chan struct{}, 1)
|
||||
|
||||
FOR_LOOP:
|
||||
ForLoop:
|
||||
for {
|
||||
select {
|
||||
case request := <-bcR.requestsCh:
|
||||
peer := bcR.Switch.Peers().Get(request.PeerID)
|
||||
if peer == nil {
|
||||
continue FOR_LOOP // Peer has since been disconnected.
|
||||
}
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{request.Height})
|
||||
queued := peer.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
// We couldn't make the request, send-queue full.
|
||||
// The pool handles timeouts, just let it go.
|
||||
continue FOR_LOOP
|
||||
|
||||
case <-trySendTicker.C: // chan time
|
||||
select {
|
||||
case doSendCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
case err := <-bcR.errorsCh:
|
||||
peer := bcR.Switch.Peers().Get(err.peerID)
|
||||
if peer != nil {
|
||||
bcR.Switch.StopPeerForError(peer, err)
|
||||
case <-doSendCh:
|
||||
// Tell FSM to make more requests.
|
||||
// The maxNumPendingRequests may be changed based on low/ high watermark thresholds for
|
||||
// - the number of blocks received and waiting to be processed,
|
||||
// - the number of blockResponse messages waiting in messagesForFSMCh, etc.
|
||||
// Currently maxNumPendingRequests value is not changed.
|
||||
msgData := bReactorMessageData{
|
||||
event: makeRequestsEv,
|
||||
data: bReactorEventData{
|
||||
maxNumRequests: maxNumPendingRequests,
|
||||
},
|
||||
}
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
|
||||
case msg := <-bcR.errorsFromFSMCh:
|
||||
bcR.reportPeerErrorToSwitch(msg.err, msg.peerID)
|
||||
if msg.err == errNoPeerResponse {
|
||||
msgData := bReactorMessageData{
|
||||
event: peerRemoveEv,
|
||||
data: bReactorEventData{
|
||||
peerId: msg.peerID,
|
||||
err: msg.err,
|
||||
},
|
||||
}
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
}
|
||||
|
||||
case <-statusUpdateTicker.C:
|
||||
// ask for status updates
|
||||
go bcR.BroadcastStatusRequest() // nolint: errcheck
|
||||
// Ask for status updates.
|
||||
go bcR.sendStatusRequest()
|
||||
|
||||
case <-switchToConsensusTicker.C:
|
||||
height, numPending, lenRequesters := bcR.pool.GetStatus()
|
||||
height, numPending, maxPeerHeight := bcR.fsm.pool.getStatus()
|
||||
outbound, inbound, _ := bcR.Switch.NumPeers()
|
||||
bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "total", lenRequesters,
|
||||
bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "maxPeerHeight", maxPeerHeight,
|
||||
"outbound", outbound, "inbound", inbound)
|
||||
if bcR.pool.IsCaughtUp() {
|
||||
if bcR.fsm.isCaughtUp() {
|
||||
bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
|
||||
bcR.pool.Stop()
|
||||
|
||||
conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
||||
if ok {
|
||||
conR.SwitchToConsensus(state, blocksSynced)
|
||||
} else {
|
||||
// should only happen during testing
|
||||
}
|
||||
|
||||
break FOR_LOOP
|
||||
bcR.fsm.stop()
|
||||
bcR.switchToConsensus()
|
||||
break ForLoop
|
||||
}
|
||||
|
||||
case <-trySyncTicker.C: // chan time
|
||||
select {
|
||||
case didProcessCh <- struct{}{}:
|
||||
case doProcessCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
case <-didProcessCh:
|
||||
// NOTE: It is a subtle mistake to process more than a single block
|
||||
// at a time (e.g. 10) here, because we only TrySend 1 request per
|
||||
// loop. The ratio mismatch can result in starving of blocks, a
|
||||
// sudden burst of requests and responses, and repeat.
|
||||
// Consequently, it is better to split these routines rather than
|
||||
// coupling them as it's written here. TODO uncouple from request
|
||||
// routine.
|
||||
|
||||
// See if there are any blocks to sync.
|
||||
first, second := bcR.pool.PeekTwoBlocks()
|
||||
//bcR.Logger.Info("TrySync peeked", "first", first, "second", second)
|
||||
if first == nil || second == nil {
|
||||
// We need both to sync the first block.
|
||||
continue FOR_LOOP
|
||||
} else {
|
||||
// Try again quickly next loop.
|
||||
didProcessCh <- struct{}{}
|
||||
case <-doProcessCh:
|
||||
err := bcR.processBlocksFromPoolRoutine()
|
||||
if err == errMissingBlocks {
|
||||
continue ForLoop
|
||||
}
|
||||
|
||||
firstParts := first.MakePartSet(types.BlockPartSizeBytes)
|
||||
firstPartsHeader := firstParts.Header()
|
||||
firstID := types.BlockID{Hash: first.Hash(), PartsHeader: firstPartsHeader}
|
||||
// Finally, verify the first block using the second's commit
|
||||
// NOTE: we can probably make this more efficient, but note that calling
|
||||
// first.Hash() doesn't verify the tx contents, so MakePartSet() is
|
||||
// currently necessary.
|
||||
err := state.Validators.VerifyCommit(
|
||||
chainID, firstID, first.Height, second.LastCommit)
|
||||
if err != nil {
|
||||
bcR.Logger.Error("Error in validation", "err", err)
|
||||
peerID := bcR.pool.RedoRequest(first.Height)
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer != nil {
|
||||
// NOTE: we've already removed the peer's request, but we
|
||||
// still need to clean up the rest.
|
||||
bcR.Switch.StopPeerForError(peer, fmt.Errorf("BlockchainReactor validation error: %v", err))
|
||||
}
|
||||
peerID2 := bcR.pool.RedoRequest(second.Height)
|
||||
peer2 := bcR.Switch.Peers().Get(peerID2)
|
||||
if peer2 != nil && peer2 != peer {
|
||||
// NOTE: we've already removed the peer's request, but we
|
||||
// still need to clean up the rest.
|
||||
bcR.Switch.StopPeerForError(peer2, fmt.Errorf("BlockchainReactor validation error: %v", err))
|
||||
}
|
||||
continue FOR_LOOP
|
||||
} else {
|
||||
bcR.pool.PopRequest()
|
||||
|
||||
// TODO: batch saves so we dont persist to disk every block
|
||||
bcR.store.SaveBlock(first, firstParts, second.LastCommit)
|
||||
|
||||
// TODO: same thing for app - but we would need a way to
|
||||
// get the hash without persisting the state
|
||||
var err error
|
||||
state, err = bcR.blockExec.ApplyBlock(state, firstID, first)
|
||||
if err != nil {
|
||||
// TODO This is bad, are we zombie?
|
||||
panic(fmt.Sprintf("Failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
|
||||
}
|
||||
blocksSynced++
|
||||
|
||||
if blocksSynced%100 == 0 {
|
||||
lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
|
||||
bcR.Logger.Info("Fast Sync Rate", "height", bcR.pool.height,
|
||||
"max_peer_height", bcR.pool.MaxPeerHeight(), "blocks/s", lastRate)
|
||||
lastHundred = time.Now()
|
||||
}
|
||||
// Notify FSM of block processing result.
|
||||
msgData := bReactorMessageData{
|
||||
event: processedBlockEv,
|
||||
data: bReactorEventData{
|
||||
err: err,
|
||||
},
|
||||
}
|
||||
continue FOR_LOOP
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
|
||||
if err == errBlockVerificationFailure {
|
||||
continue ForLoop
|
||||
}
|
||||
|
||||
doProcessCh <- struct{}{}
|
||||
|
||||
bcR.blocksSynced++
|
||||
if bcR.blocksSynced%100 == 0 {
|
||||
lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
|
||||
bcR.Logger.Info("Fast Sync Rate", "height", bcR.fsm.pool.height,
|
||||
"max_peer_height", bcR.fsm.pool.getMaxPeerHeight(), "blocks/s", lastRate)
|
||||
lastHundred = time.Now()
|
||||
}
|
||||
|
||||
case msg := <-bcR.messagesForFSMCh:
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msg)
|
||||
|
||||
case msg := <-bcR.errorsForFSMCh:
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msg)
|
||||
|
||||
case <-bcR.Quit():
|
||||
break FOR_LOOP
|
||||
break ForLoop
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) reportPeerErrorToSwitch(err error, peerID p2p.ID) {
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer != nil {
|
||||
bcR.Switch.StopPeerForError(peer, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Called by FSM and pool:
|
||||
// - pool calls when it detects slow peer or when peer times out
|
||||
// - FSM calls when:
|
||||
// - processing a block (addBlock) fails
|
||||
// - BCR process of block reports failure to FSM, FSM sends back the peers of first and second
|
||||
func (bcR *BlockchainReactor) sendPeerError(err error, peerID p2p.ID) {
|
||||
bcR.errorsFromFSMCh <- peerError{err, peerID}
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) processBlocksFromPoolRoutine() error {
|
||||
firstBP, secondBP, err := bcR.fsm.pool.getNextTwoBlocks()
|
||||
if err != nil {
|
||||
// We need both to sync the first block.
|
||||
return err
|
||||
}
|
||||
|
||||
first := firstBP.block
|
||||
second := secondBP.block
|
||||
|
||||
chainID := bcR.initialState.ChainID
|
||||
|
||||
firstParts := first.MakePartSet(types.BlockPartSizeBytes)
|
||||
firstPartsHeader := firstParts.Header()
|
||||
firstID := types.BlockID{Hash: first.Hash(), PartsHeader: firstPartsHeader}
|
||||
// Finally, verify the first block using the second's commit
|
||||
// NOTE: we can probably make this more efficient, but note that calling
|
||||
// first.Hash() doesn't verify the tx contents, so MakePartSet() is
|
||||
// currently necessary.
|
||||
err = bcR.state.Validators.VerifyCommit(
|
||||
chainID, firstID, first.Height, second.LastCommit)
|
||||
|
||||
if err != nil {
|
||||
bcR.Logger.Error("error in validation", "err", err, first.Height, second.Height)
|
||||
return errBlockVerificationFailure
|
||||
}
|
||||
|
||||
bcR.store.SaveBlock(first, firstParts, second.LastCommit)
|
||||
|
||||
// Get the hash without persisting the state.
|
||||
bcR.state, err = bcR.blockExec.ApplyBlock(bcR.state, firstID, first)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) resetStateTimer(name string, timer *time.Timer, timeout time.Duration, f func()) {
|
||||
if timer == nil {
|
||||
timer = time.AfterFunc(timeout, f)
|
||||
} else {
|
||||
timer.Reset(timeout)
|
||||
}
|
||||
}
|
||||
|
||||
// BroadcastStatusRequest broadcasts `BlockStore` height.
|
||||
func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
|
||||
func (bcR *BlockchainReactor) sendStatusRequest() {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
|
||||
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
// BlockRequest sends `BlockRequest` height.
|
||||
func (bcR *BlockchainReactor) sendBlockRequest(peerID p2p.ID, height int64) error {
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer == nil {
|
||||
return errNilPeerForBlockRequest
|
||||
}
|
||||
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{height})
|
||||
queued := peer.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
return errSendQueueFull
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) switchToConsensus() {
|
||||
|
||||
conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
||||
if ok {
|
||||
conR.SwitchToConsensus(bcR.state, bcR.blocksSynced)
|
||||
} else {
|
||||
// Should only happen during testing.
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
// Messages
|
||||
|
||||
@ -382,7 +503,7 @@ func RegisterBlockchainMessages(cdc *amino.Codec) {
|
||||
|
||||
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
|
||||
if len(bz) > maxMsgSize {
|
||||
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
||||
return msg, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
||||
}
|
||||
err = cdc.UnmarshalBinaryBare(bz, &msg)
|
||||
return
|
||||
@ -397,7 +518,7 @@ type bcBlockRequestMessage struct {
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcBlockRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -413,13 +534,13 @@ type bcNoBlockResponseMessage struct {
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (brm *bcNoBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", brm.Height)
|
||||
func (m *bcNoBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
@ -446,7 +567,7 @@ type bcStatusRequestMessage struct {
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -464,7 +585,7 @@ type bcStatusResponseMessage struct {
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"errors"
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"fmt"
|
@ -8,7 +8,6 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
@ -56,7 +55,7 @@ func makeVote(header *types.Header, blockID types.BlockID, valset *types.Validat
|
||||
BlockID: blockID,
|
||||
}
|
||||
|
||||
privVal.SignVote(header.ChainID, vote)
|
||||
_ = privVal.SignVote(header.ChainID, vote)
|
||||
|
||||
return vote
|
||||
}
|
||||
@ -109,7 +108,7 @@ func newBlockchainReactor(logger log.Logger, genDoc *types.GenesisDoc, privVals
|
||||
thisBlock := makeBlock(blockHeight, state, lastCommit)
|
||||
|
||||
thisParts := thisBlock.MakePartSet(types.BlockPartSizeBytes)
|
||||
blockID := types.BlockID{thisBlock.Hash(), thisParts.Header()}
|
||||
blockID := types.BlockID{Hash: thisBlock.Hash(), PartsHeader: thisParts.Header()}
|
||||
|
||||
state, err = blockExec.ApplyBlock(state, blockID, thisBlock)
|
||||
if err != nil {
|
||||
@ -126,7 +125,11 @@ func newBlockchainReactor(logger log.Logger, genDoc *types.GenesisDoc, privVals
|
||||
}
|
||||
|
||||
func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
peerTimeout = 15 * time.Second
|
||||
maxRequestsPerPeer = 20
|
||||
maxNumPendingRequests = 100
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_new_reactor_test")
|
||||
defer os.RemoveAll(config.RootDir)
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
|
||||
@ -134,19 +137,23 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
|
||||
reactorPairs := make([]BlockchainReactorPair, 2)
|
||||
|
||||
reactorPairs[0] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
logger := log.TestingLogger()
|
||||
reactorPairs[0] = newBlockchainReactor(logger, genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(logger, genDoc, privVals, 0)
|
||||
|
||||
p2p.MakeConnectedSwitches(config.P2P, 2, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger.With("module", moduleName))
|
||||
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
defer func() {
|
||||
for _, r := range reactorPairs {
|
||||
r.reactor.Stop()
|
||||
r.app.Stop()
|
||||
_ = r.reactor.Stop()
|
||||
_ = r.app.Stop()
|
||||
}
|
||||
}()
|
||||
|
||||
@ -161,11 +168,10 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
}
|
||||
|
||||
for {
|
||||
if reactorPairs[1].reactor.pool.IsCaughtUp() {
|
||||
time.Sleep(1 * time.Second)
|
||||
if reactorPairs[1].reactor.fsm.isCaughtUp() {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
|
||||
assert.Equal(t, maxBlockHeight, reactorPairs[0].reactor.store.Height())
|
||||
@ -186,6 +192,12 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
// Alternatively we could actually dial a TCP conn but
|
||||
// that seems extreme.
|
||||
func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
peerTimeout = 15 * time.Second
|
||||
|
||||
maxRequestsPerPeer = 20
|
||||
maxNumPendingRequests = 400
|
||||
numNodes := 4
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
defer os.RemoveAll(config.RootDir)
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
@ -194,49 +206,59 @@ func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
|
||||
otherChain := newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
defer func() {
|
||||
otherChain.reactor.Stop()
|
||||
otherChain.app.Stop()
|
||||
_ = otherChain.reactor.Stop()
|
||||
_ = otherChain.app.Stop()
|
||||
}()
|
||||
|
||||
reactorPairs := make([]BlockchainReactorPair, 4)
|
||||
reactorPairs := make([]BlockchainReactorPair, numNodes)
|
||||
|
||||
reactorPairs[0] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
reactorPairs[2] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
reactorPairs[3] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
var logger = make([]log.Logger, numNodes)
|
||||
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, 4, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
for i := 0; i < numNodes; i++ {
|
||||
logger[i] = log.TestingLogger()
|
||||
height := int64(0)
|
||||
if i == 0 {
|
||||
height = maxBlockHeight
|
||||
}
|
||||
reactorPairs[i] = newBlockchainReactor(logger[i], genDoc, privVals, height)
|
||||
}
|
||||
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, numNodes, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
defer func() {
|
||||
for _, r := range reactorPairs {
|
||||
r.reactor.Stop()
|
||||
r.app.Stop()
|
||||
_ = r.reactor.Stop()
|
||||
_ = r.app.Stop()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
if reactorPairs[3].reactor.pool.IsCaughtUp() {
|
||||
time.Sleep(1 * time.Second)
|
||||
if reactorPairs[numNodes-1].reactor.fsm.isCaughtUp() || reactorPairs[numNodes-1].reactor.Switch.Peers().Size() == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
|
||||
//at this time, reactors[0-3] is the newest
|
||||
assert.Equal(t, 3, reactorPairs[1].reactor.Switch.Peers().Size())
|
||||
assert.Equal(t, numNodes-1, reactorPairs[1].reactor.Switch.Peers().Size())
|
||||
|
||||
//mark reactorPairs[3] is an invalid peer
|
||||
reactorPairs[3].reactor.store = otherChain.reactor.store
|
||||
reactorPairs[numNodes-1].reactor.store = otherChain.reactor.store
|
||||
|
||||
lastReactorPair := newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
lastLogger := log.TestingLogger()
|
||||
lastReactorPair := newBlockchainReactor(lastLogger, genDoc, privVals, 0)
|
||||
reactorPairs = append(reactorPairs, lastReactorPair)
|
||||
|
||||
switches = append(switches, p2p.MakeConnectedSwitches(config.P2P, 1, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[len(reactorPairs)-1].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", len(reactorPairs)-1)
|
||||
reactorPairs[len(reactorPairs)-1].reactor.SetLogger(lastLogger.With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)...)
|
||||
@ -246,16 +268,13 @@ func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
}
|
||||
|
||||
for {
|
||||
if lastReactorPair.reactor.pool.IsCaughtUp() || lastReactorPair.reactor.Switch.Peers().Size() == 0 {
|
||||
time.Sleep(1 * time.Second)
|
||||
if lastReactorPair.reactor.fsm.isCaughtUp() || lastReactorPair.reactor.Switch.Peers().Size() == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
|
||||
assert.True(t, lastReactorPair.reactor.Switch.Peers().Size() < len(reactorPairs)-1)
|
||||
assert.Equal(t, lastReactorPair.reactor.pool.maxPeerHeight, lastReactorPair.reactor.pool.height)
|
||||
|
||||
}
|
||||
|
||||
func setupReactors(
|
||||
@ -279,26 +298,27 @@ func setupReactors(
|
||||
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, numReactors, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
for i := 0; i < numReactors; i++ {
|
||||
addr := reactorPairs[i].reactor.Switch.NodeInfo().ID()
|
||||
moduleName := fmt.Sprintf("blockchain-%v", addr)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName[:19]))
|
||||
}
|
||||
|
||||
return reactorPairs, switches
|
||||
}
|
||||
|
||||
// WIP - used for some scale testing, will remove
|
||||
func TestFastSyncMultiNode(t *testing.T) {
|
||||
peerTimeout = 15 * time.Second
|
||||
|
||||
numNodes := 8
|
||||
maxHeight := int64(1000)
|
||||
//numNodes := 20
|
||||
//maxHeight := int64(10000)
|
||||
|
||||
maxRequestsPerPeer = 40
|
||||
maxNumPendingRequests = 500
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
|
||||
@ -317,7 +337,7 @@ outerFor:
|
||||
for {
|
||||
i := 0
|
||||
for i < numNodes {
|
||||
if !reactorPairs[i].reactor.pool.IsCaughtUp() {
|
||||
if !reactorPairs[i].reactor.fsm.isCaughtUp() {
|
||||
break
|
||||
}
|
||||
i++
|
||||
@ -339,14 +359,12 @@ outerFor:
|
||||
|
||||
switches = append(switches, p2p.MakeConnectedSwitches(config.P2P, 1, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[len(reactorPairs)-1].reactor)
|
||||
moduleName := fmt.Sprintf("blockchainTEST-%d", len(reactorPairs)-1)
|
||||
reactorPairs[len(reactorPairs)-1].reactor.SetLogger(lastLogger.With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)...)
|
||||
|
||||
addr := lastReactorPair.reactor.Switch.NodeInfo().ID()
|
||||
moduleName := fmt.Sprintf("blockchain-%v", addr)
|
||||
lastReactorPair.reactor.SetLogger(lastLogger.With("module", moduleName[:19]))
|
||||
|
||||
start = time.Now()
|
||||
|
||||
for i := 0; i < len(reactorPairs)-1; i++ {
|
||||
@ -355,15 +373,14 @@ outerFor:
|
||||
|
||||
for {
|
||||
time.Sleep(1 * time.Second)
|
||||
if lastReactorPair.reactor.pool.IsCaughtUp() {
|
||||
if lastReactorPair.reactor.fsm.isCaughtUp() {
|
||||
fmt.Println("FAST SYNC Duration", time.Since(start))
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
assert.True(t, lastReactorPair.reactor.Switch.Peers().Size() < len(reactorPairs))
|
||||
assert.Equal(t, lastReactorPair.reactor.pool.maxPeerHeight, lastReactorPair.reactor.pool.height)
|
||||
|
||||
assert.Equal(t, lastReactorPair.reactor.fsm.pool.getMaxPeerHeight(), lastReactorPair.reactor.fsm.pool.height)
|
||||
}
|
||||
|
||||
//----------------------------------------------
|
||||
|
@ -1,340 +0,0 @@
|
||||
package blockchain_new
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
type blockData struct {
|
||||
block *types.Block
|
||||
peer *bpPeer
|
||||
}
|
||||
|
||||
func (bd *blockData) String() string {
|
||||
if bd == nil {
|
||||
return fmt.Sprintf("blockData nil")
|
||||
}
|
||||
if bd.block == nil {
|
||||
if bd.peer == nil {
|
||||
return fmt.Sprintf("block: nil peer: nil")
|
||||
}
|
||||
return fmt.Sprintf("block: nil peer: %v", bd.peer.id)
|
||||
}
|
||||
return fmt.Sprintf("block: %v peer: %v", bd.block.Height, bd.peer.id)
|
||||
}
|
||||
|
||||
type blockPool struct {
|
||||
logger log.Logger
|
||||
peers map[p2p.ID]*bpPeer
|
||||
blocks map[int64]p2p.ID
|
||||
|
||||
requests map[int64]bool // list of blocks to be assigned peers for blockRequest
|
||||
nextRequestHeight int64 // next request to be added to requests
|
||||
|
||||
height int64 // processing height
|
||||
maxPeerHeight int64 // maximum height of all peers
|
||||
numPending int32 // total numPending across peers
|
||||
toBcR bcRMessageInterface
|
||||
}
|
||||
|
||||
func newBlockPool(height int64, toBcR bcRMessageInterface) *blockPool {
|
||||
return &blockPool{
|
||||
peers: make(map[p2p.ID]*bpPeer),
|
||||
maxPeerHeight: 0,
|
||||
blocks: make(map[int64]p2p.ID),
|
||||
requests: make(map[int64]bool),
|
||||
nextRequestHeight: height,
|
||||
height: height,
|
||||
toBcR: toBcR,
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) String() string {
|
||||
peerStr := fmt.Sprintf("Pool Peers:")
|
||||
for _, p := range pool.peers {
|
||||
peerStr += fmt.Sprintf("%v,", p)
|
||||
}
|
||||
return peerStr
|
||||
}
|
||||
|
||||
func (pool *blockPool) setLogger(l log.Logger) {
|
||||
pool.logger = l
|
||||
}
|
||||
|
||||
// GetStatus returns pool's height, numPending requests and the number of
|
||||
// requests ready to be send in the future.
|
||||
func (pool *blockPool) getStatus() (height int64, numPending int32, maxPeerHeight int64) {
|
||||
return pool.height, pool.numPending, pool.maxPeerHeight
|
||||
}
|
||||
|
||||
func (pool blockPool) getMaxPeerHeight() int64 {
|
||||
return pool.maxPeerHeight
|
||||
}
|
||||
|
||||
func (pool *blockPool) reachedMaxHeight() bool {
|
||||
return pool.maxPeerHeight == 0 || pool.height >= pool.maxPeerHeight
|
||||
}
|
||||
|
||||
func (pool *blockPool) rescheduleRequest(peerID p2p.ID, height int64) {
|
||||
pool.logger.Debug("reschedule requests made to peer for height ", "peerID", peerID, "height", height)
|
||||
pool.requests[height] = true
|
||||
delete(pool.blocks, height)
|
||||
delete(pool.peers[peerID].blocks, height)
|
||||
}
|
||||
|
||||
// Updates the pool's max height. If no peers are left maxPeerHeight is set to 0.
|
||||
func (pool *blockPool) updateMaxPeerHeight() {
|
||||
var max int64
|
||||
for _, peer := range pool.peers {
|
||||
if peer.height > max {
|
||||
max = peer.height
|
||||
}
|
||||
}
|
||||
pool.maxPeerHeight = max
|
||||
}
|
||||
|
||||
// Adds a new peer or updates an existing peer with a new height.
|
||||
// If the peer is too short it is removed.
|
||||
func (pool *blockPool) updatePeer(peerID p2p.ID, height int64) error {
|
||||
pool.logger.Debug("updatePeer", "peerID", peerID, "height", height)
|
||||
peer := pool.peers[peerID]
|
||||
|
||||
if height < pool.height {
|
||||
pool.logger.Info("Peer height too small", "peer", peerID, "height", height, "fsm_height", pool.height)
|
||||
|
||||
// Don't add or update a peer that is not useful.
|
||||
if peer != nil {
|
||||
pool.logger.Info("remove short peer", "peer", peerID, "height", height, "fsm_height", pool.height)
|
||||
pool.removePeer(peerID, errPeerTooShort)
|
||||
}
|
||||
return errPeerTooShort
|
||||
}
|
||||
|
||||
if peer == nil {
|
||||
// Add new peer.
|
||||
peer = newBPPeer(peerID, height, pool.toBcR.sendPeerError)
|
||||
peer.setLogger(pool.logger.With("peer", peerID))
|
||||
pool.peers[peerID] = peer
|
||||
} else {
|
||||
// Update existing peer.
|
||||
// Remove any requests made for heights in (height, peer.height].
|
||||
for h, block := range pool.peers[peerID].blocks {
|
||||
if h <= height {
|
||||
continue
|
||||
}
|
||||
// Reschedule the requests for all blocks waiting for the peer, or received and not processed yet.
|
||||
if block == nil {
|
||||
// Since block was not yet received it is counted in numPending, decrement.
|
||||
pool.numPending--
|
||||
pool.peers[peerID].numPending--
|
||||
}
|
||||
pool.rescheduleRequest(peerID, h)
|
||||
}
|
||||
peer.height = height
|
||||
}
|
||||
|
||||
pool.updateMaxPeerHeight()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stops the peer timer and deletes the peer. Recomputes the max peer height.
|
||||
func (pool *blockPool) deletePeer(peerID p2p.ID) {
|
||||
if p, ok := pool.peers[peerID]; ok {
|
||||
if p.timeout != nil {
|
||||
p.timeout.Stop()
|
||||
}
|
||||
delete(pool.peers, peerID)
|
||||
|
||||
if p.height == pool.maxPeerHeight {
|
||||
pool.updateMaxPeerHeight()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Removes any blocks and requests associated with the peer and deletes the peer.
|
||||
// Also triggers new requests if blocks have been removed.
|
||||
func (pool *blockPool) removePeer(peerID p2p.ID, err error) {
|
||||
pool.logger.Debug("removing peer", "peerID", peerID)
|
||||
|
||||
peer := pool.peers[peerID]
|
||||
if peer == nil {
|
||||
return
|
||||
}
|
||||
// Reschedule the requests for all blocks waiting for the peer, or received and not processed yet.
|
||||
for h, block := range pool.peers[peerID].blocks {
|
||||
if block == nil {
|
||||
pool.numPending--
|
||||
}
|
||||
pool.rescheduleRequest(peerID, h)
|
||||
}
|
||||
pool.deletePeer(peerID)
|
||||
|
||||
}
|
||||
|
||||
// Called every time FSM advances its height.
|
||||
func (pool *blockPool) removeShortPeers() {
|
||||
for _, peer := range pool.peers {
|
||||
if peer.height < pool.height {
|
||||
pool.removePeer(peer.id, nil)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Validates that the block comes from the peer it was expected from and stores it in the 'blocks' map.
|
||||
func (pool *blockPool) addBlock(peerID p2p.ID, block *types.Block, blockSize int) error {
|
||||
if _, ok := pool.peers[peerID]; !ok {
|
||||
pool.logger.Error("peer doesn't exist", "peer", peerID, "block_receieved", block.Height)
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
b, ok := pool.peers[peerID].blocks[block.Height]
|
||||
if !ok {
|
||||
pool.logger.Error("peer sent us a block we didn't expect", "peer", peerID, "blockHeight", block.Height)
|
||||
if expPeerID, pok := pool.blocks[block.Height]; pok {
|
||||
pool.logger.Error("expected this block from peer", "peer", expPeerID)
|
||||
}
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
if b != nil {
|
||||
pool.logger.Error("already have a block for height", "height", block.Height)
|
||||
return errBadDataFromPeer
|
||||
}
|
||||
|
||||
pool.peers[peerID].blocks[block.Height] = block
|
||||
pool.blocks[block.Height] = peerID
|
||||
pool.numPending--
|
||||
pool.peers[peerID].decrPending(blockSize)
|
||||
pool.logger.Debug("added new block", "height", block.Height, "from_peer", peerID, "total", len(pool.blocks))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pool *blockPool) getBlockAndPeerAtHeight(height int64) (bData *blockData, err error) {
|
||||
peerID := pool.blocks[height]
|
||||
peer := pool.peers[peerID]
|
||||
if peer == nil {
|
||||
return &blockData{}, errMissingBlocks
|
||||
}
|
||||
|
||||
block, ok := peer.blocks[height]
|
||||
if !ok || block == nil {
|
||||
return &blockData{}, errMissingBlocks
|
||||
}
|
||||
|
||||
return &blockData{peer: peer, block: block}, nil
|
||||
|
||||
}
|
||||
|
||||
func (pool *blockPool) getNextTwoBlocks() (first, second *blockData, err error) {
|
||||
first, err = pool.getBlockAndPeerAtHeight(pool.height)
|
||||
second, err2 := pool.getBlockAndPeerAtHeight(pool.height + 1)
|
||||
if err == nil {
|
||||
err = err2
|
||||
}
|
||||
|
||||
if err == errMissingBlocks {
|
||||
// We need both to sync the first block.
|
||||
pool.logger.Error("missing blocks at height and/ or height+1", "height", pool.height)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Remove peers that sent us the first two blocks, blocks will also be removed by removePeer().
|
||||
func (pool *blockPool) invalidateFirstTwoBlocks(err error) {
|
||||
first, err1 := pool.getBlockAndPeerAtHeight(pool.height)
|
||||
second, err2 := pool.getBlockAndPeerAtHeight(pool.height + 1)
|
||||
|
||||
if err1 == nil {
|
||||
pool.removePeer(first.peer.id, err)
|
||||
}
|
||||
if err2 == nil {
|
||||
pool.removePeer(second.peer.id, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) processedCurrentHeightBlock() {
|
||||
peerID, peerOk := pool.blocks[pool.height]
|
||||
if peerOk {
|
||||
delete(pool.peers[peerID].blocks, pool.height)
|
||||
}
|
||||
delete(pool.blocks, pool.height)
|
||||
pool.logger.Debug("processed and removed block at height", "height", pool.height)
|
||||
pool.height++
|
||||
pool.removeShortPeers()
|
||||
}
|
||||
|
||||
func (pool *blockPool) removeBadPeers() {
|
||||
pool.removeShortPeers()
|
||||
for _, peer := range pool.peers {
|
||||
if err := peer.isGood(); err != nil {
|
||||
pool.removePeer(peer.id, err)
|
||||
if err == errSlowPeer {
|
||||
peer.errFunc(errSlowPeer, peer.id)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) makeRequestBatch(maxNumPendingRequests int32) []int {
|
||||
pool.removeBadPeers()
|
||||
// If running low on planned requests, make more.
|
||||
numNeeded := int32(cmn.MinInt(int(maxNumPendingRequests), len(pool.peers)*int(maxRequestsPerPeer))) - pool.numPending
|
||||
for int32(len(pool.requests)) < numNeeded {
|
||||
if pool.nextRequestHeight > pool.maxPeerHeight {
|
||||
break
|
||||
}
|
||||
pool.requests[pool.nextRequestHeight] = true
|
||||
pool.nextRequestHeight++
|
||||
}
|
||||
|
||||
heights := make([]int, 0, len(pool.requests))
|
||||
for k := range pool.requests {
|
||||
heights = append(heights, int(k))
|
||||
}
|
||||
sort.Ints(heights)
|
||||
return heights
|
||||
}
|
||||
|
||||
func (pool *blockPool) makeNextRequests(maxNumPendingRequests int32) {
|
||||
heights := pool.makeRequestBatch(maxNumPendingRequests)
|
||||
pool.logger.Debug("makeNextRequests will make following requests", "number", len(heights), "heights", heights)
|
||||
|
||||
for _, height := range heights {
|
||||
h := int64(height)
|
||||
if err := pool.sendRequest(h); err != nil {
|
||||
// Errors from sendRequest() are handled by this function
|
||||
return
|
||||
}
|
||||
delete(pool.requests, h)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *blockPool) sendRequest(height int64) error {
|
||||
for _, peer := range pool.peers {
|
||||
if peer.numPending >= int32(maxRequestsPerPeer) {
|
||||
continue
|
||||
}
|
||||
if peer.height < height {
|
||||
continue
|
||||
}
|
||||
pool.logger.Debug("assign request to peer", "peer", peer.id, "height", height)
|
||||
if err := pool.toBcR.sendBlockRequest(peer.id, height); err == errNilPeerForBlockRequest {
|
||||
pool.removePeer(peer.id, err)
|
||||
pool.toBcR.sendPeerError(err, peer.id)
|
||||
}
|
||||
|
||||
pool.blocks[height] = peer.id
|
||||
pool.numPending++
|
||||
|
||||
peer.blocks[height] = nil
|
||||
peer.incrPending()
|
||||
|
||||
return nil
|
||||
}
|
||||
pool.logger.Error("could not find peer to send request for block at height", "height", height)
|
||||
return errNoPeerFoundForHeight
|
||||
}
|
@ -1,575 +0,0 @@
|
||||
package blockchain_new
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
type testPeer struct {
|
||||
id p2p.ID
|
||||
height int64
|
||||
}
|
||||
|
||||
type testPeerResult struct {
|
||||
id p2p.ID
|
||||
height int64
|
||||
numPending int32
|
||||
blocks map[int64]*types.Block
|
||||
}
|
||||
|
||||
type testBcR struct {
|
||||
logger log.Logger
|
||||
}
|
||||
|
||||
type testValues struct {
|
||||
numRequestsSent int32
|
||||
}
|
||||
|
||||
var testResults testValues
|
||||
|
||||
func resetPoolTestResults() {
|
||||
testResults.numRequestsSent = 0
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendPeerError(err error, peerID p2p.ID) {
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendStatusRequest() {
|
||||
}
|
||||
|
||||
func (testR *testBcR) sendBlockRequest(peerID p2p.ID, height int64) error {
|
||||
testResults.numRequestsSent++
|
||||
return nil
|
||||
}
|
||||
|
||||
func (testR *testBcR) resetStateTimer(name string, timer *time.Timer, timeout time.Duration, f func()) {
|
||||
}
|
||||
|
||||
func (testR *testBcR) switchToConsensus() {
|
||||
}
|
||||
|
||||
func newTestBcR() *testBcR {
|
||||
testBcR := &testBcR{logger: log.TestingLogger()}
|
||||
return testBcR
|
||||
}
|
||||
|
||||
type tPBlocks struct {
|
||||
id p2p.ID
|
||||
create bool
|
||||
}
|
||||
|
||||
func makeBlockPool(bcr *testBcR, height int64, peers []bpPeer, blocks map[int64]tPBlocks) *blockPool {
|
||||
bPool := newBlockPool(height, bcr)
|
||||
txs := []types.Tx{types.Tx("foo"), types.Tx("bar")}
|
||||
|
||||
var maxH int64
|
||||
for _, p := range peers {
|
||||
if p.height > maxH {
|
||||
maxH = p.height
|
||||
}
|
||||
bPool.peers[p.id] = newBPPeer(p.id, p.height, bcr.sendPeerError)
|
||||
bPool.peers[p.id].setLogger(bcr.logger)
|
||||
|
||||
}
|
||||
bPool.maxPeerHeight = maxH
|
||||
for h, p := range blocks {
|
||||
bPool.blocks[h] = p.id
|
||||
bPool.peers[p.id].blocks[h] = nil
|
||||
if p.create {
|
||||
bPool.peers[p.id].blocks[h] = types.MakeBlock(int64(h), txs, nil, nil)
|
||||
} else {
|
||||
bPool.peers[p.id].incrPending()
|
||||
}
|
||||
}
|
||||
bPool.setLogger(bcr.logger)
|
||||
return bPool
|
||||
}
|
||||
|
||||
func poolCopy(pool *blockPool) *blockPool {
|
||||
return &blockPool{
|
||||
peers: peersCopy(pool.peers),
|
||||
logger: pool.logger,
|
||||
blocks: pool.blocks,
|
||||
requests: pool.requests,
|
||||
height: pool.height,
|
||||
nextRequestHeight: pool.height,
|
||||
maxPeerHeight: pool.maxPeerHeight,
|
||||
toBcR: pool.toBcR,
|
||||
}
|
||||
}
|
||||
|
||||
func peersCopy(peers map[p2p.ID]*bpPeer) map[p2p.ID]*bpPeer {
|
||||
peerCopy := make(map[p2p.ID]*bpPeer)
|
||||
for _, p := range peers {
|
||||
peerCopy[p.id] = newBPPeer(p.id, p.height, p.errFunc)
|
||||
}
|
||||
return peerCopy
|
||||
}
|
||||
|
||||
func TestBlockPoolUpdatePeerNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args testPeer
|
||||
errWanted error
|
||||
addWanted bool
|
||||
delWanted bool
|
||||
maxHeightWanted int64
|
||||
}{
|
||||
|
||||
{
|
||||
name: "add a first short peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 50},
|
||||
errWanted: errPeerTooShort,
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
{
|
||||
name: "add a first good peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 101},
|
||||
addWanted: true,
|
||||
maxHeightWanted: int64(101),
|
||||
},
|
||||
{
|
||||
name: "increase the height of P1 from 120 to 123",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 123},
|
||||
maxHeightWanted: int64(123),
|
||||
},
|
||||
{
|
||||
name: "decrease the height of P1 from 120 to 110",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 110},
|
||||
maxHeightWanted: int64(110),
|
||||
},
|
||||
{
|
||||
name: "decrease the height of P1 from 120 to 90",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: testPeer{"P1", 90},
|
||||
delWanted: true,
|
||||
errWanted: errPeerTooShort,
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
|
||||
beforePool := poolCopy(pool)
|
||||
err := pool.updatePeer(tt.args.id, tt.args.height)
|
||||
if err != tt.errWanted {
|
||||
t.Errorf("blockPool.updatePeer() error = %v, wantErr %v", err, tt.errWanted)
|
||||
}
|
||||
|
||||
if tt.errWanted != nil {
|
||||
// error case
|
||||
if tt.delWanted {
|
||||
assert.Equal(t, len(beforePool.peers)-1, len(pool.peers))
|
||||
return
|
||||
}
|
||||
assert.Equal(t, beforePool, pool)
|
||||
return
|
||||
}
|
||||
|
||||
if tt.addWanted {
|
||||
// add case only
|
||||
assert.Equal(t, len(beforePool.peers)+1, len(pool.peers))
|
||||
} else {
|
||||
// update case only
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
}
|
||||
|
||||
// both add and update
|
||||
assert.Equal(t, pool.peers[tt.args.id].height, tt.args.height)
|
||||
assert.Equal(t, tt.maxHeightWanted, pool.maxPeerHeight)
|
||||
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolRemovePeerNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
type args struct {
|
||||
peerID p2p.ID
|
||||
err error
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args args
|
||||
maxHeightWanted int64
|
||||
}{
|
||||
{
|
||||
name: "attempt to delete non-existing peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P99", nil},
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "delete the only peer",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P1", nil},
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
{
|
||||
name: "delete the shortest of two peers",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P1", nil},
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "delete the tallest of two peers",
|
||||
pool: makeBlockPool(testBcR, 100, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 120}}, map[int64]tPBlocks{}),
|
||||
args: args{"P2", nil},
|
||||
maxHeightWanted: int64(100),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
tt.pool.removePeer(tt.args.peerID, tt.args.err)
|
||||
assert.Equal(t, tt.maxHeightWanted, tt.pool.maxPeerHeight)
|
||||
_, ok := tt.pool.peers[tt.args.peerID]
|
||||
assert.False(t, ok)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolRemoveShortPeersNoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
maxHeightWanted int64
|
||||
noChange bool
|
||||
}{
|
||||
{
|
||||
name: "no short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 110}, {id: "P3", height: 120}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(120),
|
||||
noChange: true,
|
||||
},
|
||||
{
|
||||
name: "one short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 90}, {id: "P3", height: 120}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(120),
|
||||
},
|
||||
{
|
||||
name: "all short peers",
|
||||
pool: makeBlockPool(testBcR, 100,
|
||||
[]bpPeer{{id: "P1", height: 90}, {id: "P2", height: 91}, {id: "P3", height: 92}},
|
||||
map[int64]tPBlocks{}),
|
||||
maxHeightWanted: int64(0),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
beforePool := poolCopy(pool)
|
||||
|
||||
pool.removeShortPeers()
|
||||
assert.Equal(t, tt.maxHeightWanted, pool.maxPeerHeight)
|
||||
if tt.noChange {
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
return
|
||||
}
|
||||
for _, peer := range tt.pool.peers {
|
||||
bPeer, bok := beforePool.peers[peer.id]
|
||||
if bok && bPeer.height < beforePool.height {
|
||||
_, ok := pool.peers[peer.id]
|
||||
assert.False(t, ok)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolSendRequestBatch(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
maxRequestsPerPeer int32
|
||||
expRequests map[int64]bool
|
||||
expPeerResults []testPeerResult
|
||||
expNumPending int32
|
||||
}{
|
||||
{
|
||||
name: "one peer - send up to maxRequestsPerPeer block requests",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}}, map[int64]tPBlocks{}),
|
||||
maxRequestsPerPeer: 2,
|
||||
expRequests: map[int64]bool{10: true, 11: true},
|
||||
expPeerResults: []testPeerResult{{id: "P1", height: 100, numPending: 2, blocks: map[int64]*types.Block{10: nil, 11: nil}}},
|
||||
expNumPending: 2,
|
||||
},
|
||||
{
|
||||
name: "n peers - send n*maxRequestsPerPeer block requests",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}}, map[int64]tPBlocks{}),
|
||||
maxRequestsPerPeer: 2,
|
||||
expRequests: map[int64]bool{10: true, 11: true},
|
||||
expPeerResults: []testPeerResult{
|
||||
{id: "P1", height: 100, numPending: 2, blocks: map[int64]*types.Block{10: nil, 11: nil}},
|
||||
{id: "P2", height: 100, numPending: 2, blocks: map[int64]*types.Block{12: nil, 13: nil}}},
|
||||
expNumPending: 4,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
resetPoolTestResults()
|
||||
|
||||
pool := tt.pool
|
||||
|
||||
maxRequestsPerPeer = int32(tt.maxRequestsPerPeer)
|
||||
pool.makeNextRequests(10)
|
||||
assert.Equal(t, tt.expNumPending, pool.numPending)
|
||||
assert.Equal(t, testResults.numRequestsSent, maxRequestsPerPeer*int32(len(pool.peers)))
|
||||
|
||||
for _, tPeer := range tt.expPeerResults {
|
||||
peer := pool.peers[tPeer.id]
|
||||
assert.NotNil(t, peer)
|
||||
assert.Equal(t, tPeer.numPending, peer.numPending)
|
||||
}
|
||||
assert.Equal(t, testResults.numRequestsSent, maxRequestsPerPeer*int32(len(pool.peers)))
|
||||
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolAddBlock(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
txs := []types.Tx{types.Tx("foo"), types.Tx("bar")}
|
||||
|
||||
type args struct {
|
||||
peerID p2p.ID
|
||||
block *types.Block
|
||||
blockSize int
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
args args
|
||||
wantErr bool
|
||||
}{
|
||||
{name: "block from unknown peer",
|
||||
pool: makeBlockPool(testBcR, 10, []bpPeer{{id: "P1", height: 100}}, map[int64]tPBlocks{}),
|
||||
args: args{
|
||||
peerID: "P2",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "unexpected block 11 from known peer - waiting for 10",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", false}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(11), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "unexpected block 10 from known peer - already have 10",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: true,
|
||||
},
|
||||
{name: "expected block from known peer",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", false}}),
|
||||
args: args{
|
||||
peerID: "P1",
|
||||
block: types.MakeBlock(int64(10), txs, nil, nil),
|
||||
blockSize: 100,
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
if err := pool.addBlock(tt.args.peerID, tt.args.block, tt.args.blockSize); (err != nil) != tt.wantErr {
|
||||
t.Errorf("blockPool.addBlock() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolGetNextTwoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
firstWanted int64
|
||||
secondWanted int64
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "both blocks missing",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 0,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "second block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 18: {"P2", true}}),
|
||||
firstWanted: 15,
|
||||
secondWanted: 0,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "first block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{18: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 16,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "both blocks present",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}, 11: {"P2", true}}),
|
||||
firstWanted: 10,
|
||||
secondWanted: 11,
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
gotFirst, gotSecond, err := pool.getNextTwoBlocks()
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.firstWanted != 0 {
|
||||
peer := pool.blocks[tt.firstWanted]
|
||||
block := pool.peers[peer].blocks[tt.firstWanted]
|
||||
if !reflect.DeepEqual(gotFirst.block, block) {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() gotFirst = %v, want %v", gotFirst.block.Height, tt.firstWanted)
|
||||
}
|
||||
}
|
||||
if tt.secondWanted != 0 {
|
||||
peer := pool.blocks[tt.secondWanted]
|
||||
block := pool.peers[peer].blocks[tt.secondWanted]
|
||||
if !reflect.DeepEqual(gotSecond.block, block) {
|
||||
t.Errorf("blockPool.getNextTwoBlocks() gotFirst = %v, want %v", gotSecond.block.Height, tt.secondWanted)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolInvalidateFirstTwoBlocks(t *testing.T) {
|
||||
testBcR := newTestBcR()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
pool *blockPool
|
||||
firstWanted int64
|
||||
secondWanted int64
|
||||
wantChange bool
|
||||
}{
|
||||
{
|
||||
name: "both blocks missing",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 0,
|
||||
wantChange: false,
|
||||
},
|
||||
{
|
||||
name: "second block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{15: {"P1", true}, 18: {"P2", true}}),
|
||||
firstWanted: 15,
|
||||
secondWanted: 0,
|
||||
wantChange: true,
|
||||
},
|
||||
{
|
||||
name: "first block missing",
|
||||
pool: makeBlockPool(testBcR, 15,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{18: {"P1", true}, 16: {"P2", true}}),
|
||||
firstWanted: 0,
|
||||
secondWanted: 16,
|
||||
wantChange: true,
|
||||
},
|
||||
{
|
||||
name: "both blocks present",
|
||||
pool: makeBlockPool(testBcR, 10,
|
||||
[]bpPeer{{id: "P1", height: 100}, {id: "P2", height: 100}},
|
||||
map[int64]tPBlocks{10: {"P1", true}, 11: {"P2", true}}),
|
||||
firstWanted: 10,
|
||||
secondWanted: 11,
|
||||
wantChange: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
pool := tt.pool
|
||||
gotFirst, gotSecond, _ := pool.getNextTwoBlocks()
|
||||
|
||||
beforePool := poolCopy(pool)
|
||||
pool.invalidateFirstTwoBlocks(errNoPeerResponse)
|
||||
if !tt.wantChange {
|
||||
assert.Equal(t, len(beforePool.peers), len(pool.peers))
|
||||
return
|
||||
}
|
||||
if tt.firstWanted != 0 {
|
||||
_, ok := pool.peers[gotFirst.peer.id]
|
||||
assert.False(t, ok)
|
||||
_, ok = pool.blocks[tt.firstWanted]
|
||||
assert.False(t, ok)
|
||||
assert.True(t, pool.requests[tt.firstWanted])
|
||||
}
|
||||
if tt.secondWanted != 0 {
|
||||
_, ok := pool.peers[gotSecond.peer.id]
|
||||
assert.False(t, ok)
|
||||
_, ok = pool.blocks[tt.secondWanted]
|
||||
assert.False(t, ok)
|
||||
assert.True(t, pool.requests[tt.secondWanted])
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
@ -1,595 +0,0 @@
|
||||
package blockchain_new
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
const (
|
||||
// BlockchainChannel is a channel for blocks and status updates (`BlockStore` height)
|
||||
BlockchainChannel = byte(0x40)
|
||||
trySyncIntervalMS = 10
|
||||
trySendIntervalMS = 10
|
||||
|
||||
// stop syncing when last block's time is
|
||||
// within this much of the system time.
|
||||
// stopSyncingDurationMinutes = 10
|
||||
|
||||
// ask for best height every 10s
|
||||
statusUpdateIntervalSeconds = 10
|
||||
// check if we should switch to consensus reactor
|
||||
switchToConsensusIntervalSeconds = 1
|
||||
|
||||
// NOTE: keep up to date with bcBlockResponseMessage
|
||||
bcBlockResponseMessagePrefixSize = 4
|
||||
bcBlockResponseMessageFieldKeySize = 1
|
||||
maxMsgSize = types.MaxBlockSizeBytes +
|
||||
bcBlockResponseMessagePrefixSize +
|
||||
bcBlockResponseMessageFieldKeySize
|
||||
)
|
||||
|
||||
var (
|
||||
maxRequestsPerPeer int32 = 20
|
||||
maxNumPendingRequests int32 = 600
|
||||
)
|
||||
|
||||
type consensusReactor interface {
|
||||
// for when we switch from blockchain reactor and fast sync to
|
||||
// the consensus machine
|
||||
SwitchToConsensus(sm.State, int)
|
||||
}
|
||||
|
||||
type peerError struct {
|
||||
err error
|
||||
peerID p2p.ID
|
||||
}
|
||||
|
||||
func (e peerError) Error() string {
|
||||
return fmt.Sprintf("error with peer %v: %s", e.peerID, e.err.Error())
|
||||
}
|
||||
|
||||
// BlockchainReactor handles long-term catchup syncing.
|
||||
type BlockchainReactor struct {
|
||||
p2p.BaseReactor
|
||||
|
||||
// immutable
|
||||
initialState sm.State
|
||||
state sm.State
|
||||
|
||||
blockExec *sm.BlockExecutor
|
||||
store *BlockStore
|
||||
|
||||
fastSync bool
|
||||
|
||||
fsm *bReactorFSM
|
||||
blocksSynced int
|
||||
|
||||
// Receive goroutine forwards messages to this channel to be processed in the context of the poolRoutine.
|
||||
messagesForFSMCh chan bReactorMessageData
|
||||
|
||||
// Switch goroutine may send RemovePeer to the blockchain reactor. This is an error message that is relayed
|
||||
// to this channel to be processed in the context of the poolRoutine.
|
||||
errorsForFSMCh chan bReactorMessageData
|
||||
|
||||
// This channel is used by the FSM and indirectly the block pool to report errors to the blockchain reactor and
|
||||
// the switch.
|
||||
errorsFromFSMCh chan peerError
|
||||
}
|
||||
|
||||
type BlockRequest struct {
|
||||
Height int64
|
||||
PeerID p2p.ID
|
||||
}
|
||||
|
||||
// bReactorMessageData structure is used by the reactor when sending messages to the FSM.
|
||||
type bReactorMessageData struct {
|
||||
event bReactorEvent
|
||||
data bReactorEventData
|
||||
}
|
||||
|
||||
// NewBlockchainReactor returns new reactor instance.
|
||||
func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *BlockStore,
|
||||
fastSync bool) *BlockchainReactor {
|
||||
|
||||
if state.LastBlockHeight != store.Height() {
|
||||
panic(fmt.Sprintf("state (%v) and store (%v) height mismatch", state.LastBlockHeight,
|
||||
store.Height()))
|
||||
}
|
||||
|
||||
const capacity = 1000
|
||||
errorsFromFSMCh := make(chan peerError, capacity)
|
||||
messagesForFSMCh := make(chan bReactorMessageData, capacity)
|
||||
errorsForFSMCh := make(chan bReactorMessageData, capacity)
|
||||
|
||||
bcR := &BlockchainReactor{
|
||||
initialState: state,
|
||||
state: state,
|
||||
blockExec: blockExec,
|
||||
fastSync: fastSync,
|
||||
store: store,
|
||||
messagesForFSMCh: messagesForFSMCh,
|
||||
errorsFromFSMCh: errorsFromFSMCh,
|
||||
errorsForFSMCh: errorsForFSMCh,
|
||||
}
|
||||
fsm := NewFSM(store.Height()+1, bcR)
|
||||
bcR.fsm = fsm
|
||||
bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
|
||||
return bcR
|
||||
}
|
||||
|
||||
// SetLogger implements cmn.Service by setting the logger on reactor and pool.
|
||||
func (bcR *BlockchainReactor) SetLogger(l log.Logger) {
|
||||
bcR.BaseService.Logger = l
|
||||
bcR.fsm.setLogger(l)
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStart() error {
|
||||
if bcR.fastSync {
|
||||
go bcR.poolRoutine()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// OnStop implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStop() {
|
||||
_ = bcR.Stop()
|
||||
}
|
||||
|
||||
// GetChannels implements Reactor
|
||||
func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
|
||||
return []*p2p.ChannelDescriptor{
|
||||
{
|
||||
ID: BlockchainChannel,
|
||||
Priority: 10,
|
||||
SendQueueCapacity: 1000,
|
||||
RecvBufferCapacity: 50 * 4096,
|
||||
RecvMessageCapacity: maxMsgSize,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// AddPeer implements Reactor by sending our state to peer.
|
||||
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
if !peer.Send(BlockchainChannel, msgBytes) {
|
||||
// doing nothing, will try later in `poolRoutine`
|
||||
}
|
||||
// peer is added to the pool once we receive the first
|
||||
// bcStatusResponseMessage from the peer and call pool.SetPeerHeight
|
||||
}
|
||||
|
||||
// respondToPeer loads a block and sends it to the requesting peer,
|
||||
// if we have it. Otherwise, we'll respond saying we don't have it.
|
||||
// According to the Tendermint spec, if all nodes are honest,
|
||||
// no node should be requesting for a block that's non-existent.
|
||||
func (bcR *BlockchainReactor) sendBlockToPeer(msg *bcBlockRequestMessage,
|
||||
src p2p.Peer) (queued bool) {
|
||||
|
||||
block := bcR.store.LoadBlock(msg.Height)
|
||||
if block != nil {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
bcR.Logger.Info("peer asking for a block we don't have", "src", src, "height", msg.Height)
|
||||
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendStatusResponseToPeer(msg *bcStatusRequestMessage, src p2p.Peer) (queued bool) {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendMessageToFSMAsync(msg bReactorMessageData) {
|
||||
bcR.Logger.Error("send message to FSM for processing", "msg", msg.String())
|
||||
bcR.messagesForFSMCh <- msg
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) sendRemovePeerToFSM(peerID p2p.ID) {
|
||||
msgData := bReactorMessageData{
|
||||
event: peerRemoveEv,
|
||||
data: bReactorEventData{
|
||||
peerId: peerID,
|
||||
err: errSwitchRemovesPeer,
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
}
|
||||
|
||||
// RemovePeer implements Reactor by removing peer from the pool.
|
||||
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
bcR.sendRemovePeerToFSM(peer.ID())
|
||||
}
|
||||
|
||||
// Receive implements Reactor by handling 4 types of messages (look below).
|
||||
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
||||
msg, err := decodeMsg(msgBytes)
|
||||
if err != nil {
|
||||
bcR.Logger.Error("error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
if err = msg.ValidateBasic(); err != nil {
|
||||
bcR.Logger.Error("peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
|
||||
|
||||
switch msg := msg.(type) {
|
||||
case *bcBlockRequestMessage:
|
||||
if queued := bcR.sendBlockToPeer(msg, src); !queued {
|
||||
// Unfortunately not queued since the queue is full.
|
||||
bcR.Logger.Error("Could not send block message to peer", "src", src, "height", msg.Height)
|
||||
}
|
||||
|
||||
case *bcStatusRequestMessage:
|
||||
// Send peer our state.
|
||||
if queued := bcR.sendStatusResponseToPeer(msg, src); !queued {
|
||||
// Unfortunately not queued since the queue is full.
|
||||
bcR.Logger.Error("Could not send status message to peer", "src", src)
|
||||
}
|
||||
|
||||
case *bcBlockResponseMessage:
|
||||
msgData := bReactorMessageData{
|
||||
event: blockResponseEv,
|
||||
data: bReactorEventData{
|
||||
peerId: src.ID(),
|
||||
height: msg.Block.Height,
|
||||
block: msg.Block,
|
||||
length: len(msgBytes),
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
|
||||
case *bcStatusResponseMessage:
|
||||
// Got a peer status. Unverified.
|
||||
msgData := bReactorMessageData{
|
||||
event: statusResponseEv,
|
||||
data: bReactorEventData{
|
||||
peerId: src.ID(),
|
||||
height: msg.Height,
|
||||
length: len(msgBytes),
|
||||
},
|
||||
}
|
||||
bcR.sendMessageToFSMAsync(msgData)
|
||||
|
||||
default:
|
||||
bcR.Logger.Error(fmt.Sprintf("unknown message type %v", reflect.TypeOf(msg)))
|
||||
}
|
||||
}
|
||||
|
||||
// Handle messages from the poolReactor telling the reactor what to do.
|
||||
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
||||
func (bcR *BlockchainReactor) poolRoutine() {
|
||||
|
||||
bcR.fsm.start()
|
||||
|
||||
trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
|
||||
trySendTicker := time.NewTicker(trySendIntervalMS * time.Millisecond)
|
||||
|
||||
statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
|
||||
switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
|
||||
|
||||
lastHundred := time.Now()
|
||||
lastRate := 0.0
|
||||
|
||||
doProcessCh := make(chan struct{}, 1)
|
||||
doSendCh := make(chan struct{}, 1)
|
||||
|
||||
ForLoop:
|
||||
for {
|
||||
select {
|
||||
|
||||
case <-trySendTicker.C: // chan time
|
||||
select {
|
||||
case doSendCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
case <-doSendCh:
|
||||
// Tell FSM to make more requests.
|
||||
// The maxNumPendingRequests may be changed based on low/ high watermark thresholds for
|
||||
// - the number of blocks received and waiting to be processed,
|
||||
// - the number of blockResponse messages waiting in messagesForFSMCh, etc.
|
||||
// Currently maxNumPendingRequests value is not changed.
|
||||
msgData := bReactorMessageData{
|
||||
event: makeRequestsEv,
|
||||
data: bReactorEventData{
|
||||
maxNumRequests: maxNumPendingRequests,
|
||||
},
|
||||
}
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
|
||||
case msg := <-bcR.errorsFromFSMCh:
|
||||
bcR.reportPeerErrorToSwitch(msg.err, msg.peerID)
|
||||
if msg.err == errNoPeerResponse {
|
||||
msgData := bReactorMessageData{
|
||||
event: peerRemoveEv,
|
||||
data: bReactorEventData{
|
||||
peerId: msg.peerID,
|
||||
err: msg.err,
|
||||
},
|
||||
}
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
}
|
||||
|
||||
case <-statusUpdateTicker.C:
|
||||
// Ask for status updates.
|
||||
go bcR.sendStatusRequest()
|
||||
|
||||
case <-switchToConsensusTicker.C:
|
||||
height, numPending, maxPeerHeight := bcR.fsm.pool.getStatus()
|
||||
outbound, inbound, _ := bcR.Switch.NumPeers()
|
||||
bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "maxPeerHeight", maxPeerHeight,
|
||||
"outbound", outbound, "inbound", inbound)
|
||||
if bcR.fsm.isCaughtUp() {
|
||||
bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
|
||||
bcR.fsm.stop()
|
||||
bcR.switchToConsensus()
|
||||
break ForLoop
|
||||
}
|
||||
|
||||
case <-trySyncTicker.C: // chan time
|
||||
select {
|
||||
case doProcessCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
case <-doProcessCh:
|
||||
err := bcR.processBlocksFromPoolRoutine()
|
||||
if err == errMissingBlocks {
|
||||
continue ForLoop
|
||||
}
|
||||
|
||||
// Notify FSM of block processing result.
|
||||
msgData := bReactorMessageData{
|
||||
event: processedBlockEv,
|
||||
data: bReactorEventData{
|
||||
err: err,
|
||||
},
|
||||
}
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msgData)
|
||||
|
||||
if err == errBlockVerificationFailure {
|
||||
continue ForLoop
|
||||
}
|
||||
|
||||
doProcessCh <- struct{}{}
|
||||
|
||||
bcR.blocksSynced++
|
||||
if bcR.blocksSynced%100 == 0 {
|
||||
lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
|
||||
bcR.Logger.Info("Fast Sync Rate", "height", bcR.fsm.pool.height,
|
||||
"max_peer_height", bcR.fsm.pool.getMaxPeerHeight(), "blocks/s", lastRate)
|
||||
lastHundred = time.Now()
|
||||
}
|
||||
|
||||
case msg := <-bcR.messagesForFSMCh:
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msg)
|
||||
|
||||
case msg := <-bcR.errorsForFSMCh:
|
||||
_ = sendMessageToFSMSync(bcR.fsm, msg)
|
||||
|
||||
case <-bcR.Quit():
|
||||
break ForLoop
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) reportPeerErrorToSwitch(err error, peerID p2p.ID) {
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer != nil {
|
||||
bcR.Switch.StopPeerForError(peer, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Called by FSM and pool:
|
||||
// - pool calls when it detects slow peer or when peer times out
|
||||
// - FSM calls when:
|
||||
// - processing a block (addBlock) fails
|
||||
// - BCR process of block reports failure to FSM, FSM sends back the peers of first and second
|
||||
func (bcR *BlockchainReactor) sendPeerError(err error, peerID p2p.ID) {
|
||||
bcR.errorsFromFSMCh <- peerError{err, peerID}
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) processBlocksFromPoolRoutine() error {
|
||||
firstBP, secondBP, err := bcR.fsm.pool.getNextTwoBlocks()
|
||||
if err != nil {
|
||||
// We need both to sync the first block.
|
||||
return err
|
||||
}
|
||||
|
||||
first := firstBP.block
|
||||
second := secondBP.block
|
||||
|
||||
chainID := bcR.initialState.ChainID
|
||||
|
||||
firstParts := first.MakePartSet(types.BlockPartSizeBytes)
|
||||
firstPartsHeader := firstParts.Header()
|
||||
firstID := types.BlockID{Hash: first.Hash(), PartsHeader: firstPartsHeader}
|
||||
// Finally, verify the first block using the second's commit
|
||||
// NOTE: we can probably make this more efficient, but note that calling
|
||||
// first.Hash() doesn't verify the tx contents, so MakePartSet() is
|
||||
// currently necessary.
|
||||
err = bcR.state.Validators.VerifyCommit(
|
||||
chainID, firstID, first.Height, second.LastCommit)
|
||||
|
||||
if err != nil {
|
||||
bcR.Logger.Error("error in validation", "err", err, first.Height, second.Height)
|
||||
return errBlockVerificationFailure
|
||||
}
|
||||
|
||||
bcR.store.SaveBlock(first, firstParts, second.LastCommit)
|
||||
|
||||
// Get the hash without persisting the state.
|
||||
bcR.state, err = bcR.blockExec.ApplyBlock(bcR.state, firstID, first)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) resetStateTimer(name string, timer *time.Timer, timeout time.Duration, f func()) {
|
||||
if timer == nil {
|
||||
timer = time.AfterFunc(timeout, f)
|
||||
} else {
|
||||
timer.Reset(timeout)
|
||||
}
|
||||
}
|
||||
|
||||
// BroadcastStatusRequest broadcasts `BlockStore` height.
|
||||
func (bcR *BlockchainReactor) sendStatusRequest() {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
|
||||
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
// BlockRequest sends `BlockRequest` height.
|
||||
func (bcR *BlockchainReactor) sendBlockRequest(peerID p2p.ID, height int64) error {
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer == nil {
|
||||
return errNilPeerForBlockRequest
|
||||
}
|
||||
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{height})
|
||||
queued := peer.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
return errSendQueueFull
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bcR *BlockchainReactor) switchToConsensus() {
|
||||
|
||||
conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
||||
if ok {
|
||||
conR.SwitchToConsensus(bcR.state, bcR.blocksSynced)
|
||||
} else {
|
||||
// Should only happen during testing.
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
// Messages
|
||||
|
||||
// BlockchainMessage is a generic message for this reactor.
|
||||
type BlockchainMessage interface {
|
||||
ValidateBasic() error
|
||||
}
|
||||
|
||||
func RegisterBlockchainMessages(cdc *amino.Codec) {
|
||||
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
|
||||
cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
|
||||
cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
|
||||
cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
|
||||
cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
|
||||
cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
|
||||
}
|
||||
|
||||
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
|
||||
if len(bz) > maxMsgSize {
|
||||
return msg, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
||||
}
|
||||
err = cdc.UnmarshalBinaryBare(bz, &msg)
|
||||
return
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcBlockRequestMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcBlockRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcBlockRequestMessage) String() string {
|
||||
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
|
||||
}
|
||||
|
||||
type bcNoBlockResponseMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcNoBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcBlockResponseMessage struct {
|
||||
Block *types.Block
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcBlockResponseMessage) ValidateBasic() error {
|
||||
return m.Block.ValidateBasic()
|
||||
}
|
||||
|
||||
func (m *bcBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcStatusRequestMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcStatusRequestMessage) String() string {
|
||||
return fmt.Sprintf("[bcStatusRequestMessage %v]", m.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcStatusResponseMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcStatusResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcStatusResponseMessage %v]", m.Height)
|
||||
}
|
630
blockchain_old/pool.go
Normal file
630
blockchain_old/pool.go
Normal file
@ -0,0 +1,630 @@
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
flow "github.com/tendermint/tendermint/libs/flowrate"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
/*
|
||||
eg, L = latency = 0.1s
|
||||
P = num peers = 10
|
||||
FN = num full nodes
|
||||
BS = 1kB block size
|
||||
CB = 1 Mbit/s = 128 kB/s
|
||||
CB/P = 12.8 kB
|
||||
B/S = CB/P/BS = 12.8 blocks/s
|
||||
|
||||
12.8 * 0.1 = 1.28 blocks on conn
|
||||
*/
|
||||
|
||||
const (
|
||||
requestIntervalMS = 2
|
||||
maxTotalRequesters = 600
|
||||
maxPendingRequests = maxTotalRequesters
|
||||
maxPendingRequestsPerPeer = 20
|
||||
|
||||
// Minimum recv rate to ensure we're receiving blocks from a peer fast
|
||||
// enough. If a peer is not sending us data at at least that rate, we
|
||||
// consider them to have timedout and we disconnect.
|
||||
//
|
||||
// Assuming a DSL connection (not a good choice) 128 Kbps (upload) ~ 15 KB/s,
|
||||
// sending data across atlantic ~ 7.5 KB/s.
|
||||
minRecvRate = 7680
|
||||
|
||||
// Maximum difference between current and new block's height.
|
||||
maxDiffBetweenCurrentAndReceivedBlockHeight = 100
|
||||
)
|
||||
|
||||
var peerTimeout = 15 * time.Second // not const so we can override with tests
|
||||
|
||||
/*
|
||||
Peers self report their heights when we join the block pool.
|
||||
Starting from our latest pool.height, we request blocks
|
||||
in sequence from peers that reported higher heights than ours.
|
||||
Every so often we ask peers what height they're on so we can keep going.
|
||||
|
||||
Requests are continuously made for blocks of higher heights until
|
||||
the limit is reached. If most of the requests have no available peers, and we
|
||||
are not at peer limits, we can probably switch to consensus reactor
|
||||
*/
|
||||
|
||||
type BlockPool struct {
|
||||
cmn.BaseService
|
||||
startTime time.Time
|
||||
|
||||
mtx sync.Mutex
|
||||
// block requests
|
||||
requesters map[int64]*bpRequester
|
||||
height int64 // the lowest key in requesters.
|
||||
// peers
|
||||
peers map[p2p.ID]*bpPeer
|
||||
maxPeerHeight int64 // the biggest reported height
|
||||
|
||||
// atomic
|
||||
numPending int32 // number of requests pending assignment or block response
|
||||
|
||||
requestsCh chan<- BlockRequest
|
||||
errorsCh chan<- peerError
|
||||
}
|
||||
|
||||
// NewBlockPool returns a new BlockPool with the height equal to start. Block
|
||||
// requests and errors will be sent to requestsCh and errorsCh accordingly.
|
||||
func NewBlockPool(start int64, requestsCh chan<- BlockRequest, errorsCh chan<- peerError) *BlockPool {
|
||||
bp := &BlockPool{
|
||||
peers: make(map[p2p.ID]*bpPeer),
|
||||
|
||||
requesters: make(map[int64]*bpRequester),
|
||||
height: start,
|
||||
numPending: 0,
|
||||
|
||||
requestsCh: requestsCh,
|
||||
errorsCh: errorsCh,
|
||||
}
|
||||
bp.BaseService = *cmn.NewBaseService(nil, "BlockPool", bp)
|
||||
return bp
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service by spawning requesters routine and recording
|
||||
// pool's start time.
|
||||
func (pool *BlockPool) OnStart() error {
|
||||
go pool.makeRequestersRoutine()
|
||||
pool.startTime = time.Now()
|
||||
return nil
|
||||
}
|
||||
|
||||
// spawns requesters as needed
|
||||
func (pool *BlockPool) makeRequestersRoutine() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
break
|
||||
}
|
||||
|
||||
_, numPending, lenRequesters := pool.GetStatus()
|
||||
if numPending >= maxPendingRequests {
|
||||
// sleep for a bit.
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
// check for timed out peers
|
||||
pool.removeTimedoutPeers()
|
||||
} else if lenRequesters >= maxTotalRequesters {
|
||||
// sleep for a bit.
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
// check for timed out peers
|
||||
pool.removeTimedoutPeers()
|
||||
} else {
|
||||
// request for more blocks.
|
||||
pool.makeNextRequester()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) removeTimedoutPeers() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
for _, peer := range pool.peers {
|
||||
if !peer.didTimeout && peer.numPending > 0 {
|
||||
curRate := peer.recvMonitor.Status().CurRate
|
||||
// curRate can be 0 on start
|
||||
if curRate != 0 && curRate < minRecvRate {
|
||||
err := errors.New("peer is not sending us data fast enough")
|
||||
pool.sendError(err, peer.id)
|
||||
pool.Logger.Error("SendTimeout", "peer", peer.id,
|
||||
"reason", err,
|
||||
"curRate", fmt.Sprintf("%d KB/s", curRate/1024),
|
||||
"minRate", fmt.Sprintf("%d KB/s", minRecvRate/1024))
|
||||
peer.didTimeout = true
|
||||
}
|
||||
}
|
||||
if peer.didTimeout {
|
||||
pool.removePeer(peer.id)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// GetStatus returns pool's height, numPending requests and the number of
|
||||
// requesters.
|
||||
func (pool *BlockPool) GetStatus() (height int64, numPending int32, lenRequesters int) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
return pool.height, atomic.LoadInt32(&pool.numPending), len(pool.requesters)
|
||||
}
|
||||
|
||||
// IsCaughtUp returns true if this node is caught up, false - otherwise.
|
||||
// TODO: relax conditions, prevent abuse.
|
||||
func (pool *BlockPool) IsCaughtUp() bool {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
// Need at least 1 peer to be considered caught up.
|
||||
if len(pool.peers) == 0 {
|
||||
pool.Logger.Debug("Blockpool has no peers")
|
||||
return false
|
||||
}
|
||||
|
||||
// Some conditions to determine if we're caught up.
|
||||
// Ensures we've either received a block or waited some amount of time,
|
||||
// and that we're synced to the highest known height.
|
||||
// Note we use maxPeerHeight - 1 because to sync block H requires block H+1
|
||||
// to verify the LastCommit.
|
||||
receivedBlockOrTimedOut := pool.height > 0 || time.Since(pool.startTime) > 5*time.Second
|
||||
ourChainIsLongestAmongPeers := pool.maxPeerHeight == 0 || pool.height >= (pool.maxPeerHeight-1)
|
||||
isCaughtUp := receivedBlockOrTimedOut && ourChainIsLongestAmongPeers
|
||||
return isCaughtUp
|
||||
}
|
||||
|
||||
// We need to see the second block's Commit to validate the first block.
|
||||
// So we peek two blocks at a time.
|
||||
// The caller will verify the commit.
|
||||
func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
if r := pool.requesters[pool.height]; r != nil {
|
||||
first = r.getBlock()
|
||||
}
|
||||
if r := pool.requesters[pool.height+1]; r != nil {
|
||||
second = r.getBlock()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Pop the first block at pool.height
|
||||
// It must have been validated by 'second'.Commit from PeekTwoBlocks().
|
||||
func (pool *BlockPool) PopRequest() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
if r := pool.requesters[pool.height]; r != nil {
|
||||
/* The block can disappear at any time, due to removePeer().
|
||||
if r := pool.requesters[pool.height]; r == nil || r.block == nil {
|
||||
PanicSanity("PopRequest() requires a valid block")
|
||||
}
|
||||
*/
|
||||
r.Stop()
|
||||
delete(pool.requesters, pool.height)
|
||||
pool.height++
|
||||
} else {
|
||||
panic(fmt.Sprintf("Expected requester to pop, got nothing at height %v", pool.height))
|
||||
}
|
||||
}
|
||||
|
||||
// Invalidates the block at pool.height,
|
||||
// Remove the peer and redo request from others.
|
||||
// Returns the ID of the removed peer.
|
||||
func (pool *BlockPool) RedoRequest(height int64) p2p.ID {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
request := pool.requesters[height]
|
||||
peerID := request.getPeerID()
|
||||
if peerID != p2p.ID("") {
|
||||
// RemovePeer will redo all requesters associated with this peer.
|
||||
pool.removePeer(peerID)
|
||||
}
|
||||
return peerID
|
||||
}
|
||||
|
||||
// TODO: ensure that blocks come in order for each peer.
|
||||
func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
requester := pool.requesters[block.Height]
|
||||
if requester == nil {
|
||||
pool.Logger.Info("peer sent us a block we didn't expect", "peer", peerID, "curHeight", pool.height, "blockHeight", block.Height)
|
||||
diff := pool.height - block.Height
|
||||
if diff < 0 {
|
||||
diff *= -1
|
||||
}
|
||||
if diff > maxDiffBetweenCurrentAndReceivedBlockHeight {
|
||||
pool.sendError(errors.New("peer sent us a block we didn't expect with a height too far ahead/behind"), peerID)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if requester.setBlock(block, peerID) {
|
||||
atomic.AddInt32(&pool.numPending, -1)
|
||||
peer := pool.peers[peerID]
|
||||
if peer != nil {
|
||||
peer.decrPending(blockSize)
|
||||
}
|
||||
} else {
|
||||
pool.Logger.Info("invalid peer", "peer", peerID, "blockHeight", block.Height)
|
||||
pool.sendError(errors.New("invalid peer"), peerID)
|
||||
}
|
||||
}
|
||||
|
||||
// MaxPeerHeight returns the highest reported height.
|
||||
func (pool *BlockPool) MaxPeerHeight() int64 {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
return pool.maxPeerHeight
|
||||
}
|
||||
|
||||
// SetPeerHeight sets the peer's alleged blockchain height.
|
||||
func (pool *BlockPool) SetPeerHeight(peerID p2p.ID, height int64) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
peer := pool.peers[peerID]
|
||||
if peer != nil {
|
||||
peer.height = height
|
||||
} else {
|
||||
peer = newBPPeer(pool, peerID, height)
|
||||
peer.setLogger(pool.Logger.With("peer", peerID))
|
||||
pool.peers[peerID] = peer
|
||||
}
|
||||
|
||||
if height > pool.maxPeerHeight {
|
||||
pool.maxPeerHeight = height
|
||||
}
|
||||
}
|
||||
|
||||
// RemovePeer removes the peer with peerID from the pool. If there's no peer
|
||||
// with peerID, function is a no-op.
|
||||
func (pool *BlockPool) RemovePeer(peerID p2p.ID) {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
pool.removePeer(peerID)
|
||||
}
|
||||
|
||||
func (pool *BlockPool) removePeer(peerID p2p.ID) {
|
||||
for _, requester := range pool.requesters {
|
||||
if requester.getPeerID() == peerID {
|
||||
requester.redo(peerID)
|
||||
}
|
||||
}
|
||||
|
||||
peer, ok := pool.peers[peerID]
|
||||
if ok {
|
||||
if peer.timeout != nil {
|
||||
peer.timeout.Stop()
|
||||
}
|
||||
|
||||
delete(pool.peers, peerID)
|
||||
|
||||
// Find a new peer with the biggest height and update maxPeerHeight if the
|
||||
// peer's height was the biggest.
|
||||
if peer.height == pool.maxPeerHeight {
|
||||
pool.updateMaxPeerHeight()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If no peers are left, maxPeerHeight is set to 0.
|
||||
func (pool *BlockPool) updateMaxPeerHeight() {
|
||||
var max int64
|
||||
for _, peer := range pool.peers {
|
||||
if peer.height > max {
|
||||
max = peer.height
|
||||
}
|
||||
}
|
||||
pool.maxPeerHeight = max
|
||||
}
|
||||
|
||||
// Pick an available peer with at least the given minHeight.
|
||||
// If no peers are available, returns nil.
|
||||
func (pool *BlockPool) pickIncrAvailablePeer(minHeight int64) *bpPeer {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
for _, peer := range pool.peers {
|
||||
if peer.didTimeout {
|
||||
pool.removePeer(peer.id)
|
||||
continue
|
||||
}
|
||||
if peer.numPending >= maxPendingRequestsPerPeer {
|
||||
continue
|
||||
}
|
||||
if peer.height < minHeight {
|
||||
continue
|
||||
}
|
||||
peer.incrPending()
|
||||
return peer
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pool *BlockPool) makeNextRequester() {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
nextHeight := pool.height + pool.requestersLen()
|
||||
if nextHeight > pool.maxPeerHeight {
|
||||
return
|
||||
}
|
||||
|
||||
request := newBPRequester(pool, nextHeight)
|
||||
|
||||
pool.requesters[nextHeight] = request
|
||||
atomic.AddInt32(&pool.numPending, 1)
|
||||
|
||||
err := request.Start()
|
||||
if err != nil {
|
||||
request.Logger.Error("Error starting request", "err", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) requestersLen() int64 {
|
||||
return int64(len(pool.requesters))
|
||||
}
|
||||
|
||||
func (pool *BlockPool) sendRequest(height int64, peerID p2p.ID) {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
pool.requestsCh <- BlockRequest{height, peerID}
|
||||
}
|
||||
|
||||
func (pool *BlockPool) sendError(err error, peerID p2p.ID) {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
pool.errorsCh <- peerError{err, peerID}
|
||||
}
|
||||
|
||||
// for debugging purposes
|
||||
//nolint:unused
|
||||
func (pool *BlockPool) debug() string {
|
||||
pool.mtx.Lock()
|
||||
defer pool.mtx.Unlock()
|
||||
|
||||
str := ""
|
||||
nextHeight := pool.height + pool.requestersLen()
|
||||
for h := pool.height; h < nextHeight; h++ {
|
||||
if pool.requesters[h] == nil {
|
||||
str += fmt.Sprintf("H(%v):X ", h)
|
||||
} else {
|
||||
str += fmt.Sprintf("H(%v):", h)
|
||||
str += fmt.Sprintf("B?(%v) ", pool.requesters[h].block != nil)
|
||||
}
|
||||
}
|
||||
return str
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bpPeer struct {
|
||||
pool *BlockPool
|
||||
id p2p.ID
|
||||
recvMonitor *flow.Monitor
|
||||
|
||||
height int64
|
||||
numPending int32
|
||||
timeout *time.Timer
|
||||
didTimeout bool
|
||||
|
||||
logger log.Logger
|
||||
}
|
||||
|
||||
func newBPPeer(pool *BlockPool, peerID p2p.ID, height int64) *bpPeer {
|
||||
peer := &bpPeer{
|
||||
pool: pool,
|
||||
id: peerID,
|
||||
height: height,
|
||||
numPending: 0,
|
||||
logger: log.NewNopLogger(),
|
||||
}
|
||||
return peer
|
||||
}
|
||||
|
||||
func (peer *bpPeer) setLogger(l log.Logger) {
|
||||
peer.logger = l
|
||||
}
|
||||
|
||||
func (peer *bpPeer) resetMonitor() {
|
||||
peer.recvMonitor = flow.New(time.Second, time.Second*40)
|
||||
initialValue := float64(minRecvRate) * math.E
|
||||
peer.recvMonitor.SetREMA(initialValue)
|
||||
}
|
||||
|
||||
func (peer *bpPeer) resetTimeout() {
|
||||
if peer.timeout == nil {
|
||||
peer.timeout = time.AfterFunc(peerTimeout, peer.onTimeout)
|
||||
} else {
|
||||
peer.timeout.Reset(peerTimeout)
|
||||
}
|
||||
}
|
||||
|
||||
func (peer *bpPeer) incrPending() {
|
||||
if peer.numPending == 0 {
|
||||
peer.resetMonitor()
|
||||
peer.resetTimeout()
|
||||
}
|
||||
peer.numPending++
|
||||
}
|
||||
|
||||
func (peer *bpPeer) decrPending(recvSize int) {
|
||||
peer.numPending--
|
||||
if peer.numPending == 0 {
|
||||
peer.timeout.Stop()
|
||||
} else {
|
||||
peer.recvMonitor.Update(recvSize)
|
||||
peer.resetTimeout()
|
||||
}
|
||||
}
|
||||
|
||||
func (peer *bpPeer) onTimeout() {
|
||||
peer.pool.mtx.Lock()
|
||||
defer peer.pool.mtx.Unlock()
|
||||
|
||||
err := errors.New("peer did not send us anything")
|
||||
peer.pool.sendError(err, peer.id)
|
||||
peer.logger.Error("SendTimeout", "reason", err, "timeout", peerTimeout)
|
||||
peer.didTimeout = true
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bpRequester struct {
|
||||
cmn.BaseService
|
||||
pool *BlockPool
|
||||
height int64
|
||||
gotBlockCh chan struct{}
|
||||
redoCh chan p2p.ID //redo may send multitime, add peerId to identify repeat
|
||||
|
||||
mtx sync.Mutex
|
||||
peerID p2p.ID
|
||||
block *types.Block
|
||||
}
|
||||
|
||||
func newBPRequester(pool *BlockPool, height int64) *bpRequester {
|
||||
bpr := &bpRequester{
|
||||
pool: pool,
|
||||
height: height,
|
||||
gotBlockCh: make(chan struct{}, 1),
|
||||
redoCh: make(chan p2p.ID, 1),
|
||||
|
||||
peerID: "",
|
||||
block: nil,
|
||||
}
|
||||
bpr.BaseService = *cmn.NewBaseService(nil, "bpRequester", bpr)
|
||||
return bpr
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) OnStart() error {
|
||||
go bpr.requestRoutine()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Returns true if the peer matches and block doesn't already exist.
|
||||
func (bpr *bpRequester) setBlock(block *types.Block, peerID p2p.ID) bool {
|
||||
bpr.mtx.Lock()
|
||||
if bpr.block != nil || bpr.peerID != peerID {
|
||||
bpr.mtx.Unlock()
|
||||
return false
|
||||
}
|
||||
bpr.block = block
|
||||
bpr.mtx.Unlock()
|
||||
|
||||
select {
|
||||
case bpr.gotBlockCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) getBlock() *types.Block {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
return bpr.block
|
||||
}
|
||||
|
||||
func (bpr *bpRequester) getPeerID() p2p.ID {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
return bpr.peerID
|
||||
}
|
||||
|
||||
// This is called from the requestRoutine, upon redo().
|
||||
func (bpr *bpRequester) reset() {
|
||||
bpr.mtx.Lock()
|
||||
defer bpr.mtx.Unlock()
|
||||
|
||||
if bpr.block != nil {
|
||||
atomic.AddInt32(&bpr.pool.numPending, 1)
|
||||
}
|
||||
|
||||
bpr.peerID = ""
|
||||
bpr.block = nil
|
||||
}
|
||||
|
||||
// Tells bpRequester to pick another peer and try again.
|
||||
// NOTE: Nonblocking, and does nothing if another redo
|
||||
// was already requested.
|
||||
func (bpr *bpRequester) redo(peerId p2p.ID) {
|
||||
select {
|
||||
case bpr.redoCh <- peerId:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// Responsible for making more requests as necessary
|
||||
// Returns only when a block is found (e.g. AddBlock() is called)
|
||||
func (bpr *bpRequester) requestRoutine() {
|
||||
OUTER_LOOP:
|
||||
for {
|
||||
// Pick a peer to send request to.
|
||||
var peer *bpPeer
|
||||
PICK_PEER_LOOP:
|
||||
for {
|
||||
if !bpr.IsRunning() || !bpr.pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
peer = bpr.pool.pickIncrAvailablePeer(bpr.height)
|
||||
if peer == nil {
|
||||
//log.Info("No peers available", "height", height)
|
||||
time.Sleep(requestIntervalMS * time.Millisecond)
|
||||
continue PICK_PEER_LOOP
|
||||
}
|
||||
break PICK_PEER_LOOP
|
||||
}
|
||||
bpr.mtx.Lock()
|
||||
bpr.peerID = peer.id
|
||||
bpr.mtx.Unlock()
|
||||
|
||||
// Send request and wait.
|
||||
bpr.pool.sendRequest(bpr.height, peer.id)
|
||||
WAIT_LOOP:
|
||||
for {
|
||||
select {
|
||||
case <-bpr.pool.Quit():
|
||||
bpr.Stop()
|
||||
return
|
||||
case <-bpr.Quit():
|
||||
return
|
||||
case peerID := <-bpr.redoCh:
|
||||
if peerID == bpr.peerID {
|
||||
bpr.reset()
|
||||
continue OUTER_LOOP
|
||||
} else {
|
||||
continue WAIT_LOOP
|
||||
}
|
||||
case <-bpr.gotBlockCh:
|
||||
// We got a block!
|
||||
// Continue the for-loop and wait til Quit.
|
||||
continue WAIT_LOOP
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type BlockRequest struct {
|
||||
Height int64
|
||||
PeerID p2p.ID
|
||||
}
|
222
blockchain_old/pool_test.go
Normal file
222
blockchain_old/pool_test.go
Normal file
@ -0,0 +1,222 @@
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func init() {
|
||||
peerTimeout = 2 * time.Second
|
||||
}
|
||||
|
||||
type testPeer struct {
|
||||
id p2p.ID
|
||||
height int64
|
||||
inputChan chan inputData //make sure each peer's data is sequential
|
||||
}
|
||||
|
||||
type inputData struct {
|
||||
t *testing.T
|
||||
pool *BlockPool
|
||||
request BlockRequest
|
||||
}
|
||||
|
||||
func (p testPeer) runInputRoutine() {
|
||||
go func() {
|
||||
for input := range p.inputChan {
|
||||
p.simulateInput(input)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// Request desired, pretend like we got the block immediately.
|
||||
func (p testPeer) simulateInput(input inputData) {
|
||||
block := &types.Block{Header: types.Header{Height: input.request.Height}}
|
||||
input.pool.AddBlock(input.request.PeerID, block, 123)
|
||||
input.t.Logf("Added block from peer %v (height: %v)", input.request.PeerID, input.request.Height)
|
||||
}
|
||||
|
||||
type testPeers map[p2p.ID]testPeer
|
||||
|
||||
func (ps testPeers) start() {
|
||||
for _, v := range ps {
|
||||
v.runInputRoutine()
|
||||
}
|
||||
}
|
||||
|
||||
func (ps testPeers) stop() {
|
||||
for _, v := range ps {
|
||||
close(v.inputChan)
|
||||
}
|
||||
}
|
||||
|
||||
func makePeers(numPeers int, minHeight, maxHeight int64) testPeers {
|
||||
peers := make(testPeers, numPeers)
|
||||
for i := 0; i < numPeers; i++ {
|
||||
peerID := p2p.ID(cmn.RandStr(12))
|
||||
height := minHeight + cmn.RandInt63n(maxHeight-minHeight)
|
||||
peers[peerID] = testPeer{peerID, height, make(chan inputData, 10)}
|
||||
}
|
||||
return peers
|
||||
}
|
||||
|
||||
func TestBlockPoolBasic(t *testing.T) {
|
||||
start := int64(42)
|
||||
peers := makePeers(10, start+1, 1000)
|
||||
errorsCh := make(chan peerError, 1000)
|
||||
requestsCh := make(chan BlockRequest, 1000)
|
||||
pool := NewBlockPool(start, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
|
||||
err := pool.Start()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
defer pool.Stop()
|
||||
|
||||
peers.start()
|
||||
defer peers.stop()
|
||||
|
||||
// Introduce each peer.
|
||||
go func() {
|
||||
for _, peer := range peers {
|
||||
pool.SetPeerHeight(peer.id, peer.height)
|
||||
}
|
||||
}()
|
||||
|
||||
// Start a goroutine to pull blocks
|
||||
go func() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
first, second := pool.PeekTwoBlocks()
|
||||
if first != nil && second != nil {
|
||||
pool.PopRequest()
|
||||
} else {
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Pull from channels
|
||||
for {
|
||||
select {
|
||||
case err := <-errorsCh:
|
||||
t.Error(err)
|
||||
case request := <-requestsCh:
|
||||
t.Logf("Pulled new BlockRequest %v", request)
|
||||
if request.Height == 300 {
|
||||
return // Done!
|
||||
}
|
||||
|
||||
peers[request.PeerID].inputChan <- inputData{t, pool, request}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolTimeout(t *testing.T) {
|
||||
start := int64(42)
|
||||
peers := makePeers(10, start+1, 1000)
|
||||
errorsCh := make(chan peerError, 1000)
|
||||
requestsCh := make(chan BlockRequest, 1000)
|
||||
pool := NewBlockPool(start, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
err := pool.Start()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
defer pool.Stop()
|
||||
|
||||
for _, peer := range peers {
|
||||
t.Logf("Peer %v", peer.id)
|
||||
}
|
||||
|
||||
// Introduce each peer.
|
||||
go func() {
|
||||
for _, peer := range peers {
|
||||
pool.SetPeerHeight(peer.id, peer.height)
|
||||
}
|
||||
}()
|
||||
|
||||
// Start a goroutine to pull blocks
|
||||
go func() {
|
||||
for {
|
||||
if !pool.IsRunning() {
|
||||
return
|
||||
}
|
||||
first, second := pool.PeekTwoBlocks()
|
||||
if first != nil && second != nil {
|
||||
pool.PopRequest()
|
||||
} else {
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Pull from channels
|
||||
counter := 0
|
||||
timedOut := map[p2p.ID]struct{}{}
|
||||
for {
|
||||
select {
|
||||
case err := <-errorsCh:
|
||||
t.Log(err)
|
||||
// consider error to be always timeout here
|
||||
if _, ok := timedOut[err.peerID]; !ok {
|
||||
counter++
|
||||
if counter == len(peers) {
|
||||
return // Done!
|
||||
}
|
||||
}
|
||||
case request := <-requestsCh:
|
||||
t.Logf("Pulled new BlockRequest %+v", request)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestBlockPoolRemovePeer(t *testing.T) {
|
||||
peers := make(testPeers, 10)
|
||||
for i := 0; i < 10; i++ {
|
||||
peerID := p2p.ID(fmt.Sprintf("%d", i+1))
|
||||
height := int64(i + 1)
|
||||
peers[peerID] = testPeer{peerID, height, make(chan inputData)}
|
||||
}
|
||||
requestsCh := make(chan BlockRequest)
|
||||
errorsCh := make(chan peerError)
|
||||
|
||||
pool := NewBlockPool(1, requestsCh, errorsCh)
|
||||
pool.SetLogger(log.TestingLogger())
|
||||
err := pool.Start()
|
||||
require.NoError(t, err)
|
||||
defer pool.Stop()
|
||||
|
||||
// add peers
|
||||
for peerID, peer := range peers {
|
||||
pool.SetPeerHeight(peerID, peer.height)
|
||||
}
|
||||
assert.EqualValues(t, 10, pool.MaxPeerHeight())
|
||||
|
||||
// remove not-existing peer
|
||||
assert.NotPanics(t, func() { pool.RemovePeer(p2p.ID("Superman")) })
|
||||
|
||||
// remove peer with biggest height
|
||||
pool.RemovePeer(p2p.ID("10"))
|
||||
assert.EqualValues(t, 9, pool.MaxPeerHeight())
|
||||
|
||||
// remove all peers
|
||||
for peerID := range peers {
|
||||
pool.RemovePeer(peerID)
|
||||
}
|
||||
|
||||
assert.EqualValues(t, 0, pool.MaxPeerHeight())
|
||||
}
|
474
blockchain_old/reactor.go
Normal file
474
blockchain_old/reactor.go
Normal file
@ -0,0 +1,474 @@
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
const (
|
||||
// BlockchainChannel is a channel for blocks and status updates (`BlockStore` height)
|
||||
BlockchainChannel = byte(0x40)
|
||||
|
||||
trySyncIntervalMS = 10
|
||||
|
||||
// stop syncing when last block's time is
|
||||
// within this much of the system time.
|
||||
// stopSyncingDurationMinutes = 10
|
||||
|
||||
// ask for best height every 10s
|
||||
statusUpdateIntervalSeconds = 10
|
||||
// check if we should switch to consensus reactor
|
||||
switchToConsensusIntervalSeconds = 1
|
||||
|
||||
// NOTE: keep up to date with bcBlockResponseMessage
|
||||
bcBlockResponseMessagePrefixSize = 4
|
||||
bcBlockResponseMessageFieldKeySize = 1
|
||||
maxMsgSize = types.MaxBlockSizeBytes +
|
||||
bcBlockResponseMessagePrefixSize +
|
||||
bcBlockResponseMessageFieldKeySize
|
||||
)
|
||||
|
||||
type consensusReactor interface {
|
||||
// for when we switch from blockchain reactor and fast sync to
|
||||
// the consensus machine
|
||||
SwitchToConsensus(sm.State, int)
|
||||
}
|
||||
|
||||
type peerError struct {
|
||||
err error
|
||||
peerID p2p.ID
|
||||
}
|
||||
|
||||
func (e peerError) Error() string {
|
||||
return fmt.Sprintf("error with peer %v: %s", e.peerID, e.err.Error())
|
||||
}
|
||||
|
||||
// BlockchainReactor handles long-term catchup syncing.
|
||||
type BlockchainReactor struct {
|
||||
p2p.BaseReactor
|
||||
|
||||
// immutable
|
||||
initialState sm.State
|
||||
|
||||
blockExec *sm.BlockExecutor
|
||||
store *BlockStore
|
||||
pool *BlockPool
|
||||
fastSync bool
|
||||
|
||||
requestsCh <-chan BlockRequest
|
||||
errorsCh <-chan peerError
|
||||
}
|
||||
|
||||
// NewBlockchainReactor returns new reactor instance.
|
||||
func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *BlockStore,
|
||||
fastSync bool) *BlockchainReactor {
|
||||
|
||||
if state.LastBlockHeight != store.Height() {
|
||||
panic(fmt.Sprintf("state (%v) and store (%v) height mismatch", state.LastBlockHeight,
|
||||
store.Height()))
|
||||
}
|
||||
|
||||
requestsCh := make(chan BlockRequest, maxTotalRequesters)
|
||||
|
||||
const capacity = 1000 // must be bigger than peers count
|
||||
errorsCh := make(chan peerError, capacity) // so we don't block in #Receive#pool.AddBlock
|
||||
|
||||
pool := NewBlockPool(
|
||||
store.Height()+1,
|
||||
requestsCh,
|
||||
errorsCh,
|
||||
)
|
||||
|
||||
bcR := &BlockchainReactor{
|
||||
initialState: state,
|
||||
blockExec: blockExec,
|
||||
store: store,
|
||||
pool: pool,
|
||||
fastSync: fastSync,
|
||||
requestsCh: requestsCh,
|
||||
errorsCh: errorsCh,
|
||||
}
|
||||
bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
|
||||
return bcR
|
||||
}
|
||||
|
||||
// SetLogger implements cmn.Service by setting the logger on reactor and pool.
|
||||
func (bcR *BlockchainReactor) SetLogger(l log.Logger) {
|
||||
bcR.BaseService.Logger = l
|
||||
bcR.pool.Logger = l
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStart() error {
|
||||
if bcR.fastSync {
|
||||
err := bcR.pool.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go bcR.poolRoutine()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// OnStop implements cmn.Service.
|
||||
func (bcR *BlockchainReactor) OnStop() {
|
||||
bcR.pool.Stop()
|
||||
}
|
||||
|
||||
// GetChannels implements Reactor
|
||||
func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
|
||||
return []*p2p.ChannelDescriptor{
|
||||
{
|
||||
ID: BlockchainChannel,
|
||||
Priority: 10,
|
||||
SendQueueCapacity: 1000,
|
||||
RecvBufferCapacity: 50 * 4096,
|
||||
RecvMessageCapacity: maxMsgSize,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// AddPeer implements Reactor by sending our state to peer.
|
||||
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
if !peer.Send(BlockchainChannel, msgBytes) {
|
||||
// doing nothing, will try later in `poolRoutine`
|
||||
}
|
||||
// peer is added to the pool once we receive the first
|
||||
// bcStatusResponseMessage from the peer and call pool.SetPeerHeight
|
||||
}
|
||||
|
||||
// RemovePeer implements Reactor by removing peer from the pool.
|
||||
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
bcR.pool.RemovePeer(peer.ID())
|
||||
}
|
||||
|
||||
// respondToPeer loads a block and sends it to the requesting peer,
|
||||
// if we have it. Otherwise, we'll respond saying we don't have it.
|
||||
// According to the Tendermint spec, if all nodes are honest,
|
||||
// no node should be requesting for a block that's non-existent.
|
||||
func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
|
||||
src p2p.Peer) (queued bool) {
|
||||
|
||||
block := bcR.store.LoadBlock(msg.Height)
|
||||
if block != nil {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
|
||||
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
|
||||
return src.TrySend(BlockchainChannel, msgBytes)
|
||||
}
|
||||
|
||||
// Receive implements Reactor by handling 4 types of messages (look below).
|
||||
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
||||
msg, err := decodeMsg(msgBytes)
|
||||
if err != nil {
|
||||
bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
if err = msg.ValidateBasic(); err != nil {
|
||||
bcR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
||||
bcR.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
|
||||
|
||||
switch msg := msg.(type) {
|
||||
case *bcBlockRequestMessage:
|
||||
if queued := bcR.respondToPeer(msg, src); !queued {
|
||||
// Unfortunately not queued since the queue is full.
|
||||
}
|
||||
case *bcBlockResponseMessage:
|
||||
bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes))
|
||||
case *bcStatusRequestMessage:
|
||||
// Send peer our state.
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
||||
queued := src.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
// sorry
|
||||
}
|
||||
case *bcStatusResponseMessage:
|
||||
// Got a peer status. Unverified.
|
||||
bcR.pool.SetPeerHeight(src.ID(), msg.Height)
|
||||
default:
|
||||
bcR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
||||
}
|
||||
}
|
||||
|
||||
// Handle messages from the poolReactor telling the reactor what to do.
|
||||
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
||||
func (bcR *BlockchainReactor) poolRoutine() {
|
||||
|
||||
trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
|
||||
statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
|
||||
switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
|
||||
|
||||
blocksSynced := 0
|
||||
|
||||
chainID := bcR.initialState.ChainID
|
||||
state := bcR.initialState
|
||||
|
||||
lastHundred := time.Now()
|
||||
lastRate := 0.0
|
||||
|
||||
didProcessCh := make(chan struct{}, 1)
|
||||
|
||||
FOR_LOOP:
|
||||
for {
|
||||
select {
|
||||
case request := <-bcR.requestsCh:
|
||||
peer := bcR.Switch.Peers().Get(request.PeerID)
|
||||
if peer == nil {
|
||||
continue FOR_LOOP // Peer has since been disconnected.
|
||||
}
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{request.Height})
|
||||
queued := peer.TrySend(BlockchainChannel, msgBytes)
|
||||
if !queued {
|
||||
// We couldn't make the request, send-queue full.
|
||||
// The pool handles timeouts, just let it go.
|
||||
continue FOR_LOOP
|
||||
}
|
||||
|
||||
case err := <-bcR.errorsCh:
|
||||
peer := bcR.Switch.Peers().Get(err.peerID)
|
||||
if peer != nil {
|
||||
bcR.Switch.StopPeerForError(peer, err)
|
||||
}
|
||||
|
||||
case <-statusUpdateTicker.C:
|
||||
// ask for status updates
|
||||
go bcR.BroadcastStatusRequest() // nolint: errcheck
|
||||
|
||||
case <-switchToConsensusTicker.C:
|
||||
height, numPending, lenRequesters := bcR.pool.GetStatus()
|
||||
outbound, inbound, _ := bcR.Switch.NumPeers()
|
||||
bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "total", lenRequesters,
|
||||
"outbound", outbound, "inbound", inbound)
|
||||
if bcR.pool.IsCaughtUp() {
|
||||
bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
|
||||
bcR.pool.Stop()
|
||||
|
||||
conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
||||
if ok {
|
||||
conR.SwitchToConsensus(state, blocksSynced)
|
||||
} else {
|
||||
// should only happen during testing
|
||||
}
|
||||
|
||||
break FOR_LOOP
|
||||
}
|
||||
|
||||
case <-trySyncTicker.C: // chan time
|
||||
select {
|
||||
case didProcessCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
case <-didProcessCh:
|
||||
// NOTE: It is a subtle mistake to process more than a single block
|
||||
// at a time (e.g. 10) here, because we only TrySend 1 request per
|
||||
// loop. The ratio mismatch can result in starving of blocks, a
|
||||
// sudden burst of requests and responses, and repeat.
|
||||
// Consequently, it is better to split these routines rather than
|
||||
// coupling them as it's written here. TODO uncouple from request
|
||||
// routine.
|
||||
|
||||
// See if there are any blocks to sync.
|
||||
first, second := bcR.pool.PeekTwoBlocks()
|
||||
//bcR.Logger.Info("TrySync peeked", "first", first, "second", second)
|
||||
if first == nil || second == nil {
|
||||
// We need both to sync the first block.
|
||||
continue FOR_LOOP
|
||||
} else {
|
||||
// Try again quickly next loop.
|
||||
didProcessCh <- struct{}{}
|
||||
}
|
||||
|
||||
firstParts := first.MakePartSet(types.BlockPartSizeBytes)
|
||||
firstPartsHeader := firstParts.Header()
|
||||
firstID := types.BlockID{Hash: first.Hash(), PartsHeader: firstPartsHeader}
|
||||
// Finally, verify the first block using the second's commit
|
||||
// NOTE: we can probably make this more efficient, but note that calling
|
||||
// first.Hash() doesn't verify the tx contents, so MakePartSet() is
|
||||
// currently necessary.
|
||||
err := state.Validators.VerifyCommit(
|
||||
chainID, firstID, first.Height, second.LastCommit)
|
||||
if err != nil {
|
||||
bcR.Logger.Error("Error in validation", "err", err)
|
||||
peerID := bcR.pool.RedoRequest(first.Height)
|
||||
peer := bcR.Switch.Peers().Get(peerID)
|
||||
if peer != nil {
|
||||
// NOTE: we've already removed the peer's request, but we
|
||||
// still need to clean up the rest.
|
||||
bcR.Switch.StopPeerForError(peer, fmt.Errorf("BlockchainReactor validation error: %v", err))
|
||||
}
|
||||
peerID2 := bcR.pool.RedoRequest(second.Height)
|
||||
peer2 := bcR.Switch.Peers().Get(peerID2)
|
||||
if peer2 != nil && peer2 != peer {
|
||||
// NOTE: we've already removed the peer's request, but we
|
||||
// still need to clean up the rest.
|
||||
bcR.Switch.StopPeerForError(peer2, fmt.Errorf("BlockchainReactor validation error: %v", err))
|
||||
}
|
||||
continue FOR_LOOP
|
||||
} else {
|
||||
bcR.pool.PopRequest()
|
||||
|
||||
// TODO: batch saves so we dont persist to disk every block
|
||||
bcR.store.SaveBlock(first, firstParts, second.LastCommit)
|
||||
|
||||
// TODO: same thing for app - but we would need a way to
|
||||
// get the hash without persisting the state
|
||||
var err error
|
||||
state, err = bcR.blockExec.ApplyBlock(state, firstID, first)
|
||||
if err != nil {
|
||||
// TODO This is bad, are we zombie?
|
||||
panic(fmt.Sprintf("Failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
|
||||
}
|
||||
blocksSynced++
|
||||
|
||||
if blocksSynced%100 == 0 {
|
||||
lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
|
||||
bcR.Logger.Info("Fast Sync Rate", "height", bcR.pool.height,
|
||||
"max_peer_height", bcR.pool.MaxPeerHeight(), "blocks/s", lastRate)
|
||||
lastHundred = time.Now()
|
||||
}
|
||||
}
|
||||
continue FOR_LOOP
|
||||
|
||||
case <-bcR.Quit():
|
||||
break FOR_LOOP
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// BroadcastStatusRequest broadcasts `BlockStore` height.
|
||||
func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
|
||||
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
|
||||
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
|
||||
return nil
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
// Messages
|
||||
|
||||
// BlockchainMessage is a generic message for this reactor.
|
||||
type BlockchainMessage interface {
|
||||
ValidateBasic() error
|
||||
}
|
||||
|
||||
func RegisterBlockchainMessages(cdc *amino.Codec) {
|
||||
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
|
||||
cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
|
||||
cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
|
||||
cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
|
||||
cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
|
||||
cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
|
||||
}
|
||||
|
||||
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
|
||||
if len(bz) > maxMsgSize {
|
||||
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
||||
}
|
||||
err = cdc.UnmarshalBinaryBare(bz, &msg)
|
||||
return
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcBlockRequestMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcBlockRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcBlockRequestMessage) String() string {
|
||||
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
|
||||
}
|
||||
|
||||
type bcNoBlockResponseMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (brm *bcNoBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", brm.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcBlockResponseMessage struct {
|
||||
Block *types.Block
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcBlockResponseMessage) ValidateBasic() error {
|
||||
return m.Block.ValidateBasic()
|
||||
}
|
||||
|
||||
func (m *bcBlockResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcStatusRequestMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusRequestMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcStatusRequestMessage) String() string {
|
||||
return fmt.Sprintf("[bcStatusRequestMessage %v]", m.Height)
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
|
||||
type bcStatusResponseMessage struct {
|
||||
Height int64
|
||||
}
|
||||
|
||||
// ValidateBasic performs basic validation.
|
||||
func (m *bcStatusResponseMessage) ValidateBasic() error {
|
||||
if m.Height < 0 {
|
||||
return errors.New("Negative Height")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *bcStatusResponseMessage) String() string {
|
||||
return fmt.Sprintf("[bcStatusResponseMessage %v]", m.Height)
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
@ -8,6 +8,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
@ -55,7 +56,7 @@ func makeVote(header *types.Header, blockID types.BlockID, valset *types.Validat
|
||||
BlockID: blockID,
|
||||
}
|
||||
|
||||
_ = privVal.SignVote(header.ChainID, vote)
|
||||
privVal.SignVote(header.ChainID, vote)
|
||||
|
||||
return vote
|
||||
}
|
||||
@ -108,7 +109,7 @@ func newBlockchainReactor(logger log.Logger, genDoc *types.GenesisDoc, privVals
|
||||
thisBlock := makeBlock(blockHeight, state, lastCommit)
|
||||
|
||||
thisParts := thisBlock.MakePartSet(types.BlockPartSizeBytes)
|
||||
blockID := types.BlockID{Hash: thisBlock.Hash(), PartsHeader: thisParts.Header()}
|
||||
blockID := types.BlockID{thisBlock.Hash(), thisParts.Header()}
|
||||
|
||||
state, err = blockExec.ApplyBlock(state, blockID, thisBlock)
|
||||
if err != nil {
|
||||
@ -125,11 +126,7 @@ func newBlockchainReactor(logger log.Logger, genDoc *types.GenesisDoc, privVals
|
||||
}
|
||||
|
||||
func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
peerTimeout = 15 * time.Second
|
||||
maxRequestsPerPeer = 20
|
||||
maxNumPendingRequests = 100
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_new_reactor_test")
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
defer os.RemoveAll(config.RootDir)
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
|
||||
@ -137,23 +134,19 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
|
||||
reactorPairs := make([]BlockchainReactorPair, 2)
|
||||
|
||||
logger := log.TestingLogger()
|
||||
reactorPairs[0] = newBlockchainReactor(logger, genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(logger, genDoc, privVals, 0)
|
||||
reactorPairs[0] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
|
||||
p2p.MakeConnectedSwitches(config.P2P, 2, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger.With("module", moduleName))
|
||||
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
defer func() {
|
||||
for _, r := range reactorPairs {
|
||||
_ = r.reactor.Stop()
|
||||
_ = r.app.Stop()
|
||||
r.reactor.Stop()
|
||||
r.app.Stop()
|
||||
}
|
||||
}()
|
||||
|
||||
@ -168,10 +161,11 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
}
|
||||
|
||||
for {
|
||||
time.Sleep(1 * time.Second)
|
||||
if reactorPairs[1].reactor.fsm.isCaughtUp() {
|
||||
if reactorPairs[1].reactor.pool.IsCaughtUp() {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
|
||||
assert.Equal(t, maxBlockHeight, reactorPairs[0].reactor.store.Height())
|
||||
@ -192,12 +186,6 @@ func TestFastSyncNoBlockResponse(t *testing.T) {
|
||||
// Alternatively we could actually dial a TCP conn but
|
||||
// that seems extreme.
|
||||
func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
peerTimeout = 15 * time.Second
|
||||
|
||||
maxRequestsPerPeer = 20
|
||||
maxNumPendingRequests = 400
|
||||
numNodes := 4
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
defer os.RemoveAll(config.RootDir)
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
@ -206,59 +194,49 @@ func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
|
||||
otherChain := newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
defer func() {
|
||||
_ = otherChain.reactor.Stop()
|
||||
_ = otherChain.app.Stop()
|
||||
otherChain.reactor.Stop()
|
||||
otherChain.app.Stop()
|
||||
}()
|
||||
|
||||
reactorPairs := make([]BlockchainReactorPair, numNodes)
|
||||
reactorPairs := make([]BlockchainReactorPair, 4)
|
||||
|
||||
var logger = make([]log.Logger, numNodes)
|
||||
reactorPairs[0] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, maxBlockHeight)
|
||||
reactorPairs[1] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
reactorPairs[2] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
reactorPairs[3] = newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
|
||||
for i := 0; i < numNodes; i++ {
|
||||
logger[i] = log.TestingLogger()
|
||||
height := int64(0)
|
||||
if i == 0 {
|
||||
height = maxBlockHeight
|
||||
}
|
||||
reactorPairs[i] = newBlockchainReactor(logger[i], genDoc, privVals, height)
|
||||
}
|
||||
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, numNodes, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, 4, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
defer func() {
|
||||
for _, r := range reactorPairs {
|
||||
_ = r.reactor.Stop()
|
||||
_ = r.app.Stop()
|
||||
r.reactor.Stop()
|
||||
r.app.Stop()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
time.Sleep(1 * time.Second)
|
||||
if reactorPairs[numNodes-1].reactor.fsm.isCaughtUp() || reactorPairs[numNodes-1].reactor.Switch.Peers().Size() == 0 {
|
||||
if reactorPairs[3].reactor.pool.IsCaughtUp() {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
|
||||
//at this time, reactors[0-3] is the newest
|
||||
assert.Equal(t, numNodes-1, reactorPairs[1].reactor.Switch.Peers().Size())
|
||||
assert.Equal(t, 3, reactorPairs[1].reactor.Switch.Peers().Size())
|
||||
|
||||
//mark reactorPairs[3] is an invalid peer
|
||||
reactorPairs[numNodes-1].reactor.store = otherChain.reactor.store
|
||||
reactorPairs[3].reactor.store = otherChain.reactor.store
|
||||
|
||||
lastLogger := log.TestingLogger()
|
||||
lastReactorPair := newBlockchainReactor(lastLogger, genDoc, privVals, 0)
|
||||
lastReactorPair := newBlockchainReactor(log.TestingLogger(), genDoc, privVals, 0)
|
||||
reactorPairs = append(reactorPairs, lastReactorPair)
|
||||
|
||||
switches = append(switches, p2p.MakeConnectedSwitches(config.P2P, 1, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[len(reactorPairs)-1].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", len(reactorPairs)-1)
|
||||
reactorPairs[len(reactorPairs)-1].reactor.SetLogger(lastLogger.With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)...)
|
||||
@ -268,13 +246,16 @@ func TestFastSyncBadBlockStopsPeer(t *testing.T) {
|
||||
}
|
||||
|
||||
for {
|
||||
time.Sleep(1 * time.Second)
|
||||
if lastReactorPair.reactor.fsm.isCaughtUp() || lastReactorPair.reactor.Switch.Peers().Size() == 0 {
|
||||
if lastReactorPair.reactor.pool.IsCaughtUp() || lastReactorPair.reactor.Switch.Peers().Size() == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
|
||||
assert.True(t, lastReactorPair.reactor.Switch.Peers().Size() < len(reactorPairs)-1)
|
||||
assert.Equal(t, lastReactorPair.reactor.pool.maxPeerHeight, lastReactorPair.reactor.pool.height)
|
||||
|
||||
}
|
||||
|
||||
func setupReactors(
|
||||
@ -298,27 +279,26 @@ func setupReactors(
|
||||
|
||||
switches := p2p.MakeConnectedSwitches(config.P2P, numReactors, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[i].reactor)
|
||||
moduleName := fmt.Sprintf("blockchain-%v", i)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)
|
||||
|
||||
for i := 0; i < numReactors; i++ {
|
||||
addr := reactorPairs[i].reactor.Switch.NodeInfo().ID()
|
||||
moduleName := fmt.Sprintf("blockchain-%v", addr)
|
||||
reactorPairs[i].reactor.SetLogger(logger[i].With("module", moduleName[:19]))
|
||||
}
|
||||
|
||||
return reactorPairs, switches
|
||||
}
|
||||
|
||||
// WIP - used for some scale testing, will remove
|
||||
func TestFastSyncMultiNode(t *testing.T) {
|
||||
peerTimeout = 15 * time.Second
|
||||
|
||||
numNodes := 8
|
||||
maxHeight := int64(1000)
|
||||
//numNodes := 20
|
||||
//maxHeight := int64(10000)
|
||||
|
||||
maxRequestsPerPeer = 40
|
||||
maxNumPendingRequests = 500
|
||||
|
||||
config = cfg.ResetTestRoot("blockchain_reactor_test")
|
||||
genDoc, privVals := randGenesisDoc(1, false, 30)
|
||||
|
||||
@ -337,7 +317,7 @@ outerFor:
|
||||
for {
|
||||
i := 0
|
||||
for i < numNodes {
|
||||
if !reactorPairs[i].reactor.fsm.isCaughtUp() {
|
||||
if !reactorPairs[i].reactor.pool.IsCaughtUp() {
|
||||
break
|
||||
}
|
||||
i++
|
||||
@ -359,12 +339,14 @@ outerFor:
|
||||
|
||||
switches = append(switches, p2p.MakeConnectedSwitches(config.P2P, 1, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
s.AddReactor("BLOCKCHAIN", reactorPairs[len(reactorPairs)-1].reactor)
|
||||
moduleName := fmt.Sprintf("blockchainTEST-%d", len(reactorPairs)-1)
|
||||
reactorPairs[len(reactorPairs)-1].reactor.SetLogger(lastLogger.With("module", moduleName))
|
||||
return s
|
||||
|
||||
}, p2p.Connect2Switches)...)
|
||||
|
||||
addr := lastReactorPair.reactor.Switch.NodeInfo().ID()
|
||||
moduleName := fmt.Sprintf("blockchain-%v", addr)
|
||||
lastReactorPair.reactor.SetLogger(lastLogger.With("module", moduleName[:19]))
|
||||
|
||||
start = time.Now()
|
||||
|
||||
for i := 0; i < len(reactorPairs)-1; i++ {
|
||||
@ -373,14 +355,15 @@ outerFor:
|
||||
|
||||
for {
|
||||
time.Sleep(1 * time.Second)
|
||||
if lastReactorPair.reactor.fsm.isCaughtUp() {
|
||||
if lastReactorPair.reactor.pool.IsCaughtUp() {
|
||||
fmt.Println("FAST SYNC Duration", time.Since(start))
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
assert.True(t, lastReactorPair.reactor.Switch.Peers().Size() < len(reactorPairs))
|
||||
assert.Equal(t, lastReactorPair.reactor.fsm.pool.getMaxPeerHeight(), lastReactorPair.reactor.fsm.pool.height)
|
||||
assert.Equal(t, lastReactorPair.reactor.pool.maxPeerHeight, lastReactorPair.reactor.pool.height)
|
||||
|
||||
}
|
||||
|
||||
//----------------------------------------------
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"fmt"
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
"bytes"
|
@ -1,4 +1,4 @@
|
||||
package blockchain_new
|
||||
package blockchain_old
|
||||
|
||||
import (
|
||||
amino "github.com/tendermint/go-amino"
|
Loading…
x
Reference in New Issue
Block a user