2015-03-25 00:15:18 -07:00
|
|
|
package blockchain
|
|
|
|
|
|
|
|
import (
|
2018-11-01 07:07:18 +01:00
|
|
|
"errors"
|
2018-01-21 13:32:04 -05:00
|
|
|
"fmt"
|
2015-04-16 17:46:27 -07:00
|
|
|
"reflect"
|
2015-03-25 00:15:18 -07:00
|
|
|
"time"
|
|
|
|
|
2018-07-09 13:01:23 +04:00
|
|
|
amino "github.com/tendermint/go-amino"
|
|
|
|
|
|
|
|
"github.com/tendermint/tendermint/libs/log"
|
2017-04-08 22:04:06 -04:00
|
|
|
"github.com/tendermint/tendermint/p2p"
|
2015-04-01 17:30:16 -07:00
|
|
|
sm "github.com/tendermint/tendermint/state"
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2015-04-01 17:30:16 -07:00
|
|
|
"github.com/tendermint/tendermint/types"
|
2015-03-25 00:15:18 -07:00
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
2017-01-17 20:58:27 +04:00
|
|
|
// BlockchainChannel is a channel for blocks and status updates (`BlockStore` height)
|
|
|
|
BlockchainChannel = byte(0x40)
|
2018-06-21 01:57:35 -07:00
|
|
|
trySyncIntervalMS = 10
|
2019-04-13 09:23:43 -04:00
|
|
|
trySendIntervalMS = 10
|
2018-06-21 01:57:35 -07:00
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
// ask for best height every 10s
|
|
|
|
statusUpdateIntervalSeconds = 10
|
2018-04-06 13:46:40 -07:00
|
|
|
|
|
|
|
// NOTE: keep up to date with bcBlockResponseMessage
|
|
|
|
bcBlockResponseMessagePrefixSize = 4
|
|
|
|
bcBlockResponseMessageFieldKeySize = 1
|
2018-04-09 15:14:33 +03:00
|
|
|
maxMsgSize = types.MaxBlockSizeBytes +
|
2018-04-06 13:46:40 -07:00
|
|
|
bcBlockResponseMessagePrefixSize +
|
|
|
|
bcBlockResponseMessageFieldKeySize
|
2015-03-25 00:15:18 -07:00
|
|
|
)
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
var (
|
2019-05-15 09:08:02 -04:00
|
|
|
maxRequestsPerPeer int32 = 20
|
2019-04-15 23:26:07 -04:00
|
|
|
maxNumPendingRequests int32 = 500
|
2019-04-13 09:23:43 -04:00
|
|
|
)
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
type consensusReactor interface {
|
2015-05-27 22:06:33 -04:00
|
|
|
// for when we switch from blockchain reactor and fast sync to
|
|
|
|
// the consensus machine
|
2017-12-27 20:40:36 -05:00
|
|
|
SwitchToConsensus(sm.State, int)
|
2015-03-25 13:17:45 -07:00
|
|
|
}
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
// BlockchainReactor handles long-term catchup syncing.
|
|
|
|
type BlockchainReactor struct {
|
2015-07-20 14:40:41 -07:00
|
|
|
p2p.BaseReactor
|
|
|
|
|
2019-05-15 18:33:12 -04:00
|
|
|
initialState sm.State // immutable
|
2019-04-13 09:23:43 -04:00
|
|
|
state sm.State
|
2017-12-27 20:40:36 -05:00
|
|
|
|
2018-01-03 11:57:42 +01:00
|
|
|
blockExec *sm.BlockExecutor
|
|
|
|
store *BlockStore
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
fastSync bool
|
|
|
|
|
|
|
|
fsm *bReactorFSM
|
|
|
|
blocksSynced int
|
|
|
|
|
|
|
|
// Receive goroutine forwards messages to this channel to be processed in the context of the poolRoutine.
|
2019-05-07 21:06:43 -04:00
|
|
|
messagesForFSMCh chan bcReactorMessage
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
// 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.
|
2019-05-07 21:06:43 -04:00
|
|
|
errorsForFSMCh chan bcReactorMessage
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
// This channel is used by the FSM and indirectly the block pool to report errors to the blockchain reactor and
|
|
|
|
// the switch.
|
2019-05-07 21:06:43 -04:00
|
|
|
eventsFromFSMCh chan bcFsmMessage
|
2019-04-13 09:23:43 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
type BlockRequest struct {
|
|
|
|
Height int64
|
|
|
|
PeerID p2p.ID
|
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// bcReactorMessage is used by the reactor to send messages to the FSM.
|
|
|
|
type bcReactorMessage struct {
|
2019-04-13 09:23:43 -04:00
|
|
|
event bReactorEvent
|
|
|
|
data bReactorEventData
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
type bFsmEvent uint
|
|
|
|
|
|
|
|
const (
|
|
|
|
// message type events
|
|
|
|
peerErrorEv = iota + 1
|
|
|
|
syncFinishedEv
|
|
|
|
)
|
|
|
|
|
|
|
|
type bFsmEventData struct {
|
|
|
|
peerID p2p.ID
|
|
|
|
err error
|
|
|
|
}
|
|
|
|
|
|
|
|
// bcFsmMessage is used by the FSM to send messages to the reactor
|
|
|
|
type bcFsmMessage struct {
|
|
|
|
event bFsmEvent
|
|
|
|
data bFsmEventData
|
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// NewBlockchainReactor returns new reactor instance.
|
2018-01-03 11:29:19 +01:00
|
|
|
func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *BlockStore,
|
|
|
|
fastSync bool) *BlockchainReactor {
|
|
|
|
|
2015-12-07 16:57:33 -08:00
|
|
|
if state.LastBlockHeight != store.Height() {
|
2018-03-04 14:58:43 +04:00
|
|
|
panic(fmt.Sprintf("state (%v) and store (%v) height mismatch", state.LastBlockHeight,
|
2018-01-03 11:29:19 +01:00
|
|
|
store.Height()))
|
2015-03-25 11:33:39 -07:00
|
|
|
}
|
2017-12-27 20:40:36 -05:00
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
const capacity = 1000
|
2019-05-07 21:06:43 -04:00
|
|
|
eventsFromFSMCh := make(chan bcFsmMessage, capacity)
|
|
|
|
messagesForFSMCh := make(chan bcReactorMessage, capacity)
|
|
|
|
errorsForFSMCh := make(chan bcReactorMessage, capacity)
|
2018-03-04 14:58:43 +04:00
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
bcR := &BlockchainReactor{
|
2019-04-13 09:23:43 -04:00
|
|
|
initialState: state,
|
|
|
|
state: state,
|
|
|
|
blockExec: blockExec,
|
|
|
|
fastSync: fastSync,
|
|
|
|
store: store,
|
|
|
|
messagesForFSMCh: messagesForFSMCh,
|
2019-05-07 21:06:43 -04:00
|
|
|
eventsFromFSMCh: eventsFromFSMCh,
|
2019-04-13 09:23:43 -04:00
|
|
|
errorsForFSMCh: errorsForFSMCh,
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
2019-04-13 09:23:43 -04:00
|
|
|
fsm := NewFSM(store.Height()+1, bcR)
|
|
|
|
bcR.fsm = fsm
|
2017-05-02 11:53:32 +04:00
|
|
|
bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
|
2015-03-25 00:15:18 -07:00
|
|
|
return bcR
|
|
|
|
}
|
|
|
|
|
2017-10-20 23:56:21 +04:00
|
|
|
// SetLogger implements cmn.Service by setting the logger on reactor and pool.
|
|
|
|
func (bcR *BlockchainReactor) SetLogger(l log.Logger) {
|
|
|
|
bcR.BaseService.Logger = l
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.fsm.setLogger(l)
|
2017-10-20 23:56:21 +04:00
|
|
|
}
|
|
|
|
|
|
|
|
// OnStart implements cmn.Service.
|
2015-08-04 18:44:15 -07:00
|
|
|
func (bcR *BlockchainReactor) OnStart() error {
|
2016-03-24 18:08:18 -07:00
|
|
|
if bcR.fastSync {
|
2015-07-20 14:40:41 -07:00
|
|
|
go bcR.poolRoutine()
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
2015-08-04 18:44:15 -07:00
|
|
|
return nil
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-10-20 23:56:21 +04:00
|
|
|
// OnStop implements cmn.Service.
|
2015-07-21 18:31:01 -07:00
|
|
|
func (bcR *BlockchainReactor) OnStop() {
|
2019-04-13 09:23:43 -04:00
|
|
|
_ = bcR.Stop()
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// GetChannels implements Reactor
|
2015-03-25 00:15:18 -07:00
|
|
|
func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
|
|
|
|
return []*p2p.ChannelDescriptor{
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
2018-04-06 13:46:40 -07:00
|
|
|
ID: BlockchainChannel,
|
|
|
|
Priority: 10,
|
|
|
|
SendQueueCapacity: 1000,
|
|
|
|
RecvBufferCapacity: 50 * 4096,
|
2018-04-09 15:14:33 +03:00
|
|
|
RecvMessageCapacity: maxMsgSize,
|
2015-03-25 00:15:18 -07:00
|
|
|
},
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// AddPeer implements Reactor by sending our state to peer.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
|
2018-04-03 07:03:08 -07:00
|
|
|
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
2019-05-15 09:08:02 -04:00
|
|
|
if !peer.Send(BlockchainChannel, msgBytes) {
|
2017-01-17 20:58:27 +04:00
|
|
|
// doing nothing, will try later in `poolRoutine`
|
|
|
|
}
|
2017-11-08 02:42:27 +00:00
|
|
|
// peer is added to the pool once we receive the first
|
2019-05-07 21:06:43 -04:00
|
|
|
// bcStatusResponseMessage from the peer and call pool.updatePeer()
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// sendBlockToPeer loads a block and sends it to the requesting peer.
|
|
|
|
// If the block doesn't exist a bcNoBlockResponseMessage is sent.
|
|
|
|
// If all nodes are honest, no node should be requesting for a block that doesn't exist.
|
2019-04-13 09:23:43 -04:00
|
|
|
func (bcR *BlockchainReactor) sendBlockToPeer(msg *bcBlockRequestMessage,
|
2018-01-03 11:29:19 +01:00
|
|
|
src p2p.Peer) (queued bool) {
|
|
|
|
|
2017-08-26 02:33:19 -06:00
|
|
|
block := bcR.store.LoadBlock(msg.Height)
|
|
|
|
if block != nil {
|
2018-04-03 07:03:08 -07:00
|
|
|
msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
|
|
|
|
return src.TrySend(BlockchainChannel, msgBytes)
|
2017-08-26 02:33:19 -06:00
|
|
|
}
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.Logger.Info("peer asking for a block we don't have", "src", src, "height", msg.Height)
|
2017-08-26 02:33:19 -06:00
|
|
|
|
2018-04-03 07:03:08 -07:00
|
|
|
msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
|
|
|
|
return src.TrySend(BlockchainChannel, msgBytes)
|
2017-08-26 02:33:19 -06:00
|
|
|
}
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
func (bcR *BlockchainReactor) sendStatusResponseToPeer(msg *bcStatusRequestMessage, src p2p.Peer) (queued bool) {
|
|
|
|
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
|
|
|
|
return src.TrySend(BlockchainChannel, msgBytes)
|
|
|
|
}
|
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
// RemovePeer implements Reactor by removing peer from the pool.
|
|
|
|
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
2019-05-07 21:06:43 -04:00
|
|
|
msgData := bcReactorMessage{
|
2019-04-13 09:23:43 -04:00
|
|
|
event: peerRemoveEv,
|
|
|
|
data: bReactorEventData{
|
2019-04-28 22:03:26 -04:00
|
|
|
peerId: peer.ID(),
|
2019-04-13 09:23:43 -04:00
|
|
|
err: errSwitchRemovesPeer,
|
|
|
|
},
|
|
|
|
}
|
2019-04-30 21:46:27 -04:00
|
|
|
bcR.errorsForFSMCh <- msgData
|
2019-04-13 09:23:43 -04:00
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// Receive implements Reactor by handling 4 types of messages (look below).
|
2017-09-12 20:49:22 -04:00
|
|
|
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
2018-07-09 13:01:23 +04:00
|
|
|
msg, err := decodeMsg(msgBytes)
|
2015-03-25 00:15:18 -07:00
|
|
|
if err != nil {
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.Logger.Error("error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
2018-03-04 13:42:45 +04:00
|
|
|
bcR.Switch.StopPeerForError(src, err)
|
2015-03-25 00:15:18 -07:00
|
|
|
return
|
|
|
|
}
|
2015-03-25 17:16:49 -07:00
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
if err = msg.ValidateBasic(); err != nil {
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.Logger.Error("peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
2018-11-01 07:07:18 +01:00
|
|
|
bcR.Switch.StopPeerForError(src, err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2017-05-02 11:53:32 +04:00
|
|
|
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
|
2015-03-25 00:15:18 -07:00
|
|
|
|
2015-04-14 15:57:16 -07:00
|
|
|
switch msg := msg.(type) {
|
2015-04-16 17:46:27 -07:00
|
|
|
case *bcBlockRequestMessage:
|
2019-04-13 09:23:43 -04:00
|
|
|
if queued := bcR.sendBlockToPeer(msg, src); !queued {
|
2017-08-26 02:33:19 -06:00
|
|
|
// Unfortunately not queued since the queue is full.
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.Logger.Error("Could not send block message to peer", "src", src, "height", msg.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
case *bcStatusRequestMessage:
|
|
|
|
// Send peer our state.
|
2019-04-13 09:23:43 -04:00
|
|
|
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)
|
2015-04-21 19:51:23 -07:00
|
|
|
}
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
case *bcBlockResponseMessage:
|
2019-05-08 22:23:23 -04:00
|
|
|
msgForFSM := bcReactorMessage{
|
2019-04-13 09:23:43 -04:00
|
|
|
event: blockResponseEv,
|
|
|
|
data: bReactorEventData{
|
|
|
|
peerId: src.ID(),
|
|
|
|
height: msg.Block.Height,
|
|
|
|
block: msg.Block,
|
|
|
|
length: len(msgBytes),
|
|
|
|
},
|
|
|
|
}
|
2019-05-08 22:23:23 -04:00
|
|
|
bcR.messagesForFSMCh <- msgForFSM
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
case *bcStatusResponseMessage:
|
|
|
|
// Got a peer status. Unverified.
|
2019-05-08 22:23:23 -04:00
|
|
|
msgForFSM := bcReactorMessage{
|
2019-04-13 09:23:43 -04:00
|
|
|
event: statusResponseEv,
|
|
|
|
data: bReactorEventData{
|
|
|
|
peerId: src.ID(),
|
|
|
|
height: msg.Height,
|
|
|
|
length: len(msgBytes),
|
|
|
|
},
|
|
|
|
}
|
2019-05-08 22:23:23 -04:00
|
|
|
bcR.messagesForFSMCh <- msgForFSM
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
default:
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.Logger.Error(fmt.Sprintf("unknown message type %v", reflect.TypeOf(msg)))
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// poolRoutine receives and handles messages from the Receive() routine and from the FSM
|
2015-03-25 00:15:18 -07:00
|
|
|
func (bcR *BlockchainReactor) poolRoutine() {
|
2015-03-25 11:33:39 -07:00
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
bcR.fsm.start()
|
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
processReceivedBlockTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
|
|
|
|
sendBlockRequestTicker := time.NewTicker(trySendIntervalMS * time.Millisecond)
|
2015-04-21 19:51:23 -07:00
|
|
|
statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
|
2019-05-08 22:23:23 -04:00
|
|
|
doProcessBlockCh := make(chan struct{}, 1)
|
2015-03-25 11:33:39 -07:00
|
|
|
|
2017-10-26 18:29:23 -04:00
|
|
|
lastHundred := time.Now()
|
|
|
|
lastRate := 0.0
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
ForLoop:
|
2015-03-25 00:15:18 -07:00
|
|
|
for {
|
|
|
|
select {
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
case <-sendBlockRequestTicker.C:
|
2019-04-13 09:23:43 -04:00
|
|
|
// Tell FSM to make more requests.
|
2019-05-15 09:08:02 -04:00
|
|
|
// Approximate the number of new rquests based on:
|
2019-04-13 09:23:43 -04:00
|
|
|
// - the number of blocks received and waiting to be processed,
|
2019-05-15 09:08:02 -04:00
|
|
|
blocksToBeProcessed := bcR.fsm.getNumberOfBlocksAdded()
|
|
|
|
// - the approximate number of blockResponse messages waiting in messagesForFSMCh, etc.
|
|
|
|
// (queue may include Status response messages)
|
|
|
|
blocksToBeAdded := len(bcR.messagesForFSMCh)
|
|
|
|
numRequests := maxNumPendingRequests - blocksToBeProcessed - int32(blocksToBeAdded)
|
|
|
|
if numRequests <= 0 {
|
|
|
|
continue
|
|
|
|
}
|
2019-05-07 21:06:43 -04:00
|
|
|
_ = bcR.fsm.handle(&bcReactorMessage{
|
2019-04-13 09:23:43 -04:00
|
|
|
event: makeRequestsEv,
|
|
|
|
data: bReactorEventData{
|
2019-05-15 09:08:02 -04:00
|
|
|
maxNumRequests: numRequests}})
|
2018-06-21 01:57:35 -07:00
|
|
|
|
2017-05-29 23:11:40 -04:00
|
|
|
case <-statusUpdateTicker.C:
|
2019-04-13 09:23:43 -04:00
|
|
|
// Ask for status updates.
|
|
|
|
go bcR.sendStatusRequest()
|
2018-06-21 01:57:35 -07:00
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
case <-processReceivedBlockTicker.C: // chan time
|
2018-06-21 01:57:35 -07:00
|
|
|
select {
|
2019-04-28 22:03:26 -04:00
|
|
|
case doProcessBlockCh <- struct{}{}:
|
2018-06-21 01:57:35 -07:00
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
case <-doProcessBlockCh:
|
2019-05-07 21:06:43 -04:00
|
|
|
err := bcR.processBlock()
|
2019-04-13 09:23:43 -04:00
|
|
|
if err == errMissingBlocks {
|
2019-05-07 21:06:43 -04:00
|
|
|
continue
|
2018-06-21 01:57:35 -07:00
|
|
|
}
|
2019-04-13 09:23:43 -04:00
|
|
|
// Notify FSM of block processing result.
|
2019-05-07 21:06:43 -04:00
|
|
|
_ = bcR.fsm.handle(&bcReactorMessage{
|
2019-04-13 09:23:43 -04:00
|
|
|
event: processedBlockEv,
|
|
|
|
data: bReactorEventData{
|
|
|
|
err: err,
|
|
|
|
},
|
2019-04-30 21:46:27 -04:00
|
|
|
})
|
2018-06-21 01:57:35 -07:00
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
if err == errBlockVerificationFailure {
|
2019-05-07 21:06:43 -04:00
|
|
|
continue
|
2015-03-25 11:33:39 -07:00
|
|
|
}
|
2019-04-13 09:23:43 -04:00
|
|
|
|
2019-04-28 22:03:26 -04:00
|
|
|
doProcessBlockCh <- struct{}{}
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
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:
|
2019-05-08 22:23:23 -04:00
|
|
|
// Sent from the Receive() routine when status (statusResponseEv) and
|
|
|
|
// block (blockResponseEv) response events are received
|
2019-04-30 21:46:27 -04:00
|
|
|
_ = bcR.fsm.handle(&msg)
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
case msg := <-bcR.errorsForFSMCh:
|
2019-05-08 22:23:23 -04:00
|
|
|
// Sent from the switch.RemovePeer() routine (RemovePeerEv) and
|
|
|
|
// FSM state timer expiry routine (stateTimeoutEv).
|
2019-04-30 21:46:27 -04:00
|
|
|
_ = bcR.fsm.handle(&msg)
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
case msg := <-bcR.eventsFromFSMCh:
|
|
|
|
switch msg.event {
|
2019-05-08 18:28:18 -04:00
|
|
|
case syncFinishedEv:
|
2019-05-08 22:23:23 -04:00
|
|
|
// Sent from the FSM when it enters finished state.
|
2019-05-08 18:28:18 -04:00
|
|
|
break ForLoop
|
2019-05-07 21:06:43 -04:00
|
|
|
case peerErrorEv:
|
2019-05-08 22:23:23 -04:00
|
|
|
// Sent from the FSM when it detects peer error
|
2019-05-07 21:06:43 -04:00
|
|
|
bcR.reportPeerErrorToSwitch(msg.data.err, msg.data.peerID)
|
|
|
|
if msg.data.err == errNoPeerResponse {
|
2019-05-08 22:23:23 -04:00
|
|
|
// Sent from the peer timeout handler routine
|
2019-05-07 21:06:43 -04:00
|
|
|
_ = bcR.fsm.handle(&bcReactorMessage{
|
|
|
|
event: peerRemoveEv,
|
|
|
|
data: bReactorEventData{
|
|
|
|
peerId: msg.data.peerID,
|
|
|
|
err: msg.data.err,
|
|
|
|
},
|
|
|
|
})
|
2019-05-08 22:23:23 -04:00
|
|
|
} else {
|
|
|
|
// For slow peers, or errors due to blocks received from wrong peer
|
|
|
|
// the FSM had already removed the peers
|
2019-05-07 21:06:43 -04:00
|
|
|
}
|
|
|
|
default:
|
|
|
|
bcR.Logger.Error("Event from FSM not supported", "type", msg.event)
|
2019-04-30 21:46:27 -04:00
|
|
|
}
|
2018-06-21 01:57:35 -07:00
|
|
|
|
2018-02-12 14:31:52 +04:00
|
|
|
case <-bcR.Quit():
|
2019-04-13 09:23:43 -04:00
|
|
|
break ForLoop
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
func (bcR *BlockchainReactor) reportPeerErrorToSwitch(err error, peerID p2p.ID) {
|
|
|
|
peer := bcR.Switch.Peers().Get(peerID)
|
|
|
|
if peer != nil {
|
|
|
|
bcR.Switch.StopPeerForError(peer, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
func (bcR *BlockchainReactor) processBlock() error {
|
2019-04-13 09:23:43 -04:00
|
|
|
|
|
|
|
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
|
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// Implements bcRMessageInterface
|
|
|
|
// sendStatusRequest broadcasts `BlockStore` height.
|
2019-04-13 09:23:43 -04:00
|
|
|
func (bcR *BlockchainReactor) sendStatusRequest() {
|
2018-04-03 07:03:08 -07:00
|
|
|
msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
|
|
|
|
bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
|
2019-04-13 09:23:43 -04:00
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// Implements bcRMessageInterface
|
2019-04-13 09:23:43 -04:00
|
|
|
// 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
|
|
|
|
}
|
2015-03-25 00:15:18 -07:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-05-08 18:28:18 -04:00
|
|
|
// Implements bcRMessageInterface
|
2019-04-13 09:23:43 -04:00
|
|
|
func (bcR *BlockchainReactor) switchToConsensus() {
|
|
|
|
conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
|
|
|
if ok {
|
|
|
|
conR.SwitchToConsensus(bcR.state, bcR.blocksSynced)
|
2019-05-07 21:06:43 -04:00
|
|
|
bcR.eventsFromFSMCh <- bcFsmMessage{event: syncFinishedEv}
|
2019-04-13 09:23:43 -04:00
|
|
|
} else {
|
|
|
|
// Should only happen during testing.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-07 21:06:43 -04:00
|
|
|
// Implements bcRMessageInterface
|
|
|
|
// Called by FSM and pool:
|
|
|
|
// - pool calls when it detects slow peer or when peer times out
|
|
|
|
// - FSM calls when:
|
|
|
|
// - adding a block (addBlock) fails
|
|
|
|
// - reactor processing of a block reports failure and FSM sends back the peers of first and second blocks
|
|
|
|
func (bcR *BlockchainReactor) sendPeerError(err error, peerID p2p.ID) {
|
|
|
|
msgData := bcFsmMessage{
|
|
|
|
event: peerErrorEv,
|
|
|
|
data: bFsmEventData{
|
|
|
|
peerID: peerID,
|
|
|
|
err: err,
|
|
|
|
},
|
|
|
|
}
|
|
|
|
bcR.eventsFromFSMCh <- msgData
|
|
|
|
}
|
|
|
|
|
|
|
|
// Implements bcRMessageInterface
|
|
|
|
func (bcR *BlockchainReactor) resetStateTimer(name string, timer **time.Timer, timeout time.Duration) {
|
|
|
|
if timer == nil {
|
|
|
|
panic("nil timer pointer parameter")
|
|
|
|
}
|
|
|
|
if *timer == nil {
|
|
|
|
*timer = time.AfterFunc(timeout, func() {
|
|
|
|
msg := bcReactorMessage{
|
|
|
|
event: stateTimeoutEv,
|
|
|
|
data: bReactorEventData{
|
|
|
|
stateName: name,
|
|
|
|
},
|
|
|
|
}
|
2019-05-08 22:23:23 -04:00
|
|
|
bcR.errorsForFSMCh <- msg
|
2019-05-07 21:06:43 -04:00
|
|
|
})
|
|
|
|
} else {
|
|
|
|
(*timer).Reset(timeout)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
//-----------------------------------------------------------------------------
|
|
|
|
// Messages
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// BlockchainMessage is a generic message for this reactor.
|
2018-11-01 07:07:18 +01:00
|
|
|
type BlockchainMessage interface {
|
|
|
|
ValidateBasic() error
|
|
|
|
}
|
2015-04-14 15:57:16 -07:00
|
|
|
|
2018-04-03 07:03:08 -07:00
|
|
|
func RegisterBlockchainMessages(cdc *amino.Codec) {
|
|
|
|
cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
|
2018-08-13 21:40:49 +08:00
|
|
|
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)
|
2018-04-03 07:03:08 -07:00
|
|
|
}
|
2015-04-14 15:57:16 -07:00
|
|
|
|
2018-07-09 13:01:23 +04:00
|
|
|
func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
|
2018-04-09 15:14:33 +03:00
|
|
|
if len(bz) > maxMsgSize {
|
2019-04-13 09:23:43 -04:00
|
|
|
return msg, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
2018-04-09 15:14:33 +03:00
|
|
|
}
|
2018-04-03 07:03:08 -07:00
|
|
|
err = cdc.UnmarshalBinaryBare(bz, &msg)
|
2015-03-25 00:15:18 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2015-03-25 17:16:49 -07:00
|
|
|
type bcBlockRequestMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *bcBlockRequestMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
2019-04-13 09:23:43 -04:00
|
|
|
return errors.New("negative Height")
|
2018-11-01 07:07:18 +01:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-04-16 17:46:27 -07:00
|
|
|
func (m *bcBlockRequestMessage) String() string {
|
2018-08-10 00:25:57 -05:00
|
|
|
return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-08-26 02:33:19 -06:00
|
|
|
type bcNoBlockResponseMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2017-08-26 02:33:19 -06:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *bcNoBlockResponseMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
2019-04-13 09:23:43 -04:00
|
|
|
return errors.New("negative Height")
|
2018-11-01 07:07:18 +01:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-04-13 09:23:43 -04:00
|
|
|
func (m *bcNoBlockResponseMessage) String() string {
|
|
|
|
return fmt.Sprintf("[bcNoBlockResponseMessage %d]", m.Height)
|
2017-08-26 02:33:19 -06:00
|
|
|
}
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
//-------------------------------------
|
|
|
|
|
2015-03-25 17:16:49 -07:00
|
|
|
type bcBlockResponseMessage struct {
|
2015-03-25 00:15:18 -07:00
|
|
|
Block *types.Block
|
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *bcBlockResponseMessage) ValidateBasic() error {
|
2018-12-17 11:51:53 -05:00
|
|
|
return m.Block.ValidateBasic()
|
2018-11-01 07:07:18 +01:00
|
|
|
}
|
|
|
|
|
2015-04-16 17:46:27 -07:00
|
|
|
func (m *bcBlockResponseMessage) String() string {
|
2018-08-10 00:25:57 -05:00
|
|
|
return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
type bcStatusRequestMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-04-21 19:51:23 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *bcStatusRequestMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
2019-04-13 09:23:43 -04:00
|
|
|
return errors.New("negative Height")
|
2018-11-01 07:07:18 +01:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
func (m *bcStatusRequestMessage) String() string {
|
2018-08-10 00:25:57 -05:00
|
|
|
return fmt.Sprintf("[bcStatusRequestMessage %v]", m.Height)
|
2015-04-21 19:51:23 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
|
|
|
type bcStatusResponseMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *bcStatusResponseMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
2019-04-13 09:23:43 -04:00
|
|
|
return errors.New("negative Height")
|
2018-11-01 07:07:18 +01:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
func (m *bcStatusResponseMessage) String() string {
|
2018-08-10 00:25:57 -05:00
|
|
|
return fmt.Sprintf("[bcStatusResponseMessage %v]", m.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|