tendermint/consensus/common_test.go

482 lines
14 KiB
Go
Raw Normal View History

2015-12-01 20:12:01 -08:00
package consensus
import (
"bytes"
"fmt"
2016-11-23 15:20:46 -08:00
"io/ioutil"
2017-01-12 11:44:42 -08:00
"os"
"path"
2015-12-01 20:12:01 -08:00
"sort"
"sync"
2015-12-01 20:12:01 -08:00
"testing"
"time"
abcicli "github.com/tendermint/abci/client"
abci "github.com/tendermint/abci/types"
2015-12-01 20:12:01 -08:00
bc "github.com/tendermint/tendermint/blockchain"
2017-05-01 21:43:49 -07:00
cfg "github.com/tendermint/tendermint/config"
2015-12-01 20:12:01 -08:00
mempl "github.com/tendermint/tendermint/mempool"
2017-04-08 19:04:06 -07:00
"github.com/tendermint/tendermint/p2p"
2015-12-01 20:12:01 -08:00
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
2017-04-08 19:04:06 -07:00
. "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
2017-05-02 00:53:32 -07:00
"github.com/tendermint/tmlibs/log"
2015-12-01 20:12:01 -08:00
2017-01-12 12:53:32 -08:00
"github.com/tendermint/abci/example/counter"
"github.com/tendermint/abci/example/dummy"
"github.com/go-kit/kit/log/term"
2015-12-01 20:12:01 -08:00
)
// genesis, chain_id, priv_val
2017-05-04 19:33:08 -07:00
var config *cfg.Config // NOTE: must be reset for each _test.go file
2017-08-08 13:35:25 -07:00
var ensureTimeout = time.Second * 2
2017-01-12 11:44:42 -08:00
func ensureDir(dir string, mode os.FileMode) {
if err := EnsureDir(dir, mode); err != nil {
panic(err)
}
}
2017-05-04 19:33:08 -07:00
func ResetConfig(name string) *cfg.Config {
return cfg.ResetTestRoot(name)
2017-05-01 21:43:49 -07:00
}
//-------------------------------------------------------------------------------
// validator stub (a dummy consensus peer we control)
2015-12-01 20:12:01 -08:00
type validatorStub struct {
Index int // Validator index. NOTE: we don't assume validator set changes.
2015-12-01 20:12:01 -08:00
Height int
Round int
2017-09-18 20:16:14 -07:00
types.PrivValidator
2015-12-01 20:12:01 -08:00
}
2016-11-23 15:20:46 -08:00
var testMinPower = 10
2017-09-18 20:16:14 -07:00
func NewValidatorStub(privValidator types.PrivValidator, valIndex int) *validatorStub {
2015-12-01 20:12:01 -08:00
return &validatorStub{
Index: valIndex,
2015-12-01 20:12:01 -08:00
PrivValidator: privValidator,
}
}
func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
vote := &types.Vote{
ValidatorIndex: vs.Index,
2017-09-18 20:16:14 -07:00
ValidatorAddress: vs.PrivValidator.Address(),
2015-12-01 20:12:01 -08:00
Height: vs.Height,
Round: vs.Round,
Type: voteType,
2016-08-16 14:59:19 -07:00
BlockID: types.BlockID{hash, header},
2015-12-01 20:12:01 -08:00
}
2017-05-01 21:43:49 -07:00
err := vs.PrivValidator.SignVote(config.ChainID, vote)
2015-12-01 20:12:01 -08:00
return vote, err
}
// Sign vote for type/hash/header
2015-12-01 20:12:01 -08:00
func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSetHeader) *types.Vote {
v, err := vs.signVote(voteType, hash, header)
if err != nil {
panic(fmt.Errorf("failed to sign vote: %v", err))
}
return v
}
func signVotes(voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
votes := make([]*types.Vote, len(vss))
for i, vs := range vss {
votes[i] = signVote(vs, voteType, hash, header)
}
return votes
}
func incrementHeight(vss ...*validatorStub) {
for _, vs := range vss {
vs.Height += 1
}
}
func incrementRound(vss ...*validatorStub) {
for _, vs := range vss {
vs.Round += 1
}
}
//-------------------------------------------------------------------------------
// Functions for transitioning the consensus state
func startTestRound(cs *ConsensusState, height, round int) {
cs.enterNewRound(height, round)
cs.startRoutines(0)
}
// Create proposal block from cs1 but sign it with vs
func decideProposal(cs1 *ConsensusState, vs *validatorStub, height, round int) (proposal *types.Proposal, block *types.Block) {
2015-12-01 20:12:01 -08:00
block, blockParts := cs1.createProposalBlock()
if block == nil { // on error
panic("error creating proposal block")
}
// Make proposal
polRound, polBlockID := cs1.Votes.POLInfo()
proposal = types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
2017-05-01 21:43:49 -07:00
if err := vs.SignProposal(config.ChainID, proposal); err != nil {
2015-12-01 20:12:01 -08:00
panic(err)
}
return
}
func addVotes(to *ConsensusState, votes ...*types.Vote) {
for _, vote := range votes {
to.peerMsgQueue <- msgInfo{Msg: &VoteMessage{vote}}
2015-12-01 20:12:01 -08:00
}
}
func signAddVotes(to *ConsensusState, voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) {
votes := signVotes(voteType, hash, header, vss...)
addVotes(to, votes...)
2015-12-01 20:12:01 -08:00
}
func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *validatorStub, blockHash []byte) {
prevotes := cs.Votes.Prevotes(round)
var vote *types.Vote
2017-09-18 20:16:14 -07:00
if vote = prevotes.GetByAddress(privVal.Address()); vote == nil {
2015-12-01 20:12:01 -08:00
panic("Failed to find prevote from validator")
}
if blockHash == nil {
2016-08-16 14:59:19 -07:00
if vote.BlockID.Hash != nil {
panic(fmt.Sprintf("Expected prevote to be for nil, got %X", vote.BlockID.Hash))
2015-12-01 20:12:01 -08:00
}
} else {
2016-08-16 14:59:19 -07:00
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected prevote to be for %X, got %X", blockHash, vote.BlockID.Hash))
2015-12-01 20:12:01 -08:00
}
}
}
2015-12-13 11:56:05 -08:00
func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorStub, blockHash []byte) {
votes := cs.LastCommit
var vote *types.Vote
2017-09-18 20:16:14 -07:00
if vote = votes.GetByAddress(privVal.Address()); vote == nil {
2015-12-13 11:56:05 -08:00
panic("Failed to find precommit from validator")
2015-12-01 20:12:01 -08:00
}
2016-08-16 14:59:19 -07:00
if !bytes.Equal(vote.BlockID.Hash, blockHash) {
panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockID.Hash))
2015-12-01 20:12:01 -08:00
}
}
func validatePrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound int, privVal *validatorStub, votedBlockHash, lockedBlockHash []byte) {
precommits := cs.Votes.Precommits(thisRound)
var vote *types.Vote
2017-09-18 20:16:14 -07:00
if vote = precommits.GetByAddress(privVal.Address()); vote == nil {
2015-12-01 20:12:01 -08:00
panic("Failed to find precommit from validator")
}
if votedBlockHash == nil {
2016-08-16 14:59:19 -07:00
if vote.BlockID.Hash != nil {
2015-12-01 20:12:01 -08:00
panic("Expected precommit to be for nil")
}
} else {
2016-08-16 14:59:19 -07:00
if !bytes.Equal(vote.BlockID.Hash, votedBlockHash) {
2015-12-01 20:12:01 -08:00
panic("Expected precommit to be for proposal block")
}
}
if lockedBlockHash == nil {
if cs.LockedRound != lockRound || cs.LockedBlock != nil {
panic(fmt.Sprintf("Expected to be locked on nil at round %d. Got locked at round %d with block %v", lockRound, cs.LockedRound, cs.LockedBlock))
}
} else {
if cs.LockedRound != lockRound || !bytes.Equal(cs.LockedBlock.Hash(), lockedBlockHash) {
panic(fmt.Sprintf("Expected block to be locked on round %d, got %d. Got locked block %X, expected %X", lockRound, cs.LockedRound, cs.LockedBlock.Hash(), lockedBlockHash))
}
}
}
func validatePrevoteAndPrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound int, privVal *validatorStub, votedBlockHash, lockedBlockHash []byte) {
// verify the prevote
validatePrevote(t, cs, thisRound, privVal, votedBlockHash)
// verify precommit
cs.mtx.Lock()
validatePrecommit(t, cs, thisRound, lockRound, privVal, votedBlockHash, lockedBlockHash)
cs.mtx.Unlock()
}
// genesis
func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
voteCh0 := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 1)
voteCh := make(chan interface{})
go func() {
for {
v := <-voteCh0
2017-04-28 14:57:06 -07:00
vote := v.(types.TMEventData).Unwrap().(types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
voteCh <- v
}
}
}()
return voteCh
2016-01-18 12:57:57 -08:00
}
2015-12-01 20:12:01 -08:00
//-------------------------------------------------------------------------------
// consensus states
2017-09-18 20:16:14 -07:00
func newConsensusState(state *sm.State, pv types.PrivValidator, app abci.Application) *ConsensusState {
return newConsensusStateWithConfig(config, state, pv, app)
2016-10-11 09:51:48 -07:00
}
2017-09-18 20:16:14 -07:00
func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv types.PrivValidator, app abci.Application) *ConsensusState {
2015-12-01 20:12:01 -08:00
// Get BlockStore
blockDB := dbm.NewMemDB()
blockStore := bc.NewBlockStore(blockDB)
// one for mempool, one for consensus
mtx := new(sync.Mutex)
2017-01-12 12:53:32 -08:00
proxyAppConnMem := abcicli.NewLocalClient(mtx, app)
proxyAppConnCon := abcicli.NewLocalClient(mtx, app)
2015-12-01 20:12:01 -08:00
// Make Mempool
mempool := mempl.NewMempool(thisConfig.Mempool, proxyAppConnMem, 0)
2017-05-02 00:53:32 -07:00
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
if thisConfig.Consensus.WaitForTxs() {
mempool.EnableTxsAvailable()
}
2015-12-01 20:12:01 -08:00
// Make ConsensusReactor
2017-05-01 21:43:49 -07:00
cs := NewConsensusState(thisConfig.Consensus, state, proxyAppConnCon, blockStore, mempool)
2017-05-02 00:53:32 -07:00
cs.SetLogger(log.TestingLogger())
2016-01-18 12:57:57 -08:00
cs.SetPrivValidator(pv)
2015-12-01 20:12:01 -08:00
2016-10-09 23:58:13 -07:00
evsw := types.NewEventSwitch()
2017-05-02 00:53:32 -07:00
evsw.SetLogger(log.TestingLogger().With("module", "events"))
cs.SetEventSwitch(evsw)
evsw.Start()
2016-01-18 12:57:57 -08:00
return cs
}
2015-12-01 20:12:01 -08:00
2017-09-18 20:16:14 -07:00
func loadPrivValidator(config *cfg.Config) *types.PrivValidatorFS {
2017-05-04 19:33:08 -07:00
privValidatorFile := config.PrivValidatorFile()
2017-01-12 11:44:42 -08:00
ensureDir(path.Dir(privValidatorFile), 0700)
2017-09-18 20:16:14 -07:00
privValidator := types.LoadOrGenPrivValidatorFS(privValidatorFile)
privValidator.Reset()
return privValidator
}
func fixedConsensusStateDummy() *ConsensusState {
stateDB := dbm.NewMemDB()
2017-05-04 19:33:08 -07:00
state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
2017-05-02 00:53:32 -07:00
state.SetLogger(log.TestingLogger().With("module", "state"))
privValidator := loadPrivValidator(config)
cs := newConsensusState(state, privValidator, dummy.NewDummyApplication())
2017-05-02 00:53:32 -07:00
cs.SetLogger(log.TestingLogger())
return cs
2016-09-13 19:25:11 -07:00
}
2016-01-18 12:57:57 -08:00
func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
// Get State
state, privVals := randGenesisState(nValidators, false, 10)
vss := make([]*validatorStub, nValidators)
2015-12-01 20:12:01 -08:00
cs := newConsensusState(state, privVals[0], counter.NewCounterApplication(true))
2017-05-02 00:53:32 -07:00
cs.SetLogger(log.TestingLogger())
2015-12-08 13:00:59 -08:00
2015-12-01 20:12:01 -08:00
for i := 0; i < nValidators; i++ {
vss[i] = NewValidatorStub(privVals[i], i)
2015-12-01 20:12:01 -08:00
}
// since cs1 starts at 1
incrementHeight(vss[1:]...)
return cs, vss
}
//-------------------------------------------------------------------------------
func ensureNoNewStep(stepCh chan interface{}) {
2017-08-08 13:35:25 -07:00
timer := time.NewTimer(ensureTimeout)
select {
2017-07-13 12:03:19 -07:00
case <-timer.C:
break
case <-stepCh:
2017-07-13 12:03:19 -07:00
panic("We should be stuck waiting, not moving to the next step")
}
}
func ensureNewStep(stepCh chan interface{}) {
2017-08-08 13:35:25 -07:00
timer := time.NewTimer(ensureTimeout)
2017-07-13 12:03:19 -07:00
select {
case <-timer.C:
panic("We shouldnt be stuck waiting")
case <-stepCh:
break
}
}
//-------------------------------------------------------------------------------
// consensus nets
// consensusLogger is a TestingLogger which uses a different
// color for each validator ("validator" key must exist).
func consensusLogger() log.Logger {
return log.TestingLoggerWithColorFn(func(keyvals ...interface{}) term.FgBgColor {
for i := 0; i < len(keyvals)-1; i += 2 {
if keyvals[i] == "validator" {
return term.FgBgColor{Fg: term.Color(uint8(keyvals[i+1].(int) + 1))}
}
}
return term.FgBgColor{}
})
}
2017-08-09 22:09:04 -07:00
func randConsensusNet(nValidators int, testName string, tickerFunc func() TimeoutTicker, appFunc func() abci.Application, configOpts ...func(*cfg.Config)) []*ConsensusState {
2016-06-25 21:40:53 -07:00
genDoc, privVals := randGenesisDoc(nValidators, false, 10)
css := make([]*ConsensusState, nValidators)
logger := consensusLogger()
2016-06-25 21:40:53 -07:00
for i := 0; i < nValidators; i++ {
db := dbm.NewMemDB() // each state needs its own db
state := sm.MakeGenesisState(db, genDoc)
state.SetLogger(logger.With("module", "state", "validator", i))
2016-06-25 21:40:53 -07:00
state.Save()
2017-05-01 21:43:49 -07:00
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
2017-08-09 22:09:04 -07:00
for _, opt := range configOpts {
opt(thisConfig)
}
2017-05-04 19:33:08 -07:00
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
css[i] = newConsensusStateWithConfig(thisConfig, state, privVals[i], appFunc())
css[i].SetLogger(logger.With("validator", i))
css[i].SetTimeoutTicker(tickerFunc())
2016-06-25 21:40:53 -07:00
}
return css
}
2016-11-23 15:20:46 -08:00
// nPeers = nValidators + nNotValidator
2017-01-12 12:53:32 -08:00
func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerFunc func() TimeoutTicker, appFunc func() abci.Application) []*ConsensusState {
2016-11-23 15:20:46 -08:00
genDoc, privVals := randGenesisDoc(nValidators, false, int64(testMinPower))
css := make([]*ConsensusState, nPeers)
for i := 0; i < nPeers; i++ {
db := dbm.NewMemDB() // each state needs its own db
state := sm.MakeGenesisState(db, genDoc)
2017-05-02 00:53:32 -07:00
state.SetLogger(log.TestingLogger().With("module", "state"))
2016-11-23 15:20:46 -08:00
state.Save()
2017-05-01 21:43:49 -07:00
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
2017-05-04 19:33:08 -07:00
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
2017-09-18 20:16:14 -07:00
var privVal types.PrivValidator
2016-11-23 15:20:46 -08:00
if i < nValidators {
privVal = privVals[i]
} else {
_, tempFilePath := Tempfile("priv_validator_")
2017-09-18 20:16:14 -07:00
privVal = types.GenPrivValidatorFS(tempFilePath)
2016-11-23 15:20:46 -08:00
}
css[i] = newConsensusStateWithConfig(thisConfig, state, privVal, appFunc())
2017-05-02 00:53:32 -07:00
css[i].SetLogger(log.TestingLogger())
css[i].SetTimeoutTicker(tickerFunc())
2016-11-23 15:20:46 -08:00
}
return css
}
2017-09-12 17:49:22 -07:00
func getSwitchIndex(switches []*p2p.Switch, peer p2p.Peer) int {
for i, s := range switches {
2017-09-12 17:49:22 -07:00
if bytes.Equal(peer.NodeInfo().PubKey.Address(), s.NodeInfo().PubKey.Address()) {
return i
2016-07-11 17:40:48 -07:00
}
}
panic("didnt find peer in switches")
return -1
2016-07-11 17:40:48 -07:00
}
//-------------------------------------------------------------------------------
// genesis
2015-12-01 20:12:01 -08:00
2017-09-18 20:16:14 -07:00
func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []*types.PrivValidatorFS) {
2015-12-01 20:12:01 -08:00
validators := make([]types.GenesisValidator, numValidators)
2017-09-18 20:16:14 -07:00
privValidators := make([]*types.PrivValidatorFS, numValidators)
2015-12-01 20:12:01 -08:00
for i := 0; i < numValidators; i++ {
val, privVal := types.RandValidator(randPower, minPower)
validators[i] = types.GenesisValidator{
PubKey: val.PubKey,
2017-09-21 11:37:34 -07:00
Power: val.VotingPower,
2015-12-01 20:12:01 -08:00
}
privValidators[i] = privVal
}
sort.Sort(types.PrivValidatorsByAddress(privValidators))
return &types.GenesisDoc{
GenesisTime: time.Now(),
2017-05-01 21:43:49 -07:00
ChainID: config.ChainID,
2015-12-01 20:12:01 -08:00
Validators: validators,
}, privValidators
}
2016-11-23 15:20:46 -08:00
2017-09-18 20:16:14 -07:00
func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.State, []*types.PrivValidatorFS) {
genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower)
db := dbm.NewMemDB()
s0 := sm.MakeGenesisState(db, genDoc)
2017-05-02 00:53:32 -07:00
s0.SetLogger(log.TestingLogger().With("module", "state"))
s0.Save()
return s0, privValidators
2016-11-23 15:20:46 -08:00
}
2016-12-06 16:54:10 -08:00
//------------------------------------
// mock ticker
func newMockTickerFunc(onlyOnce bool) func() TimeoutTicker {
return func() TimeoutTicker {
return &mockTicker{
c: make(chan timeoutInfo, 10),
onlyOnce: onlyOnce,
}
}
}
2016-12-22 18:51:58 -08:00
// mock ticker only fires on RoundStepNewHeight
// and only once if onlyOnce=true
type mockTicker struct {
c chan timeoutInfo
mtx sync.Mutex
onlyOnce bool
fired bool
}
func (m *mockTicker) Start() (bool, error) {
return true, nil
}
func (m *mockTicker) Stop() bool {
return true
}
func (m *mockTicker) ScheduleTimeout(ti timeoutInfo) {
m.mtx.Lock()
defer m.mtx.Unlock()
if m.onlyOnce && m.fired {
return
}
if ti.Step == RoundStepNewHeight {
m.c <- ti
m.fired = true
}
}
func (m *mockTicker) Chan() <-chan timeoutInfo {
return m.c
}
2017-05-12 14:07:53 -07:00
func (mockTicker) SetLogger(log.Logger) {
}
//------------------------------------
2017-01-12 12:53:32 -08:00
func newCounter() abci.Application {
return counter.NewCounterApplication(true)
}
2017-01-12 12:53:32 -08:00
func newPersistentDummy() abci.Application {
dir, _ := ioutil.TempDir("/tmp", "persistent-dummy")
return dummy.NewPersistentDummyApplication(dir)
}