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.

855 lines
24 KiB

  1. package statesync
  2. import (
  3. "context"
  4. "fmt"
  5. "strings"
  6. "sync"
  7. "testing"
  8. "time"
  9. "github.com/fortytw2/leaktest"
  10. "github.com/stretchr/testify/mock"
  11. "github.com/stretchr/testify/require"
  12. dbm "github.com/tendermint/tm-db"
  13. abci "github.com/tendermint/tendermint/abci/types"
  14. "github.com/tendermint/tendermint/config"
  15. "github.com/tendermint/tendermint/internal/p2p"
  16. "github.com/tendermint/tendermint/internal/proxy"
  17. proxymocks "github.com/tendermint/tendermint/internal/proxy/mocks"
  18. smmocks "github.com/tendermint/tendermint/internal/state/mocks"
  19. "github.com/tendermint/tendermint/internal/statesync/mocks"
  20. "github.com/tendermint/tendermint/internal/store"
  21. "github.com/tendermint/tendermint/internal/test/factory"
  22. "github.com/tendermint/tendermint/libs/log"
  23. "github.com/tendermint/tendermint/light/provider"
  24. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  25. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  26. "github.com/tendermint/tendermint/types"
  27. )
  28. var (
  29. m = PrometheusMetrics(config.TestConfig().Instrumentation.Namespace)
  30. )
  31. type reactorTestSuite struct {
  32. reactor *Reactor
  33. syncer *syncer
  34. conn *proxymocks.AppConnSnapshot
  35. connQuery *proxymocks.AppConnQuery
  36. stateProvider *mocks.StateProvider
  37. snapshotChannel *p2p.Channel
  38. snapshotInCh chan p2p.Envelope
  39. snapshotOutCh chan p2p.Envelope
  40. snapshotPeerErrCh chan p2p.PeerError
  41. chunkChannel *p2p.Channel
  42. chunkInCh chan p2p.Envelope
  43. chunkOutCh chan p2p.Envelope
  44. chunkPeerErrCh chan p2p.PeerError
  45. blockChannel *p2p.Channel
  46. blockInCh chan p2p.Envelope
  47. blockOutCh chan p2p.Envelope
  48. blockPeerErrCh chan p2p.PeerError
  49. paramsChannel *p2p.Channel
  50. paramsInCh chan p2p.Envelope
  51. paramsOutCh chan p2p.Envelope
  52. paramsPeerErrCh chan p2p.PeerError
  53. peerUpdateCh chan p2p.PeerUpdate
  54. peerUpdates *p2p.PeerUpdates
  55. stateStore *smmocks.Store
  56. blockStore *store.BlockStore
  57. }
  58. func setup(
  59. t *testing.T,
  60. conn *proxymocks.AppConnSnapshot,
  61. connQuery *proxymocks.AppConnQuery,
  62. stateProvider *mocks.StateProvider,
  63. chBuf uint,
  64. ) *reactorTestSuite {
  65. t.Helper()
  66. if conn == nil {
  67. conn = &proxymocks.AppConnSnapshot{}
  68. }
  69. if connQuery == nil {
  70. connQuery = &proxymocks.AppConnQuery{}
  71. }
  72. if stateProvider == nil {
  73. stateProvider = &mocks.StateProvider{}
  74. }
  75. rts := &reactorTestSuite{
  76. snapshotInCh: make(chan p2p.Envelope, chBuf),
  77. snapshotOutCh: make(chan p2p.Envelope, chBuf),
  78. snapshotPeerErrCh: make(chan p2p.PeerError, chBuf),
  79. chunkInCh: make(chan p2p.Envelope, chBuf),
  80. chunkOutCh: make(chan p2p.Envelope, chBuf),
  81. chunkPeerErrCh: make(chan p2p.PeerError, chBuf),
  82. blockInCh: make(chan p2p.Envelope, chBuf),
  83. blockOutCh: make(chan p2p.Envelope, chBuf),
  84. blockPeerErrCh: make(chan p2p.PeerError, chBuf),
  85. paramsInCh: make(chan p2p.Envelope, chBuf),
  86. paramsOutCh: make(chan p2p.Envelope, chBuf),
  87. paramsPeerErrCh: make(chan p2p.PeerError, chBuf),
  88. conn: conn,
  89. connQuery: connQuery,
  90. stateProvider: stateProvider,
  91. }
  92. rts.peerUpdateCh = make(chan p2p.PeerUpdate, chBuf)
  93. rts.peerUpdates = p2p.NewPeerUpdates(rts.peerUpdateCh, int(chBuf))
  94. rts.snapshotChannel = p2p.NewChannel(
  95. SnapshotChannel,
  96. new(ssproto.Message),
  97. rts.snapshotInCh,
  98. rts.snapshotOutCh,
  99. rts.snapshotPeerErrCh,
  100. )
  101. rts.chunkChannel = p2p.NewChannel(
  102. ChunkChannel,
  103. new(ssproto.Message),
  104. rts.chunkInCh,
  105. rts.chunkOutCh,
  106. rts.chunkPeerErrCh,
  107. )
  108. rts.blockChannel = p2p.NewChannel(
  109. LightBlockChannel,
  110. new(ssproto.Message),
  111. rts.blockInCh,
  112. rts.blockOutCh,
  113. rts.blockPeerErrCh,
  114. )
  115. rts.paramsChannel = p2p.NewChannel(
  116. ParamsChannel,
  117. new(ssproto.Message),
  118. rts.paramsInCh,
  119. rts.paramsOutCh,
  120. rts.paramsPeerErrCh,
  121. )
  122. rts.stateStore = &smmocks.Store{}
  123. rts.blockStore = store.NewBlockStore(dbm.NewMemDB())
  124. cfg := config.DefaultStateSyncConfig()
  125. rts.reactor = NewReactor(
  126. factory.DefaultTestChainID,
  127. 1,
  128. *cfg,
  129. log.TestingLogger(),
  130. conn,
  131. connQuery,
  132. rts.snapshotChannel,
  133. rts.chunkChannel,
  134. rts.blockChannel,
  135. rts.paramsChannel,
  136. rts.peerUpdates,
  137. rts.stateStore,
  138. rts.blockStore,
  139. "",
  140. m,
  141. )
  142. rts.syncer = newSyncer(
  143. *cfg,
  144. log.NewNopLogger(),
  145. conn,
  146. connQuery,
  147. stateProvider,
  148. rts.snapshotOutCh,
  149. rts.chunkOutCh,
  150. rts.snapshotChannel.Done(),
  151. "",
  152. rts.reactor.metrics,
  153. )
  154. require.NoError(t, rts.reactor.Start())
  155. require.True(t, rts.reactor.IsRunning())
  156. t.Cleanup(func() {
  157. require.NoError(t, rts.reactor.Stop())
  158. require.False(t, rts.reactor.IsRunning())
  159. })
  160. return rts
  161. }
  162. func TestReactor_Sync(t *testing.T) {
  163. const snapshotHeight = 7
  164. rts := setup(t, nil, nil, nil, 2)
  165. chain := buildLightBlockChain(t, 1, 10, time.Now())
  166. // app accepts any snapshot
  167. rts.conn.On("OfferSnapshotSync", ctx, mock.AnythingOfType("types.RequestOfferSnapshot")).
  168. Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, nil)
  169. // app accepts every chunk
  170. rts.conn.On("ApplySnapshotChunkSync", ctx, mock.AnythingOfType("types.RequestApplySnapshotChunk")).
  171. Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  172. // app query returns valid state app hash
  173. rts.connQuery.On("InfoSync", ctx, proxy.RequestInfo).Return(&abci.ResponseInfo{
  174. AppVersion: 9,
  175. LastBlockHeight: snapshotHeight,
  176. LastBlockAppHash: chain[snapshotHeight+1].AppHash,
  177. }, nil)
  178. // store accepts state and validator sets
  179. rts.stateStore.On("Bootstrap", mock.AnythingOfType("state.State")).Return(nil)
  180. rts.stateStore.On("SaveValidatorSets", mock.AnythingOfType("int64"), mock.AnythingOfType("int64"),
  181. mock.AnythingOfType("*types.ValidatorSet")).Return(nil)
  182. closeCh := make(chan struct{})
  183. defer close(closeCh)
  184. go handleLightBlockRequests(t, chain, rts.blockOutCh,
  185. rts.blockInCh, closeCh, 0)
  186. go graduallyAddPeers(rts.peerUpdateCh, closeCh, 1*time.Second)
  187. go handleSnapshotRequests(t, rts.snapshotOutCh, rts.snapshotInCh, closeCh, []snapshot{
  188. {
  189. Height: uint64(snapshotHeight),
  190. Format: 1,
  191. Chunks: 1,
  192. },
  193. })
  194. go handleChunkRequests(t, rts.chunkOutCh, rts.chunkInCh, closeCh, []byte("abc"))
  195. go handleConsensusParamsRequest(t, rts.paramsOutCh, rts.paramsInCh, closeCh)
  196. // update the config to use the p2p provider
  197. rts.reactor.cfg.UseP2P = true
  198. rts.reactor.cfg.TrustHeight = 1
  199. rts.reactor.cfg.TrustHash = fmt.Sprintf("%X", chain[1].Hash())
  200. rts.reactor.cfg.DiscoveryTime = 1 * time.Second
  201. // Run state sync
  202. _, err := rts.reactor.Sync(context.Background())
  203. require.NoError(t, err)
  204. }
  205. func TestReactor_ChunkRequest_InvalidRequest(t *testing.T) {
  206. rts := setup(t, nil, nil, nil, 2)
  207. rts.chunkInCh <- p2p.Envelope{
  208. From: types.NodeID("aa"),
  209. Message: &ssproto.SnapshotsRequest{},
  210. }
  211. response := <-rts.chunkPeerErrCh
  212. require.Error(t, response.Err)
  213. require.Empty(t, rts.chunkOutCh)
  214. require.Contains(t, response.Err.Error(), "received unknown message")
  215. require.Equal(t, types.NodeID("aa"), response.NodeID)
  216. }
  217. func TestReactor_ChunkRequest(t *testing.T) {
  218. testcases := map[string]struct {
  219. request *ssproto.ChunkRequest
  220. chunk []byte
  221. expectResponse *ssproto.ChunkResponse
  222. }{
  223. "chunk is returned": {
  224. &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
  225. []byte{1, 2, 3},
  226. &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 2, 3}},
  227. },
  228. "empty chunk is returned, as empty": {
  229. &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
  230. []byte{},
  231. &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Chunk: []byte{}},
  232. },
  233. "nil (missing) chunk is returned as missing": {
  234. &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
  235. nil,
  236. &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true},
  237. },
  238. "invalid request": {
  239. &ssproto.ChunkRequest{Height: 1, Format: 1, Index: 1},
  240. nil,
  241. &ssproto.ChunkResponse{Height: 1, Format: 1, Index: 1, Missing: true},
  242. },
  243. }
  244. for name, tc := range testcases {
  245. tc := tc
  246. t.Run(name, func(t *testing.T) {
  247. // mock ABCI connection to return local snapshots
  248. conn := &proxymocks.AppConnSnapshot{}
  249. conn.On("LoadSnapshotChunkSync", context.Background(), abci.RequestLoadSnapshotChunk{
  250. Height: tc.request.Height,
  251. Format: tc.request.Format,
  252. Chunk: tc.request.Index,
  253. }).Return(&abci.ResponseLoadSnapshotChunk{Chunk: tc.chunk}, nil)
  254. rts := setup(t, conn, nil, nil, 2)
  255. rts.chunkInCh <- p2p.Envelope{
  256. From: types.NodeID("aa"),
  257. Message: tc.request,
  258. }
  259. response := <-rts.chunkOutCh
  260. require.Equal(t, tc.expectResponse, response.Message)
  261. require.Empty(t, rts.chunkOutCh)
  262. conn.AssertExpectations(t)
  263. })
  264. }
  265. }
  266. func TestReactor_SnapshotsRequest_InvalidRequest(t *testing.T) {
  267. rts := setup(t, nil, nil, nil, 2)
  268. rts.snapshotInCh <- p2p.Envelope{
  269. From: types.NodeID("aa"),
  270. Message: &ssproto.ChunkRequest{},
  271. }
  272. response := <-rts.snapshotPeerErrCh
  273. require.Error(t, response.Err)
  274. require.Empty(t, rts.snapshotOutCh)
  275. require.Contains(t, response.Err.Error(), "received unknown message")
  276. require.Equal(t, types.NodeID("aa"), response.NodeID)
  277. }
  278. func TestReactor_SnapshotsRequest(t *testing.T) {
  279. testcases := map[string]struct {
  280. snapshots []*abci.Snapshot
  281. expectResponses []*ssproto.SnapshotsResponse
  282. }{
  283. "no snapshots": {nil, []*ssproto.SnapshotsResponse{}},
  284. ">10 unordered snapshots": {
  285. []*abci.Snapshot{
  286. {Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1}},
  287. {Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}},
  288. {Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}},
  289. {Height: 1, Format: 1, Chunks: 7, Hash: []byte{1, 1}, Metadata: []byte{4}},
  290. {Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}},
  291. {Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}},
  292. {Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}},
  293. {Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}},
  294. {Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}},
  295. {Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}},
  296. {Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}},
  297. {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
  298. },
  299. []*ssproto.SnapshotsResponse{
  300. {Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}},
  301. {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
  302. {Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}},
  303. {Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}},
  304. {Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}},
  305. {Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}},
  306. {Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}},
  307. {Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}},
  308. {Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}},
  309. {Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}},
  310. },
  311. },
  312. }
  313. for name, tc := range testcases {
  314. tc := tc
  315. t.Run(name, func(t *testing.T) {
  316. // mock ABCI connection to return local snapshots
  317. conn := &proxymocks.AppConnSnapshot{}
  318. conn.On("ListSnapshotsSync", context.Background(), abci.RequestListSnapshots{}).Return(&abci.ResponseListSnapshots{
  319. Snapshots: tc.snapshots,
  320. }, nil)
  321. rts := setup(t, conn, nil, nil, 100)
  322. rts.snapshotInCh <- p2p.Envelope{
  323. From: types.NodeID("aa"),
  324. Message: &ssproto.SnapshotsRequest{},
  325. }
  326. if len(tc.expectResponses) > 0 {
  327. retryUntil(t, func() bool { return len(rts.snapshotOutCh) == len(tc.expectResponses) }, time.Second)
  328. }
  329. responses := make([]*ssproto.SnapshotsResponse, len(tc.expectResponses))
  330. for i := 0; i < len(tc.expectResponses); i++ {
  331. e := <-rts.snapshotOutCh
  332. responses[i] = e.Message.(*ssproto.SnapshotsResponse)
  333. }
  334. require.Equal(t, tc.expectResponses, responses)
  335. require.Empty(t, rts.snapshotOutCh)
  336. })
  337. }
  338. }
  339. func TestReactor_LightBlockResponse(t *testing.T) {
  340. rts := setup(t, nil, nil, nil, 2)
  341. var height int64 = 10
  342. h := factory.MakeRandomHeader()
  343. h.Height = height
  344. blockID := factory.MakeBlockIDWithHash(h.Hash())
  345. vals, pv := factory.RandValidatorSet(1, 10)
  346. vote, err := factory.MakeVote(pv[0], h.ChainID, 0, h.Height, 0, 2,
  347. blockID, factory.DefaultTestTime)
  348. require.NoError(t, err)
  349. sh := &types.SignedHeader{
  350. Header: h,
  351. Commit: &types.Commit{
  352. Height: h.Height,
  353. BlockID: blockID,
  354. Signatures: []types.CommitSig{
  355. vote.CommitSig(),
  356. },
  357. },
  358. }
  359. lb := &types.LightBlock{
  360. SignedHeader: sh,
  361. ValidatorSet: vals,
  362. }
  363. require.NoError(t, rts.blockStore.SaveSignedHeader(sh, blockID))
  364. rts.stateStore.On("LoadValidators", height).Return(vals, nil)
  365. rts.blockInCh <- p2p.Envelope{
  366. From: types.NodeID("aa"),
  367. Message: &ssproto.LightBlockRequest{
  368. Height: 10,
  369. },
  370. }
  371. require.Empty(t, rts.blockPeerErrCh)
  372. select {
  373. case response := <-rts.blockOutCh:
  374. require.Equal(t, types.NodeID("aa"), response.To)
  375. res, ok := response.Message.(*ssproto.LightBlockResponse)
  376. require.True(t, ok)
  377. receivedLB, err := types.LightBlockFromProto(res.LightBlock)
  378. require.NoError(t, err)
  379. require.Equal(t, lb, receivedLB)
  380. case <-time.After(1 * time.Second):
  381. t.Fatal("expected light block response")
  382. }
  383. }
  384. func TestReactor_BlockProviders(t *testing.T) {
  385. rts := setup(t, nil, nil, nil, 2)
  386. rts.peerUpdateCh <- p2p.PeerUpdate{
  387. NodeID: types.NodeID("aa"),
  388. Status: p2p.PeerStatusUp,
  389. }
  390. rts.peerUpdateCh <- p2p.PeerUpdate{
  391. NodeID: types.NodeID("bb"),
  392. Status: p2p.PeerStatusUp,
  393. }
  394. closeCh := make(chan struct{})
  395. defer close(closeCh)
  396. chain := buildLightBlockChain(t, 1, 10, time.Now())
  397. go handleLightBlockRequests(t, chain, rts.blockOutCh, rts.blockInCh, closeCh, 0)
  398. peers := rts.reactor.peers.All()
  399. require.Len(t, peers, 2)
  400. providers := make([]provider.Provider, len(peers))
  401. for idx, peer := range peers {
  402. providers[idx] = NewBlockProvider(peer, factory.DefaultTestChainID, rts.reactor.dispatcher)
  403. }
  404. wg := sync.WaitGroup{}
  405. for _, p := range providers {
  406. wg.Add(1)
  407. go func(t *testing.T, p provider.Provider) {
  408. defer wg.Done()
  409. for height := 2; height < 10; height++ {
  410. lb, err := p.LightBlock(context.Background(), int64(height))
  411. require.NoError(t, err)
  412. require.NotNil(t, lb)
  413. require.Equal(t, height, int(lb.Height))
  414. }
  415. }(t, p)
  416. }
  417. ctx, cancel := context.WithCancel(context.Background())
  418. go func() { wg.Wait(); cancel() }()
  419. select {
  420. case <-time.After(time.Second):
  421. // not all of the requests to the dispatcher were responded to
  422. // within the timeout
  423. t.Fail()
  424. case <-ctx.Done():
  425. }
  426. }
  427. func TestReactor_StateProviderP2P(t *testing.T) {
  428. rts := setup(t, nil, nil, nil, 2)
  429. // make syncer non nil else test won't think we are state syncing
  430. rts.reactor.syncer = rts.syncer
  431. peerA := types.NodeID(strings.Repeat("a", 2*types.NodeIDByteLength))
  432. peerB := types.NodeID(strings.Repeat("b", 2*types.NodeIDByteLength))
  433. rts.peerUpdateCh <- p2p.PeerUpdate{
  434. NodeID: peerA,
  435. Status: p2p.PeerStatusUp,
  436. }
  437. rts.peerUpdateCh <- p2p.PeerUpdate{
  438. NodeID: peerB,
  439. Status: p2p.PeerStatusUp,
  440. }
  441. closeCh := make(chan struct{})
  442. defer close(closeCh)
  443. chain := buildLightBlockChain(t, 1, 10, time.Now())
  444. go handleLightBlockRequests(t, chain, rts.blockOutCh, rts.blockInCh, closeCh, 0)
  445. go handleConsensusParamsRequest(t, rts.paramsOutCh, rts.paramsInCh, closeCh)
  446. rts.reactor.cfg.UseP2P = true
  447. rts.reactor.cfg.TrustHeight = 1
  448. rts.reactor.cfg.TrustHash = fmt.Sprintf("%X", chain[1].Hash())
  449. for _, p := range []types.NodeID{peerA, peerB} {
  450. if !rts.reactor.peers.Contains(p) {
  451. rts.reactor.peers.Append(p)
  452. }
  453. }
  454. require.True(t, rts.reactor.peers.Len() >= 2, "peer network not configured")
  455. bctx, cancel := context.WithCancel(context.Background())
  456. defer cancel()
  457. ictx, cancel := context.WithTimeout(bctx, time.Second)
  458. defer cancel()
  459. rts.reactor.mtx.Lock()
  460. err := rts.reactor.initStateProvider(ictx, factory.DefaultTestChainID, 1)
  461. rts.reactor.mtx.Unlock()
  462. require.NoError(t, err)
  463. rts.reactor.syncer.stateProvider = rts.reactor.stateProvider
  464. actx, cancel := context.WithTimeout(bctx, 10*time.Second)
  465. defer cancel()
  466. appHash, err := rts.reactor.stateProvider.AppHash(actx, 5)
  467. require.NoError(t, err)
  468. require.Len(t, appHash, 32)
  469. state, err := rts.reactor.stateProvider.State(actx, 5)
  470. require.NoError(t, err)
  471. require.Equal(t, appHash, state.AppHash)
  472. require.Equal(t, types.DefaultConsensusParams(), &state.ConsensusParams)
  473. commit, err := rts.reactor.stateProvider.Commit(actx, 5)
  474. require.NoError(t, err)
  475. require.Equal(t, commit.BlockID, state.LastBlockID)
  476. added, err := rts.reactor.syncer.AddSnapshot(peerA, &snapshot{
  477. Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1},
  478. })
  479. require.NoError(t, err)
  480. require.True(t, added)
  481. }
  482. func TestReactor_Backfill(t *testing.T) {
  483. // test backfill algorithm with varying failure rates [0, 10]
  484. failureRates := []int{0, 2, 9}
  485. for _, failureRate := range failureRates {
  486. failureRate := failureRate
  487. t.Run(fmt.Sprintf("failure rate: %d", failureRate), func(t *testing.T) {
  488. t.Cleanup(leaktest.CheckTimeout(t, 1*time.Minute))
  489. rts := setup(t, nil, nil, nil, 21)
  490. var (
  491. startHeight int64 = 20
  492. stopHeight int64 = 10
  493. stopTime = time.Date(2020, 1, 1, 0, 100, 0, 0, time.UTC)
  494. )
  495. peers := []string{"a", "b", "c", "d"}
  496. for _, peer := range peers {
  497. rts.peerUpdateCh <- p2p.PeerUpdate{
  498. NodeID: types.NodeID(peer),
  499. Status: p2p.PeerStatusUp,
  500. }
  501. }
  502. trackingHeight := startHeight
  503. rts.stateStore.On("SaveValidatorSets", mock.AnythingOfType("int64"), mock.AnythingOfType("int64"),
  504. mock.AnythingOfType("*types.ValidatorSet")).Return(func(lh, uh int64, vals *types.ValidatorSet) error {
  505. require.Equal(t, trackingHeight, lh)
  506. require.Equal(t, lh, uh)
  507. require.GreaterOrEqual(t, lh, stopHeight)
  508. trackingHeight--
  509. return nil
  510. })
  511. chain := buildLightBlockChain(t, stopHeight-1, startHeight+1, stopTime)
  512. closeCh := make(chan struct{})
  513. defer close(closeCh)
  514. go handleLightBlockRequests(t, chain, rts.blockOutCh,
  515. rts.blockInCh, closeCh, failureRate)
  516. err := rts.reactor.backfill(
  517. context.Background(),
  518. factory.DefaultTestChainID,
  519. startHeight,
  520. stopHeight,
  521. 1,
  522. factory.MakeBlockIDWithHash(chain[startHeight].Header.Hash()),
  523. stopTime,
  524. )
  525. if failureRate > 3 {
  526. require.Error(t, err)
  527. require.NotEqual(t, rts.reactor.backfilledBlocks, rts.reactor.backfillBlockTotal)
  528. require.Equal(t, startHeight-stopHeight+1, rts.reactor.backfillBlockTotal)
  529. } else {
  530. require.NoError(t, err)
  531. for height := startHeight; height <= stopHeight; height++ {
  532. blockMeta := rts.blockStore.LoadBlockMeta(height)
  533. require.NotNil(t, blockMeta)
  534. }
  535. require.Nil(t, rts.blockStore.LoadBlockMeta(stopHeight-1))
  536. require.Nil(t, rts.blockStore.LoadBlockMeta(startHeight+1))
  537. require.Equal(t, startHeight-stopHeight+1, rts.reactor.backfilledBlocks)
  538. require.Equal(t, startHeight-stopHeight+1, rts.reactor.backfillBlockTotal)
  539. }
  540. require.Equal(t, rts.reactor.backfilledBlocks, rts.reactor.BackFilledBlocks())
  541. require.Equal(t, rts.reactor.backfillBlockTotal, rts.reactor.BackFillBlocksTotal())
  542. })
  543. }
  544. }
  545. // retryUntil will continue to evaluate fn and will return successfully when true
  546. // or fail when the timeout is reached.
  547. func retryUntil(t *testing.T, fn func() bool, timeout time.Duration) {
  548. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  549. defer cancel()
  550. for {
  551. if fn() {
  552. return
  553. }
  554. require.NoError(t, ctx.Err())
  555. }
  556. }
  557. func handleLightBlockRequests(t *testing.T,
  558. chain map[int64]*types.LightBlock,
  559. receiving chan p2p.Envelope,
  560. sending chan p2p.Envelope,
  561. close chan struct{},
  562. failureRate int) {
  563. requests := 0
  564. errorCount := 0
  565. for {
  566. select {
  567. case envelope := <-receiving:
  568. if msg, ok := envelope.Message.(*ssproto.LightBlockRequest); ok {
  569. if requests%10 >= failureRate {
  570. lb, err := chain[int64(msg.Height)].ToProto()
  571. require.NoError(t, err)
  572. sending <- p2p.Envelope{
  573. From: envelope.To,
  574. Message: &ssproto.LightBlockResponse{
  575. LightBlock: lb,
  576. },
  577. }
  578. } else {
  579. switch errorCount % 3 {
  580. case 0: // send a different block
  581. vals, pv := factory.RandValidatorSet(3, 10)
  582. _, _, lb := mockLB(t, int64(msg.Height), factory.DefaultTestTime, factory.MakeBlockID(), vals, pv)
  583. differntLB, err := lb.ToProto()
  584. require.NoError(t, err)
  585. sending <- p2p.Envelope{
  586. From: envelope.To,
  587. Message: &ssproto.LightBlockResponse{
  588. LightBlock: differntLB,
  589. },
  590. }
  591. case 1: // send nil block i.e. pretend we don't have it
  592. sending <- p2p.Envelope{
  593. From: envelope.To,
  594. Message: &ssproto.LightBlockResponse{
  595. LightBlock: nil,
  596. },
  597. }
  598. case 2: // don't do anything
  599. }
  600. errorCount++
  601. }
  602. }
  603. case <-close:
  604. return
  605. }
  606. requests++
  607. }
  608. }
  609. func handleConsensusParamsRequest(t *testing.T, receiving, sending chan p2p.Envelope, closeCh chan struct{}) {
  610. t.Helper()
  611. params := types.DefaultConsensusParams()
  612. paramsProto := params.ToProto()
  613. for {
  614. select {
  615. case envelope := <-receiving:
  616. t.Log("received consensus params request")
  617. msg, ok := envelope.Message.(*ssproto.ParamsRequest)
  618. require.True(t, ok)
  619. sending <- p2p.Envelope{
  620. From: envelope.To,
  621. Message: &ssproto.ParamsResponse{
  622. Height: msg.Height,
  623. ConsensusParams: paramsProto,
  624. },
  625. }
  626. case <-closeCh:
  627. return
  628. }
  629. }
  630. }
  631. func buildLightBlockChain(t *testing.T, fromHeight, toHeight int64, startTime time.Time) map[int64]*types.LightBlock {
  632. chain := make(map[int64]*types.LightBlock, toHeight-fromHeight)
  633. lastBlockID := factory.MakeBlockID()
  634. blockTime := startTime.Add(time.Duration(fromHeight-toHeight) * time.Minute)
  635. vals, pv := factory.RandValidatorSet(3, 10)
  636. for height := fromHeight; height < toHeight; height++ {
  637. vals, pv, chain[height] = mockLB(t, height, blockTime, lastBlockID, vals, pv)
  638. lastBlockID = factory.MakeBlockIDWithHash(chain[height].Header.Hash())
  639. blockTime = blockTime.Add(1 * time.Minute)
  640. }
  641. return chain
  642. }
  643. func mockLB(t *testing.T, height int64, time time.Time, lastBlockID types.BlockID,
  644. currentVals *types.ValidatorSet, currentPrivVals []types.PrivValidator,
  645. ) (*types.ValidatorSet, []types.PrivValidator, *types.LightBlock) {
  646. header, err := factory.MakeHeader(&types.Header{
  647. Height: height,
  648. LastBlockID: lastBlockID,
  649. Time: time,
  650. })
  651. require.NoError(t, err)
  652. nextVals, nextPrivVals := factory.RandValidatorSet(3, 10)
  653. header.ValidatorsHash = currentVals.Hash()
  654. header.NextValidatorsHash = nextVals.Hash()
  655. header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams()
  656. lastBlockID = factory.MakeBlockIDWithHash(header.Hash())
  657. voteSet := types.NewVoteSet(factory.DefaultTestChainID, height, 0, tmproto.PrecommitType, currentVals)
  658. commit, err := factory.MakeCommit(lastBlockID, height, 0, voteSet, currentPrivVals, time)
  659. require.NoError(t, err)
  660. return nextVals, nextPrivVals, &types.LightBlock{
  661. SignedHeader: &types.SignedHeader{
  662. Header: header,
  663. Commit: commit,
  664. },
  665. ValidatorSet: currentVals,
  666. }
  667. }
  668. // graduallyAddPeers delivers a new randomly-generated peer update on peerUpdateCh once
  669. // per interval, until closeCh is closed. Each peer update is assigned a random node ID.
  670. func graduallyAddPeers(
  671. peerUpdateCh chan p2p.PeerUpdate,
  672. closeCh chan struct{},
  673. interval time.Duration,
  674. ) {
  675. ticker := time.NewTicker(interval)
  676. for {
  677. select {
  678. case <-ticker.C:
  679. peerUpdateCh <- p2p.PeerUpdate{
  680. NodeID: factory.RandomNodeID(),
  681. Status: p2p.PeerStatusUp,
  682. }
  683. case <-closeCh:
  684. return
  685. }
  686. }
  687. }
  688. func handleSnapshotRequests(
  689. t *testing.T,
  690. receivingCh chan p2p.Envelope,
  691. sendingCh chan p2p.Envelope,
  692. closeCh chan struct{},
  693. snapshots []snapshot,
  694. ) {
  695. t.Helper()
  696. for {
  697. select {
  698. case envelope := <-receivingCh:
  699. _, ok := envelope.Message.(*ssproto.SnapshotsRequest)
  700. require.True(t, ok)
  701. for _, snapshot := range snapshots {
  702. sendingCh <- p2p.Envelope{
  703. From: envelope.To,
  704. Message: &ssproto.SnapshotsResponse{
  705. Height: snapshot.Height,
  706. Format: snapshot.Format,
  707. Chunks: snapshot.Chunks,
  708. Hash: snapshot.Hash,
  709. Metadata: snapshot.Metadata,
  710. },
  711. }
  712. }
  713. case <-closeCh:
  714. return
  715. }
  716. }
  717. }
  718. func handleChunkRequests(
  719. t *testing.T,
  720. receivingCh chan p2p.Envelope,
  721. sendingCh chan p2p.Envelope,
  722. closeCh chan struct{},
  723. chunk []byte,
  724. ) {
  725. t.Helper()
  726. for {
  727. select {
  728. case envelope := <-receivingCh:
  729. msg, ok := envelope.Message.(*ssproto.ChunkRequest)
  730. require.True(t, ok)
  731. sendingCh <- p2p.Envelope{
  732. From: envelope.To,
  733. Message: &ssproto.ChunkResponse{
  734. Height: msg.Height,
  735. Format: msg.Format,
  736. Index: msg.Index,
  737. Chunk: chunk,
  738. Missing: false,
  739. },
  740. }
  741. case <-closeCh:
  742. return
  743. }
  744. }
  745. }