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.

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