mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-24 18:21:38 +00:00
Merge pull request #788 from tendermint/feature/548-indexing-tags
new pubsub package
This commit is contained in:
@ -1,16 +1,17 @@
|
||||
package consensus
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
crypto "github.com/tendermint/go-crypto"
|
||||
data "github.com/tendermint/go-wire/data"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
"github.com/tendermint/tmlibs/events"
|
||||
)
|
||||
|
||||
func init() {
|
||||
@ -41,18 +42,8 @@ func TestByzantine(t *testing.T) {
|
||||
switches[i].SetLogger(p2pLogger.With("validator", i))
|
||||
}
|
||||
|
||||
reactors := make([]p2p.Reactor, N)
|
||||
defer func() {
|
||||
for _, r := range reactors {
|
||||
if rr, ok := r.(*ByzantineReactor); ok {
|
||||
rr.reactor.Switch.Stop()
|
||||
} else {
|
||||
r.(*ConsensusReactor).Switch.Stop()
|
||||
}
|
||||
}
|
||||
}()
|
||||
eventChans := make([]chan interface{}, N)
|
||||
eventLogger := logger.With("module", "events")
|
||||
reactors := make([]p2p.Reactor, N)
|
||||
for i := 0; i < N; i++ {
|
||||
if i == 0 {
|
||||
css[i].privValidator = NewByzantinePrivValidator(css[i].privValidator)
|
||||
@ -65,17 +56,19 @@ func TestByzantine(t *testing.T) {
|
||||
css[i].doPrevote = func(height, round int) {}
|
||||
}
|
||||
|
||||
eventSwitch := events.NewEventSwitch()
|
||||
eventSwitch.SetLogger(eventLogger.With("validator", i))
|
||||
_, err := eventSwitch.Start()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start switch: %v", err)
|
||||
}
|
||||
eventChans[i] = subscribeToEvent(eventSwitch, "tester", types.EventStringNewBlock(), 1)
|
||||
eventBus := types.NewEventBus()
|
||||
eventBus.SetLogger(logger.With("module", "events", "validator", i))
|
||||
_, err := eventBus.Start()
|
||||
require.NoError(t, err)
|
||||
defer eventBus.Stop()
|
||||
|
||||
eventChans[i] = make(chan interface{}, 1)
|
||||
err = eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, eventChans[i])
|
||||
require.NoError(t, err)
|
||||
|
||||
conR := NewConsensusReactor(css[i], true) // so we dont start the consensus states
|
||||
conR.SetLogger(logger.With("validator", i))
|
||||
conR.SetEventSwitch(eventSwitch)
|
||||
conR.SetEventBus(eventBus)
|
||||
|
||||
var conRI p2p.Reactor
|
||||
conRI = conR
|
||||
@ -86,6 +79,16 @@ func TestByzantine(t *testing.T) {
|
||||
reactors[i] = conRI
|
||||
}
|
||||
|
||||
defer func() {
|
||||
for _, r := range reactors {
|
||||
if rr, ok := r.(*ByzantineReactor); ok {
|
||||
rr.reactor.Switch.Stop()
|
||||
} else {
|
||||
r.(*ConsensusReactor).Switch.Stop()
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
p2p.MakeConnectedSwitches(config.P2P, N, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
// ignore new switch s, we already made ours
|
||||
switches[i].AddReactor("CONSENSUS", reactors[i])
|
||||
|
@ -1,35 +0,0 @@
|
||||
package consensus
|
||||
|
||||
import (
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
// XXX: WARNING: these functions can halt the consensus as firing events is synchronous.
|
||||
// Make sure to read off the channels, and in the case of subscribeToEventRespond, to write back on it
|
||||
|
||||
// NOTE: if chanCap=0, this blocks on the event being consumed
|
||||
func subscribeToEvent(evsw types.EventSwitch, receiver, eventID string, chanCap int) chan interface{} {
|
||||
// listen for event
|
||||
ch := make(chan interface{}, chanCap)
|
||||
types.AddListenerForEvent(evsw, receiver, eventID, func(data types.TMEventData) {
|
||||
ch <- data
|
||||
})
|
||||
return ch
|
||||
}
|
||||
|
||||
// NOTE: this blocks on receiving a response after the event is consumed
|
||||
func subscribeToEventRespond(evsw types.EventSwitch, receiver, eventID string) chan interface{} {
|
||||
// listen for event
|
||||
ch := make(chan interface{})
|
||||
types.AddListenerForEvent(evsw, receiver, eventID, func(data types.TMEventData) {
|
||||
ch <- data
|
||||
<-ch
|
||||
})
|
||||
return ch
|
||||
}
|
||||
|
||||
func discardFromChan(ch chan interface{}, n int) {
|
||||
for i := 0; i < n; i++ {
|
||||
<-ch
|
||||
}
|
||||
}
|
@ -2,6 +2,7 @@ package consensus
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
@ -30,6 +31,10 @@ import (
|
||||
"github.com/go-kit/kit/log/term"
|
||||
)
|
||||
|
||||
const (
|
||||
testSubscriber = "test-client"
|
||||
)
|
||||
|
||||
// genesis, chain_id, priv_val
|
||||
var config *cfg.Config // NOTE: must be reset for each _test.go file
|
||||
var ensureTimeout = time.Second * 2
|
||||
@ -208,11 +213,14 @@ func validatePrevoteAndPrecommit(t *testing.T, cs *ConsensusState, thisRound, lo
|
||||
|
||||
// genesis
|
||||
func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
|
||||
voteCh0 := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 1)
|
||||
voteCh0 := make(chan interface{})
|
||||
err := cs.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryVote, voteCh0)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to subscribe %s to %v", testSubscriber, types.EventQueryVote))
|
||||
}
|
||||
voteCh := make(chan interface{})
|
||||
go func() {
|
||||
for {
|
||||
v := <-voteCh0
|
||||
for v := range voteCh0 {
|
||||
vote := v.(types.TMEventData).Unwrap().(types.EventDataVote)
|
||||
// we only fire for our own votes
|
||||
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
|
||||
@ -231,8 +239,12 @@ func newConsensusState(state *sm.State, pv types.PrivValidator, app abci.Applica
|
||||
}
|
||||
|
||||
func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv types.PrivValidator, app abci.Application) *ConsensusState {
|
||||
// Get BlockStore
|
||||
blockDB := dbm.NewMemDB()
|
||||
return newConsensusStateWithConfigAndBlockStore(thisConfig, state, pv, app, blockDB)
|
||||
}
|
||||
|
||||
func newConsensusStateWithConfigAndBlockStore(thisConfig *cfg.Config, state *sm.State, pv types.PrivValidator, app abci.Application, blockDB dbm.DB) *ConsensusState {
|
||||
// Get BlockStore
|
||||
blockStore := bc.NewBlockStore(blockDB)
|
||||
|
||||
// one for mempool, one for consensus
|
||||
@ -252,10 +264,11 @@ func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv typ
|
||||
cs.SetLogger(log.TestingLogger())
|
||||
cs.SetPrivValidator(pv)
|
||||
|
||||
evsw := types.NewEventSwitch()
|
||||
evsw.SetLogger(log.TestingLogger().With("module", "events"))
|
||||
cs.SetEventSwitch(evsw)
|
||||
evsw.Start()
|
||||
eventBus := types.NewEventBus()
|
||||
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
|
||||
eventBus.Start()
|
||||
cs.SetEventBus(eventBus)
|
||||
|
||||
return cs
|
||||
}
|
||||
|
||||
@ -267,13 +280,13 @@ func loadPrivValidator(config *cfg.Config) *types.PrivValidatorFS {
|
||||
return privValidator
|
||||
}
|
||||
|
||||
func fixedConsensusStateDummy() *ConsensusState {
|
||||
func fixedConsensusStateDummy(config *cfg.Config, logger log.Logger) *ConsensusState {
|
||||
stateDB := dbm.NewMemDB()
|
||||
state, _ := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
|
||||
state.SetLogger(log.TestingLogger().With("module", "state"))
|
||||
state.SetLogger(logger.With("module", "state"))
|
||||
privValidator := loadPrivValidator(config)
|
||||
cs := newConsensusState(state, privValidator, dummy.NewDummyApplication())
|
||||
cs.SetLogger(log.TestingLogger())
|
||||
cs.SetLogger(logger)
|
||||
return cs
|
||||
}
|
||||
|
||||
@ -297,7 +310,7 @@ func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
|
||||
|
||||
//-------------------------------------------------------------------------------
|
||||
|
||||
func ensureNoNewStep(stepCh chan interface{}) {
|
||||
func ensureNoNewStep(stepCh <-chan interface{}) {
|
||||
timer := time.NewTimer(ensureTimeout)
|
||||
select {
|
||||
case <-timer.C:
|
||||
@ -307,7 +320,7 @@ func ensureNoNewStep(stepCh chan interface{}) {
|
||||
}
|
||||
}
|
||||
|
||||
func ensureNewStep(stepCh chan interface{}) {
|
||||
func ensureNewStep(stepCh <-chan interface{}) {
|
||||
timer := time.NewTimer(ensureTimeout)
|
||||
select {
|
||||
case <-timer.C:
|
||||
@ -362,10 +375,11 @@ func randConsensusNet(nValidators int, testName string, tickerFunc func() Timeou
|
||||
func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerFunc func() TimeoutTicker, appFunc func() abci.Application) []*ConsensusState {
|
||||
genDoc, privVals := randGenesisDoc(nValidators, false, int64(testMinPower))
|
||||
css := make([]*ConsensusState, nPeers)
|
||||
logger := consensusLogger()
|
||||
for i := 0; i < nPeers; i++ {
|
||||
db := dbm.NewMemDB() // each state needs its own db
|
||||
state, _ := sm.MakeGenesisState(db, genDoc)
|
||||
state.SetLogger(log.TestingLogger().With("module", "state"))
|
||||
state.SetLogger(logger.With("module", "state", "validator", i))
|
||||
state.Save()
|
||||
thisConfig := ResetConfig(cmn.Fmt("%s_%d", testName, i))
|
||||
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
|
||||
@ -382,7 +396,7 @@ func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerF
|
||||
app.InitChain(abci.RequestInitChain{Validators: vals})
|
||||
|
||||
css[i] = newConsensusStateWithConfig(thisConfig, state, privVal, app)
|
||||
css[i].SetLogger(log.TestingLogger())
|
||||
css[i].SetLogger(logger.With("validator", i))
|
||||
css[i].SetTimeoutTicker(tickerFunc())
|
||||
}
|
||||
return css
|
||||
|
@ -7,7 +7,6 @@ import (
|
||||
|
||||
abci "github.com/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
)
|
||||
|
||||
@ -22,7 +21,7 @@ func TestNoProgressUntilTxsAvailable(t *testing.T) {
|
||||
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
|
||||
cs.mempool.EnableTxsAvailable()
|
||||
height, round := cs.Height, cs.Round
|
||||
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
|
||||
startTestRound(cs, height, round)
|
||||
|
||||
ensureNewStep(newBlockCh) // first block gets committed
|
||||
@ -41,7 +40,7 @@ func TestProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
|
||||
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
|
||||
cs.mempool.EnableTxsAvailable()
|
||||
height, round := cs.Height, cs.Round
|
||||
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
|
||||
startTestRound(cs, height, round)
|
||||
|
||||
ensureNewStep(newBlockCh) // first block gets committed
|
||||
@ -56,9 +55,9 @@ func TestProgressInHigherRound(t *testing.T) {
|
||||
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
|
||||
cs.mempool.EnableTxsAvailable()
|
||||
height, round := cs.Height, cs.Round
|
||||
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
newRoundCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
timeoutCh := subscribeToEvent(cs.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
|
||||
newRoundCh := subscribe(cs.eventBus, types.EventQueryNewRound)
|
||||
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
|
||||
cs.setProposal = func(proposal *types.Proposal) error {
|
||||
if cs.Height == 2 && cs.Round == 0 {
|
||||
// dont set the proposal in round 0 so we timeout and
|
||||
@ -92,11 +91,10 @@ func deliverTxsRange(cs *ConsensusState, start, end int) {
|
||||
}
|
||||
|
||||
func TestTxConcurrentWithCommit(t *testing.T) {
|
||||
|
||||
state, privVals := randGenesisState(1, false, 10)
|
||||
cs := newConsensusState(state, privVals[0], NewCounterApplication())
|
||||
height, round := cs.Height, cs.Round
|
||||
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
|
||||
|
||||
NTxs := 10000
|
||||
go deliverTxsRange(cs, 0, NTxs)
|
||||
|
@ -2,12 +2,14 @@ package consensus
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
wire "github.com/tendermint/go-wire"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
@ -34,10 +36,10 @@ type ConsensusReactor struct {
|
||||
p2p.BaseReactor // BaseService + p2p.Switch
|
||||
|
||||
conS *ConsensusState
|
||||
evsw types.EventSwitch
|
||||
|
||||
mtx sync.RWMutex
|
||||
fastSync bool
|
||||
eventBus *types.EventBus
|
||||
}
|
||||
|
||||
// NewConsensusReactor returns a new ConsensusReactor with the given consensusState.
|
||||
@ -55,9 +57,10 @@ func (conR *ConsensusReactor) OnStart() error {
|
||||
conR.Logger.Info("ConsensusReactor ", "fastSync", conR.FastSync())
|
||||
conR.BaseReactor.OnStart()
|
||||
|
||||
// callbacks for broadcasting new steps and votes to peers
|
||||
// upon their respective events (ie. uses evsw)
|
||||
conR.registerEventCallbacks()
|
||||
err := conR.startBroadcastRoutine()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !conR.FastSync() {
|
||||
_, err := conR.conS.Start()
|
||||
@ -65,6 +68,7 @@ func (conR *ConsensusReactor) OnStart() error {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -306,10 +310,10 @@ func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
|
||||
}
|
||||
}
|
||||
|
||||
// SetEventSwitch implements events.Eventable
|
||||
func (conR *ConsensusReactor) SetEventSwitch(evsw types.EventSwitch) {
|
||||
conR.evsw = evsw
|
||||
conR.conS.SetEventSwitch(evsw)
|
||||
// SetEventBus sets event bus.
|
||||
func (conR *ConsensusReactor) SetEventBus(b *types.EventBus) {
|
||||
conR.eventBus = b
|
||||
conR.conS.SetEventBus(b)
|
||||
}
|
||||
|
||||
// FastSync returns whether the consensus reactor is in fast-sync mode.
|
||||
@ -321,24 +325,60 @@ func (conR *ConsensusReactor) FastSync() bool {
|
||||
|
||||
//--------------------------------------
|
||||
|
||||
// Listens for new steps and votes,
|
||||
// broadcasting the result to peers
|
||||
func (conR *ConsensusReactor) registerEventCallbacks() {
|
||||
// startBroadcastRoutine subscribes for new round steps, votes and proposal
|
||||
// heartbeats using the event bus and starts a go routine to broadcasts events
|
||||
// to peers upon receiving them.
|
||||
func (conR *ConsensusReactor) startBroadcastRoutine() error {
|
||||
const subscriber = "consensus-reactor"
|
||||
ctx := context.Background()
|
||||
|
||||
types.AddListenerForEvent(conR.evsw, "conR", types.EventStringNewRoundStep(), func(data types.TMEventData) {
|
||||
rs := data.Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState)
|
||||
conR.broadcastNewRoundStep(rs)
|
||||
})
|
||||
// new round steps
|
||||
stepsCh := make(chan interface{})
|
||||
err := conR.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep, stepsCh)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to subscribe %s to %s", subscriber, types.EventQueryNewRoundStep)
|
||||
}
|
||||
|
||||
types.AddListenerForEvent(conR.evsw, "conR", types.EventStringVote(), func(data types.TMEventData) {
|
||||
edv := data.Unwrap().(types.EventDataVote)
|
||||
conR.broadcastHasVoteMessage(edv.Vote)
|
||||
})
|
||||
// votes
|
||||
votesCh := make(chan interface{})
|
||||
err = conR.eventBus.Subscribe(ctx, subscriber, types.EventQueryVote, votesCh)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to subscribe %s to %s", subscriber, types.EventQueryVote)
|
||||
}
|
||||
|
||||
types.AddListenerForEvent(conR.evsw, "conR", types.EventStringProposalHeartbeat(), func(data types.TMEventData) {
|
||||
heartbeat := data.Unwrap().(types.EventDataProposalHeartbeat)
|
||||
conR.broadcastProposalHeartbeatMessage(heartbeat)
|
||||
})
|
||||
// proposal heartbeats
|
||||
heartbeatsCh := make(chan interface{})
|
||||
err = conR.eventBus.Subscribe(ctx, subscriber, types.EventQueryProposalHeartbeat, heartbeatsCh)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to subscribe %s to %s", subscriber, types.EventQueryProposalHeartbeat)
|
||||
}
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case data, ok := <-stepsCh:
|
||||
if ok { // a receive from a closed channel returns the zero value immediately
|
||||
edrs := data.(types.TMEventData).Unwrap().(types.EventDataRoundState)
|
||||
conR.broadcastNewRoundStep(edrs.RoundState.(*cstypes.RoundState))
|
||||
}
|
||||
case data, ok := <-votesCh:
|
||||
if ok {
|
||||
edv := data.(types.TMEventData).Unwrap().(types.EventDataVote)
|
||||
conR.broadcastHasVoteMessage(edv.Vote)
|
||||
}
|
||||
case data, ok := <-heartbeatsCh:
|
||||
if ok {
|
||||
edph := data.(types.TMEventData).Unwrap().(types.EventDataProposalHeartbeat)
|
||||
conR.broadcastProposalHeartbeatMessage(edph)
|
||||
}
|
||||
case <-conR.Quit:
|
||||
conR.eventBus.UnsubscribeAll(ctx, subscriber)
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (conR *ConsensusReactor) broadcastProposalHeartbeatMessage(heartbeat types.EventDataProposalHeartbeat) {
|
||||
|
@ -1,17 +1,19 @@
|
||||
package consensus
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/abci/example/dummy"
|
||||
"github.com/tendermint/tmlibs/events"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func init() {
|
||||
@ -21,27 +23,25 @@ func init() {
|
||||
//----------------------------------------------
|
||||
// in-process testnets
|
||||
|
||||
func startConsensusNet(t *testing.T, css []*ConsensusState, N int, subscribeEventRespond bool) ([]*ConsensusReactor, []chan interface{}) {
|
||||
func startConsensusNet(t *testing.T, css []*ConsensusState, N int) ([]*ConsensusReactor, []chan interface{}, []*types.EventBus) {
|
||||
reactors := make([]*ConsensusReactor, N)
|
||||
eventChans := make([]chan interface{}, N)
|
||||
eventBuses := make([]*types.EventBus, N)
|
||||
logger := consensusLogger()
|
||||
for i := 0; i < N; i++ {
|
||||
reactors[i] = NewConsensusReactor(css[i], true) // so we dont start the consensus states
|
||||
reactors[i].SetLogger(logger.With("validator", i))
|
||||
|
||||
eventSwitch := events.NewEventSwitch()
|
||||
eventSwitch.SetLogger(logger.With("module", "events", "validator", i))
|
||||
_, err := eventSwitch.Start()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start switch: %v", err)
|
||||
}
|
||||
eventBuses[i] = types.NewEventBus()
|
||||
eventBuses[i].SetLogger(logger.With("module", "events", "validator", i))
|
||||
_, err := eventBuses[i].Start()
|
||||
require.NoError(t, err)
|
||||
|
||||
reactors[i].SetEventSwitch(eventSwitch)
|
||||
if subscribeEventRespond {
|
||||
eventChans[i] = subscribeToEventRespond(eventSwitch, "tester", types.EventStringNewBlock())
|
||||
} else {
|
||||
eventChans[i] = subscribeToEvent(eventSwitch, "tester", types.EventStringNewBlock(), 1)
|
||||
}
|
||||
reactors[i].SetEventBus(eventBuses[i])
|
||||
|
||||
eventChans[i] = make(chan interface{}, 1)
|
||||
err = eventBuses[i].Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, eventChans[i])
|
||||
require.NoError(t, err)
|
||||
}
|
||||
// make connected switches and start all reactors
|
||||
p2p.MakeConnectedSwitches(config.P2P, N, func(i int, s *p2p.Switch) *p2p.Switch {
|
||||
@ -56,21 +56,24 @@ func startConsensusNet(t *testing.T, css []*ConsensusState, N int, subscribeEven
|
||||
s := reactors[i].conS.GetState()
|
||||
reactors[i].SwitchToConsensus(s, 0)
|
||||
}
|
||||
return reactors, eventChans
|
||||
return reactors, eventChans, eventBuses
|
||||
}
|
||||
|
||||
func stopConsensusNet(reactors []*ConsensusReactor) {
|
||||
func stopConsensusNet(reactors []*ConsensusReactor, eventBuses []*types.EventBus) {
|
||||
for _, r := range reactors {
|
||||
r.Switch.Stop()
|
||||
}
|
||||
for _, b := range eventBuses {
|
||||
b.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure a testnet makes blocks
|
||||
func TestReactor(t *testing.T) {
|
||||
N := 4
|
||||
css := randConsensusNet(N, "consensus_reactor_test", newMockTickerFunc(true), newCounter)
|
||||
reactors, eventChans := startConsensusNet(t, css, N, false)
|
||||
defer stopConsensusNet(reactors)
|
||||
reactors, eventChans, eventBuses := startConsensusNet(t, css, N)
|
||||
defer stopConsensusNet(reactors, eventBuses)
|
||||
// wait till everyone makes the first new block
|
||||
timeoutWaitGroup(t, N, func(wg *sync.WaitGroup, j int) {
|
||||
<-eventChans[j]
|
||||
@ -85,11 +88,14 @@ func TestReactorProposalHeartbeats(t *testing.T) {
|
||||
func(c *cfg.Config) {
|
||||
c.Consensus.CreateEmptyBlocks = false
|
||||
})
|
||||
reactors, eventChans := startConsensusNet(t, css, N, false)
|
||||
defer stopConsensusNet(reactors)
|
||||
reactors, eventChans, eventBuses := startConsensusNet(t, css, N)
|
||||
defer stopConsensusNet(reactors, eventBuses)
|
||||
heartbeatChans := make([]chan interface{}, N)
|
||||
var err error
|
||||
for i := 0; i < N; i++ {
|
||||
heartbeatChans[i] = subscribeToEvent(css[i].evsw, "tester", types.EventStringProposalHeartbeat(), 1)
|
||||
heartbeatChans[i] = make(chan interface{}, 1)
|
||||
err = eventBuses[i].Subscribe(context.Background(), testSubscriber, types.EventQueryProposalHeartbeat, heartbeatChans[i])
|
||||
require.NoError(t, err)
|
||||
}
|
||||
// wait till everyone sends a proposal heartbeat
|
||||
timeoutWaitGroup(t, N, func(wg *sync.WaitGroup, j int) {
|
||||
@ -113,8 +119,8 @@ func TestReactorProposalHeartbeats(t *testing.T) {
|
||||
func TestVotingPowerChange(t *testing.T) {
|
||||
nVals := 4
|
||||
css := randConsensusNet(nVals, "consensus_voting_power_changes_test", newMockTickerFunc(true), newPersistentDummy)
|
||||
reactors, eventChans := startConsensusNet(t, css, nVals, true)
|
||||
defer stopConsensusNet(reactors)
|
||||
reactors, eventChans, eventBuses := startConsensusNet(t, css, nVals)
|
||||
defer stopConsensusNet(reactors, eventBuses)
|
||||
|
||||
// map of active validators
|
||||
activeVals := make(map[string]struct{})
|
||||
@ -125,7 +131,6 @@ func TestVotingPowerChange(t *testing.T) {
|
||||
// wait till everyone makes block 1
|
||||
timeoutWaitGroup(t, nVals, func(wg *sync.WaitGroup, j int) {
|
||||
<-eventChans[j]
|
||||
eventChans[j] <- struct{}{}
|
||||
wg.Done()
|
||||
}, css)
|
||||
|
||||
@ -174,8 +179,9 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
nPeers := 7
|
||||
nVals := 4
|
||||
css := randConsensusNetWithPeers(nVals, nPeers, "consensus_val_set_changes_test", newMockTickerFunc(true), newPersistentDummy)
|
||||
reactors, eventChans := startConsensusNet(t, css, nPeers, true)
|
||||
defer stopConsensusNet(reactors)
|
||||
|
||||
reactors, eventChans, eventBuses := startConsensusNet(t, css, nPeers)
|
||||
defer stopConsensusNet(reactors, eventBuses)
|
||||
|
||||
// map of active validators
|
||||
activeVals := make(map[string]struct{})
|
||||
@ -186,7 +192,6 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
// wait till everyone makes block 1
|
||||
timeoutWaitGroup(t, nPeers, func(wg *sync.WaitGroup, j int) {
|
||||
<-eventChans[j]
|
||||
eventChans[j] <- struct{}{}
|
||||
wg.Done()
|
||||
}, css)
|
||||
|
||||
@ -214,7 +219,7 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
|
||||
// wait till everyone makes block 5
|
||||
// it includes the commit for block 4, which should have the updated validator set
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
t.Log("---------------------------- Testing changing the voting power of one validator")
|
||||
@ -226,7 +231,7 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css, updateValidatorTx1)
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
|
||||
|
||||
if css[nVals].GetRoundState().LastValidators.TotalVotingPower() == previousTotalVotingPower {
|
||||
t.Errorf("expected voting power to change (before: %d, after: %d)", previousTotalVotingPower, css[nVals].GetRoundState().LastValidators.TotalVotingPower())
|
||||
@ -246,7 +251,7 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
activeVals[string(newValidatorPubKey2.Address())] = struct{}{}
|
||||
activeVals[string(newValidatorPubKey3.Address())] = struct{}{}
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
t.Log("---------------------------- Testing removing two validators at once")
|
||||
@ -259,7 +264,7 @@ func TestValidatorSetChanges(t *testing.T) {
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
delete(activeVals, string(newValidatorPubKey2.Address()))
|
||||
delete(activeVals, string(newValidatorPubKey3.Address()))
|
||||
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
|
||||
waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
|
||||
}
|
||||
|
||||
// Check we can make blocks with skip_timeout_commit=false
|
||||
@ -271,8 +276,8 @@ func TestReactorWithTimeoutCommit(t *testing.T) {
|
||||
css[i].config.SkipTimeoutCommit = false
|
||||
}
|
||||
|
||||
reactors, eventChans := startConsensusNet(t, css, N-1, false)
|
||||
defer stopConsensusNet(reactors)
|
||||
reactors, eventChans, eventBuses := startConsensusNet(t, css, N-1)
|
||||
defer stopConsensusNet(reactors, eventBuses)
|
||||
|
||||
// wait till everyone makes the first new block
|
||||
timeoutWaitGroup(t, N-1, func(wg *sync.WaitGroup, j int) {
|
||||
@ -285,16 +290,40 @@ func waitForAndValidateBlock(t *testing.T, n int, activeVals map[string]struct{}
|
||||
timeoutWaitGroup(t, n, func(wg *sync.WaitGroup, j int) {
|
||||
newBlockI := <-eventChans[j]
|
||||
newBlock := newBlockI.(types.TMEventData).Unwrap().(types.EventDataNewBlock).Block
|
||||
t.Logf("[WARN] Got block height=%v validator=%v", newBlock.Height, j)
|
||||
t.Logf("Got block height=%v validator=%v", newBlock.Height, j)
|
||||
err := validateBlock(newBlock, activeVals)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, tx := range txs {
|
||||
css[j].mempool.CheckTx(tx, nil)
|
||||
if err = css[j].mempool.CheckTx(tx, nil); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
wg.Done()
|
||||
}, css)
|
||||
}
|
||||
|
||||
func waitForBlockWithUpdatedValsAndValidateIt(t *testing.T, n int, updatedVals map[string]struct{}, eventChans []chan interface{}, css []*ConsensusState, txs ...[]byte) {
|
||||
timeoutWaitGroup(t, n, func(wg *sync.WaitGroup, j int) {
|
||||
var newBlock *types.Block
|
||||
LOOP:
|
||||
for {
|
||||
newBlockI := <-eventChans[j]
|
||||
newBlock = newBlockI.(types.TMEventData).Unwrap().(types.EventDataNewBlock).Block
|
||||
if newBlock.LastCommit.Size() == len(updatedVals) {
|
||||
t.Logf("Block with new validators height=%v validator=%v", newBlock.Height, j)
|
||||
break LOOP
|
||||
} else {
|
||||
t.Logf("Block with no new validators height=%v validator=%v. Skipping...", newBlock.Height, j)
|
||||
}
|
||||
}
|
||||
|
||||
err := validateBlock(newBlock, updatedVals)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
eventChans[j] <- struct{}{}
|
||||
wg.Done()
|
||||
}, css)
|
||||
}
|
||||
|
@ -91,7 +91,6 @@ func (cs *ConsensusState) readReplayMessage(msg *TimedWALMessage, newStepCh chan
|
||||
// replay only those messages since the last block.
|
||||
// timeoutRoutine should run concurrently to read off tickChan
|
||||
func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
|
||||
// set replayMode
|
||||
cs.replayMode = true
|
||||
defer func() { cs.replayMode = false }()
|
||||
@ -104,7 +103,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
|
||||
gr.Close()
|
||||
}
|
||||
if found {
|
||||
return errors.New(cmn.Fmt("WAL should not contain #ENDHEIGHT %d.", csHeight))
|
||||
return fmt.Errorf("WAL should not contain #ENDHEIGHT %d.", csHeight)
|
||||
}
|
||||
|
||||
// Search for last height marker
|
||||
@ -334,11 +333,10 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store
|
||||
func (h *Handshaker) replayBlock(height int, proxyApp proxy.AppConnConsensus) ([]byte, error) {
|
||||
mempool := types.MockMempool{}
|
||||
|
||||
var eventCache types.Fireable // nil
|
||||
block := h.store.LoadBlock(height)
|
||||
meta := h.store.LoadBlockMeta(height)
|
||||
|
||||
if err := h.state.ApplyBlock(eventCache, proxyApp, block, meta.BlockID.PartsHeader, mempool); err != nil {
|
||||
if err := h.state.ApplyBlock(types.NopEventBus{}, proxyApp, block, meta.BlockID.PartsHeader, mempool); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -2,13 +2,15 @@ package consensus
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"errors"
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
bc "github.com/tendermint/tendermint/blockchain"
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
@ -18,6 +20,11 @@ import (
|
||||
dbm "github.com/tendermint/tmlibs/db"
|
||||
)
|
||||
|
||||
const (
|
||||
// event bus subscriber
|
||||
subscriber = "replay-file"
|
||||
)
|
||||
|
||||
//--------------------------------------------------------
|
||||
// replay messages interactively or all at once
|
||||
|
||||
@ -42,7 +49,14 @@ func (cs *ConsensusState) ReplayFile(file string, console bool) error {
|
||||
cs.startForReplay()
|
||||
|
||||
// ensure all new step events are regenerated as expected
|
||||
newStepCh := subscribeToEvent(cs.evsw, "replay-test", types.EventStringNewRoundStep(), 1)
|
||||
newStepCh := make(chan interface{}, 1)
|
||||
|
||||
ctx := context.Background()
|
||||
err := cs.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep, newStepCh)
|
||||
if err != nil {
|
||||
return errors.Errorf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep)
|
||||
}
|
||||
defer cs.eventBus.Unsubscribe(ctx, subscriber, types.EventQueryNewRoundStep)
|
||||
|
||||
// just open the file for reading, no need to use wal
|
||||
fp, err := os.OpenFile(file, os.O_RDONLY, 0666)
|
||||
@ -106,12 +120,11 @@ func newPlayback(fileName string, fp *os.File, cs *ConsensusState, genState *sm.
|
||||
|
||||
// go back count steps by resetting the state and running (pb.count - count) steps
|
||||
func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
|
||||
|
||||
pb.cs.Stop()
|
||||
pb.cs.Wait()
|
||||
|
||||
newCS := NewConsensusState(pb.cs.config, pb.genesisState.Copy(), pb.cs.proxyAppConn, pb.cs.blockStore, pb.cs.mempool)
|
||||
newCS.SetEventSwitch(pb.cs.evsw)
|
||||
newCS.SetEventBus(pb.cs.eventBus)
|
||||
newCS.startForReplay()
|
||||
|
||||
pb.fp.Close()
|
||||
@ -196,8 +209,16 @@ func (pb *playback) replayConsoleLoop() int {
|
||||
// NOTE: "back" is not supported in the state machine design,
|
||||
// so we restart and replay up to
|
||||
|
||||
ctx := context.Background()
|
||||
// ensure all new step events are regenerated as expected
|
||||
newStepCh := subscribeToEvent(pb.cs.evsw, "replay-test", types.EventStringNewRoundStep(), 1)
|
||||
newStepCh := make(chan interface{}, 1)
|
||||
|
||||
err := pb.cs.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep, newStepCh)
|
||||
if err != nil {
|
||||
cmn.Exit(fmt.Sprintf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep))
|
||||
}
|
||||
defer pb.cs.eventBus.Unsubscribe(ctx, subscriber, types.EventQueryNewRoundStep)
|
||||
|
||||
if len(tokens) == 1 {
|
||||
pb.replayReset(1, newStepCh)
|
||||
} else {
|
||||
@ -270,14 +291,13 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo
|
||||
cmn.Exit(cmn.Fmt("Error starting proxy app conns: %v", err))
|
||||
}
|
||||
|
||||
// Make event switch
|
||||
eventSwitch := types.NewEventSwitch()
|
||||
if _, err := eventSwitch.Start(); err != nil {
|
||||
cmn.Exit(cmn.Fmt("Failed to start event switch: %v", err))
|
||||
eventBus := types.NewEventBus()
|
||||
if _, err := eventBus.Start(); err != nil {
|
||||
cmn.Exit(cmn.Fmt("Failed to start event bus: %v", err))
|
||||
}
|
||||
|
||||
consensusState := NewConsensusState(csConfig, state.Copy(), proxyApp.Consensus(), blockStore, types.MockMempool{})
|
||||
|
||||
consensusState.SetEventSwitch(eventSwitch)
|
||||
consensusState.SetEventBus(eventBus)
|
||||
return consensusState
|
||||
}
|
||||
|
@ -2,19 +2,24 @@ package consensus
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path"
|
||||
"runtime"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/abci/example/dummy"
|
||||
abci "github.com/tendermint/abci/types"
|
||||
crypto "github.com/tendermint/go-crypto"
|
||||
wire "github.com/tendermint/go-wire"
|
||||
auto "github.com/tendermint/tmlibs/autofile"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
dbm "github.com/tendermint/tmlibs/db"
|
||||
|
||||
@ -25,8 +30,10 @@ import (
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
)
|
||||
|
||||
var consensusReplayConfig *cfg.Config
|
||||
|
||||
func init() {
|
||||
config = ResetConfig("consensus_replay_test")
|
||||
consensusReplayConfig = ResetConfig("consensus_replay_test")
|
||||
}
|
||||
|
||||
// These tests ensure we can always recover from failure at any part of the consensus process.
|
||||
@ -39,8 +46,7 @@ func init() {
|
||||
|
||||
// NOTE: Files in this dir are generated by running the `build.sh` therein.
|
||||
// It's a simple way to generate wals for a single block, or multiple blocks, with random transactions,
|
||||
// and different part sizes. The output is not deterministic, and the stepChanges may need to be adjusted
|
||||
// after running it (eg. sometimes small_block2 will have 5 block parts, sometimes 6).
|
||||
// and different part sizes. The output is not deterministic.
|
||||
// It should only have to be re-run if there is some breaking change to the consensus data structures (eg. blocks, votes)
|
||||
// or to the behaviour of the app (eg. computes app hash differently)
|
||||
var data_dir = path.Join(cmn.GoPath(), "src/github.com/tendermint/tendermint/consensus", "test_data")
|
||||
@ -52,230 +58,209 @@ var data_dir = path.Join(cmn.GoPath(), "src/github.com/tendermint/tendermint/con
|
||||
// and which ones we need the wal for - then we'd also be able to only flush the
|
||||
// wal writer when we need to, instead of with every message.
|
||||
|
||||
// the priv validator changes step at these lines for a block with 1 val and 1 part
|
||||
var baseStepChanges = []int{3, 6, 8}
|
||||
func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight int, blockDB dbm.DB, stateDB dbm.DB) {
|
||||
logger := log.TestingLogger()
|
||||
state, _ := sm.GetState(stateDB, consensusReplayConfig.GenesisFile())
|
||||
state.SetLogger(logger.With("module", "state"))
|
||||
privValidator := loadPrivValidator(consensusReplayConfig)
|
||||
cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, dummy.NewDummyApplication(), blockDB)
|
||||
cs.SetLogger(logger)
|
||||
|
||||
// test recovery from each line in each testCase
|
||||
var testCases = []*testCase{
|
||||
newTestCase("empty_block", baseStepChanges), // empty block (has 1 block part)
|
||||
newTestCase("small_block1", baseStepChanges), // small block with txs in 1 block part
|
||||
newTestCase("small_block2", []int{3, 12, 14}), // small block with txs across 6 smaller block parts
|
||||
}
|
||||
bytes, _ := ioutil.ReadFile(cs.config.WalFile())
|
||||
// fmt.Printf("====== WAL: \n\r%s\n", bytes)
|
||||
t.Logf("====== WAL: \n\r%s\n", bytes)
|
||||
|
||||
type testCase struct {
|
||||
name string
|
||||
log []byte //full cs wal
|
||||
stepMap map[int]int8 // map lines of log to privval step
|
||||
_, err := cs.Start()
|
||||
require.NoError(t, err)
|
||||
defer func() {
|
||||
cs.Stop()
|
||||
}()
|
||||
|
||||
proposeLine int
|
||||
prevoteLine int
|
||||
precommitLine int
|
||||
}
|
||||
|
||||
func newTestCase(name string, stepChanges []int) *testCase {
|
||||
if len(stepChanges) != 3 {
|
||||
panic(cmn.Fmt("a full wal has 3 step changes! Got array %v", stepChanges))
|
||||
}
|
||||
return &testCase{
|
||||
name: name,
|
||||
log: readWAL(path.Join(data_dir, name+".cswal")),
|
||||
stepMap: newMapFromChanges(stepChanges),
|
||||
|
||||
proposeLine: stepChanges[0],
|
||||
prevoteLine: stepChanges[1],
|
||||
precommitLine: stepChanges[2],
|
||||
}
|
||||
}
|
||||
|
||||
func newMapFromChanges(changes []int) map[int]int8 {
|
||||
changes = append(changes, changes[2]+1) // so we add the last step change to the map
|
||||
m := make(map[int]int8)
|
||||
var count int
|
||||
for changeNum, nextChange := range changes {
|
||||
for ; count < nextChange; count++ {
|
||||
m[count] = int8(changeNum)
|
||||
}
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
func readWAL(p string) []byte {
|
||||
b, err := ioutil.ReadFile(p)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
func writeWAL(walMsgs []byte) string {
|
||||
walFile, err := ioutil.TempFile("", "wal")
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to create temp WAL file: %v", err))
|
||||
}
|
||||
_, err = walFile.Write(walMsgs)
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to write to temp WAL file: %v", err))
|
||||
}
|
||||
if err := walFile.Close(); err != nil {
|
||||
panic(fmt.Errorf("failed to close temp WAL file: %v", err))
|
||||
}
|
||||
return walFile.Name()
|
||||
}
|
||||
|
||||
func waitForBlock(newBlockCh chan interface{}, thisCase *testCase, i int) {
|
||||
after := time.After(time.Second * 10)
|
||||
// This is just a signal that we haven't halted; its not something contained
|
||||
// in the WAL itself. Assuming the consensus state is running, replay of any
|
||||
// WAL, including the empty one, should eventually be followed by a new
|
||||
// block, or else something is wrong.
|
||||
newBlockCh := make(chan interface{}, 1)
|
||||
err = cs.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, newBlockCh)
|
||||
require.NoError(t, err)
|
||||
select {
|
||||
case <-newBlockCh:
|
||||
case <-after:
|
||||
panic(cmn.Fmt("Timed out waiting for new block for case '%s' line %d", thisCase.name, i))
|
||||
case <-time.After(10 * time.Second):
|
||||
t.Fatalf("Timed out waiting for new block (see trace above)")
|
||||
}
|
||||
}
|
||||
|
||||
func runReplayTest(t *testing.T, cs *ConsensusState, walFile string, newBlockCh chan interface{},
|
||||
thisCase *testCase, i int) {
|
||||
|
||||
cs.config.SetWalFile(walFile)
|
||||
started, err := cs.Start()
|
||||
if err != nil {
|
||||
t.Fatalf("Cannot start consensus: %v", err)
|
||||
}
|
||||
if !started {
|
||||
t.Error("Consensus did not start")
|
||||
}
|
||||
// Wait to make a new block.
|
||||
// This is just a signal that we haven't halted; its not something contained in the WAL itself.
|
||||
// Assuming the consensus state is running, replay of any WAL, including the empty one,
|
||||
// should eventually be followed by a new block, or else something is wrong
|
||||
waitForBlock(newBlockCh, thisCase, i)
|
||||
cs.evsw.Stop()
|
||||
cs.Stop()
|
||||
LOOP:
|
||||
func sendTxs(cs *ConsensusState, ctx context.Context) {
|
||||
i := 0
|
||||
for {
|
||||
select {
|
||||
case <-newBlockCh:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
break LOOP
|
||||
}
|
||||
}
|
||||
cs.Wait()
|
||||
}
|
||||
|
||||
func toPV(pv types.PrivValidator) *types.PrivValidatorFS {
|
||||
return pv.(*types.PrivValidatorFS)
|
||||
}
|
||||
|
||||
func setupReplayTest(t *testing.T, thisCase *testCase, nLines int, crashAfter bool) (*ConsensusState, chan interface{}, []byte, string) {
|
||||
t.Log("-------------------------------------")
|
||||
t.Logf("Starting replay test %v (of %d lines of WAL). Crash after = %v", thisCase.name, nLines, crashAfter)
|
||||
|
||||
lineStep := nLines
|
||||
if crashAfter {
|
||||
lineStep -= 1
|
||||
}
|
||||
|
||||
split := bytes.Split(thisCase.log, walSeparator)
|
||||
lastMsg := split[nLines]
|
||||
|
||||
// we write those lines up to (not including) one with the signature
|
||||
b := bytes.Join(split[:nLines], walSeparator)
|
||||
b = append(b, walSeparator...)
|
||||
walFile := writeWAL(b)
|
||||
|
||||
cs := fixedConsensusStateDummy()
|
||||
|
||||
// set the last step according to when we crashed vs the wal
|
||||
toPV(cs.privValidator).LastHeight = 1 // first block
|
||||
toPV(cs.privValidator).LastStep = thisCase.stepMap[lineStep]
|
||||
|
||||
t.Logf("[WARN] setupReplayTest LastStep=%v", toPV(cs.privValidator).LastStep)
|
||||
|
||||
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
|
||||
return cs, newBlockCh, lastMsg, walFile
|
||||
}
|
||||
|
||||
func readTimedWALMessage(t *testing.T, rawMsg []byte) TimedWALMessage {
|
||||
b := bytes.NewBuffer(rawMsg)
|
||||
// because rawMsg does not contain a separator and WALDecoder#Decode expects it
|
||||
_, err := b.Write(walSeparator)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
dec := NewWALDecoder(b)
|
||||
msg, err := dec.Decode()
|
||||
if err != nil {
|
||||
t.Fatalf("Error reading json data: %v", err)
|
||||
}
|
||||
return *msg
|
||||
}
|
||||
|
||||
//-----------------------------------------------
|
||||
// Test the log at every iteration, and set the privVal last step
|
||||
// as if the log was written after signing, before the crash
|
||||
|
||||
func TestWALCrashAfterWrite(t *testing.T) {
|
||||
for _, thisCase := range testCases {
|
||||
splitSize := bytes.Count(thisCase.log, walSeparator)
|
||||
for i := 0; i < splitSize-1; i++ {
|
||||
t.Run(fmt.Sprintf("%s:%d", thisCase.name, i), func(t *testing.T) {
|
||||
cs, newBlockCh, _, walFile := setupReplayTest(t, thisCase, i+1, true)
|
||||
cs.config.TimeoutPropose = 100
|
||||
runReplayTest(t, cs, walFile, newBlockCh, thisCase, i+1)
|
||||
// cleanup
|
||||
os.Remove(walFile)
|
||||
})
|
||||
cs.mempool.CheckTx([]byte{byte(i)}, nil)
|
||||
i++
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------
|
||||
// Test the log as if we crashed after signing but before writing.
|
||||
// This relies on privValidator.LastSignature being set
|
||||
// TestWALCrash uses crashing WAL to test we can recover from any WAL failure.
|
||||
func TestWALCrash(t *testing.T) {
|
||||
testCases := []struct {
|
||||
name string
|
||||
initFn func(*ConsensusState, context.Context)
|
||||
heightToStop uint64
|
||||
}{
|
||||
{"empty block",
|
||||
func(cs *ConsensusState, ctx context.Context) {},
|
||||
1},
|
||||
{"block with a smaller part size",
|
||||
func(cs *ConsensusState, ctx context.Context) {
|
||||
// XXX: is there a better way to change BlockPartSizeBytes?
|
||||
params := cs.state.Params
|
||||
params.BlockPartSizeBytes = 512
|
||||
cs.state.Params = params
|
||||
sendTxs(cs, ctx)
|
||||
},
|
||||
1},
|
||||
{"many non-empty blocks",
|
||||
sendTxs,
|
||||
3},
|
||||
}
|
||||
|
||||
func TestWALCrashBeforeWritePropose(t *testing.T) {
|
||||
for _, thisCase := range testCases {
|
||||
lineNum := thisCase.proposeLine
|
||||
t.Run(fmt.Sprintf("%s:%d", thisCase.name, lineNum), func(t *testing.T) {
|
||||
// setup replay test where last message is a proposal
|
||||
cs, newBlockCh, proposalMsg, walFile := setupReplayTest(t, thisCase, lineNum, false)
|
||||
cs.config.TimeoutPropose = 100
|
||||
msg := readTimedWALMessage(t, proposalMsg)
|
||||
proposal := msg.Msg.(msgInfo).Msg.(*ProposalMessage)
|
||||
// Set LastSig
|
||||
toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, proposal.Proposal)
|
||||
toPV(cs.privValidator).LastSignature = proposal.Proposal.Signature
|
||||
runReplayTest(t, cs, walFile, newBlockCh, thisCase, lineNum)
|
||||
// cleanup
|
||||
os.Remove(walFile)
|
||||
for _, tc := range testCases {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
crashWALandCheckLiveness(t, tc.initFn, tc.heightToStop)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestWALCrashBeforeWritePrevote(t *testing.T) {
|
||||
for _, thisCase := range testCases {
|
||||
testReplayCrashBeforeWriteVote(t, thisCase, thisCase.prevoteLine, types.EventStringCompleteProposal())
|
||||
func crashWALandCheckLiveness(t *testing.T, initFn func(*ConsensusState, context.Context), heightToStop uint64) {
|
||||
walPaniced := make(chan error)
|
||||
crashingWal := &crashingWAL{panicCh: walPaniced, heightToStop: heightToStop}
|
||||
|
||||
i := 1
|
||||
LOOP:
|
||||
for {
|
||||
// fmt.Printf("====== LOOP %d\n", i)
|
||||
t.Logf("====== LOOP %d\n", i)
|
||||
|
||||
// create consensus state from a clean slate
|
||||
logger := log.NewNopLogger()
|
||||
stateDB := dbm.NewMemDB()
|
||||
state, _ := sm.MakeGenesisStateFromFile(stateDB, consensusReplayConfig.GenesisFile())
|
||||
state.SetLogger(logger.With("module", "state"))
|
||||
privValidator := loadPrivValidator(consensusReplayConfig)
|
||||
blockDB := dbm.NewMemDB()
|
||||
cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, dummy.NewDummyApplication(), blockDB)
|
||||
cs.SetLogger(logger)
|
||||
|
||||
// start sending transactions
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
go initFn(cs, ctx)
|
||||
|
||||
// clean up WAL file from the previous iteration
|
||||
walFile := cs.config.WalFile()
|
||||
os.Remove(walFile)
|
||||
|
||||
// set crashing WAL
|
||||
csWal, err := cs.OpenWAL(walFile)
|
||||
require.NoError(t, err)
|
||||
crashingWal.next = csWal
|
||||
// reset the message counter
|
||||
crashingWal.msgIndex = 1
|
||||
cs.wal = crashingWal
|
||||
|
||||
// start consensus state
|
||||
_, err = cs.Start()
|
||||
require.NoError(t, err)
|
||||
|
||||
i++
|
||||
|
||||
select {
|
||||
case err := <-walPaniced:
|
||||
t.Logf("WAL paniced: %v", err)
|
||||
|
||||
// make sure we can make blocks after a crash
|
||||
startNewConsensusStateAndWaitForBlock(t, cs.Height, blockDB, stateDB)
|
||||
|
||||
// stop consensus state and transactions sender (initFn)
|
||||
cs.Stop()
|
||||
cancel()
|
||||
|
||||
// if we reached the required height, exit
|
||||
if _, ok := err.(ReachedHeightToStopError); ok {
|
||||
break LOOP
|
||||
}
|
||||
case <-time.After(10 * time.Second):
|
||||
t.Fatal("WAL did not panic for 10 seconds (check the log)")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestWALCrashBeforeWritePrecommit(t *testing.T) {
|
||||
for _, thisCase := range testCases {
|
||||
testReplayCrashBeforeWriteVote(t, thisCase, thisCase.precommitLine, types.EventStringPolka())
|
||||
// crashingWAL is a WAL which crashes or rather simulates a crash during Save
|
||||
// (before and after). It remembers a message for which we last panicked
|
||||
// (lastPanicedForMsgIndex), so we don't panic for it in subsequent iterations.
|
||||
type crashingWAL struct {
|
||||
next WAL
|
||||
panicCh chan error
|
||||
heightToStop uint64
|
||||
|
||||
msgIndex int // current message index
|
||||
lastPanicedForMsgIndex int // last message for which we panicked
|
||||
}
|
||||
|
||||
// WALWriteError indicates a WAL crash.
|
||||
type WALWriteError struct {
|
||||
msg string
|
||||
}
|
||||
|
||||
func (e WALWriteError) Error() string {
|
||||
return e.msg
|
||||
}
|
||||
|
||||
// ReachedHeightToStopError indicates we've reached the required consensus
|
||||
// height and may exit.
|
||||
type ReachedHeightToStopError struct {
|
||||
height uint64
|
||||
}
|
||||
|
||||
func (e ReachedHeightToStopError) Error() string {
|
||||
return fmt.Sprintf("reached height to stop %d", e.height)
|
||||
}
|
||||
|
||||
// Save simulate WAL's crashing by sending an error to the panicCh and then
|
||||
// exiting the cs.receiveRoutine.
|
||||
func (w *crashingWAL) Save(m WALMessage) {
|
||||
if endMsg, ok := m.(EndHeightMessage); ok {
|
||||
if endMsg.Height == w.heightToStop {
|
||||
w.panicCh <- ReachedHeightToStopError{endMsg.Height}
|
||||
runtime.Goexit()
|
||||
} else {
|
||||
w.next.Save(m)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if w.msgIndex > w.lastPanicedForMsgIndex {
|
||||
w.lastPanicedForMsgIndex = w.msgIndex
|
||||
_, file, line, _ := runtime.Caller(1)
|
||||
w.panicCh <- WALWriteError{fmt.Sprintf("failed to write %T to WAL (fileline: %s:%d)", m, file, line)}
|
||||
runtime.Goexit()
|
||||
} else {
|
||||
w.msgIndex++
|
||||
w.next.Save(m)
|
||||
}
|
||||
}
|
||||
|
||||
func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum int, eventString string) {
|
||||
// setup replay test where last message is a vote
|
||||
cs, newBlockCh, voteMsg, walFile := setupReplayTest(t, thisCase, lineNum, false)
|
||||
types.AddListenerForEvent(cs.evsw, "tester", eventString, func(data types.TMEventData) {
|
||||
msg := readTimedWALMessage(t, voteMsg)
|
||||
vote := msg.Msg.(msgInfo).Msg.(*VoteMessage)
|
||||
// Set LastSig
|
||||
toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote)
|
||||
toPV(cs.privValidator).LastSignature = vote.Vote.Signature
|
||||
})
|
||||
runReplayTest(t, cs, walFile, newBlockCh, thisCase, lineNum)
|
||||
func (w *crashingWAL) Group() *auto.Group { return w.next.Group() }
|
||||
func (w *crashingWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) {
|
||||
return w.next.SearchForEndHeight(height)
|
||||
}
|
||||
|
||||
func (w *crashingWAL) Start() (bool, error) { return w.next.Start() }
|
||||
func (w *crashingWAL) Stop() bool { return w.next.Stop() }
|
||||
func (w *crashingWAL) Wait() { w.next.Wait() }
|
||||
|
||||
//------------------------------------------------------------------------------------------
|
||||
// Handshake Tests
|
||||
|
||||
@ -320,6 +305,21 @@ func TestHandshakeReplayNone(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func writeWAL(walMsgs []byte) string {
|
||||
walFile, err := ioutil.TempFile("", "wal")
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to create temp WAL file: %v", err))
|
||||
}
|
||||
_, err = walFile.Write(walMsgs)
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to write to temp WAL file: %v", err))
|
||||
}
|
||||
if err := walFile.Close(); err != nil {
|
||||
panic(fmt.Errorf("failed to close temp WAL file: %v", err))
|
||||
}
|
||||
return walFile.Name()
|
||||
}
|
||||
|
||||
// Make some blocks. Start a fresh app and apply nBlocks blocks. Then restart the app and sync it up with the remaining blocks
|
||||
func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
|
||||
config := ResetConfig("proxy_test_")
|
||||
@ -397,7 +397,7 @@ func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
|
||||
|
||||
func applyBlock(st *sm.State, blk *types.Block, proxyApp proxy.AppConns) {
|
||||
testPartSize := st.Params.BlockPartSizeBytes
|
||||
err := st.ApplyBlock(nil, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool)
|
||||
err := st.ApplyBlock(types.NopEventBus{}, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
@ -477,7 +477,7 @@ func buildTMStateFromChain(config *cfg.Config, state *sm.State, chain []*types.B
|
||||
//--------------------------
|
||||
// utils for making blocks
|
||||
|
||||
func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) {
|
||||
func makeBlockchainFromWAL(wal WAL) ([]*types.Block, []*types.Commit, error) {
|
||||
// Search for height marker
|
||||
gr, found, err := wal.SearchForEndHeight(0)
|
||||
if err != nil {
|
||||
|
@ -4,7 +4,6 @@ import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"runtime/debug"
|
||||
"sync"
|
||||
@ -91,13 +90,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 +124,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 +148,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 +212,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 +265,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 +278,17 @@ 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)
|
||||
if err != nil {
|
||||
cs.Logger.Error("Error ensuring ConsensusState wal dir", "err", err.Error())
|
||||
return err
|
||||
}
|
||||
|
||||
cs.mtx.Lock()
|
||||
defer cs.mtx.Unlock()
|
||||
func (cs *ConsensusState) OpenWAL(walFile string) (WAL, error) {
|
||||
wal, err := NewWAL(walFile, cs.config.WalLight)
|
||||
if err != nil {
|
||||
return err
|
||||
cs.Logger.Error("Failed to open WAL for consensus state", "wal", walFile, "err", 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 +477,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 +533,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 +602,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 +668,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 +721,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 +879,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 +981,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 +998,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 +1010,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 +1025,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 +1041,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 +1185,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)
|
||||
err := cmn.Kill()
|
||||
@ -1224,9 +1216,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
|
||||
|
||||
@ -1361,7 +1356,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() {
|
||||
@ -1379,7 +1374,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:
|
||||
@ -1397,7 +1392,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() {
|
||||
|
@ -2,6 +2,7 @@ package consensus
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
@ -9,6 +10,7 @@ import (
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
tmpubsub "github.com/tendermint/tmlibs/pubsub"
|
||||
)
|
||||
|
||||
func init() {
|
||||
@ -56,8 +58,8 @@ func TestProposerSelection0(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4)
|
||||
height, round := cs1.Height, cs1.Round
|
||||
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
|
||||
startTestRound(cs1, height, round)
|
||||
|
||||
@ -89,7 +91,7 @@ func TestProposerSelection0(t *testing.T) {
|
||||
func TestProposerSelection2(t *testing.T) {
|
||||
cs1, vss := randConsensusState(4) // test needs more work for more than 3 validators
|
||||
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
|
||||
// this time we jump in at round 2
|
||||
incrementRound(vss[1:]...)
|
||||
@ -121,7 +123,7 @@ func TestEnterProposeNoPrivValidator(t *testing.T) {
|
||||
height, round := cs.Height, cs.Round
|
||||
|
||||
// Listen for propose timeout event
|
||||
timeoutCh := subscribeToEvent(cs.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
|
||||
|
||||
startTestRound(cs, height, round)
|
||||
|
||||
@ -146,8 +148,8 @@ func TestEnterProposeYesPrivValidator(t *testing.T) {
|
||||
|
||||
// Listen for propose timeout event
|
||||
|
||||
timeoutCh := subscribeToEvent(cs.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
proposalCh := subscribeToEvent(cs.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
|
||||
proposalCh := subscribe(cs.eventBus, types.EventQueryCompleteProposal)
|
||||
|
||||
cs.enterNewRound(height, round)
|
||||
cs.startRoutines(3)
|
||||
@ -183,8 +185,8 @@ func TestBadProposal(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
|
||||
|
||||
propBlock, _ := cs1.createProposalBlock() //changeProposer(t, cs1, vs2)
|
||||
|
||||
@ -238,9 +240,9 @@ func TestFullRound1(t *testing.T) {
|
||||
cs, vss := randConsensusState(1)
|
||||
height, round := cs.Height, cs.Round
|
||||
|
||||
voteCh := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 0)
|
||||
propCh := subscribeToEvent(cs.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
newRoundCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
voteCh := subscribe(cs.eventBus, types.EventQueryVote)
|
||||
propCh := subscribe(cs.eventBus, types.EventQueryCompleteProposal)
|
||||
newRoundCh := subscribe(cs.eventBus, types.EventQueryNewRound)
|
||||
|
||||
startTestRound(cs, height, round)
|
||||
|
||||
@ -251,8 +253,6 @@ func TestFullRound1(t *testing.T) {
|
||||
propBlockHash := re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*cstypes.RoundState).ProposalBlock.Hash()
|
||||
|
||||
<-voteCh // wait for prevote
|
||||
// NOTE: voteChan cap of 0 ensures we can complete this
|
||||
// before consensus can move to the next height (and cause a race condition)
|
||||
validatePrevote(t, cs, round, vss[0], propBlockHash)
|
||||
|
||||
<-voteCh // wait for precommit
|
||||
@ -268,7 +268,7 @@ func TestFullRoundNil(t *testing.T) {
|
||||
cs, vss := randConsensusState(1)
|
||||
height, round := cs.Height, cs.Round
|
||||
|
||||
voteCh := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 1)
|
||||
voteCh := subscribe(cs.eventBus, types.EventQueryVote)
|
||||
|
||||
cs.enterPrevote(height, round)
|
||||
cs.startRoutines(4)
|
||||
@ -287,8 +287,8 @@ func TestFullRound2(t *testing.T) {
|
||||
vs2 := vss[1]
|
||||
height, round := cs1.Height, cs1.Round
|
||||
|
||||
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
||||
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
|
||||
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlock)
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
startTestRound(cs1, height, round)
|
||||
@ -330,11 +330,11 @@ func TestLockNoPOL(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
|
||||
/*
|
||||
Round1 (cs1, B) // B B // B B2
|
||||
@ -496,12 +496,12 @@ func TestLockPOLRelock(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlockHeader(), 1)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader)
|
||||
|
||||
// everything done from perspective of cs1
|
||||
|
||||
@ -609,11 +609,11 @@ func TestLockPOLUnlock(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
unlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringUnlock(), 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// everything done from perspective of cs1
|
||||
@ -704,10 +704,10 @@ func TestLockPOLSafety1(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
@ -802,7 +802,7 @@ func TestLockPOLSafety1(t *testing.T) {
|
||||
// we should prevote what we're locked on
|
||||
validatePrevote(t, cs1, 2, vss[0], propBlockHash)
|
||||
|
||||
newStepCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRoundStep(), 1)
|
||||
newStepCh := subscribe(cs1.eventBus, types.EventQueryNewRoundStep)
|
||||
|
||||
// add prevotes from the earlier round
|
||||
addVotes(cs1, prevotes...)
|
||||
@ -825,11 +825,11 @@ func TestLockPOLSafety2(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
unlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringUnlock(), 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// the block for R0: gets polkad but we miss it
|
||||
@ -919,9 +919,9 @@ func TestSlashingPrevotes(t *testing.T) {
|
||||
vs2 := vss[1]
|
||||
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringCompleteProposal() , 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringTimeoutWait() , 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringNewRound() , 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
@ -954,9 +954,9 @@ func TestSlashingPrecommits(t *testing.T) {
|
||||
vs2 := vss[1]
|
||||
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringCompleteProposal() , 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringTimeoutWait() , 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw,"tester",types.EventStringNewRound() , 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
@ -1000,10 +1000,10 @@ func TestHalt1(t *testing.T) {
|
||||
|
||||
partSize := cs1.state.Params.BlockPartSizeBytes
|
||||
|
||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
|
||||
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlock(), 1)
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlock)
|
||||
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
|
||||
|
||||
// start round and wait for propose and prevote
|
||||
@ -1057,3 +1057,20 @@ func TestHalt1(t *testing.T) {
|
||||
panic("expected height to increment")
|
||||
}
|
||||
}
|
||||
|
||||
// subscribe subscribes test client to the given query and returns a channel with cap = 1.
|
||||
func subscribe(eventBus *types.EventBus, q tmpubsub.Query) <-chan interface{} {
|
||||
out := make(chan interface{}, 1)
|
||||
err := eventBus.Subscribe(context.Background(), testSubscriber, q, out)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to subscribe %s to %v", testSubscriber, q))
|
||||
}
|
||||
return out
|
||||
}
|
||||
|
||||
// discardFromChan reads n values from the channel.
|
||||
func discardFromChan(ch <-chan interface{}, n int) {
|
||||
for i := 0; i < n; i++ {
|
||||
<-ch
|
||||
}
|
||||
}
|
||||
|
@ -52,19 +52,19 @@ function reset(){
|
||||
|
||||
reset
|
||||
|
||||
function empty_block(){
|
||||
echo "==> Starting tendermint..."
|
||||
tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
sleep 5
|
||||
echo "==> Killing tendermint..."
|
||||
killall tendermint
|
||||
# function empty_block(){
|
||||
# echo "==> Starting tendermint..."
|
||||
# tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
# sleep 5
|
||||
# echo "==> Killing tendermint..."
|
||||
# killall tendermint
|
||||
|
||||
echo "==> Copying WAL log..."
|
||||
$cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_empty_block.cswal
|
||||
mv consensus/test_data/new_empty_block.cswal consensus/test_data/empty_block.cswal
|
||||
# echo "==> Copying WAL log..."
|
||||
# $cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_empty_block.cswal
|
||||
# mv consensus/test_data/new_empty_block.cswal consensus/test_data/empty_block.cswal
|
||||
|
||||
reset
|
||||
}
|
||||
# reset
|
||||
# }
|
||||
|
||||
function many_blocks(){
|
||||
bash scripts/txs/random.sh 1000 36657 &> /dev/null &
|
||||
@ -84,63 +84,63 @@ function many_blocks(){
|
||||
}
|
||||
|
||||
|
||||
function small_block1(){
|
||||
bash scripts/txs/random.sh 1000 36657 &> /dev/null &
|
||||
PID=$!
|
||||
echo "==> Starting tendermint..."
|
||||
tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
sleep 10
|
||||
echo "==> Killing tendermint..."
|
||||
kill -9 $PID
|
||||
killall tendermint
|
||||
# function small_block1(){
|
||||
# bash scripts/txs/random.sh 1000 36657 &> /dev/null &
|
||||
# PID=$!
|
||||
# echo "==> Starting tendermint..."
|
||||
# tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
# sleep 10
|
||||
# echo "==> Killing tendermint..."
|
||||
# kill -9 $PID
|
||||
# killall tendermint
|
||||
|
||||
echo "==> Copying WAL log..."
|
||||
$cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_small_block1.cswal
|
||||
mv consensus/test_data/new_small_block1.cswal consensus/test_data/small_block1.cswal
|
||||
# echo "==> Copying WAL log..."
|
||||
# $cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_small_block1.cswal
|
||||
# mv consensus/test_data/new_small_block1.cswal consensus/test_data/small_block1.cswal
|
||||
|
||||
reset
|
||||
}
|
||||
# reset
|
||||
# }
|
||||
|
||||
|
||||
# block part size = 512
|
||||
function small_block2(){
|
||||
cat "$TMHOME/genesis.json" | jq '. + {consensus_params: {block_size_params: {max_bytes: 22020096}, block_gossip_params: {block_part_size_bytes: 512}}}' > "$TMHOME/new_genesis.json"
|
||||
mv "$TMHOME/new_genesis.json" "$TMHOME/genesis.json"
|
||||
bash scripts/txs/random.sh 1000 36657 &> /dev/null &
|
||||
PID=$!
|
||||
echo "==> Starting tendermint..."
|
||||
tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
sleep 5
|
||||
echo "==> Killing tendermint..."
|
||||
kill -9 $PID
|
||||
killall tendermint
|
||||
# # block part size = 512
|
||||
# function small_block2(){
|
||||
# cat "$TMHOME/genesis.json" | jq '. + {consensus_params: {block_size_params: {max_bytes: 22020096}, block_gossip_params: {block_part_size_bytes: 512}}}' > "$TMHOME/new_genesis.json"
|
||||
# mv "$TMHOME/new_genesis.json" "$TMHOME/genesis.json"
|
||||
# bash scripts/txs/random.sh 1000 36657 &> /dev/null &
|
||||
# PID=$!
|
||||
# echo "==> Starting tendermint..."
|
||||
# tendermint node --proxy_app=persistent_dummy &> /dev/null &
|
||||
# sleep 5
|
||||
# echo "==> Killing tendermint..."
|
||||
# kill -9 $PID
|
||||
# killall tendermint
|
||||
|
||||
echo "==> Copying WAL log..."
|
||||
$cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_small_block2.cswal
|
||||
mv consensus/test_data/new_small_block2.cswal consensus/test_data/small_block2.cswal
|
||||
# echo "==> Copying WAL log..."
|
||||
# $cutWALUntil "$TMHOME/data/cs.wal/wal" 1 consensus/test_data/new_small_block2.cswal
|
||||
# mv consensus/test_data/new_small_block2.cswal consensus/test_data/small_block2.cswal
|
||||
|
||||
reset
|
||||
}
|
||||
# reset
|
||||
# }
|
||||
|
||||
|
||||
|
||||
case "$1" in
|
||||
"small_block1")
|
||||
small_block1
|
||||
;;
|
||||
"small_block2")
|
||||
small_block2
|
||||
;;
|
||||
"empty_block")
|
||||
empty_block
|
||||
;;
|
||||
# "small_block1")
|
||||
# small_block1
|
||||
# ;;
|
||||
# "small_block2")
|
||||
# small_block2
|
||||
# ;;
|
||||
# "empty_block")
|
||||
# empty_block
|
||||
# ;;
|
||||
"many_blocks")
|
||||
many_blocks
|
||||
;;
|
||||
*)
|
||||
small_block1
|
||||
small_block2
|
||||
empty_block
|
||||
# small_block1
|
||||
# small_block2
|
||||
# empty_block
|
||||
many_blocks
|
||||
esac
|
||||
|
||||
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
@ -76,11 +76,14 @@ type RoundState struct {
|
||||
|
||||
// RoundStateEvent returns the H/R/S of the RoundState as an event.
|
||||
func (rs *RoundState) RoundStateEvent() types.EventDataRoundState {
|
||||
// XXX: copy the RoundState
|
||||
// if we want to avoid this, we may need synchronous events after all
|
||||
rs_ := *rs
|
||||
edrs := types.EventDataRoundState{
|
||||
Height: rs.Height,
|
||||
Round: rs.Round,
|
||||
Step: rs.Step.String(),
|
||||
RoundState: rs,
|
||||
RoundState: &rs_,
|
||||
}
|
||||
return edrs
|
||||
}
|
||||
|
@ -6,8 +6,11 @@ import (
|
||||
"fmt"
|
||||
"hash/crc32"
|
||||
"io"
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
wire "github.com/tendermint/go-wire"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
auto "github.com/tendermint/tmlibs/autofile"
|
||||
@ -45,11 +48,22 @@ var _ = wire.RegisterInterface(
|
||||
//--------------------------------------------------------
|
||||
// Simple write-ahead logger
|
||||
|
||||
// WAL is an interface for any write-ahead logger.
|
||||
type WAL interface {
|
||||
Save(WALMessage)
|
||||
Group() *auto.Group
|
||||
SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error)
|
||||
|
||||
Start() (bool, error)
|
||||
Stop() bool
|
||||
Wait()
|
||||
}
|
||||
|
||||
// Write ahead logger writes msgs to disk before they are processed.
|
||||
// Can be used for crash-recovery and deterministic replay
|
||||
// TODO: currently the wal is overwritten during replay catchup
|
||||
// give it a mode so it's either reading or appending - must read to end to start appending again
|
||||
type WAL struct {
|
||||
type baseWAL struct {
|
||||
cmn.BaseService
|
||||
|
||||
group *auto.Group
|
||||
@ -58,21 +72,30 @@ type WAL struct {
|
||||
enc *WALEncoder
|
||||
}
|
||||
|
||||
func NewWAL(walFile string, light bool) (*WAL, error) {
|
||||
func NewWAL(walFile string, light bool) (*baseWAL, error) {
|
||||
err := cmn.EnsureDir(filepath.Dir(walFile), 0700)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to ensure WAL directory is in place")
|
||||
}
|
||||
|
||||
group, err := auto.OpenGroup(walFile)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
wal := &WAL{
|
||||
wal := &baseWAL{
|
||||
group: group,
|
||||
light: light,
|
||||
enc: NewWALEncoder(group),
|
||||
}
|
||||
wal.BaseService = *cmn.NewBaseService(nil, "WAL", wal)
|
||||
wal.BaseService = *cmn.NewBaseService(nil, "baseWAL", wal)
|
||||
return wal, nil
|
||||
}
|
||||
|
||||
func (wal *WAL) OnStart() error {
|
||||
func (wal *baseWAL) Group() *auto.Group {
|
||||
return wal.group
|
||||
}
|
||||
|
||||
func (wal *baseWAL) OnStart() error {
|
||||
size, err := wal.group.Head.Size()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -83,13 +106,13 @@ func (wal *WAL) OnStart() error {
|
||||
return err
|
||||
}
|
||||
|
||||
func (wal *WAL) OnStop() {
|
||||
func (wal *baseWAL) OnStop() {
|
||||
wal.BaseService.OnStop()
|
||||
wal.group.Stop()
|
||||
}
|
||||
|
||||
// called in newStep and for each pass in receiveRoutine
|
||||
func (wal *WAL) Save(msg WALMessage) {
|
||||
func (wal *baseWAL) Save(msg WALMessage) {
|
||||
if wal == nil {
|
||||
return
|
||||
}
|
||||
@ -119,7 +142,7 @@ func (wal *WAL) Save(msg WALMessage) {
|
||||
// Group reader will be nil if found equals false.
|
||||
//
|
||||
// CONTRACT: caller must close group reader.
|
||||
func (wal *WAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) {
|
||||
func (wal *baseWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) {
|
||||
var msg *TimedWALMessage
|
||||
|
||||
// NOTE: starting from the last file in the group because we're usually
|
||||
@ -277,3 +300,14 @@ func readSeparator(r io.Reader) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type nilWAL struct{}
|
||||
|
||||
func (nilWAL) Save(m WALMessage) {}
|
||||
func (nilWAL) Group() *auto.Group { return nil }
|
||||
func (nilWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) {
|
||||
return nil, false, nil
|
||||
}
|
||||
func (nilWAL) Start() (bool, error) { return true, nil }
|
||||
func (nilWAL) Stop() bool { return true }
|
||||
func (nilWAL) Wait() {}
|
||||
|
Reference in New Issue
Block a user