You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

454 lines
13 KiB

8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
9 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
  1. package consensus
  2. import (
  3. "bytes"
  4. "fmt"
  5. "io/ioutil"
  6. "os"
  7. "path"
  8. "sort"
  9. "sync"
  10. "testing"
  11. "time"
  12. abcicli "github.com/tendermint/abci/client"
  13. abci "github.com/tendermint/abci/types"
  14. bc "github.com/tendermint/tendermint/blockchain"
  15. cfg "github.com/tendermint/tendermint/config"
  16. mempl "github.com/tendermint/tendermint/mempool"
  17. "github.com/tendermint/tendermint/p2p"
  18. sm "github.com/tendermint/tendermint/state"
  19. "github.com/tendermint/tendermint/types"
  20. . "github.com/tendermint/tmlibs/common"
  21. dbm "github.com/tendermint/tmlibs/db"
  22. "github.com/tendermint/abci/example/counter"
  23. "github.com/tendermint/abci/example/dummy"
  24. )
  25. // genesis, chain_id, priv_val
  26. var config *cfg.Config // NOTE: must be reset for each _test.go file
  27. var ensureTimeout = time.Duration(2)
  28. func ensureDir(dir string, mode os.FileMode) {
  29. if err := EnsureDir(dir, mode); err != nil {
  30. panic(err)
  31. }
  32. }
  33. func ResetConfig(name string) *cfg.Config {
  34. return cfg.ResetTestRoot(name)
  35. }
  36. //-------------------------------------------------------------------------------
  37. // validator stub (a dummy consensus peer we control)
  38. type validatorStub struct {
  39. Index int // Validator index. NOTE: we don't assume validator set changes.
  40. Height int
  41. Round int
  42. *types.PrivValidator
  43. }
  44. var testMinPower = 10
  45. func NewValidatorStub(privValidator *types.PrivValidator, valIndex int) *validatorStub {
  46. return &validatorStub{
  47. Index: valIndex,
  48. PrivValidator: privValidator,
  49. }
  50. }
  51. func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
  52. vote := &types.Vote{
  53. ValidatorIndex: vs.Index,
  54. ValidatorAddress: vs.PrivValidator.Address,
  55. Height: vs.Height,
  56. Round: vs.Round,
  57. Type: voteType,
  58. BlockID: types.BlockID{hash, header},
  59. }
  60. err := vs.PrivValidator.SignVote(config.ChainID, vote)
  61. return vote, err
  62. }
  63. // Sign vote for type/hash/header
  64. func signVote(vs *validatorStub, voteType byte, hash []byte, header types.PartSetHeader) *types.Vote {
  65. v, err := vs.signVote(voteType, hash, header)
  66. if err != nil {
  67. panic(fmt.Errorf("failed to sign vote: %v", err))
  68. }
  69. return v
  70. }
  71. func signVotes(voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) []*types.Vote {
  72. votes := make([]*types.Vote, len(vss))
  73. for i, vs := range vss {
  74. votes[i] = signVote(vs, voteType, hash, header)
  75. }
  76. return votes
  77. }
  78. func incrementHeight(vss ...*validatorStub) {
  79. for _, vs := range vss {
  80. vs.Height += 1
  81. }
  82. }
  83. func incrementRound(vss ...*validatorStub) {
  84. for _, vs := range vss {
  85. vs.Round += 1
  86. }
  87. }
  88. //-------------------------------------------------------------------------------
  89. // Functions for transitioning the consensus state
  90. func startTestRound(cs *ConsensusState, height, round int) {
  91. cs.enterNewRound(height, round)
  92. cs.startRoutines(0)
  93. }
  94. // Create proposal block from cs1 but sign it with vs
  95. func decideProposal(cs1 *ConsensusState, vs *validatorStub, height, round int) (proposal *types.Proposal, block *types.Block) {
  96. block, blockParts := cs1.createProposalBlock()
  97. if block == nil { // on error
  98. panic("error creating proposal block")
  99. }
  100. // Make proposal
  101. polRound, polBlockID := cs1.Votes.POLInfo()
  102. proposal = types.NewProposal(height, round, blockParts.Header(), polRound, polBlockID)
  103. if err := vs.SignProposal(config.ChainID, proposal); err != nil {
  104. panic(err)
  105. }
  106. return
  107. }
  108. func addVotes(to *ConsensusState, votes ...*types.Vote) {
  109. for _, vote := range votes {
  110. to.peerMsgQueue <- msgInfo{Msg: &VoteMessage{vote}}
  111. }
  112. }
  113. func signAddVotes(to *ConsensusState, voteType byte, hash []byte, header types.PartSetHeader, vss ...*validatorStub) {
  114. votes := signVotes(voteType, hash, header, vss...)
  115. addVotes(to, votes...)
  116. }
  117. func validatePrevote(t *testing.T, cs *ConsensusState, round int, privVal *validatorStub, blockHash []byte) {
  118. prevotes := cs.Votes.Prevotes(round)
  119. var vote *types.Vote
  120. if vote = prevotes.GetByAddress(privVal.Address); vote == nil {
  121. panic("Failed to find prevote from validator")
  122. }
  123. if blockHash == nil {
  124. if vote.BlockID.Hash != nil {
  125. panic(fmt.Sprintf("Expected prevote to be for nil, got %X", vote.BlockID.Hash))
  126. }
  127. } else {
  128. if !bytes.Equal(vote.BlockID.Hash, blockHash) {
  129. panic(fmt.Sprintf("Expected prevote to be for %X, got %X", blockHash, vote.BlockID.Hash))
  130. }
  131. }
  132. }
  133. func validateLastPrecommit(t *testing.T, cs *ConsensusState, privVal *validatorStub, blockHash []byte) {
  134. votes := cs.LastCommit
  135. var vote *types.Vote
  136. if vote = votes.GetByAddress(privVal.Address); vote == nil {
  137. panic("Failed to find precommit from validator")
  138. }
  139. if !bytes.Equal(vote.BlockID.Hash, blockHash) {
  140. panic(fmt.Sprintf("Expected precommit to be for %X, got %X", blockHash, vote.BlockID.Hash))
  141. }
  142. }
  143. func validatePrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound int, privVal *validatorStub, votedBlockHash, lockedBlockHash []byte) {
  144. precommits := cs.Votes.Precommits(thisRound)
  145. var vote *types.Vote
  146. if vote = precommits.GetByAddress(privVal.Address); vote == nil {
  147. panic("Failed to find precommit from validator")
  148. }
  149. if votedBlockHash == nil {
  150. if vote.BlockID.Hash != nil {
  151. panic("Expected precommit to be for nil")
  152. }
  153. } else {
  154. if !bytes.Equal(vote.BlockID.Hash, votedBlockHash) {
  155. panic("Expected precommit to be for proposal block")
  156. }
  157. }
  158. if lockedBlockHash == nil {
  159. if cs.LockedRound != lockRound || cs.LockedBlock != nil {
  160. 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))
  161. }
  162. } else {
  163. if cs.LockedRound != lockRound || !bytes.Equal(cs.LockedBlock.Hash(), lockedBlockHash) {
  164. 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))
  165. }
  166. }
  167. }
  168. func validatePrevoteAndPrecommit(t *testing.T, cs *ConsensusState, thisRound, lockRound int, privVal *validatorStub, votedBlockHash, lockedBlockHash []byte) {
  169. // verify the prevote
  170. validatePrevote(t, cs, thisRound, privVal, votedBlockHash)
  171. // verify precommit
  172. cs.mtx.Lock()
  173. validatePrecommit(t, cs, thisRound, lockRound, privVal, votedBlockHash, lockedBlockHash)
  174. cs.mtx.Unlock()
  175. }
  176. // genesis
  177. func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} {
  178. voteCh0 := subscribeToEvent(cs.evsw, "tester", types.EventStringVote(), 1)
  179. voteCh := make(chan interface{})
  180. go func() {
  181. for {
  182. v := <-voteCh0
  183. vote := v.(types.TMEventData).Unwrap().(types.EventDataVote)
  184. // we only fire for our own votes
  185. if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
  186. voteCh <- v
  187. }
  188. }
  189. }()
  190. return voteCh
  191. }
  192. func readVotes(ch chan interface{}, reads int) chan struct{} {
  193. wg := make(chan struct{})
  194. go func() {
  195. for i := 0; i < reads; i++ {
  196. <-ch // read the precommit event
  197. }
  198. close(wg)
  199. }()
  200. return wg
  201. }
  202. //-------------------------------------------------------------------------------
  203. // consensus states
  204. func newConsensusState(state *sm.State, pv *types.PrivValidator, app abci.Application) *ConsensusState {
  205. return newConsensusStateWithConfig(config, state, pv, app)
  206. }
  207. func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv *types.PrivValidator, app abci.Application) *ConsensusState {
  208. // Get BlockStore
  209. blockDB := dbm.NewMemDB()
  210. blockStore := bc.NewBlockStore(blockDB)
  211. // one for mempool, one for consensus
  212. mtx := new(sync.Mutex)
  213. proxyAppConnMem := abcicli.NewLocalClient(mtx, app)
  214. proxyAppConnCon := abcicli.NewLocalClient(mtx, app)
  215. // Make Mempool
  216. mempool := mempl.NewMempool(thisConfig.Mempool, proxyAppConnMem)
  217. // Make ConsensusReactor
  218. cs := NewConsensusState(thisConfig.Consensus, state, proxyAppConnCon, blockStore, mempool)
  219. cs.SetPrivValidator(pv)
  220. evsw := types.NewEventSwitch()
  221. cs.SetEventSwitch(evsw)
  222. evsw.Start()
  223. return cs
  224. }
  225. func loadPrivValidator(config *cfg.Config) *types.PrivValidator {
  226. privValidatorFile := config.PrivValidatorFile()
  227. ensureDir(path.Dir(privValidatorFile), 0700)
  228. privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
  229. privValidator.Reset()
  230. return privValidator
  231. }
  232. func fixedConsensusState() *ConsensusState {
  233. stateDB := dbm.NewMemDB()
  234. state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
  235. privValidator := loadPrivValidator(config)
  236. cs := newConsensusState(state, privValidator, counter.NewCounterApplication(true))
  237. return cs
  238. }
  239. func fixedConsensusStateDummy() *ConsensusState {
  240. stateDB := dbm.NewMemDB()
  241. state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
  242. privValidator := loadPrivValidator(config)
  243. cs := newConsensusState(state, privValidator, dummy.NewDummyApplication())
  244. return cs
  245. }
  246. func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
  247. // Get State
  248. state, privVals := randGenesisState(nValidators, false, 10)
  249. vss := make([]*validatorStub, nValidators)
  250. cs := newConsensusState(state, privVals[0], counter.NewCounterApplication(true))
  251. for i := 0; i < nValidators; i++ {
  252. vss[i] = NewValidatorStub(privVals[i], i)
  253. }
  254. // since cs1 starts at 1
  255. incrementHeight(vss[1:]...)
  256. return cs, vss
  257. }
  258. //-------------------------------------------------------------------------------
  259. func ensureNoNewStep(stepCh chan interface{}) {
  260. timeout := time.NewTicker(ensureTimeout * time.Second)
  261. select {
  262. case <-timeout.C:
  263. break
  264. case <-stepCh:
  265. panic("We should be stuck waiting for more votes, not moving to the next step")
  266. }
  267. }
  268. //-------------------------------------------------------------------------------
  269. // consensus nets
  270. func randConsensusNet(nValidators int, testName string, tickerFunc func() TimeoutTicker, appFunc func() abci.Application) []*ConsensusState {
  271. genDoc, privVals := randGenesisDoc(nValidators, false, 10)
  272. css := make([]*ConsensusState, nValidators)
  273. for i := 0; i < nValidators; i++ {
  274. db := dbm.NewMemDB() // each state needs its own db
  275. state := sm.MakeGenesisState(db, genDoc)
  276. state.Save()
  277. thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
  278. ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
  279. css[i] = newConsensusStateWithConfig(thisConfig, state, privVals[i], appFunc())
  280. css[i].SetTimeoutTicker(tickerFunc())
  281. }
  282. return css
  283. }
  284. // nPeers = nValidators + nNotValidator
  285. func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerFunc func() TimeoutTicker, appFunc func() abci.Application) []*ConsensusState {
  286. genDoc, privVals := randGenesisDoc(nValidators, false, int64(testMinPower))
  287. css := make([]*ConsensusState, nPeers)
  288. for i := 0; i < nPeers; i++ {
  289. db := dbm.NewMemDB() // each state needs its own db
  290. state := sm.MakeGenesisState(db, genDoc)
  291. state.Save()
  292. thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
  293. ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
  294. var privVal *types.PrivValidator
  295. if i < nValidators {
  296. privVal = privVals[i]
  297. } else {
  298. privVal = types.GenPrivValidator()
  299. _, tempFilePath := Tempfile("priv_validator_")
  300. privVal.SetFile(tempFilePath)
  301. }
  302. css[i] = newConsensusStateWithConfig(thisConfig, state, privVal, appFunc())
  303. css[i].SetTimeoutTicker(tickerFunc())
  304. }
  305. return css
  306. }
  307. func getSwitchIndex(switches []*p2p.Switch, peer *p2p.Peer) int {
  308. for i, s := range switches {
  309. if bytes.Equal(peer.NodeInfo.PubKey.Address(), s.NodeInfo().PubKey.Address()) {
  310. return i
  311. }
  312. }
  313. panic("didnt find peer in switches")
  314. return -1
  315. }
  316. //-------------------------------------------------------------------------------
  317. // genesis
  318. func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.GenesisDoc, []*types.PrivValidator) {
  319. validators := make([]types.GenesisValidator, numValidators)
  320. privValidators := make([]*types.PrivValidator, numValidators)
  321. for i := 0; i < numValidators; i++ {
  322. val, privVal := types.RandValidator(randPower, minPower)
  323. validators[i] = types.GenesisValidator{
  324. PubKey: val.PubKey,
  325. Amount: val.VotingPower,
  326. }
  327. privValidators[i] = privVal
  328. }
  329. sort.Sort(types.PrivValidatorsByAddress(privValidators))
  330. return &types.GenesisDoc{
  331. GenesisTime: time.Now(),
  332. ChainID: config.ChainID,
  333. Validators: validators,
  334. }, privValidators
  335. }
  336. func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.State, []*types.PrivValidator) {
  337. genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower)
  338. db := dbm.NewMemDB()
  339. s0 := sm.MakeGenesisState(db, genDoc)
  340. s0.Save()
  341. return s0, privValidators
  342. }
  343. //------------------------------------
  344. // mock ticker
  345. func newMockTickerFunc(onlyOnce bool) func() TimeoutTicker {
  346. return func() TimeoutTicker {
  347. return &mockTicker{
  348. c: make(chan timeoutInfo, 10),
  349. onlyOnce: onlyOnce,
  350. }
  351. }
  352. }
  353. // mock ticker only fires on RoundStepNewHeight
  354. // and only once if onlyOnce=true
  355. type mockTicker struct {
  356. c chan timeoutInfo
  357. mtx sync.Mutex
  358. onlyOnce bool
  359. fired bool
  360. }
  361. func (m *mockTicker) Start() (bool, error) {
  362. return true, nil
  363. }
  364. func (m *mockTicker) Stop() bool {
  365. return true
  366. }
  367. func (m *mockTicker) ScheduleTimeout(ti timeoutInfo) {
  368. m.mtx.Lock()
  369. defer m.mtx.Unlock()
  370. if m.onlyOnce && m.fired {
  371. return
  372. }
  373. if ti.Step == RoundStepNewHeight {
  374. m.c <- ti
  375. m.fired = true
  376. }
  377. }
  378. func (m *mockTicker) Chan() <-chan timeoutInfo {
  379. return m.c
  380. }
  381. //------------------------------------
  382. func newCounter() abci.Application {
  383. return counter.NewCounterApplication(true)
  384. }
  385. func newPersistentDummy() abci.Application {
  386. dir, _ := ioutil.TempDir("/tmp", "persistent-dummy")
  387. return dummy.NewPersistentDummyApplication(dir)
  388. }