2015-03-25 00:15:18 -07:00
|
|
|
package blockchain
|
|
|
|
|
|
|
|
import (
|
|
|
|
"bytes"
|
|
|
|
"errors"
|
2015-04-16 17:46:27 -07:00
|
|
|
"reflect"
|
2015-03-25 00:15:18 -07:00
|
|
|
"time"
|
|
|
|
|
2017-05-02 11:53:32 +04:00
|
|
|
wire "github.com/tendermint/go-wire"
|
2017-04-08 22:04:06 -04:00
|
|
|
"github.com/tendermint/tendermint/p2p"
|
2015-12-01 20:12:01 -08:00
|
|
|
"github.com/tendermint/tendermint/proxy"
|
2015-04-01 17:30:16 -07:00
|
|
|
sm "github.com/tendermint/tendermint/state"
|
|
|
|
"github.com/tendermint/tendermint/types"
|
2017-04-08 22:04:06 -04:00
|
|
|
cmn "github.com/tendermint/tmlibs/common"
|
2017-10-20 23:56:21 +04:00
|
|
|
"github.com/tendermint/tmlibs/log"
|
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)
|
|
|
|
|
2017-10-26 18:29:23 -04:00
|
|
|
defaultChannelCapacity = 1000
|
|
|
|
trySyncIntervalMS = 50
|
2015-03-25 11:33:39 -07:00
|
|
|
// stop syncing when last block's time is
|
|
|
|
// within this much of the system time.
|
2015-04-23 14:59:12 -07:00
|
|
|
// stopSyncingDurationMinutes = 10
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
// ask for best height every 10s
|
|
|
|
statusUpdateIntervalSeconds = 10
|
|
|
|
// check if we should switch to consensus reactor
|
2015-08-18 10:51:55 -07:00
|
|
|
switchToConsensusIntervalSeconds = 1
|
2015-03-25 00:15:18 -07: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-10-26 18:29:23 -04: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
|
|
|
|
|
2016-01-06 17:14:20 -08:00
|
|
|
state *sm.State
|
2016-08-17 22:28:08 -04:00
|
|
|
proxyAppConn proxy.AppConnConsensus // same as consensus.proxyAppConn
|
2016-01-06 17:14:20 -08:00
|
|
|
store *BlockStore
|
|
|
|
pool *BlockPool
|
2016-03-24 18:08:18 -07:00
|
|
|
fastSync bool
|
2016-01-06 17:14:20 -08:00
|
|
|
requestsCh chan BlockRequest
|
|
|
|
timeoutsCh chan string
|
2015-04-02 17:35:27 -07:00
|
|
|
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
eventBus *types.EventBus
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// NewBlockchainReactor returns new reactor instance.
|
2017-04-28 23:50:24 -04:00
|
|
|
func NewBlockchainReactor(state *sm.State, proxyAppConn proxy.AppConnConsensus, store *BlockStore, fastSync bool) *BlockchainReactor {
|
2015-12-07 16:57:33 -08:00
|
|
|
if state.LastBlockHeight == store.Height()-1 {
|
2017-01-17 20:58:27 +04:00
|
|
|
store.height-- // XXX HACK, make this better
|
2015-12-07 16:57:33 -08:00
|
|
|
}
|
|
|
|
if state.LastBlockHeight != store.Height() {
|
2017-01-17 20:58:27 +04:00
|
|
|
cmn.PanicSanity(cmn.Fmt("state (%v) and store (%v) height mismatch", state.LastBlockHeight, store.Height()))
|
2015-03-25 11:33:39 -07:00
|
|
|
}
|
2015-03-25 00:15:18 -07:00
|
|
|
requestsCh := make(chan BlockRequest, defaultChannelCapacity)
|
|
|
|
timeoutsCh := make(chan string, defaultChannelCapacity)
|
|
|
|
pool := NewBlockPool(
|
|
|
|
store.Height()+1,
|
|
|
|
requestsCh,
|
|
|
|
timeoutsCh,
|
|
|
|
)
|
|
|
|
bcR := &BlockchainReactor{
|
2016-01-06 17:14:20 -08:00
|
|
|
state: state,
|
|
|
|
proxyAppConn: proxyAppConn,
|
|
|
|
store: store,
|
|
|
|
pool: pool,
|
2016-03-24 18:08:18 -07:00
|
|
|
fastSync: fastSync,
|
2016-01-06 17:14:20 -08:00
|
|
|
requestsCh: requestsCh,
|
|
|
|
timeoutsCh: timeoutsCh,
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
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
|
|
|
|
bcR.pool.Logger = l
|
|
|
|
}
|
|
|
|
|
|
|
|
// OnStart implements cmn.Service.
|
2015-08-04 18:44:15 -07:00
|
|
|
func (bcR *BlockchainReactor) OnStart() error {
|
2017-09-06 11:50:43 -04:00
|
|
|
if err := bcR.BaseReactor.OnStart(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2016-03-24 18:08:18 -07:00
|
|
|
if bcR.fastSync {
|
2015-08-04 19:04:00 -07:00
|
|
|
_, err := bcR.pool.Start()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
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() {
|
|
|
|
bcR.BaseReactor.OnStop()
|
2015-07-20 14:40:41 -07:00
|
|
|
bcR.pool.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
|
|
|
{
|
2015-08-18 10:51:55 -07:00
|
|
|
ID: BlockchainChannel,
|
2017-10-26 18:29:23 -04:00
|
|
|
Priority: 10,
|
|
|
|
SendQueueCapacity: 1000,
|
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) {
|
2017-01-17 20:58:27 +04:00
|
|
|
if !peer.Send(BlockchainChannel, struct{ BlockchainMessage }{&bcStatusResponseMessage{bcR.store.Height()}}) {
|
|
|
|
// 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
|
|
|
|
// bcStatusResponseMessage from the peer and call pool.SetPeerHeight
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// RemovePeer implements Reactor by removing peer from the pool.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
|
|
|
bcR.pool.RemovePeer(peer.Key())
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-08-26 02:33:19 -06:00
|
|
|
// 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 {
|
|
|
|
msg := &bcBlockResponseMessage{Block: block}
|
|
|
|
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg})
|
|
|
|
}
|
|
|
|
|
|
|
|
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
|
|
|
|
|
|
|
|
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{
|
|
|
|
&bcNoBlockResponseMessage{Height: msg.Height},
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
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) {
|
2017-09-11 16:28:00 -04:00
|
|
|
_, msg, err := DecodeMessage(msgBytes, bcR.maxMsgSize())
|
2015-03-25 00:15:18 -07:00
|
|
|
if err != nil {
|
2017-06-14 12:50:49 +04:00
|
|
|
bcR.Logger.Error("Error decoding message", "err", err)
|
2015-03-25 00:15:18 -07:00
|
|
|
return
|
|
|
|
}
|
2015-03-25 17:16:49 -07:00
|
|
|
|
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
|
|
|
|
2017-06-23 21:36:47 -04:00
|
|
|
// TODO: improve logic to satisfy megacheck
|
2015-04-14 15:57:16 -07:00
|
|
|
switch msg := msg.(type) {
|
2015-04-16 17:46:27 -07:00
|
|
|
case *bcBlockRequestMessage:
|
2017-08-26 02:33:19 -06:00
|
|
|
if queued := bcR.respondToPeer(msg, src); !queued {
|
|
|
|
// Unfortunately not queued since the queue is full.
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
2015-04-16 17:46:27 -07:00
|
|
|
case *bcBlockResponseMessage:
|
2015-03-25 00:15:18 -07:00
|
|
|
// Got a block.
|
2017-09-12 20:49:22 -04:00
|
|
|
bcR.pool.AddBlock(src.Key(), msg.Block, len(msgBytes))
|
2015-04-21 19:51:23 -07:00
|
|
|
case *bcStatusRequestMessage:
|
|
|
|
// Send peer our state.
|
2015-12-21 14:48:44 -08:00
|
|
|
queued := src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{&bcStatusResponseMessage{bcR.store.Height()}})
|
2015-04-21 19:51:23 -07:00
|
|
|
if !queued {
|
|
|
|
// sorry
|
|
|
|
}
|
|
|
|
case *bcStatusResponseMessage:
|
|
|
|
// Got a peer status. Unverified.
|
2017-09-12 20:49:22 -04:00
|
|
|
bcR.pool.SetPeerHeight(src.Key(), msg.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
default:
|
2017-05-02 11:53:32 +04:00
|
|
|
bcR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-09-11 16:28:00 -04:00
|
|
|
// maxMsgSize returns the maximum allowable size of a
|
|
|
|
// message on the blockchain reactor.
|
|
|
|
func (bcR *BlockchainReactor) maxMsgSize() int {
|
2017-10-12 14:50:09 +04:00
|
|
|
return bcR.state.Params.BlockSizeParams.MaxBytes + 2
|
2017-09-11 16:28:00 -04:00
|
|
|
}
|
|
|
|
|
2015-03-25 11:33:39 -07:00
|
|
|
// Handle messages from the poolReactor telling the reactor what to do.
|
2015-05-28 02:18:53 -07:00
|
|
|
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
|
|
|
// (Except for the SYNC_LOOP, which is the primary purpose and must be synchronous.)
|
2015-03-25 00:15:18 -07:00
|
|
|
func (bcR *BlockchainReactor) poolRoutine() {
|
2015-03-25 11:33:39 -07:00
|
|
|
|
|
|
|
trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
|
2015-04-21 19:51:23 -07:00
|
|
|
statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
|
|
|
|
switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
|
2015-03-25 11:33:39 -07:00
|
|
|
|
2017-10-26 18:29:23 -04:00
|
|
|
blocksSynced := 0
|
|
|
|
|
2017-10-12 14:50:09 +04:00
|
|
|
chainID := bcR.state.ChainID
|
2017-10-05 16:50:05 +04:00
|
|
|
|
2017-10-26 18:29:23 -04:00
|
|
|
lastHundred := time.Now()
|
|
|
|
lastRate := 0.0
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
FOR_LOOP:
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case request := <-bcR.requestsCh: // chan BlockRequest
|
2015-08-18 10:51:55 -07:00
|
|
|
peer := bcR.Switch.Peers().Get(request.PeerID)
|
2015-03-25 00:15:18 -07:00
|
|
|
if peer == nil {
|
2015-08-18 10:51:55 -07:00
|
|
|
continue FOR_LOOP // Peer has since been disconnected.
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
2015-04-16 17:46:27 -07:00
|
|
|
msg := &bcBlockRequestMessage{request.Height}
|
2015-12-21 14:48:44 -08:00
|
|
|
queued := peer.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg})
|
2015-03-25 00:15:18 -07:00
|
|
|
if !queued {
|
2015-05-28 02:18:53 -07:00
|
|
|
// We couldn't make the request, send-queue full.
|
2015-08-18 10:51:55 -07:00
|
|
|
// The pool handles timeouts, just let it go.
|
2015-03-25 00:15:18 -07:00
|
|
|
continue FOR_LOOP
|
|
|
|
}
|
2015-08-18 10:51:55 -07:00
|
|
|
case peerID := <-bcR.timeoutsCh: // chan string
|
2015-03-25 00:15:18 -07:00
|
|
|
// Peer timed out.
|
2015-08-18 10:51:55 -07:00
|
|
|
peer := bcR.Switch.Peers().Get(peerID)
|
2015-03-25 12:21:52 -07:00
|
|
|
if peer != nil {
|
2015-07-20 14:40:41 -07:00
|
|
|
bcR.Switch.StopPeerForError(peer, errors.New("BlockchainReactor Timeout"))
|
2015-03-25 12:21:52 -07:00
|
|
|
}
|
2017-05-29 23:11:40 -04:00
|
|
|
case <-statusUpdateTicker.C:
|
2015-04-21 19:51:23 -07:00
|
|
|
// ask for status updates
|
2017-10-03 18:49:20 -04:00
|
|
|
go bcR.BroadcastStatusRequest() // nolint: errcheck
|
2017-05-29 23:11:40 -04:00
|
|
|
case <-switchToConsensusTicker.C:
|
2017-10-20 21:19:25 -04:00
|
|
|
height, numPending, lenRequesters := bcR.pool.GetStatus()
|
2015-07-20 14:40:41 -07:00
|
|
|
outbound, inbound, _ := bcR.Switch.NumPeers()
|
2017-10-26 18:29:23 -04:00
|
|
|
bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "total", lenRequesters,
|
2015-08-18 10:51:55 -07:00
|
|
|
"outbound", outbound, "inbound", inbound)
|
|
|
|
if bcR.pool.IsCaughtUp() {
|
2017-05-02 11:53:32 +04:00
|
|
|
bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
|
2015-04-21 19:51:23 -07:00
|
|
|
bcR.pool.Stop()
|
|
|
|
|
2015-07-20 14:40:41 -07:00
|
|
|
conR := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
|
2017-10-26 18:29:23 -04:00
|
|
|
conR.SwitchToConsensus(bcR.state, blocksSynced)
|
2015-04-21 19:51:23 -07:00
|
|
|
|
|
|
|
break FOR_LOOP
|
|
|
|
}
|
2017-05-29 23:11:40 -04:00
|
|
|
case <-trySyncTicker.C: // chan time
|
2015-05-28 02:18:53 -07:00
|
|
|
// This loop can be slow as long as it's doing syncing work.
|
2015-03-25 11:33:39 -07:00
|
|
|
SYNC_LOOP:
|
|
|
|
for i := 0; i < 10; i++ {
|
|
|
|
// See if there are any blocks to sync.
|
|
|
|
first, second := bcR.pool.PeekTwoBlocks()
|
2017-05-02 11:53:32 +04:00
|
|
|
//bcR.Logger.Info("TrySync peeked", "first", first, "second", second)
|
2015-03-25 11:33:39 -07:00
|
|
|
if first == nil || second == nil {
|
|
|
|
// We need both to sync the first block.
|
|
|
|
break SYNC_LOOP
|
|
|
|
}
|
2017-10-12 14:50:09 +04:00
|
|
|
firstParts := first.MakePartSet(bcR.state.Params.BlockPartSizeBytes)
|
2015-03-25 17:16:49 -07:00
|
|
|
firstPartsHeader := firstParts.Header()
|
2016-04-02 09:10:16 -07:00
|
|
|
// Finally, verify the first block using the second's commit
|
2016-08-16 14:59:19 -07:00
|
|
|
// 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.
|
2017-10-05 16:50:05 +04:00
|
|
|
err := bcR.state.Validators.VerifyCommit(
|
2017-09-22 18:17:21 -06:00
|
|
|
chainID, types.BlockID{first.Hash(), firstPartsHeader}, first.Height, second.LastCommit)
|
2015-03-25 11:33:39 -07:00
|
|
|
if err != nil {
|
2017-10-02 15:24:30 -04:00
|
|
|
bcR.Logger.Error("Error in validation", "err", err)
|
2015-03-25 11:33:39 -07:00
|
|
|
bcR.pool.RedoRequest(first.Height)
|
|
|
|
break SYNC_LOOP
|
|
|
|
} else {
|
|
|
|
bcR.pool.PopRequest()
|
2016-12-06 23:01:55 -05:00
|
|
|
|
|
|
|
bcR.store.SaveBlock(first, firstParts, second.LastCommit)
|
|
|
|
|
2016-11-19 19:32:35 -05:00
|
|
|
// TODO: should we be firing events? need to fire NewBlock events manually ...
|
2016-08-27 14:15:18 -04:00
|
|
|
// NOTE: we could improve performance if we
|
|
|
|
// didn't make the app commit to disk every block
|
|
|
|
// ... but we would need a way to get the hash without it persisting
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
err := bcR.state.ApplyBlock(bcR.eventBus, bcR.proxyAppConn, first, firstPartsHeader, types.MockMempool{})
|
2016-12-06 23:01:55 -05:00
|
|
|
if err != nil {
|
|
|
|
// TODO This is bad, are we zombie?
|
2017-01-17 20:58:27 +04:00
|
|
|
cmn.PanicQ(cmn.Fmt("Failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
|
2015-03-25 11:33:39 -07:00
|
|
|
}
|
2017-10-26 18:29:23 -04:00
|
|
|
blocksSynced += 1
|
|
|
|
|
|
|
|
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()
|
|
|
|
}
|
2015-03-25 11:33:39 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
continue FOR_LOOP
|
2015-07-20 14:40:41 -07:00
|
|
|
case <-bcR.Quit:
|
2015-03-25 00:15:18 -07:00
|
|
|
break FOR_LOOP
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// BroadcastStatusRequest broadcasts `BlockStore` height.
|
2015-04-21 19:51:23 -07:00
|
|
|
func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
|
2015-12-21 15:18:16 -08:00
|
|
|
bcR.Switch.Broadcast(BlockchainChannel, struct{ BlockchainMessage }{&bcStatusRequestMessage{bcR.store.Height()}})
|
2015-03-25 00:15:18 -07:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
// SetEventBus sets event bus.
|
|
|
|
func (bcR *BlockchainReactor) SetEventBus(b *types.EventBus) {
|
|
|
|
bcR.eventBus = b
|
2015-04-02 17:35:27 -07:00
|
|
|
}
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
//-----------------------------------------------------------------------------
|
|
|
|
// Messages
|
|
|
|
|
|
|
|
const (
|
2017-08-26 02:33:19 -06:00
|
|
|
msgTypeBlockRequest = byte(0x10)
|
|
|
|
msgTypeBlockResponse = byte(0x11)
|
|
|
|
msgTypeNoBlockResponse = byte(0x12)
|
|
|
|
msgTypeStatusResponse = byte(0x20)
|
|
|
|
msgTypeStatusRequest = byte(0x21)
|
2015-03-25 00:15:18 -07:00
|
|
|
)
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// BlockchainMessage is a generic message for this reactor.
|
2015-04-14 15:57:16 -07:00
|
|
|
type BlockchainMessage interface{}
|
|
|
|
|
2015-07-25 15:45:45 -07:00
|
|
|
var _ = wire.RegisterInterface(
|
2015-04-14 15:57:16 -07:00
|
|
|
struct{ BlockchainMessage }{},
|
2015-07-25 15:45:45 -07:00
|
|
|
wire.ConcreteType{&bcBlockRequestMessage{}, msgTypeBlockRequest},
|
|
|
|
wire.ConcreteType{&bcBlockResponseMessage{}, msgTypeBlockResponse},
|
2017-08-26 02:33:19 -06:00
|
|
|
wire.ConcreteType{&bcNoBlockResponseMessage{}, msgTypeNoBlockResponse},
|
2015-07-25 15:45:45 -07:00
|
|
|
wire.ConcreteType{&bcStatusResponseMessage{}, msgTypeStatusResponse},
|
|
|
|
wire.ConcreteType{&bcStatusRequestMessage{}, msgTypeStatusRequest},
|
2015-04-14 15:57:16 -07:00
|
|
|
)
|
|
|
|
|
2017-01-17 20:58:27 +04:00
|
|
|
// DecodeMessage decodes BlockchainMessage.
|
2015-08-18 10:51:55 -07:00
|
|
|
// TODO: ensure that bz is completely read.
|
2017-09-11 16:28:00 -04:00
|
|
|
func DecodeMessage(bz []byte, maxSize int) (msgType byte, msg BlockchainMessage, err error) {
|
2015-03-25 00:15:18 -07:00
|
|
|
msgType = bz[0]
|
2015-11-10 13:10:43 -08:00
|
|
|
n := int(0)
|
2015-03-25 00:15:18 -07:00
|
|
|
r := bytes.NewReader(bz)
|
2017-09-11 16:28:00 -04:00
|
|
|
msg = wire.ReadBinary(struct{ BlockchainMessage }{}, r, maxSize, &n, &err).(struct{ BlockchainMessage }).BlockchainMessage
|
2015-11-10 13:10:43 -08:00
|
|
|
if err != nil && n != len(bz) {
|
2017-01-17 20:58:27 +04:00
|
|
|
err = errors.New("DecodeMessage() had bytes left over")
|
2015-08-18 10:51:55 -07:00
|
|
|
}
|
2015-03-25 00:15:18 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2015-03-25 17:16:49 -07:00
|
|
|
type bcBlockRequestMessage struct {
|
2015-06-25 20:28:34 -07:00
|
|
|
Height int
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2015-04-16 17:46:27 -07:00
|
|
|
func (m *bcBlockRequestMessage) String() string {
|
2017-01-17 20:58:27 +04:00
|
|
|
return cmn.Fmt("[bcBlockRequestMessage %v]", m.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2017-08-26 02:33:19 -06:00
|
|
|
type bcNoBlockResponseMessage struct {
|
|
|
|
Height int
|
|
|
|
}
|
|
|
|
|
|
|
|
func (brm *bcNoBlockResponseMessage) String() string {
|
|
|
|
return cmn.Fmt("[bcNoBlockResponseMessage %d]", brm.Height)
|
|
|
|
}
|
|
|
|
|
2015-03-25 00:15:18 -07:00
|
|
|
//-------------------------------------
|
|
|
|
|
2015-11-10 13:10:43 -08:00
|
|
|
// NOTE: keep up-to-date with maxBlockchainResponseSize
|
2015-03-25 17:16:49 -07:00
|
|
|
type bcBlockResponseMessage struct {
|
2015-03-25 00:15:18 -07:00
|
|
|
Block *types.Block
|
|
|
|
}
|
|
|
|
|
2015-04-16 17:46:27 -07:00
|
|
|
func (m *bcBlockResponseMessage) String() string {
|
2017-01-17 20:58:27 +04:00
|
|
|
return cmn.Fmt("[bcBlockResponseMessage %v]", m.Block.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
type bcStatusRequestMessage struct {
|
2015-06-25 20:28:34 -07:00
|
|
|
Height int
|
2015-04-21 19:51:23 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
func (m *bcStatusRequestMessage) String() string {
|
2017-01-17 20:58:27 +04:00
|
|
|
return cmn.Fmt("[bcStatusRequestMessage %v]", m.Height)
|
2015-04-21 19:51:23 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
|
|
|
type bcStatusResponseMessage struct {
|
2015-06-25 20:28:34 -07:00
|
|
|
Height int
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|
|
|
|
|
2015-04-21 19:51:23 -07:00
|
|
|
func (m *bcStatusResponseMessage) String() string {
|
2017-01-17 20:58:27 +04:00
|
|
|
return cmn.Fmt("[bcStatusResponseMessage %v]", m.Height)
|
2015-03-25 00:15:18 -07:00
|
|
|
}
|