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.

538 lines
20 KiB

  1. package consensus
  2. import (
  3. "context"
  4. "fmt"
  5. "os"
  6. "runtime"
  7. "runtime/pprof"
  8. "sync"
  9. "testing"
  10. "time"
  11. "github.com/tendermint/abci/example/kvstore"
  12. cmn "github.com/tendermint/tmlibs/common"
  13. "github.com/tendermint/tmlibs/log"
  14. cfg "github.com/tendermint/tendermint/config"
  15. "github.com/tendermint/tendermint/p2p"
  16. p2pdummy "github.com/tendermint/tendermint/p2p/dummy"
  17. "github.com/tendermint/tendermint/types"
  18. "github.com/stretchr/testify/assert"
  19. "github.com/stretchr/testify/require"
  20. )
  21. func init() {
  22. config = ResetConfig("consensus_reactor_test")
  23. }
  24. //----------------------------------------------
  25. // in-process testnets
  26. func startConsensusNet(t *testing.T, css []*ConsensusState, N int) ([]*ConsensusReactor, []chan interface{}, []*types.EventBus) {
  27. reactors := make([]*ConsensusReactor, N)
  28. eventChans := make([]chan interface{}, N)
  29. eventBuses := make([]*types.EventBus, N)
  30. for i := 0; i < N; i++ {
  31. /*logger, err := tmflags.ParseLogLevel("consensus:info,*:error", logger, "info")
  32. if err != nil { t.Fatal(err)}*/
  33. reactors[i] = NewConsensusReactor(css[i], true) // so we dont start the consensus states
  34. reactors[i].SetLogger(css[i].Logger)
  35. // eventBus is already started with the cs
  36. eventBuses[i] = css[i].eventBus
  37. reactors[i].SetEventBus(eventBuses[i])
  38. eventChans[i] = make(chan interface{}, 1)
  39. err := eventBuses[i].Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, eventChans[i])
  40. require.NoError(t, err)
  41. }
  42. // make connected switches and start all reactors
  43. p2p.MakeConnectedSwitches(config.P2P, N, func(i int, s *p2p.Switch) *p2p.Switch {
  44. s.AddReactor("CONSENSUS", reactors[i])
  45. s.SetLogger(reactors[i].conS.Logger.With("module", "p2p"))
  46. return s
  47. }, p2p.Connect2Switches)
  48. // now that everyone is connected, start the state machines
  49. // If we started the state machines before everyone was connected,
  50. // we'd block when the cs fires NewBlockEvent and the peers are trying to start their reactors
  51. // TODO: is this still true with new pubsub?
  52. for i := 0; i < N; i++ {
  53. s := reactors[i].conS.GetState()
  54. reactors[i].SwitchToConsensus(s, 0)
  55. }
  56. return reactors, eventChans, eventBuses
  57. }
  58. func stopConsensusNet(logger log.Logger, reactors []*ConsensusReactor, eventBuses []*types.EventBus) {
  59. logger.Info("stopConsensusNet", "n", len(reactors))
  60. for i, r := range reactors {
  61. logger.Info("stopConsensusNet: Stopping ConsensusReactor", "i", i)
  62. r.Switch.Stop()
  63. }
  64. for i, b := range eventBuses {
  65. logger.Info("stopConsensusNet: Stopping eventBus", "i", i)
  66. b.Stop()
  67. }
  68. logger.Info("stopConsensusNet: DONE", "n", len(reactors))
  69. }
  70. // Ensure a testnet makes blocks
  71. func TestReactorBasic(t *testing.T) {
  72. N := 4
  73. css := randConsensusNet(N, "consensus_reactor_test", newMockTickerFunc(true), newCounter)
  74. reactors, eventChans, eventBuses := startConsensusNet(t, css, N)
  75. defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
  76. // wait till everyone makes the first new block
  77. timeoutWaitGroup(t, N, func(j int) {
  78. <-eventChans[j]
  79. }, css)
  80. }
  81. // Ensure a testnet sends proposal heartbeats and makes blocks when there are txs
  82. func TestReactorProposalHeartbeats(t *testing.T) {
  83. N := 4
  84. css := randConsensusNet(N, "consensus_reactor_test", newMockTickerFunc(true), newCounter,
  85. func(c *cfg.Config) {
  86. c.Consensus.CreateEmptyBlocks = false
  87. })
  88. reactors, eventChans, eventBuses := startConsensusNet(t, css, N)
  89. defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
  90. heartbeatChans := make([]chan interface{}, N)
  91. var err error
  92. for i := 0; i < N; i++ {
  93. heartbeatChans[i] = make(chan interface{}, 1)
  94. err = eventBuses[i].Subscribe(context.Background(), testSubscriber, types.EventQueryProposalHeartbeat, heartbeatChans[i])
  95. require.NoError(t, err)
  96. }
  97. // wait till everyone sends a proposal heartbeat
  98. timeoutWaitGroup(t, N, func(j int) {
  99. <-heartbeatChans[j]
  100. }, css)
  101. // send a tx
  102. if err := css[3].mempool.CheckTx([]byte{1, 2, 3}, nil); err != nil {
  103. //t.Fatal(err)
  104. }
  105. // wait till everyone makes the first new block
  106. timeoutWaitGroup(t, N, func(j int) {
  107. <-eventChans[j]
  108. }, css)
  109. }
  110. // Test we record block parts from other peers
  111. func TestReactorRecordsBlockParts(t *testing.T) {
  112. // create dummy peer
  113. peer := p2pdummy.NewPeer()
  114. ps := NewPeerState(peer).SetLogger(log.TestingLogger())
  115. peer.Set(types.PeerStateKey, ps)
  116. // create reactor
  117. css := randConsensusNet(1, "consensus_reactor_records_block_parts_test", newMockTickerFunc(true), newPersistentKVStore)
  118. reactor := NewConsensusReactor(css[0], false) // so we dont start the consensus states
  119. reactor.SetEventBus(css[0].eventBus)
  120. reactor.SetLogger(log.TestingLogger())
  121. sw := p2p.MakeSwitch(cfg.DefaultP2PConfig(), 1, "testing", "123.123.123", func(i int, sw *p2p.Switch) *p2p.Switch { return sw })
  122. reactor.SetSwitch(sw)
  123. err := reactor.Start()
  124. require.NoError(t, err)
  125. defer reactor.Stop()
  126. // 1) new block part
  127. parts := types.NewPartSetFromData(cmn.RandBytes(100), 10)
  128. msg := &BlockPartMessage{
  129. Height: 2,
  130. Round: 0,
  131. Part: parts.GetPart(0),
  132. }
  133. bz, err := cdc.MarshalBinaryBare(msg)
  134. require.NoError(t, err)
  135. reactor.Receive(DataChannel, peer, bz)
  136. require.Equal(t, 1, ps.BlockPartsSent(), "number of block parts sent should have increased by 1")
  137. // 2) block part with the same height, but different round
  138. msg.Round = 1
  139. bz, err = cdc.MarshalBinaryBare(msg)
  140. require.NoError(t, err)
  141. reactor.Receive(DataChannel, peer, bz)
  142. require.Equal(t, 1, ps.BlockPartsSent(), "number of block parts sent should stay the same")
  143. // 3) block part from earlier height
  144. msg.Height = 1
  145. msg.Round = 0
  146. bz, err = cdc.MarshalBinaryBare(msg)
  147. require.NoError(t, err)
  148. reactor.Receive(DataChannel, peer, bz)
  149. require.Equal(t, 1, ps.BlockPartsSent(), "number of block parts sent should stay the same")
  150. }
  151. // Test we record votes from other peers
  152. func TestReactorRecordsVotes(t *testing.T) {
  153. // create dummy peer
  154. peer := p2pdummy.NewPeer()
  155. ps := NewPeerState(peer).SetLogger(log.TestingLogger())
  156. peer.Set(types.PeerStateKey, ps)
  157. // create reactor
  158. css := randConsensusNet(1, "consensus_reactor_records_votes_test", newMockTickerFunc(true), newPersistentKVStore)
  159. reactor := NewConsensusReactor(css[0], false) // so we dont start the consensus states
  160. reactor.SetEventBus(css[0].eventBus)
  161. reactor.SetLogger(log.TestingLogger())
  162. sw := p2p.MakeSwitch(cfg.DefaultP2PConfig(), 1, "testing", "123.123.123", func(i int, sw *p2p.Switch) *p2p.Switch { return sw })
  163. reactor.SetSwitch(sw)
  164. err := reactor.Start()
  165. require.NoError(t, err)
  166. defer reactor.Stop()
  167. _, val := css[0].state.Validators.GetByIndex(0)
  168. // 1) new vote
  169. vote := &types.Vote{
  170. ValidatorIndex: 0,
  171. ValidatorAddress: val.Address,
  172. Height: 2,
  173. Round: 0,
  174. Timestamp: time.Now().UTC(),
  175. Type: types.VoteTypePrevote,
  176. BlockID: types.BlockID{},
  177. }
  178. bz, err := cdc.MarshalBinaryBare(&VoteMessage{vote})
  179. require.NoError(t, err)
  180. reactor.Receive(VoteChannel, peer, bz)
  181. assert.Equal(t, 1, ps.VotesSent(), "number of votes sent should have increased by 1")
  182. // 2) vote with the same height, but different round
  183. vote.Round = 1
  184. bz, err = cdc.MarshalBinaryBare(&VoteMessage{vote})
  185. require.NoError(t, err)
  186. reactor.Receive(VoteChannel, peer, bz)
  187. assert.Equal(t, 1, ps.VotesSent(), "number of votes sent should stay the same")
  188. // 3) vote from earlier height
  189. vote.Height = 1
  190. vote.Round = 0
  191. bz, err = cdc.MarshalBinaryBare(&VoteMessage{vote})
  192. require.NoError(t, err)
  193. reactor.Receive(VoteChannel, peer, bz)
  194. assert.Equal(t, 1, ps.VotesSent(), "number of votes sent should stay the same")
  195. }
  196. //-------------------------------------------------------------
  197. // ensure we can make blocks despite cycling a validator set
  198. func TestReactorVotingPowerChange(t *testing.T) {
  199. nVals := 4
  200. logger := log.TestingLogger()
  201. css := randConsensusNet(nVals, "consensus_voting_power_changes_test", newMockTickerFunc(true), newPersistentKVStore)
  202. reactors, eventChans, eventBuses := startConsensusNet(t, css, nVals)
  203. defer stopConsensusNet(logger, reactors, eventBuses)
  204. // map of active validators
  205. activeVals := make(map[string]struct{})
  206. for i := 0; i < nVals; i++ {
  207. activeVals[string(css[i].privValidator.GetAddress())] = struct{}{}
  208. }
  209. // wait till everyone makes block 1
  210. timeoutWaitGroup(t, nVals, func(j int) {
  211. <-eventChans[j]
  212. }, css)
  213. //---------------------------------------------------------------------------
  214. logger.Debug("---------------------------- Testing changing the voting power of one validator a few times")
  215. val1PubKey := css[0].privValidator.GetPubKey()
  216. val1PubKeyABCI := types.TM2PB.PubKey(val1PubKey)
  217. updateValidatorTx := kvstore.MakeValSetChangeTx(val1PubKeyABCI, 25)
  218. previousTotalVotingPower := css[0].GetRoundState().LastValidators.TotalVotingPower()
  219. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  220. waitForAndValidateBlockWithTx(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  221. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  222. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  223. if css[0].GetRoundState().LastValidators.TotalVotingPower() == previousTotalVotingPower {
  224. t.Fatalf("expected voting power to change (before: %d, after: %d)", previousTotalVotingPower, css[0].GetRoundState().LastValidators.TotalVotingPower())
  225. }
  226. updateValidatorTx = kvstore.MakeValSetChangeTx(val1PubKeyABCI, 2)
  227. previousTotalVotingPower = css[0].GetRoundState().LastValidators.TotalVotingPower()
  228. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  229. waitForAndValidateBlockWithTx(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  230. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  231. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  232. if css[0].GetRoundState().LastValidators.TotalVotingPower() == previousTotalVotingPower {
  233. t.Fatalf("expected voting power to change (before: %d, after: %d)", previousTotalVotingPower, css[0].GetRoundState().LastValidators.TotalVotingPower())
  234. }
  235. updateValidatorTx = kvstore.MakeValSetChangeTx(val1PubKeyABCI, 26)
  236. previousTotalVotingPower = css[0].GetRoundState().LastValidators.TotalVotingPower()
  237. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  238. waitForAndValidateBlockWithTx(t, nVals, activeVals, eventChans, css, updateValidatorTx)
  239. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  240. waitForAndValidateBlock(t, nVals, activeVals, eventChans, css)
  241. if css[0].GetRoundState().LastValidators.TotalVotingPower() == previousTotalVotingPower {
  242. t.Fatalf("expected voting power to change (before: %d, after: %d)", previousTotalVotingPower, css[0].GetRoundState().LastValidators.TotalVotingPower())
  243. }
  244. }
  245. func TestReactorValidatorSetChanges(t *testing.T) {
  246. nPeers := 7
  247. nVals := 4
  248. css := randConsensusNetWithPeers(nVals, nPeers, "consensus_val_set_changes_test", newMockTickerFunc(true), newPersistentKVStore)
  249. logger := log.TestingLogger()
  250. reactors, eventChans, eventBuses := startConsensusNet(t, css, nPeers)
  251. defer stopConsensusNet(logger, reactors, eventBuses)
  252. // map of active validators
  253. activeVals := make(map[string]struct{})
  254. for i := 0; i < nVals; i++ {
  255. activeVals[string(css[i].privValidator.GetAddress())] = struct{}{}
  256. }
  257. // wait till everyone makes block 1
  258. timeoutWaitGroup(t, nPeers, func(j int) {
  259. <-eventChans[j]
  260. }, css)
  261. //---------------------------------------------------------------------------
  262. logger.Info("---------------------------- Testing adding one validator")
  263. newValidatorPubKey1 := css[nVals].privValidator.GetPubKey()
  264. valPubKey1ABCI := types.TM2PB.PubKey(newValidatorPubKey1)
  265. newValidatorTx1 := kvstore.MakeValSetChangeTx(valPubKey1ABCI, testMinPower)
  266. // wait till everyone makes block 2
  267. // ensure the commit includes all validators
  268. // send newValTx to change vals in block 3
  269. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css, newValidatorTx1)
  270. // wait till everyone makes block 3.
  271. // it includes the commit for block 2, which is by the original validator set
  272. waitForAndValidateBlockWithTx(t, nPeers, activeVals, eventChans, css, newValidatorTx1)
  273. // wait till everyone makes block 4.
  274. // it includes the commit for block 3, which is by the original validator set
  275. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
  276. // the commits for block 4 should be with the updated validator set
  277. activeVals[string(newValidatorPubKey1.Address())] = struct{}{}
  278. // wait till everyone makes block 5
  279. // it includes the commit for block 4, which should have the updated validator set
  280. waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
  281. //---------------------------------------------------------------------------
  282. logger.Info("---------------------------- Testing changing the voting power of one validator")
  283. updateValidatorPubKey1 := css[nVals].privValidator.GetPubKey()
  284. updatePubKey1ABCI := types.TM2PB.PubKey(updateValidatorPubKey1)
  285. updateValidatorTx1 := kvstore.MakeValSetChangeTx(updatePubKey1ABCI, 25)
  286. previousTotalVotingPower := css[nVals].GetRoundState().LastValidators.TotalVotingPower()
  287. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css, updateValidatorTx1)
  288. waitForAndValidateBlockWithTx(t, nPeers, activeVals, eventChans, css, updateValidatorTx1)
  289. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
  290. waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
  291. if css[nVals].GetRoundState().LastValidators.TotalVotingPower() == previousTotalVotingPower {
  292. t.Errorf("expected voting power to change (before: %d, after: %d)", previousTotalVotingPower, css[nVals].GetRoundState().LastValidators.TotalVotingPower())
  293. }
  294. //---------------------------------------------------------------------------
  295. logger.Info("---------------------------- Testing adding two validators at once")
  296. newValidatorPubKey2 := css[nVals+1].privValidator.GetPubKey()
  297. newVal2ABCI := types.TM2PB.PubKey(newValidatorPubKey2)
  298. newValidatorTx2 := kvstore.MakeValSetChangeTx(newVal2ABCI, testMinPower)
  299. newValidatorPubKey3 := css[nVals+2].privValidator.GetPubKey()
  300. newVal3ABCI := types.TM2PB.PubKey(newValidatorPubKey3)
  301. newValidatorTx3 := kvstore.MakeValSetChangeTx(newVal3ABCI, testMinPower)
  302. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css, newValidatorTx2, newValidatorTx3)
  303. waitForAndValidateBlockWithTx(t, nPeers, activeVals, eventChans, css, newValidatorTx2, newValidatorTx3)
  304. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
  305. activeVals[string(newValidatorPubKey2.Address())] = struct{}{}
  306. activeVals[string(newValidatorPubKey3.Address())] = struct{}{}
  307. waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
  308. //---------------------------------------------------------------------------
  309. logger.Info("---------------------------- Testing removing two validators at once")
  310. removeValidatorTx2 := kvstore.MakeValSetChangeTx(newVal2ABCI, 0)
  311. removeValidatorTx3 := kvstore.MakeValSetChangeTx(newVal3ABCI, 0)
  312. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css, removeValidatorTx2, removeValidatorTx3)
  313. waitForAndValidateBlockWithTx(t, nPeers, activeVals, eventChans, css, removeValidatorTx2, removeValidatorTx3)
  314. waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
  315. delete(activeVals, string(newValidatorPubKey2.Address()))
  316. delete(activeVals, string(newValidatorPubKey3.Address()))
  317. waitForBlockWithUpdatedValsAndValidateIt(t, nPeers, activeVals, eventChans, css)
  318. }
  319. // Check we can make blocks with skip_timeout_commit=false
  320. func TestReactorWithTimeoutCommit(t *testing.T) {
  321. N := 4
  322. css := randConsensusNet(N, "consensus_reactor_with_timeout_commit_test", newMockTickerFunc(false), newCounter)
  323. // override default SkipTimeoutCommit == true for tests
  324. for i := 0; i < N; i++ {
  325. css[i].config.SkipTimeoutCommit = false
  326. }
  327. reactors, eventChans, eventBuses := startConsensusNet(t, css, N-1)
  328. defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
  329. // wait till everyone makes the first new block
  330. timeoutWaitGroup(t, N-1, func(j int) {
  331. <-eventChans[j]
  332. }, css)
  333. }
  334. func waitForAndValidateBlock(t *testing.T, n int, activeVals map[string]struct{}, eventChans []chan interface{}, css []*ConsensusState, txs ...[]byte) {
  335. timeoutWaitGroup(t, n, func(j int) {
  336. css[j].Logger.Debug("waitForAndValidateBlock")
  337. newBlockI, ok := <-eventChans[j]
  338. if !ok {
  339. return
  340. }
  341. newBlock := newBlockI.(types.EventDataNewBlock).Block
  342. css[j].Logger.Debug("waitForAndValidateBlock: Got block", "height", newBlock.Height)
  343. err := validateBlock(newBlock, activeVals)
  344. assert.Nil(t, err)
  345. for _, tx := range txs {
  346. css[j].mempool.CheckTx(tx, nil)
  347. assert.Nil(t, err)
  348. }
  349. }, css)
  350. }
  351. func waitForAndValidateBlockWithTx(t *testing.T, n int, activeVals map[string]struct{}, eventChans []chan interface{}, css []*ConsensusState, txs ...[]byte) {
  352. timeoutWaitGroup(t, n, func(j int) {
  353. ntxs := 0
  354. BLOCK_TX_LOOP:
  355. for {
  356. css[j].Logger.Debug("waitForAndValidateBlockWithTx", "ntxs", ntxs)
  357. newBlockI, ok := <-eventChans[j]
  358. if !ok {
  359. return
  360. }
  361. newBlock := newBlockI.(types.EventDataNewBlock).Block
  362. css[j].Logger.Debug("waitForAndValidateBlockWithTx: Got block", "height", newBlock.Height)
  363. err := validateBlock(newBlock, activeVals)
  364. assert.Nil(t, err)
  365. // check that txs match the txs we're waiting for.
  366. // note they could be spread over multiple blocks,
  367. // but they should be in order.
  368. for _, tx := range newBlock.Data.Txs {
  369. assert.EqualValues(t, txs[ntxs], tx)
  370. ntxs++
  371. }
  372. if ntxs == len(txs) {
  373. break BLOCK_TX_LOOP
  374. }
  375. }
  376. }, css)
  377. }
  378. func waitForBlockWithUpdatedValsAndValidateIt(t *testing.T, n int, updatedVals map[string]struct{}, eventChans []chan interface{}, css []*ConsensusState) {
  379. timeoutWaitGroup(t, n, func(j int) {
  380. var newBlock *types.Block
  381. LOOP:
  382. for {
  383. css[j].Logger.Debug("waitForBlockWithUpdatedValsAndValidateIt")
  384. newBlockI, ok := <-eventChans[j]
  385. if !ok {
  386. return
  387. }
  388. newBlock = newBlockI.(types.EventDataNewBlock).Block
  389. if newBlock.LastCommit.Size() == len(updatedVals) {
  390. css[j].Logger.Debug("waitForBlockWithUpdatedValsAndValidateIt: Got block", "height", newBlock.Height)
  391. break LOOP
  392. } else {
  393. css[j].Logger.Debug("waitForBlockWithUpdatedValsAndValidateIt: Got block with no new validators. Skipping", "height", newBlock.Height)
  394. }
  395. }
  396. err := validateBlock(newBlock, updatedVals)
  397. assert.Nil(t, err)
  398. }, css)
  399. }
  400. // expects high synchrony!
  401. func validateBlock(block *types.Block, activeVals map[string]struct{}) error {
  402. if block.LastCommit.Size() != len(activeVals) {
  403. return fmt.Errorf("Commit size doesn't match number of active validators. Got %d, expected %d", block.LastCommit.Size(), len(activeVals))
  404. }
  405. for _, vote := range block.LastCommit.Precommits {
  406. if _, ok := activeVals[string(vote.ValidatorAddress)]; !ok {
  407. return fmt.Errorf("Found vote for unactive validator %X", vote.ValidatorAddress)
  408. }
  409. }
  410. return nil
  411. }
  412. func timeoutWaitGroup(t *testing.T, n int, f func(int), css []*ConsensusState) {
  413. wg := new(sync.WaitGroup)
  414. wg.Add(n)
  415. for i := 0; i < n; i++ {
  416. go func(j int) {
  417. f(j)
  418. wg.Done()
  419. }(i)
  420. }
  421. done := make(chan struct{})
  422. go func() {
  423. wg.Wait()
  424. close(done)
  425. }()
  426. // we're running many nodes in-process, possibly in in a virtual machine,
  427. // and spewing debug messages - making a block could take a while,
  428. timeout := time.Second * 300
  429. select {
  430. case <-done:
  431. case <-time.After(timeout):
  432. for i, cs := range css {
  433. t.Log("#################")
  434. t.Log("Validator", i)
  435. t.Log(cs.GetRoundState())
  436. t.Log("")
  437. }
  438. os.Stdout.Write([]byte("pprof.Lookup('goroutine'):\n"))
  439. pprof.Lookup("goroutine").WriteTo(os.Stdout, 1)
  440. capture()
  441. panic("Timed out waiting for all validators to commit a block")
  442. }
  443. }
  444. func capture() {
  445. trace := make([]byte, 10240000)
  446. count := runtime.Stack(trace, true)
  447. fmt.Printf("Stack of %d bytes: %s\n", count, trace)
  448. }