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.

685 lines
20 KiB

  1. package consensus
  2. import (
  3. "bytes"
  4. "context"
  5. "fmt"
  6. "io"
  7. "io/ioutil"
  8. "os"
  9. "path"
  10. "runtime"
  11. "testing"
  12. "time"
  13. "github.com/stretchr/testify/assert"
  14. "github.com/stretchr/testify/require"
  15. "github.com/tendermint/tendermint/abci/example/kvstore"
  16. abci "github.com/tendermint/tendermint/abci/types"
  17. crypto "github.com/tendermint/tendermint/crypto"
  18. auto "github.com/tendermint/tendermint/libs/autofile"
  19. dbm "github.com/tendermint/tendermint/libs/db"
  20. cfg "github.com/tendermint/tendermint/config"
  21. "github.com/tendermint/tendermint/libs/log"
  22. "github.com/tendermint/tendermint/privval"
  23. "github.com/tendermint/tendermint/proxy"
  24. sm "github.com/tendermint/tendermint/state"
  25. "github.com/tendermint/tendermint/types"
  26. )
  27. var consensusReplayConfig *cfg.Config
  28. func init() {
  29. consensusReplayConfig = ResetConfig("consensus_replay_test")
  30. }
  31. // These tests ensure we can always recover from failure at any part of the consensus process.
  32. // There are two general failure scenarios: failure during consensus, and failure while applying the block.
  33. // Only the latter interacts with the app and store,
  34. // but the former has to deal with restrictions on re-use of priv_validator keys.
  35. // The `WAL Tests` are for failures during the consensus;
  36. // the `Handshake Tests` are for failures in applying the block.
  37. // With the help of the WAL, we can recover from it all!
  38. //------------------------------------------------------------------------------------------
  39. // WAL Tests
  40. // TODO: It would be better to verify explicitly which states we can recover from without the wal
  41. // and which ones we need the wal for - then we'd also be able to only flush the
  42. // wal writer when we need to, instead of with every message.
  43. func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight int64, blockDB dbm.DB, stateDB dbm.DB) {
  44. logger := log.TestingLogger()
  45. state, _ := sm.LoadStateFromDBOrGenesisFile(stateDB, consensusReplayConfig.GenesisFile())
  46. privValidator := loadPrivValidator(consensusReplayConfig)
  47. cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, kvstore.NewKVStoreApplication(), blockDB)
  48. cs.SetLogger(logger)
  49. bytes, _ := ioutil.ReadFile(cs.config.WalFile())
  50. // fmt.Printf("====== WAL: \n\r%s\n", bytes)
  51. t.Logf("====== WAL: \n\r%X\n", bytes)
  52. err := cs.Start()
  53. require.NoError(t, err)
  54. defer cs.Stop()
  55. // This is just a signal that we haven't halted; its not something contained
  56. // in the WAL itself. Assuming the consensus state is running, replay of any
  57. // WAL, including the empty one, should eventually be followed by a new
  58. // block, or else something is wrong.
  59. newBlockCh := make(chan interface{}, 1)
  60. err = cs.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, newBlockCh)
  61. require.NoError(t, err)
  62. select {
  63. case <-newBlockCh:
  64. case <-time.After(60 * time.Second):
  65. t.Fatalf("Timed out waiting for new block (see trace above)")
  66. }
  67. }
  68. func sendTxs(cs *ConsensusState, ctx context.Context) {
  69. for i := 0; i < 256; i++ {
  70. select {
  71. case <-ctx.Done():
  72. return
  73. default:
  74. tx := []byte{byte(i)}
  75. cs.mempool.CheckTx(tx, nil)
  76. i++
  77. }
  78. }
  79. }
  80. // TestWALCrash uses crashing WAL to test we can recover from any WAL failure.
  81. func TestWALCrash(t *testing.T) {
  82. testCases := []struct {
  83. name string
  84. initFn func(dbm.DB, *ConsensusState, context.Context)
  85. heightToStop int64
  86. }{
  87. {"empty block",
  88. func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {},
  89. 1},
  90. {"block with a smaller part size",
  91. func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {
  92. // XXX: is there a better way to change BlockPartSizeBytes?
  93. cs.state.ConsensusParams.BlockPartSizeBytes = 512
  94. sm.SaveState(stateDB, cs.state)
  95. go sendTxs(cs, ctx)
  96. },
  97. 1},
  98. {"many non-empty blocks",
  99. func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {
  100. go sendTxs(cs, ctx)
  101. },
  102. 3},
  103. }
  104. for _, tc := range testCases {
  105. t.Run(tc.name, func(t *testing.T) {
  106. crashWALandCheckLiveness(t, tc.initFn, tc.heightToStop)
  107. })
  108. }
  109. }
  110. func crashWALandCheckLiveness(t *testing.T, initFn func(dbm.DB, *ConsensusState, context.Context), heightToStop int64) {
  111. walPaniced := make(chan error)
  112. crashingWal := &crashingWAL{panicCh: walPaniced, heightToStop: heightToStop}
  113. i := 1
  114. LOOP:
  115. for {
  116. // fmt.Printf("====== LOOP %d\n", i)
  117. t.Logf("====== LOOP %d\n", i)
  118. // create consensus state from a clean slate
  119. logger := log.NewNopLogger()
  120. stateDB := dbm.NewMemDB()
  121. state, _ := sm.MakeGenesisStateFromFile(consensusReplayConfig.GenesisFile())
  122. privValidator := loadPrivValidator(consensusReplayConfig)
  123. blockDB := dbm.NewMemDB()
  124. cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, kvstore.NewKVStoreApplication(), blockDB)
  125. cs.SetLogger(logger)
  126. // start sending transactions
  127. ctx, cancel := context.WithCancel(context.Background())
  128. initFn(stateDB, cs, ctx)
  129. // clean up WAL file from the previous iteration
  130. walFile := cs.config.WalFile()
  131. os.Remove(walFile)
  132. // set crashing WAL
  133. csWal, err := cs.OpenWAL(walFile)
  134. require.NoError(t, err)
  135. crashingWal.next = csWal
  136. // reset the message counter
  137. crashingWal.msgIndex = 1
  138. cs.wal = crashingWal
  139. // start consensus state
  140. err = cs.Start()
  141. require.NoError(t, err)
  142. i++
  143. select {
  144. case err := <-walPaniced:
  145. t.Logf("WAL paniced: %v", err)
  146. // make sure we can make blocks after a crash
  147. startNewConsensusStateAndWaitForBlock(t, cs.Height, blockDB, stateDB)
  148. // stop consensus state and transactions sender (initFn)
  149. cs.Stop()
  150. cancel()
  151. // if we reached the required height, exit
  152. if _, ok := err.(ReachedHeightToStopError); ok {
  153. break LOOP
  154. }
  155. case <-time.After(10 * time.Second):
  156. t.Fatal("WAL did not panic for 10 seconds (check the log)")
  157. }
  158. }
  159. }
  160. // crashingWAL is a WAL which crashes or rather simulates a crash during Save
  161. // (before and after). It remembers a message for which we last panicked
  162. // (lastPanicedForMsgIndex), so we don't panic for it in subsequent iterations.
  163. type crashingWAL struct {
  164. next WAL
  165. panicCh chan error
  166. heightToStop int64
  167. msgIndex int // current message index
  168. lastPanicedForMsgIndex int // last message for which we panicked
  169. }
  170. // WALWriteError indicates a WAL crash.
  171. type WALWriteError struct {
  172. msg string
  173. }
  174. func (e WALWriteError) Error() string {
  175. return e.msg
  176. }
  177. // ReachedHeightToStopError indicates we've reached the required consensus
  178. // height and may exit.
  179. type ReachedHeightToStopError struct {
  180. height int64
  181. }
  182. func (e ReachedHeightToStopError) Error() string {
  183. return fmt.Sprintf("reached height to stop %d", e.height)
  184. }
  185. // Write simulate WAL's crashing by sending an error to the panicCh and then
  186. // exiting the cs.receiveRoutine.
  187. func (w *crashingWAL) Write(m WALMessage) {
  188. if endMsg, ok := m.(EndHeightMessage); ok {
  189. if endMsg.Height == w.heightToStop {
  190. w.panicCh <- ReachedHeightToStopError{endMsg.Height}
  191. runtime.Goexit()
  192. } else {
  193. w.next.Write(m)
  194. }
  195. return
  196. }
  197. if w.msgIndex > w.lastPanicedForMsgIndex {
  198. w.lastPanicedForMsgIndex = w.msgIndex
  199. _, file, line, _ := runtime.Caller(1)
  200. w.panicCh <- WALWriteError{fmt.Sprintf("failed to write %T to WAL (fileline: %s:%d)", m, file, line)}
  201. runtime.Goexit()
  202. } else {
  203. w.msgIndex++
  204. w.next.Write(m)
  205. }
  206. }
  207. func (w *crashingWAL) WriteSync(m WALMessage) {
  208. w.Write(m)
  209. }
  210. func (w *crashingWAL) Group() *auto.Group { return w.next.Group() }
  211. func (w *crashingWAL) SearchForEndHeight(height int64, options *WALSearchOptions) (gr *auto.GroupReader, found bool, err error) {
  212. return w.next.SearchForEndHeight(height, options)
  213. }
  214. func (w *crashingWAL) Start() error { return w.next.Start() }
  215. func (w *crashingWAL) Stop() error { return w.next.Stop() }
  216. func (w *crashingWAL) Wait() { w.next.Wait() }
  217. //------------------------------------------------------------------------------------------
  218. // Handshake Tests
  219. const (
  220. NUM_BLOCKS = 6
  221. )
  222. var (
  223. mempool = sm.MockMempool{}
  224. evpool = sm.MockEvidencePool{}
  225. )
  226. //---------------------------------------
  227. // Test handshake/replay
  228. // 0 - all synced up
  229. // 1 - saved block but app and state are behind
  230. // 2 - save block and committed but state is behind
  231. var modes = []uint{0, 1, 2}
  232. // Sync from scratch
  233. func TestHandshakeReplayAll(t *testing.T) {
  234. for _, m := range modes {
  235. testHandshakeReplay(t, 0, m)
  236. }
  237. }
  238. // Sync many, not from scratch
  239. func TestHandshakeReplaySome(t *testing.T) {
  240. for _, m := range modes {
  241. testHandshakeReplay(t, 1, m)
  242. }
  243. }
  244. // Sync from lagging by one
  245. func TestHandshakeReplayOne(t *testing.T) {
  246. for _, m := range modes {
  247. testHandshakeReplay(t, NUM_BLOCKS-1, m)
  248. }
  249. }
  250. // Sync from caught up
  251. func TestHandshakeReplayNone(t *testing.T) {
  252. for _, m := range modes {
  253. testHandshakeReplay(t, NUM_BLOCKS, m)
  254. }
  255. }
  256. func tempWALWithData(data []byte) string {
  257. walFile, err := ioutil.TempFile("", "wal")
  258. if err != nil {
  259. panic(fmt.Errorf("failed to create temp WAL file: %v", err))
  260. }
  261. _, err = walFile.Write(data)
  262. if err != nil {
  263. panic(fmt.Errorf("failed to write to temp WAL file: %v", err))
  264. }
  265. if err := walFile.Close(); err != nil {
  266. panic(fmt.Errorf("failed to close temp WAL file: %v", err))
  267. }
  268. return walFile.Name()
  269. }
  270. // Make some blocks. Start a fresh app and apply nBlocks blocks. Then restart the app and sync it up with the remaining blocks
  271. func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
  272. config := ResetConfig("proxy_test_")
  273. walBody, err := WALWithNBlocks(NUM_BLOCKS)
  274. if err != nil {
  275. t.Fatal(err)
  276. }
  277. walFile := tempWALWithData(walBody)
  278. config.Consensus.SetWalFile(walFile)
  279. privVal := privval.LoadFilePV(config.PrivValidatorFile())
  280. wal, err := NewWAL(walFile)
  281. if err != nil {
  282. t.Fatal(err)
  283. }
  284. wal.SetLogger(log.TestingLogger())
  285. if err := wal.Start(); err != nil {
  286. t.Fatal(err)
  287. }
  288. defer wal.Stop()
  289. chain, commits, err := makeBlockchainFromWAL(wal)
  290. if err != nil {
  291. t.Fatalf(err.Error())
  292. }
  293. stateDB, state, store := stateAndStore(config, privVal.GetPubKey())
  294. store.chain = chain
  295. store.commits = commits
  296. // run the chain through state.ApplyBlock to build up the tendermint state
  297. state = buildTMStateFromChain(config, stateDB, state, chain, mode)
  298. latestAppHash := state.AppHash
  299. // make a new client creator
  300. kvstoreApp := kvstore.NewPersistentKVStoreApplication(path.Join(config.DBDir(), "2"))
  301. clientCreator2 := proxy.NewLocalClientCreator(kvstoreApp)
  302. if nBlocks > 0 {
  303. // run nBlocks against a new client to build up the app state.
  304. // use a throwaway tendermint state
  305. proxyApp := proxy.NewAppConns(clientCreator2, nil)
  306. stateDB, state, _ := stateAndStore(config, privVal.GetPubKey())
  307. buildAppStateFromChain(proxyApp, stateDB, state, chain, nBlocks, mode)
  308. }
  309. // now start the app using the handshake - it should sync
  310. genDoc, _ := sm.MakeGenesisDocFromFile(config.GenesisFile())
  311. handshaker := NewHandshaker(stateDB, state, store, genDoc)
  312. proxyApp := proxy.NewAppConns(clientCreator2, handshaker)
  313. if err := proxyApp.Start(); err != nil {
  314. t.Fatalf("Error starting proxy app connections: %v", err)
  315. }
  316. defer proxyApp.Stop()
  317. // get the latest app hash from the app
  318. res, err := proxyApp.Query().InfoSync(abci.RequestInfo{Version: ""})
  319. if err != nil {
  320. t.Fatal(err)
  321. }
  322. // the app hash should be synced up
  323. if !bytes.Equal(latestAppHash, res.LastBlockAppHash) {
  324. t.Fatalf("Expected app hashes to match after handshake/replay. got %X, expected %X", res.LastBlockAppHash, latestAppHash)
  325. }
  326. expectedBlocksToSync := NUM_BLOCKS - nBlocks
  327. if nBlocks == NUM_BLOCKS && mode > 0 {
  328. expectedBlocksToSync++
  329. } else if nBlocks > 0 && mode == 1 {
  330. expectedBlocksToSync++
  331. }
  332. if handshaker.NBlocks() != expectedBlocksToSync {
  333. t.Fatalf("Expected handshake to sync %d blocks, got %d", expectedBlocksToSync, handshaker.NBlocks())
  334. }
  335. }
  336. func applyBlock(stateDB dbm.DB, st sm.State, blk *types.Block, proxyApp proxy.AppConns) sm.State {
  337. testPartSize := st.ConsensusParams.BlockPartSizeBytes
  338. blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
  339. blkID := types.BlockID{blk.Hash(), blk.MakePartSet(testPartSize).Header()}
  340. newState, err := blockExec.ApplyBlock(st, blkID, blk)
  341. if err != nil {
  342. panic(err)
  343. }
  344. return newState
  345. }
  346. func buildAppStateFromChain(proxyApp proxy.AppConns, stateDB dbm.DB,
  347. state sm.State, chain []*types.Block, nBlocks int, mode uint) {
  348. // start a new app without handshake, play nBlocks blocks
  349. if err := proxyApp.Start(); err != nil {
  350. panic(err)
  351. }
  352. defer proxyApp.Stop()
  353. validators := types.TM2PB.Validators(state.Validators)
  354. if _, err := proxyApp.Consensus().InitChainSync(abci.RequestInitChain{
  355. Validators: validators,
  356. }); err != nil {
  357. panic(err)
  358. }
  359. switch mode {
  360. case 0:
  361. for i := 0; i < nBlocks; i++ {
  362. block := chain[i]
  363. state = applyBlock(stateDB, state, block, proxyApp)
  364. }
  365. case 1, 2:
  366. for i := 0; i < nBlocks-1; i++ {
  367. block := chain[i]
  368. state = applyBlock(stateDB, state, block, proxyApp)
  369. }
  370. if mode == 2 {
  371. // update the kvstore height and apphash
  372. // as if we ran commit but not
  373. state = applyBlock(stateDB, state, chain[nBlocks-1], proxyApp)
  374. }
  375. }
  376. }
  377. func buildTMStateFromChain(config *cfg.Config, stateDB dbm.DB, state sm.State, chain []*types.Block, mode uint) sm.State {
  378. // run the whole chain against this client to build up the tendermint state
  379. clientCreator := proxy.NewLocalClientCreator(kvstore.NewPersistentKVStoreApplication(path.Join(config.DBDir(), "1")))
  380. proxyApp := proxy.NewAppConns(clientCreator, nil) // sm.NewHandshaker(config, state, store, ReplayLastBlock))
  381. if err := proxyApp.Start(); err != nil {
  382. panic(err)
  383. }
  384. defer proxyApp.Stop()
  385. validators := types.TM2PB.Validators(state.Validators)
  386. if _, err := proxyApp.Consensus().InitChainSync(abci.RequestInitChain{
  387. Validators: validators,
  388. }); err != nil {
  389. panic(err)
  390. }
  391. switch mode {
  392. case 0:
  393. // sync right up
  394. for _, block := range chain {
  395. state = applyBlock(stateDB, state, block, proxyApp)
  396. }
  397. case 1, 2:
  398. // sync up to the penultimate as if we stored the block.
  399. // whether we commit or not depends on the appHash
  400. for _, block := range chain[:len(chain)-1] {
  401. state = applyBlock(stateDB, state, block, proxyApp)
  402. }
  403. // apply the final block to a state copy so we can
  404. // get the right next appHash but keep the state back
  405. applyBlock(stateDB, state, chain[len(chain)-1], proxyApp)
  406. }
  407. return state
  408. }
  409. //--------------------------
  410. // utils for making blocks
  411. func makeBlockchainFromWAL(wal WAL) ([]*types.Block, []*types.Commit, error) {
  412. // Search for height marker
  413. gr, found, err := wal.SearchForEndHeight(0, &WALSearchOptions{})
  414. if err != nil {
  415. return nil, nil, err
  416. }
  417. if !found {
  418. return nil, nil, fmt.Errorf("WAL does not contain height %d.", 1)
  419. }
  420. defer gr.Close() // nolint: errcheck
  421. // log.Notice("Build a blockchain by reading from the WAL")
  422. var blocks []*types.Block
  423. var commits []*types.Commit
  424. var thisBlockParts *types.PartSet
  425. var thisBlockCommit *types.Commit
  426. var height int64
  427. dec := NewWALDecoder(gr)
  428. for {
  429. msg, err := dec.Decode()
  430. if err == io.EOF {
  431. break
  432. } else if err != nil {
  433. return nil, nil, err
  434. }
  435. piece := readPieceFromWAL(msg)
  436. if piece == nil {
  437. continue
  438. }
  439. switch p := piece.(type) {
  440. case EndHeightMessage:
  441. // if its not the first one, we have a full block
  442. if thisBlockParts != nil {
  443. var block = new(types.Block)
  444. _, err = cdc.UnmarshalBinaryReader(thisBlockParts.GetReader(), block, 0)
  445. if err != nil {
  446. panic(err)
  447. }
  448. if block.Height != height+1 {
  449. panic(fmt.Sprintf("read bad block from wal. got height %d, expected %d", block.Height, height+1))
  450. }
  451. commitHeight := thisBlockCommit.Precommits[0].Height
  452. if commitHeight != height+1 {
  453. panic(fmt.Sprintf("commit doesnt match. got height %d, expected %d", commitHeight, height+1))
  454. }
  455. blocks = append(blocks, block)
  456. commits = append(commits, thisBlockCommit)
  457. height++
  458. }
  459. case *types.PartSetHeader:
  460. thisBlockParts = types.NewPartSetFromHeader(*p)
  461. case *types.Part:
  462. _, err := thisBlockParts.AddPart(p)
  463. if err != nil {
  464. return nil, nil, err
  465. }
  466. case *types.Vote:
  467. if p.Type == types.VoteTypePrecommit {
  468. thisBlockCommit = &types.Commit{
  469. BlockID: p.BlockID,
  470. Precommits: []*types.Vote{p},
  471. }
  472. }
  473. }
  474. }
  475. // grab the last block too
  476. var block = new(types.Block)
  477. _, err = cdc.UnmarshalBinaryReader(thisBlockParts.GetReader(), block, 0)
  478. if err != nil {
  479. panic(err)
  480. }
  481. if block.Height != height+1 {
  482. panic(fmt.Sprintf("read bad block from wal. got height %d, expected %d", block.Height, height+1))
  483. }
  484. commitHeight := thisBlockCommit.Precommits[0].Height
  485. if commitHeight != height+1 {
  486. panic(fmt.Sprintf("commit doesnt match. got height %d, expected %d", commitHeight, height+1))
  487. }
  488. blocks = append(blocks, block)
  489. commits = append(commits, thisBlockCommit)
  490. return blocks, commits, nil
  491. }
  492. func readPieceFromWAL(msg *TimedWALMessage) interface{} {
  493. // for logging
  494. switch m := msg.Msg.(type) {
  495. case msgInfo:
  496. switch msg := m.Msg.(type) {
  497. case *ProposalMessage:
  498. return &msg.Proposal.BlockPartsHeader
  499. case *BlockPartMessage:
  500. return msg.Part
  501. case *VoteMessage:
  502. return msg.Vote
  503. }
  504. case EndHeightMessage:
  505. return m
  506. }
  507. return nil
  508. }
  509. // fresh state and mock store
  510. func stateAndStore(config *cfg.Config, pubKey crypto.PubKey) (dbm.DB, sm.State, *mockBlockStore) {
  511. stateDB := dbm.NewMemDB()
  512. state, _ := sm.MakeGenesisStateFromFile(config.GenesisFile())
  513. store := NewMockBlockStore(config, state.ConsensusParams)
  514. return stateDB, state, store
  515. }
  516. //----------------------------------
  517. // mock block store
  518. type mockBlockStore struct {
  519. config *cfg.Config
  520. params types.ConsensusParams
  521. chain []*types.Block
  522. commits []*types.Commit
  523. }
  524. // TODO: NewBlockStore(db.NewMemDB) ...
  525. func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBlockStore {
  526. return &mockBlockStore{config, params, nil, nil}
  527. }
  528. func (bs *mockBlockStore) Height() int64 { return int64(len(bs.chain)) }
  529. func (bs *mockBlockStore) LoadBlock(height int64) *types.Block { return bs.chain[height-1] }
  530. func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
  531. block := bs.chain[height-1]
  532. return &types.BlockMeta{
  533. BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.params.BlockPartSizeBytes).Header()},
  534. Header: block.Header,
  535. }
  536. }
  537. func (bs *mockBlockStore) LoadBlockPart(height int64, index int) *types.Part { return nil }
  538. func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) {
  539. }
  540. func (bs *mockBlockStore) LoadBlockCommit(height int64) *types.Commit {
  541. return bs.commits[height-1]
  542. }
  543. func (bs *mockBlockStore) LoadSeenCommit(height int64) *types.Commit {
  544. return bs.commits[height-1]
  545. }
  546. //----------------------------------------
  547. func TestInitChainUpdateValidators(t *testing.T) {
  548. val, _ := types.RandValidator(true, 10)
  549. vals := types.NewValidatorSet([]*types.Validator{val})
  550. app := &initChainApp{vals: types.TM2PB.Validators(vals)}
  551. clientCreator := proxy.NewLocalClientCreator(app)
  552. config := ResetConfig("proxy_test_")
  553. privVal := privval.LoadFilePV(config.PrivValidatorFile())
  554. stateDB, state, store := stateAndStore(config, privVal.GetPubKey())
  555. oldValAddr := state.Validators.Validators[0].Address
  556. // now start the app using the handshake - it should sync
  557. genDoc, _ := sm.MakeGenesisDocFromFile(config.GenesisFile())
  558. handshaker := NewHandshaker(stateDB, state, store, genDoc)
  559. proxyApp := proxy.NewAppConns(clientCreator, handshaker)
  560. if err := proxyApp.Start(); err != nil {
  561. t.Fatalf("Error starting proxy app connections: %v", err)
  562. }
  563. defer proxyApp.Stop()
  564. // reload the state, check the validator set was updated
  565. state = sm.LoadState(stateDB)
  566. newValAddr := state.Validators.Validators[0].Address
  567. expectValAddr := val.Address
  568. assert.NotEqual(t, oldValAddr, newValAddr)
  569. assert.Equal(t, newValAddr, expectValAddr)
  570. }
  571. func newInitChainApp(vals []abci.Validator) *initChainApp {
  572. return &initChainApp{
  573. vals: vals,
  574. }
  575. }
  576. // returns the vals on InitChain
  577. type initChainApp struct {
  578. abci.BaseApplication
  579. vals []abci.Validator
  580. }
  581. func (ica *initChainApp) InitChain(req abci.RequestInitChain) abci.ResponseInitChain {
  582. return abci.ResponseInitChain{
  583. Validators: ica.vals,
  584. }
  585. }