mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-12 12:51:22 +00: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
This commit is contained in:
@ -91,13 +91,13 @@ type ConsensusState struct {
|
||||
internalMsgQueue chan msgInfo
|
||||
timeoutTicker TimeoutTicker
|
||||
|
||||
// we use PubSub to trigger msg broadcasts in the reactor,
|
||||
// we use eventBus to trigger msg broadcasts in the reactor,
|
||||
// and to notify external subscribers, eg. through a websocket
|
||||
evsw types.EventSwitch
|
||||
eventBus *types.EventBus
|
||||
|
||||
// a Write-Ahead Log ensures we can recover from any kind of crash
|
||||
// and helps us avoid signing conflicting votes
|
||||
wal *WAL
|
||||
wal WAL
|
||||
replayMode bool // so we don't log signing errors during replay
|
||||
doWALCatchup bool // determines if we even try to do the catchup
|
||||
|
||||
@ -125,6 +125,7 @@ func NewConsensusState(config *cfg.ConsensusConfig, state *sm.State, proxyAppCon
|
||||
timeoutTicker: NewTimeoutTicker(),
|
||||
done: make(chan struct{}),
|
||||
doWALCatchup: true,
|
||||
wal: nilWAL{},
|
||||
}
|
||||
// set function defaults (may be overwritten before calling Start)
|
||||
cs.decideProposal = cs.defaultDecideProposal
|
||||
@ -148,9 +149,9 @@ func (cs *ConsensusState) SetLogger(l log.Logger) {
|
||||
cs.timeoutTicker.SetLogger(l)
|
||||
}
|
||||
|
||||
// SetEventSwitch implements events.Eventable
|
||||
func (cs *ConsensusState) SetEventSwitch(evsw types.EventSwitch) {
|
||||
cs.evsw = evsw
|
||||
// SetEventBus sets event bus.
|
||||
func (cs *ConsensusState) SetEventBus(b *types.EventBus) {
|
||||
cs.eventBus = b
|
||||
}
|
||||
|
||||
// String returns a string.
|
||||
@ -212,11 +213,16 @@ func (cs *ConsensusState) LoadCommit(height int) *types.Commit {
|
||||
// OnStart implements cmn.Service.
|
||||
// It loads the latest state via the WAL, and starts the timeout and receive routines.
|
||||
func (cs *ConsensusState) OnStart() error {
|
||||
|
||||
walFile := cs.config.WalFile()
|
||||
if err := cs.OpenWAL(walFile); err != nil {
|
||||
cs.Logger.Error("Error loading ConsensusState wal", "err", err.Error())
|
||||
return err
|
||||
// we may set the WAL in testing before calling Start,
|
||||
// so only OpenWAL if its still the nilWAL
|
||||
if _, ok := cs.wal.(nilWAL); ok {
|
||||
walFile := cs.config.WalFile()
|
||||
wal, err := cs.OpenWAL(walFile)
|
||||
if err != nil {
|
||||
cs.Logger.Error("Error loading ConsensusState wal", "err", err.Error())
|
||||
return err
|
||||
}
|
||||
cs.wal = wal
|
||||
}
|
||||
|
||||
// we need the timeoutRoutine for replay so
|
||||
@ -260,7 +266,7 @@ func (cs *ConsensusState) OnStop() {
|
||||
cs.timeoutTicker.Stop()
|
||||
|
||||
// Make BaseService.Wait() wait until cs.wal.Wait()
|
||||
if cs.wal != nil && cs.IsRunning() {
|
||||
if cs.IsRunning() {
|
||||
cs.wal.Wait()
|
||||
}
|
||||
}
|
||||
@ -273,25 +279,22 @@ func (cs *ConsensusState) Wait() {
|
||||
}
|
||||
|
||||
// OpenWAL opens a file to log all consensus messages and timeouts for deterministic accountability
|
||||
func (cs *ConsensusState) OpenWAL(walFile string) (err error) {
|
||||
err = cmn.EnsureDir(filepath.Dir(walFile), 0700)
|
||||
func (cs *ConsensusState) OpenWAL(walFile string) (WAL, error) {
|
||||
err := cmn.EnsureDir(filepath.Dir(walFile), 0700)
|
||||
if err != nil {
|
||||
cs.Logger.Error("Error ensuring ConsensusState wal dir", "err", err.Error())
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
wal, err := NewWAL(walFile, cs.config.WalLight)
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
wal.SetLogger(cs.Logger.With("wal", walFile))
|
||||
if _, err := wal.Start(); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
cs.wal = wal
|
||||
return nil
|
||||
return wal, nil
|
||||
}
|
||||
|
||||
//------------------------------------------------------------
|
||||
@ -480,9 +483,9 @@ func (cs *ConsensusState) newStep() {
|
||||
rs := cs.RoundStateEvent()
|
||||
cs.wal.Save(rs)
|
||||
cs.nSteps += 1
|
||||
// newStep is called by updateToStep in NewConsensusState before the evsw is set!
|
||||
if cs.evsw != nil {
|
||||
types.FireEventNewRoundStep(cs.evsw, rs)
|
||||
// newStep is called by updateToStep in NewConsensusState before the eventBus is set!
|
||||
if cs.eventBus != nil {
|
||||
cs.eventBus.PublishEventNewRoundStep(rs)
|
||||
}
|
||||
}
|
||||
|
||||
@ -536,9 +539,7 @@ func (cs *ConsensusState) receiveRoutine(maxSteps int) {
|
||||
// priv_val tracks LastSig
|
||||
|
||||
// close wal now that we're done writing to it
|
||||
if cs.wal != nil {
|
||||
cs.wal.Stop()
|
||||
}
|
||||
cs.wal.Stop()
|
||||
|
||||
close(cs.done)
|
||||
return
|
||||
@ -607,13 +608,13 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs cstypes.RoundState) {
|
||||
case cstypes.RoundStepNewRound:
|
||||
cs.enterPropose(ti.Height, 0)
|
||||
case cstypes.RoundStepPropose:
|
||||
types.FireEventTimeoutPropose(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventTimeoutPropose(cs.RoundStateEvent())
|
||||
cs.enterPrevote(ti.Height, ti.Round)
|
||||
case cstypes.RoundStepPrevoteWait:
|
||||
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventTimeoutWait(cs.RoundStateEvent())
|
||||
cs.enterPrecommit(ti.Height, ti.Round)
|
||||
case cstypes.RoundStepPrecommitWait:
|
||||
types.FireEventTimeoutWait(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventTimeoutWait(cs.RoundStateEvent())
|
||||
cs.enterNewRound(ti.Height, ti.Round+1)
|
||||
default:
|
||||
panic(cmn.Fmt("Invalid timeout step: %v", ti.Step))
|
||||
@ -673,7 +674,7 @@ func (cs *ConsensusState) enterNewRound(height int, round int) {
|
||||
}
|
||||
cs.Votes.SetRound(round + 1) // also track next round (round+1) to allow round-skipping
|
||||
|
||||
types.FireEventNewRound(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventNewRound(cs.RoundStateEvent())
|
||||
|
||||
// Wait for txs to be available in the mempool
|
||||
// before we enterPropose in round 0. If the last block changed the app hash,
|
||||
@ -726,8 +727,7 @@ func (cs *ConsensusState) proposalHeartbeat(height, round int) {
|
||||
ValidatorIndex: valIndex,
|
||||
}
|
||||
cs.privValidator.SignHeartbeat(chainID, heartbeat)
|
||||
heartbeatEvent := types.EventDataProposalHeartbeat{heartbeat}
|
||||
types.FireEventProposalHeartbeat(cs.evsw, heartbeatEvent)
|
||||
cs.eventBus.PublishEventProposalHeartbeat(types.EventDataProposalHeartbeat{heartbeat})
|
||||
counter += 1
|
||||
time.Sleep(proposalHeartbeatIntervalSeconds * time.Second)
|
||||
}
|
||||
@ -885,7 +885,7 @@ func (cs *ConsensusState) enterPrevote(height int, round int) {
|
||||
|
||||
// fire event for how we got here
|
||||
if cs.isProposalComplete() {
|
||||
types.FireEventCompleteProposal(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventCompleteProposal(cs.RoundStateEvent())
|
||||
} else {
|
||||
// we received +2/3 prevotes for a future round
|
||||
// TODO: catchup event?
|
||||
@ -987,7 +987,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
}
|
||||
|
||||
// At this point +2/3 prevoted for a particular block or nil
|
||||
types.FireEventPolka(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventPolka(cs.RoundStateEvent())
|
||||
|
||||
// the latest POLRound should be this round
|
||||
polRound, _ := cs.Votes.POLInfo()
|
||||
@ -1004,7 +1004,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
cs.LockedRound = 0
|
||||
cs.LockedBlock = nil
|
||||
cs.LockedBlockParts = nil
|
||||
types.FireEventUnlock(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
|
||||
}
|
||||
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
|
||||
return
|
||||
@ -1016,7 +1016,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
if cs.LockedBlock.HashesTo(blockID.Hash) {
|
||||
cs.Logger.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
|
||||
cs.LockedRound = round
|
||||
types.FireEventRelock(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventRelock(cs.RoundStateEvent())
|
||||
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
|
||||
return
|
||||
}
|
||||
@ -1031,7 +1031,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
cs.LockedRound = round
|
||||
cs.LockedBlock = cs.ProposalBlock
|
||||
cs.LockedBlockParts = cs.ProposalBlockParts
|
||||
types.FireEventLock(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventLock(cs.RoundStateEvent())
|
||||
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
|
||||
return
|
||||
}
|
||||
@ -1047,7 +1047,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
|
||||
cs.ProposalBlock = nil
|
||||
cs.ProposalBlockParts = types.NewPartSetFromHeader(blockID.PartsHeader)
|
||||
}
|
||||
types.FireEventUnlock(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
|
||||
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
|
||||
}
|
||||
|
||||
@ -1191,21 +1191,19 @@ func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
// WAL replay for blocks with an #ENDHEIGHT
|
||||
// As is, ConsensusState should not be started again
|
||||
// until we successfully call ApplyBlock (ie. here or in Handshake after restart)
|
||||
if cs.wal != nil {
|
||||
cs.wal.Save(EndHeightMessage{uint64(height)})
|
||||
}
|
||||
cs.wal.Save(EndHeightMessage{uint64(height)})
|
||||
|
||||
fail.Fail() // XXX
|
||||
|
||||
// Create a copy of the state for staging
|
||||
// and an event cache for txs
|
||||
stateCopy := cs.state.Copy()
|
||||
eventCache := types.NewEventCache(cs.evsw)
|
||||
txEventBuffer := types.NewTxEventBuffer(cs.eventBus, block.NumTxs)
|
||||
|
||||
// Execute and commit the block, update and save the state, and update the mempool.
|
||||
// All calls to the proxyAppConn come here.
|
||||
// NOTE: the block.AppHash wont reflect these txs until the next block
|
||||
err := stateCopy.ApplyBlock(eventCache, cs.proxyAppConn, block, blockParts.Header(), cs.mempool)
|
||||
err := stateCopy.ApplyBlock(txEventBuffer, cs.proxyAppConn, block, blockParts.Header(), cs.mempool)
|
||||
if err != nil {
|
||||
cs.Logger.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "err", err)
|
||||
return
|
||||
@ -1220,9 +1218,12 @@ func (cs *ConsensusState) finalizeCommit(height int) {
|
||||
// * Fire before persisting state, in ApplyBlock
|
||||
// * Fire on start up if we haven't written any new WAL msgs
|
||||
// Both options mean we may fire more than once. Is that fine ?
|
||||
types.FireEventNewBlock(cs.evsw, types.EventDataNewBlock{block})
|
||||
types.FireEventNewBlockHeader(cs.evsw, types.EventDataNewBlockHeader{block.Header})
|
||||
eventCache.Flush()
|
||||
cs.eventBus.PublishEventNewBlock(types.EventDataNewBlock{block})
|
||||
cs.eventBus.PublishEventNewBlockHeader(types.EventDataNewBlockHeader{block.Header})
|
||||
err = txEventBuffer.Flush()
|
||||
if err != nil {
|
||||
cs.Logger.Error("Failed to flush event buffer", "err", err)
|
||||
}
|
||||
|
||||
fail.Fail() // XXX
|
||||
|
||||
@ -1357,7 +1358,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
added, err = cs.LastCommit.AddVote(vote)
|
||||
if added {
|
||||
cs.Logger.Info(cmn.Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
|
||||
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
|
||||
cs.eventBus.PublishEventVote(types.EventDataVote{vote})
|
||||
|
||||
// if we can skip timeoutCommit and have all the votes now,
|
||||
if cs.config.SkipTimeoutCommit && cs.LastCommit.HasAll() {
|
||||
@ -1375,7 +1376,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
height := cs.Height
|
||||
added, err = cs.Votes.AddVote(vote, peerKey)
|
||||
if added {
|
||||
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
|
||||
cs.eventBus.PublishEventVote(types.EventDataVote{vote})
|
||||
|
||||
switch vote.Type {
|
||||
case types.VoteTypePrevote:
|
||||
@ -1393,7 +1394,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
|
||||
cs.LockedRound = 0
|
||||
cs.LockedBlock = nil
|
||||
cs.LockedBlockParts = nil
|
||||
types.FireEventUnlock(cs.evsw, cs.RoundStateEvent())
|
||||
cs.eventBus.PublishEventUnlock(cs.RoundStateEvent())
|
||||
}
|
||||
}
|
||||
if cs.Round <= vote.Round && prevotes.HasTwoThirdsAny() {
|
||||
|
Reference in New Issue
Block a user