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.

783 lines
27 KiB

statesync: remove deadlock on init fail (#7029) When statesync is stopped during shutdown, it has the possibility of deadlocking. A dump of goroutines reveals that this is related to the peerUpdates channel not returning anything on its `Done()` channel when `OnStop` is called. As this is occuring, `processPeerUpdate` is attempting to acquire the reactor lock. It appears that this lock can never be acquired. I looked for the places where the lock may remain locked accidentally and cleaned them up in hopes to eradicate the issue. Dumps of the relevant goroutines may be found below. Note that the line numbers below are relative to the code in the `v0.35.0-rc1` tag. ``` goroutine 36 [chan receive]: github.com/tendermint/tendermint/internal/statesync.(*Reactor).OnStop(0xc00058f200) github.com/tendermint/tendermint/internal/statesync/reactor.go:243 +0x117 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00058f200, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc0001ea240) github.com/tendermint/tendermint/node/node.go:769 +0x132 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0001ea240, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/cmd/tendermint/commands.NewRunNodeCmd.func1.1() github.com/tendermint/tendermint/cmd/tendermint/commands/run_node.go:143 +0x62 github.com/tendermint/tendermint/libs/os.TrapSignal.func1(0xc000629500, 0x7fdb52f96358, 0xc0002b5030, 0xc00000daa0) github.com/tendermint/tendermint/libs/os/os.go:26 +0x102 created by github.com/tendermint/tendermint/libs/os.TrapSignal github.com/tendermint/tendermint/libs/os/os.go:22 +0xe6 goroutine 188 [semacquire]: sync.runtime_SemacquireMutex(0xc00026b1cc, 0x0, 0x1) runtime/sema.go:71 +0x47 sync.(*Mutex).lockSlow(0xc00026b1c8) sync/mutex.go:138 +0x105 sync.(*Mutex).Lock(...) sync/mutex.go:81 sync.(*RWMutex).Lock(0xc00026b1c8) sync/rwmutex.go:111 +0x90 github.com/tendermint/tendermint/internal/statesync.(*Reactor).processPeerUpdate(0xc00026b080, 0xc000650008, 0x28, 0x124de90, 0x4) github.com/tendermint/tendermint/internal/statesync/reactor.go:849 +0x1a5 github.com/tendermint/tendermint/internal/statesync.(*Reactor).processPeerUpdates(0xc00026b080) github.com/tendermint/tendermint/internal/statesync/reactor.go:883 +0xab created by github.com/tendermint/tendermint/internal/statesync.(*Reactor.OnStart github.com/tendermint/tendermint/internal/statesync/reactor.go:219 +0xcd) ```
3 years ago
statesync: remove deadlock on init fail (#7029) When statesync is stopped during shutdown, it has the possibility of deadlocking. A dump of goroutines reveals that this is related to the peerUpdates channel not returning anything on its `Done()` channel when `OnStop` is called. As this is occuring, `processPeerUpdate` is attempting to acquire the reactor lock. It appears that this lock can never be acquired. I looked for the places where the lock may remain locked accidentally and cleaned them up in hopes to eradicate the issue. Dumps of the relevant goroutines may be found below. Note that the line numbers below are relative to the code in the `v0.35.0-rc1` tag. ``` goroutine 36 [chan receive]: github.com/tendermint/tendermint/internal/statesync.(*Reactor).OnStop(0xc00058f200) github.com/tendermint/tendermint/internal/statesync/reactor.go:243 +0x117 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00058f200, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc0001ea240) github.com/tendermint/tendermint/node/node.go:769 +0x132 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0001ea240, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/cmd/tendermint/commands.NewRunNodeCmd.func1.1() github.com/tendermint/tendermint/cmd/tendermint/commands/run_node.go:143 +0x62 github.com/tendermint/tendermint/libs/os.TrapSignal.func1(0xc000629500, 0x7fdb52f96358, 0xc0002b5030, 0xc00000daa0) github.com/tendermint/tendermint/libs/os/os.go:26 +0x102 created by github.com/tendermint/tendermint/libs/os.TrapSignal github.com/tendermint/tendermint/libs/os/os.go:22 +0xe6 goroutine 188 [semacquire]: sync.runtime_SemacquireMutex(0xc00026b1cc, 0x0, 0x1) runtime/sema.go:71 +0x47 sync.(*Mutex).lockSlow(0xc00026b1c8) sync/mutex.go:138 +0x105 sync.(*Mutex).Lock(...) sync/mutex.go:81 sync.(*RWMutex).Lock(0xc00026b1c8) sync/rwmutex.go:111 +0x90 github.com/tendermint/tendermint/internal/statesync.(*Reactor).processPeerUpdate(0xc00026b080, 0xc000650008, 0x28, 0x124de90, 0x4) github.com/tendermint/tendermint/internal/statesync/reactor.go:849 +0x1a5 github.com/tendermint/tendermint/internal/statesync.(*Reactor).processPeerUpdates(0xc00026b080) github.com/tendermint/tendermint/internal/statesync/reactor.go:883 +0xab created by github.com/tendermint/tendermint/internal/statesync.(*Reactor.OnStart github.com/tendermint/tendermint/internal/statesync/reactor.go:219 +0xcd) ```
3 years ago
  1. package statesync
  2. import (
  3. "context"
  4. "errors"
  5. "sync"
  6. "testing"
  7. "time"
  8. "github.com/stretchr/testify/assert"
  9. "github.com/stretchr/testify/mock"
  10. "github.com/stretchr/testify/require"
  11. clientmocks "github.com/tendermint/tendermint/abci/client/mocks"
  12. abci "github.com/tendermint/tendermint/abci/types"
  13. "github.com/tendermint/tendermint/internal/proxy"
  14. sm "github.com/tendermint/tendermint/internal/state"
  15. "github.com/tendermint/tendermint/internal/statesync/mocks"
  16. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  17. "github.com/tendermint/tendermint/types"
  18. "github.com/tendermint/tendermint/version"
  19. )
  20. func TestSyncer_SyncAny(t *testing.T) {
  21. ctx, cancel := context.WithCancel(context.Background())
  22. defer cancel()
  23. state := sm.State{
  24. ChainID: "chain",
  25. Version: sm.Version{
  26. Consensus: version.Consensus{
  27. Block: version.BlockProtocol,
  28. App: testAppVersion,
  29. },
  30. Software: version.TMVersion,
  31. },
  32. LastBlockHeight: 1,
  33. LastBlockID: types.BlockID{Hash: []byte("blockhash")},
  34. LastBlockTime: time.Now(),
  35. LastResultsHash: []byte("last_results_hash"),
  36. AppHash: []byte("app_hash"),
  37. LastValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val1")}},
  38. Validators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val2")}},
  39. NextValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val3")}},
  40. ConsensusParams: *types.DefaultConsensusParams(),
  41. LastHeightConsensusParamsChanged: 1,
  42. }
  43. commit := &types.Commit{BlockID: types.BlockID{Hash: []byte("blockhash")}}
  44. chunks := []*chunk{
  45. {Height: 1, Format: 1, Index: 0, Chunk: []byte{1, 1, 0}},
  46. {Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 1, 1}},
  47. {Height: 1, Format: 1, Index: 2, Chunk: []byte{1, 1, 2}},
  48. }
  49. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  50. stateProvider := &mocks.StateProvider{}
  51. stateProvider.On("AppHash", mock.Anything, uint64(1)).Return(state.AppHash, nil)
  52. stateProvider.On("AppHash", mock.Anything, uint64(2)).Return([]byte("app_hash_2"), nil)
  53. stateProvider.On("Commit", mock.Anything, uint64(1)).Return(commit, nil)
  54. stateProvider.On("State", mock.Anything, uint64(1)).Return(state, nil)
  55. conn := &clientmocks.Client{}
  56. peerAID := types.NodeID("aa")
  57. peerBID := types.NodeID("bb")
  58. peerCID := types.NodeID("cc")
  59. rts := setup(ctx, t, conn, stateProvider, 4)
  60. rts.reactor.syncer = rts.syncer
  61. // Adding a chunk should error when no sync is in progress
  62. _, err := rts.syncer.AddChunk(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}})
  63. require.Error(t, err)
  64. // Adding a couple of peers should trigger snapshot discovery messages
  65. err = rts.syncer.AddPeer(ctx, peerAID)
  66. require.NoError(t, err)
  67. e := <-rts.snapshotOutCh
  68. require.Equal(t, &ssproto.SnapshotsRequest{}, e.Message)
  69. require.Equal(t, peerAID, e.To)
  70. err = rts.syncer.AddPeer(ctx, peerBID)
  71. require.NoError(t, err)
  72. e = <-rts.snapshotOutCh
  73. require.Equal(t, &ssproto.SnapshotsRequest{}, e.Message)
  74. require.Equal(t, peerBID, e.To)
  75. // Both peers report back with snapshots. One of them also returns a snapshot we don't want, in
  76. // format 2, which will be rejected by the ABCI application.
  77. new, err := rts.syncer.AddSnapshot(peerAID, s)
  78. require.NoError(t, err)
  79. require.True(t, new)
  80. new, err = rts.syncer.AddSnapshot(peerBID, s)
  81. require.NoError(t, err)
  82. require.False(t, new)
  83. s2 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1}}
  84. new, err = rts.syncer.AddSnapshot(peerBID, s2)
  85. require.NoError(t, err)
  86. require.True(t, new)
  87. new, err = rts.syncer.AddSnapshot(peerCID, s2)
  88. require.NoError(t, err)
  89. require.False(t, new)
  90. // We start a sync, with peers sending back chunks when requested. We first reject the snapshot
  91. // with height 2 format 2, and accept the snapshot at height 1.
  92. conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  93. Snapshot: &abci.Snapshot{
  94. Height: 2,
  95. Format: 2,
  96. Chunks: 3,
  97. Hash: []byte{1},
  98. },
  99. AppHash: []byte("app_hash_2"),
  100. }).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT}, nil)
  101. conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  102. Snapshot: &abci.Snapshot{
  103. Height: s.Height,
  104. Format: s.Format,
  105. Chunks: s.Chunks,
  106. Hash: s.Hash,
  107. Metadata: s.Metadata,
  108. },
  109. AppHash: []byte("app_hash"),
  110. }).Times(2).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, nil)
  111. chunkRequests := make(map[uint32]int)
  112. chunkRequestsMtx := sync.Mutex{}
  113. chunkProcessDone := make(chan struct{})
  114. go func() {
  115. defer close(chunkProcessDone)
  116. var seen int
  117. for {
  118. if seen >= 4 {
  119. return
  120. }
  121. select {
  122. case <-ctx.Done():
  123. t.Logf("sent %d chunks", seen)
  124. return
  125. case e := <-rts.chunkOutCh:
  126. msg, ok := e.Message.(*ssproto.ChunkRequest)
  127. assert.True(t, ok)
  128. assert.EqualValues(t, 1, msg.Height)
  129. assert.EqualValues(t, 1, msg.Format)
  130. assert.LessOrEqual(t, msg.Index, uint32(len(chunks)))
  131. added, err := rts.syncer.AddChunk(chunks[msg.Index])
  132. assert.NoError(t, err)
  133. assert.True(t, added)
  134. chunkRequestsMtx.Lock()
  135. chunkRequests[msg.Index]++
  136. chunkRequestsMtx.Unlock()
  137. seen++
  138. t.Logf("added chunk (%d of 4): %d", seen, msg.Index)
  139. }
  140. }
  141. }()
  142. // The first time we're applying chunk 2 we tell it to retry the snapshot and discard chunk 1,
  143. // which should cause it to keep the existing chunk 0 and 2, and restart restoration from
  144. // beginning. We also wait for a little while, to exercise the retry logic in fetchChunks().
  145. conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  146. Index: 2, Chunk: []byte{1, 1, 2},
  147. }).Once().Run(func(args mock.Arguments) { time.Sleep(1 * time.Second) }).Return(
  148. &abci.ResponseApplySnapshotChunk{
  149. Result: abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT,
  150. RefetchChunks: []uint32{1},
  151. }, nil)
  152. conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  153. Index: 0, Chunk: []byte{1, 1, 0},
  154. }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  155. conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  156. Index: 1, Chunk: []byte{1, 1, 1},
  157. }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  158. conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  159. Index: 2, Chunk: []byte{1, 1, 2},
  160. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  161. conn.On("Info", mock.Anything, proxy.RequestInfo).Return(&abci.ResponseInfo{
  162. AppVersion: testAppVersion,
  163. LastBlockHeight: 1,
  164. LastBlockAppHash: []byte("app_hash"),
  165. }, nil)
  166. newState, lastCommit, err := rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  167. require.NoError(t, err)
  168. <-chunkProcessDone
  169. chunkRequestsMtx.Lock()
  170. require.Equal(t, map[uint32]int{0: 1, 1: 2, 2: 1}, chunkRequests)
  171. chunkRequestsMtx.Unlock()
  172. expectState := state
  173. require.Equal(t, expectState, newState)
  174. require.Equal(t, commit, lastCommit)
  175. require.Equal(t, len(chunks), int(rts.syncer.processingSnapshot.Chunks))
  176. require.Equal(t, expectState.LastBlockHeight, rts.syncer.lastSyncedSnapshotHeight)
  177. require.True(t, rts.syncer.avgChunkTime > 0)
  178. require.Equal(t, int64(rts.syncer.processingSnapshot.Chunks), rts.reactor.SnapshotChunksTotal())
  179. require.Equal(t, rts.syncer.lastSyncedSnapshotHeight, rts.reactor.SnapshotHeight())
  180. require.Equal(t, time.Duration(rts.syncer.avgChunkTime), rts.reactor.ChunkProcessAvgTime())
  181. require.Equal(t, int64(len(rts.syncer.snapshots.snapshots)), rts.reactor.TotalSnapshots())
  182. require.Equal(t, int64(0), rts.reactor.SnapshotChunksCount())
  183. conn.AssertExpectations(t)
  184. }
  185. func TestSyncer_SyncAny_noSnapshots(t *testing.T) {
  186. stateProvider := &mocks.StateProvider{}
  187. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  188. ctx, cancel := context.WithCancel(context.Background())
  189. defer cancel()
  190. rts := setup(ctx, t, nil, stateProvider, 2)
  191. _, _, err := rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  192. require.Equal(t, errNoSnapshots, err)
  193. }
  194. func TestSyncer_SyncAny_abort(t *testing.T) {
  195. stateProvider := &mocks.StateProvider{}
  196. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  197. ctx, cancel := context.WithCancel(context.Background())
  198. defer cancel()
  199. rts := setup(ctx, t, nil, stateProvider, 2)
  200. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  201. peerID := types.NodeID("aa")
  202. _, err := rts.syncer.AddSnapshot(peerID, s)
  203. require.NoError(t, err)
  204. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  205. Snapshot: toABCI(s), AppHash: []byte("app_hash"),
  206. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT}, nil)
  207. _, _, err = rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  208. require.Equal(t, errAbort, err)
  209. rts.conn.AssertExpectations(t)
  210. }
  211. func TestSyncer_SyncAny_reject(t *testing.T) {
  212. stateProvider := &mocks.StateProvider{}
  213. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  214. ctx, cancel := context.WithCancel(context.Background())
  215. defer cancel()
  216. rts := setup(ctx, t, nil, stateProvider, 2)
  217. // s22 is tried first, then s12, then s11, then errNoSnapshots
  218. s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  219. s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  220. s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  221. peerID := types.NodeID("aa")
  222. _, err := rts.syncer.AddSnapshot(peerID, s22)
  223. require.NoError(t, err)
  224. _, err = rts.syncer.AddSnapshot(peerID, s12)
  225. require.NoError(t, err)
  226. _, err = rts.syncer.AddSnapshot(peerID, s11)
  227. require.NoError(t, err)
  228. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  229. Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
  230. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  231. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  232. Snapshot: toABCI(s12), AppHash: []byte("app_hash"),
  233. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  234. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  235. Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
  236. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  237. _, _, err = rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  238. require.Equal(t, errNoSnapshots, err)
  239. rts.conn.AssertExpectations(t)
  240. }
  241. func TestSyncer_SyncAny_reject_format(t *testing.T) {
  242. stateProvider := &mocks.StateProvider{}
  243. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  244. ctx, cancel := context.WithCancel(context.Background())
  245. defer cancel()
  246. rts := setup(ctx, t, nil, stateProvider, 2)
  247. // s22 is tried first, which reject s22 and s12, then s11 will abort.
  248. s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  249. s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  250. s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  251. peerID := types.NodeID("aa")
  252. _, err := rts.syncer.AddSnapshot(peerID, s22)
  253. require.NoError(t, err)
  254. _, err = rts.syncer.AddSnapshot(peerID, s12)
  255. require.NoError(t, err)
  256. _, err = rts.syncer.AddSnapshot(peerID, s11)
  257. require.NoError(t, err)
  258. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  259. Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
  260. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT}, nil)
  261. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  262. Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
  263. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT}, nil)
  264. _, _, err = rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  265. require.Equal(t, errAbort, err)
  266. rts.conn.AssertExpectations(t)
  267. }
  268. func TestSyncer_SyncAny_reject_sender(t *testing.T) {
  269. stateProvider := &mocks.StateProvider{}
  270. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  271. ctx, cancel := context.WithCancel(context.Background())
  272. defer cancel()
  273. rts := setup(ctx, t, nil, stateProvider, 2)
  274. peerAID := types.NodeID("aa")
  275. peerBID := types.NodeID("bb")
  276. peerCID := types.NodeID("cc")
  277. // sbc will be offered first, which will be rejected with reject_sender, causing all snapshots
  278. // submitted by both b and c (i.e. sb, sc, sbc) to be rejected. Finally, sa will reject and
  279. // errNoSnapshots is returned.
  280. sa := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  281. sb := &snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  282. sc := &snapshot{Height: 3, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  283. sbc := &snapshot{Height: 4, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  284. _, err := rts.syncer.AddSnapshot(peerAID, sa)
  285. require.NoError(t, err)
  286. _, err = rts.syncer.AddSnapshot(peerBID, sb)
  287. require.NoError(t, err)
  288. _, err = rts.syncer.AddSnapshot(peerCID, sc)
  289. require.NoError(t, err)
  290. _, err = rts.syncer.AddSnapshot(peerBID, sbc)
  291. require.NoError(t, err)
  292. _, err = rts.syncer.AddSnapshot(peerCID, sbc)
  293. require.NoError(t, err)
  294. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  295. Snapshot: toABCI(sbc), AppHash: []byte("app_hash"),
  296. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_SENDER}, nil)
  297. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  298. Snapshot: toABCI(sa), AppHash: []byte("app_hash"),
  299. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  300. _, _, err = rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  301. require.Equal(t, errNoSnapshots, err)
  302. rts.conn.AssertExpectations(t)
  303. }
  304. func TestSyncer_SyncAny_abciError(t *testing.T) {
  305. stateProvider := &mocks.StateProvider{}
  306. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  307. ctx, cancel := context.WithCancel(context.Background())
  308. defer cancel()
  309. rts := setup(ctx, t, nil, stateProvider, 2)
  310. errBoom := errors.New("boom")
  311. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  312. peerID := types.NodeID("aa")
  313. _, err := rts.syncer.AddSnapshot(peerID, s)
  314. require.NoError(t, err)
  315. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  316. Snapshot: toABCI(s), AppHash: []byte("app_hash"),
  317. }).Once().Return(nil, errBoom)
  318. _, _, err = rts.syncer.SyncAny(ctx, 0, func() error { return nil })
  319. require.True(t, errors.Is(err, errBoom))
  320. rts.conn.AssertExpectations(t)
  321. }
  322. func TestSyncer_offerSnapshot(t *testing.T) {
  323. unknownErr := errors.New("unknown error")
  324. boom := errors.New("boom")
  325. testcases := map[string]struct {
  326. result abci.ResponseOfferSnapshot_Result
  327. err error
  328. expectErr error
  329. }{
  330. "accept": {abci.ResponseOfferSnapshot_ACCEPT, nil, nil},
  331. "abort": {abci.ResponseOfferSnapshot_ABORT, nil, errAbort},
  332. "reject": {abci.ResponseOfferSnapshot_REJECT, nil, errRejectSnapshot},
  333. "reject_format": {abci.ResponseOfferSnapshot_REJECT_FORMAT, nil, errRejectFormat},
  334. "reject_sender": {abci.ResponseOfferSnapshot_REJECT_SENDER, nil, errRejectSender},
  335. "unknown": {abci.ResponseOfferSnapshot_UNKNOWN, nil, unknownErr},
  336. "error": {0, boom, boom},
  337. "unknown non-zero": {9, nil, unknownErr},
  338. }
  339. ctx, cancel := context.WithCancel(context.Background())
  340. defer cancel()
  341. for name, tc := range testcases {
  342. tc := tc
  343. t.Run(name, func(t *testing.T) {
  344. ctx, cancel := context.WithCancel(ctx)
  345. defer cancel()
  346. stateProvider := &mocks.StateProvider{}
  347. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  348. rts := setup(ctx, t, nil, stateProvider, 2)
  349. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
  350. rts.conn.On("OfferSnapshot", mock.Anything, abci.RequestOfferSnapshot{
  351. Snapshot: toABCI(s),
  352. AppHash: []byte("app_hash"),
  353. }).Return(&abci.ResponseOfferSnapshot{Result: tc.result}, tc.err)
  354. err := rts.syncer.offerSnapshot(ctx, s)
  355. if tc.expectErr == unknownErr {
  356. require.Error(t, err)
  357. } else {
  358. unwrapped := errors.Unwrap(err)
  359. if unwrapped != nil {
  360. err = unwrapped
  361. }
  362. require.Equal(t, tc.expectErr, err)
  363. }
  364. })
  365. }
  366. }
  367. func TestSyncer_applyChunks_Results(t *testing.T) {
  368. unknownErr := errors.New("unknown error")
  369. boom := errors.New("boom")
  370. testcases := map[string]struct {
  371. result abci.ResponseApplySnapshotChunk_Result
  372. err error
  373. expectErr error
  374. }{
  375. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT, nil, nil},
  376. "abort": {abci.ResponseApplySnapshotChunk_ABORT, nil, errAbort},
  377. "retry": {abci.ResponseApplySnapshotChunk_RETRY, nil, nil},
  378. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT, nil, errRetrySnapshot},
  379. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT, nil, errRejectSnapshot},
  380. "unknown": {abci.ResponseApplySnapshotChunk_UNKNOWN, nil, unknownErr},
  381. "error": {0, boom, boom},
  382. "unknown non-zero": {9, nil, unknownErr},
  383. }
  384. ctx, cancel := context.WithCancel(context.Background())
  385. defer cancel()
  386. for name, tc := range testcases {
  387. tc := tc
  388. t.Run(name, func(t *testing.T) {
  389. ctx, cancel := context.WithCancel(ctx)
  390. defer cancel()
  391. stateProvider := &mocks.StateProvider{}
  392. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  393. rts := setup(ctx, t, nil, stateProvider, 2)
  394. body := []byte{1, 2, 3}
  395. chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 1}, t.TempDir())
  396. require.NoError(t, err)
  397. fetchStartTime := time.Now()
  398. _, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: body})
  399. require.NoError(t, err)
  400. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  401. Index: 0, Chunk: body,
  402. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: tc.result}, tc.err)
  403. if tc.result == abci.ResponseApplySnapshotChunk_RETRY {
  404. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  405. Index: 0, Chunk: body,
  406. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  407. Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  408. }
  409. err = rts.syncer.applyChunks(ctx, chunks, fetchStartTime)
  410. if tc.expectErr == unknownErr {
  411. require.Error(t, err)
  412. } else {
  413. unwrapped := errors.Unwrap(err)
  414. if unwrapped != nil {
  415. err = unwrapped
  416. }
  417. require.Equal(t, tc.expectErr, err)
  418. }
  419. rts.conn.AssertExpectations(t)
  420. })
  421. }
  422. }
  423. func TestSyncer_applyChunks_RefetchChunks(t *testing.T) {
  424. // Discarding chunks via refetch_chunks should work the same for all results
  425. testcases := map[string]struct {
  426. result abci.ResponseApplySnapshotChunk_Result
  427. }{
  428. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT},
  429. "abort": {abci.ResponseApplySnapshotChunk_ABORT},
  430. "retry": {abci.ResponseApplySnapshotChunk_RETRY},
  431. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT},
  432. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT},
  433. }
  434. ctx, cancel := context.WithCancel(context.Background())
  435. defer cancel()
  436. for name, tc := range testcases {
  437. tc := tc
  438. t.Run(name, func(t *testing.T) {
  439. ctx, cancel := context.WithCancel(ctx)
  440. defer cancel()
  441. stateProvider := &mocks.StateProvider{}
  442. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  443. rts := setup(ctx, t, nil, stateProvider, 2)
  444. chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 3}, t.TempDir())
  445. require.NoError(t, err)
  446. fetchStartTime := time.Now()
  447. added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}})
  448. require.True(t, added)
  449. require.NoError(t, err)
  450. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}})
  451. require.True(t, added)
  452. require.NoError(t, err)
  453. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}})
  454. require.True(t, added)
  455. require.NoError(t, err)
  456. // The first two chunks are accepted, before the last one asks for 1 to be refetched
  457. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  458. Index: 0, Chunk: []byte{0},
  459. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  460. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  461. Index: 1, Chunk: []byte{1},
  462. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  463. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  464. Index: 2, Chunk: []byte{2},
  465. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  466. Result: tc.result,
  467. RefetchChunks: []uint32{1},
  468. }, nil)
  469. // Since removing the chunk will cause Next() to block, we spawn a goroutine, then
  470. // check the queue contents, and finally close the queue to end the goroutine.
  471. // We don't really care about the result of applyChunks, since it has separate test.
  472. go func() {
  473. rts.syncer.applyChunks(ctx, chunks, fetchStartTime) //nolint:errcheck // purposefully ignore error
  474. }()
  475. time.Sleep(50 * time.Millisecond)
  476. require.True(t, chunks.Has(0))
  477. require.False(t, chunks.Has(1))
  478. require.True(t, chunks.Has(2))
  479. require.NoError(t, chunks.Close())
  480. })
  481. }
  482. }
  483. func TestSyncer_applyChunks_RejectSenders(t *testing.T) {
  484. // Banning chunks senders via ban_chunk_senders should work the same for all results
  485. testcases := map[string]struct {
  486. result abci.ResponseApplySnapshotChunk_Result
  487. }{
  488. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT},
  489. "abort": {abci.ResponseApplySnapshotChunk_ABORT},
  490. "retry": {abci.ResponseApplySnapshotChunk_RETRY},
  491. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT},
  492. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT},
  493. }
  494. ctx, cancel := context.WithCancel(context.Background())
  495. defer cancel()
  496. for name, tc := range testcases {
  497. tc := tc
  498. t.Run(name, func(t *testing.T) {
  499. ctx, cancel := context.WithCancel(ctx)
  500. defer cancel()
  501. stateProvider := &mocks.StateProvider{}
  502. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  503. rts := setup(ctx, t, nil, stateProvider, 2)
  504. // Set up three peers across two snapshots, and ask for one of them to be banned.
  505. // It should be banned from all snapshots.
  506. peerAID := types.NodeID("aa")
  507. peerBID := types.NodeID("bb")
  508. peerCID := types.NodeID("cc")
  509. s1 := &snapshot{Height: 1, Format: 1, Chunks: 3}
  510. s2 := &snapshot{Height: 2, Format: 1, Chunks: 3}
  511. _, err := rts.syncer.AddSnapshot(peerAID, s1)
  512. require.NoError(t, err)
  513. _, err = rts.syncer.AddSnapshot(peerAID, s2)
  514. require.NoError(t, err)
  515. _, err = rts.syncer.AddSnapshot(peerBID, s1)
  516. require.NoError(t, err)
  517. _, err = rts.syncer.AddSnapshot(peerBID, s2)
  518. require.NoError(t, err)
  519. _, err = rts.syncer.AddSnapshot(peerCID, s1)
  520. require.NoError(t, err)
  521. _, err = rts.syncer.AddSnapshot(peerCID, s2)
  522. require.NoError(t, err)
  523. chunks, err := newChunkQueue(s1, t.TempDir())
  524. require.NoError(t, err)
  525. fetchStartTime := time.Now()
  526. added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}, Sender: peerAID})
  527. require.True(t, added)
  528. require.NoError(t, err)
  529. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}, Sender: peerBID})
  530. require.True(t, added)
  531. require.NoError(t, err)
  532. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}, Sender: peerCID})
  533. require.True(t, added)
  534. require.NoError(t, err)
  535. // The first two chunks are accepted, before the last one asks for b sender to be rejected
  536. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  537. Index: 0, Chunk: []byte{0}, Sender: "aa",
  538. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  539. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  540. Index: 1, Chunk: []byte{1}, Sender: "bb",
  541. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  542. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  543. Index: 2, Chunk: []byte{2}, Sender: "cc",
  544. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  545. Result: tc.result,
  546. RejectSenders: []string{string(peerBID)},
  547. }, nil)
  548. // On retry, the last chunk will be tried again, so we just accept it then.
  549. if tc.result == abci.ResponseApplySnapshotChunk_RETRY {
  550. rts.conn.On("ApplySnapshotChunk", mock.Anything, abci.RequestApplySnapshotChunk{
  551. Index: 2, Chunk: []byte{2}, Sender: "cc",
  552. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  553. }
  554. // We don't really care about the result of applyChunks, since it has separate test.
  555. // However, it will block on e.g. retry result, so we spawn a goroutine that will
  556. // be shut down when the chunk queue closes.
  557. go func() {
  558. rts.syncer.applyChunks(ctx, chunks, fetchStartTime) //nolint:errcheck // purposefully ignore error
  559. }()
  560. time.Sleep(50 * time.Millisecond)
  561. s1peers := rts.syncer.snapshots.GetPeers(s1)
  562. require.Len(t, s1peers, 2)
  563. require.EqualValues(t, "aa", s1peers[0])
  564. require.EqualValues(t, "cc", s1peers[1])
  565. rts.syncer.snapshots.GetPeers(s1)
  566. require.Len(t, s1peers, 2)
  567. require.EqualValues(t, "aa", s1peers[0])
  568. require.EqualValues(t, "cc", s1peers[1])
  569. require.NoError(t, chunks.Close())
  570. })
  571. }
  572. }
  573. func TestSyncer_verifyApp(t *testing.T) {
  574. boom := errors.New("boom")
  575. const appVersion = 9
  576. appVersionMismatchErr := errors.New("app version mismatch. Expected: 9, got: 2")
  577. s := &snapshot{Height: 3, Format: 1, Chunks: 5, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
  578. testcases := map[string]struct {
  579. response *abci.ResponseInfo
  580. err error
  581. expectErr error
  582. }{
  583. "verified": {&abci.ResponseInfo{
  584. LastBlockHeight: 3,
  585. LastBlockAppHash: []byte("app_hash"),
  586. AppVersion: appVersion,
  587. }, nil, nil},
  588. "invalid app version": {&abci.ResponseInfo{
  589. LastBlockHeight: 3,
  590. LastBlockAppHash: []byte("app_hash"),
  591. AppVersion: 2,
  592. }, nil, appVersionMismatchErr},
  593. "invalid height": {&abci.ResponseInfo{
  594. LastBlockHeight: 5,
  595. LastBlockAppHash: []byte("app_hash"),
  596. AppVersion: appVersion,
  597. }, nil, errVerifyFailed},
  598. "invalid hash": {&abci.ResponseInfo{
  599. LastBlockHeight: 3,
  600. LastBlockAppHash: []byte("xxx"),
  601. AppVersion: appVersion,
  602. }, nil, errVerifyFailed},
  603. "error": {nil, boom, boom},
  604. }
  605. ctx, cancel := context.WithCancel(context.Background())
  606. defer cancel()
  607. for name, tc := range testcases {
  608. tc := tc
  609. t.Run(name, func(t *testing.T) {
  610. ctx, cancel := context.WithCancel(ctx)
  611. defer cancel()
  612. rts := setup(ctx, t, nil, nil, 2)
  613. rts.conn.On("Info", mock.Anything, proxy.RequestInfo).Return(tc.response, tc.err)
  614. err := rts.syncer.verifyApp(ctx, s, appVersion)
  615. unwrapped := errors.Unwrap(err)
  616. if unwrapped != nil {
  617. err = unwrapped
  618. }
  619. require.Equal(t, tc.expectErr, err)
  620. })
  621. }
  622. }
  623. func toABCI(s *snapshot) *abci.Snapshot {
  624. return &abci.Snapshot{
  625. Height: s.Height,
  626. Format: s.Format,
  627. Chunks: s.Chunks,
  628. Hash: s.Hash,
  629. Metadata: s.Metadata,
  630. }
  631. }