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.

724 lines
26 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. abci "github.com/tendermint/tendermint/abci/types"
  12. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  13. "github.com/tendermint/tendermint/internal/proxy"
  14. proxymocks "github.com/tendermint/tendermint/internal/proxy/mocks"
  15. sm "github.com/tendermint/tendermint/internal/state"
  16. "github.com/tendermint/tendermint/internal/statesync/mocks"
  17. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  18. "github.com/tendermint/tendermint/types"
  19. "github.com/tendermint/tendermint/version"
  20. )
  21. var ctx = context.Background()
  22. func TestSyncer_SyncAny(t *testing.T) {
  23. state := sm.State{
  24. ChainID: "chain",
  25. Version: sm.Version{
  26. Consensus: version.Consensus{
  27. Block: version.BlockProtocol,
  28. App: 0,
  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. connSnapshot := &proxymocks.AppConnSnapshot{}
  56. connQuery := &proxymocks.AppConnQuery{}
  57. peerAID := types.NodeID("aa")
  58. peerBID := types.NodeID("bb")
  59. peerCID := types.NodeID("cc")
  60. rts := setup(t, connSnapshot, connQuery, stateProvider, 3)
  61. rts.reactor.syncer = rts.syncer
  62. // Adding a chunk should error when no sync is in progress
  63. _, err := rts.syncer.AddChunk(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}})
  64. require.Error(t, err)
  65. // Adding a couple of peers should trigger snapshot discovery messages
  66. err = rts.syncer.AddPeer(peerAID)
  67. require.NoError(t, err)
  68. e := <-rts.snapshotOutCh
  69. require.Equal(t, &ssproto.SnapshotsRequest{}, e.Message)
  70. require.Equal(t, peerAID, e.To)
  71. err = rts.syncer.AddPeer(peerBID)
  72. require.NoError(t, err)
  73. e = <-rts.snapshotOutCh
  74. require.Equal(t, &ssproto.SnapshotsRequest{}, e.Message)
  75. require.Equal(t, peerBID, e.To)
  76. // Both peers report back with snapshots. One of them also returns a snapshot we don't want, in
  77. // format 2, which will be rejected by the ABCI application.
  78. new, err := rts.syncer.AddSnapshot(peerAID, s)
  79. require.NoError(t, err)
  80. require.True(t, new)
  81. new, err = rts.syncer.AddSnapshot(peerBID, s)
  82. require.NoError(t, err)
  83. require.False(t, new)
  84. s2 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1}}
  85. new, err = rts.syncer.AddSnapshot(peerBID, s2)
  86. require.NoError(t, err)
  87. require.True(t, new)
  88. new, err = rts.syncer.AddSnapshot(peerCID, s2)
  89. require.NoError(t, err)
  90. require.False(t, new)
  91. // We start a sync, with peers sending back chunks when requested. We first reject the snapshot
  92. // with height 2 format 2, and accept the snapshot at height 1.
  93. connSnapshot.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  94. Snapshot: &abci.Snapshot{
  95. Height: 2,
  96. Format: 2,
  97. Chunks: 3,
  98. Hash: []byte{1},
  99. },
  100. AppHash: []byte("app_hash_2"),
  101. }).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT}, nil)
  102. connSnapshot.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  103. Snapshot: &abci.Snapshot{
  104. Height: s.Height,
  105. Format: s.Format,
  106. Chunks: s.Chunks,
  107. Hash: s.Hash,
  108. Metadata: s.Metadata,
  109. },
  110. AppHash: []byte("app_hash"),
  111. }).Times(2).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, nil)
  112. chunkRequests := make(map[uint32]int)
  113. chunkRequestsMtx := tmsync.Mutex{}
  114. var wg sync.WaitGroup
  115. wg.Add(4)
  116. go func() {
  117. for e := range rts.chunkOutCh {
  118. msg, ok := e.Message.(*ssproto.ChunkRequest)
  119. assert.True(t, ok)
  120. assert.EqualValues(t, 1, msg.Height)
  121. assert.EqualValues(t, 1, msg.Format)
  122. assert.LessOrEqual(t, msg.Index, uint32(len(chunks)))
  123. added, err := rts.syncer.AddChunk(chunks[msg.Index])
  124. assert.NoError(t, err)
  125. assert.True(t, added)
  126. chunkRequestsMtx.Lock()
  127. chunkRequests[msg.Index]++
  128. chunkRequestsMtx.Unlock()
  129. wg.Done()
  130. }
  131. }()
  132. // The first time we're applying chunk 2 we tell it to retry the snapshot and discard chunk 1,
  133. // which should cause it to keep the existing chunk 0 and 2, and restart restoration from
  134. // beginning. We also wait for a little while, to exercise the retry logic in fetchChunks().
  135. connSnapshot.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  136. Index: 2, Chunk: []byte{1, 1, 2},
  137. }).Once().Run(func(args mock.Arguments) { time.Sleep(2 * time.Second) }).Return(
  138. &abci.ResponseApplySnapshotChunk{
  139. Result: abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT,
  140. RefetchChunks: []uint32{1},
  141. }, nil)
  142. connSnapshot.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  143. Index: 0, Chunk: []byte{1, 1, 0},
  144. }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  145. connSnapshot.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  146. Index: 1, Chunk: []byte{1, 1, 1},
  147. }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  148. connSnapshot.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  149. Index: 2, Chunk: []byte{1, 1, 2},
  150. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  151. connQuery.On("InfoSync", ctx, proxy.RequestInfo).Return(&abci.ResponseInfo{
  152. AppVersion: 9,
  153. LastBlockHeight: 1,
  154. LastBlockAppHash: []byte("app_hash"),
  155. }, nil)
  156. newState, lastCommit, err := rts.syncer.SyncAny(ctx, 0, func() {})
  157. require.NoError(t, err)
  158. wg.Wait()
  159. chunkRequestsMtx.Lock()
  160. require.Equal(t, map[uint32]int{0: 1, 1: 2, 2: 1}, chunkRequests)
  161. chunkRequestsMtx.Unlock()
  162. // The syncer should have updated the state app version from the ABCI info response.
  163. expectState := state
  164. expectState.Version.Consensus.App = 9
  165. require.Equal(t, expectState, newState)
  166. require.Equal(t, commit, lastCommit)
  167. require.Equal(t, len(chunks), int(rts.syncer.processingSnapshot.Chunks))
  168. require.Equal(t, expectState.LastBlockHeight, rts.syncer.lastSyncedSnapshotHeight)
  169. require.True(t, rts.syncer.avgChunkTime > 0)
  170. require.Equal(t, int64(rts.syncer.processingSnapshot.Chunks), rts.reactor.SnapshotChunksTotal())
  171. require.Equal(t, rts.syncer.lastSyncedSnapshotHeight, rts.reactor.SnapshotHeight())
  172. require.Equal(t, time.Duration(rts.syncer.avgChunkTime), rts.reactor.ChunkProcessAvgTime())
  173. require.Equal(t, int64(len(rts.syncer.snapshots.snapshots)), rts.reactor.TotalSnapshots())
  174. require.Equal(t, int64(0), rts.reactor.SnapshotChunksCount())
  175. connSnapshot.AssertExpectations(t)
  176. connQuery.AssertExpectations(t)
  177. }
  178. func TestSyncer_SyncAny_noSnapshots(t *testing.T) {
  179. stateProvider := &mocks.StateProvider{}
  180. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  181. rts := setup(t, nil, nil, stateProvider, 2)
  182. _, _, err := rts.syncer.SyncAny(ctx, 0, func() {})
  183. require.Equal(t, errNoSnapshots, err)
  184. }
  185. func TestSyncer_SyncAny_abort(t *testing.T) {
  186. stateProvider := &mocks.StateProvider{}
  187. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  188. rts := setup(t, nil, nil, stateProvider, 2)
  189. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  190. peerID := types.NodeID("aa")
  191. _, err := rts.syncer.AddSnapshot(peerID, s)
  192. require.NoError(t, err)
  193. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  194. Snapshot: toABCI(s), AppHash: []byte("app_hash"),
  195. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT}, nil)
  196. _, _, err = rts.syncer.SyncAny(ctx, 0, func() {})
  197. require.Equal(t, errAbort, err)
  198. rts.conn.AssertExpectations(t)
  199. }
  200. func TestSyncer_SyncAny_reject(t *testing.T) {
  201. stateProvider := &mocks.StateProvider{}
  202. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  203. rts := setup(t, nil, nil, stateProvider, 2)
  204. // s22 is tried first, then s12, then s11, then errNoSnapshots
  205. s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  206. s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  207. s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  208. peerID := types.NodeID("aa")
  209. _, err := rts.syncer.AddSnapshot(peerID, s22)
  210. require.NoError(t, err)
  211. _, err = rts.syncer.AddSnapshot(peerID, s12)
  212. require.NoError(t, err)
  213. _, err = rts.syncer.AddSnapshot(peerID, s11)
  214. require.NoError(t, err)
  215. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  216. Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
  217. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  218. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  219. Snapshot: toABCI(s12), AppHash: []byte("app_hash"),
  220. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  221. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  222. Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
  223. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  224. _, _, err = rts.syncer.SyncAny(ctx, 0, func() {})
  225. require.Equal(t, errNoSnapshots, err)
  226. rts.conn.AssertExpectations(t)
  227. }
  228. func TestSyncer_SyncAny_reject_format(t *testing.T) {
  229. stateProvider := &mocks.StateProvider{}
  230. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  231. rts := setup(t, nil, nil, stateProvider, 2)
  232. // s22 is tried first, which reject s22 and s12, then s11 will abort.
  233. s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  234. s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
  235. s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  236. peerID := types.NodeID("aa")
  237. _, err := rts.syncer.AddSnapshot(peerID, s22)
  238. require.NoError(t, err)
  239. _, err = rts.syncer.AddSnapshot(peerID, s12)
  240. require.NoError(t, err)
  241. _, err = rts.syncer.AddSnapshot(peerID, s11)
  242. require.NoError(t, err)
  243. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  244. Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
  245. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT}, nil)
  246. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  247. Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
  248. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT}, nil)
  249. _, _, err = rts.syncer.SyncAny(ctx, 0, func() {})
  250. require.Equal(t, errAbort, err)
  251. rts.conn.AssertExpectations(t)
  252. }
  253. func TestSyncer_SyncAny_reject_sender(t *testing.T) {
  254. stateProvider := &mocks.StateProvider{}
  255. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  256. rts := setup(t, nil, nil, stateProvider, 2)
  257. peerAID := types.NodeID("aa")
  258. peerBID := types.NodeID("bb")
  259. peerCID := types.NodeID("cc")
  260. // sbc will be offered first, which will be rejected with reject_sender, causing all snapshots
  261. // submitted by both b and c (i.e. sb, sc, sbc) to be rejected. Finally, sa will reject and
  262. // errNoSnapshots is returned.
  263. sa := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  264. sb := &snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  265. sc := &snapshot{Height: 3, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  266. sbc := &snapshot{Height: 4, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  267. _, err := rts.syncer.AddSnapshot(peerAID, sa)
  268. require.NoError(t, err)
  269. _, err = rts.syncer.AddSnapshot(peerBID, sb)
  270. require.NoError(t, err)
  271. _, err = rts.syncer.AddSnapshot(peerCID, sc)
  272. require.NoError(t, err)
  273. _, err = rts.syncer.AddSnapshot(peerBID, sbc)
  274. require.NoError(t, err)
  275. _, err = rts.syncer.AddSnapshot(peerCID, sbc)
  276. require.NoError(t, err)
  277. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  278. Snapshot: toABCI(sbc), AppHash: []byte("app_hash"),
  279. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_SENDER}, nil)
  280. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  281. Snapshot: toABCI(sa), AppHash: []byte("app_hash"),
  282. }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT}, nil)
  283. _, _, err = rts.syncer.SyncAny(ctx, 0, func() {})
  284. require.Equal(t, errNoSnapshots, err)
  285. rts.conn.AssertExpectations(t)
  286. }
  287. func TestSyncer_SyncAny_abciError(t *testing.T) {
  288. stateProvider := &mocks.StateProvider{}
  289. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  290. rts := setup(t, nil, nil, stateProvider, 2)
  291. errBoom := errors.New("boom")
  292. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
  293. peerID := types.NodeID("aa")
  294. _, err := rts.syncer.AddSnapshot(peerID, s)
  295. require.NoError(t, err)
  296. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  297. Snapshot: toABCI(s), AppHash: []byte("app_hash"),
  298. }).Once().Return(nil, errBoom)
  299. _, _, err = rts.syncer.SyncAny(ctx, 0, func() {})
  300. require.True(t, errors.Is(err, errBoom))
  301. rts.conn.AssertExpectations(t)
  302. }
  303. func TestSyncer_offerSnapshot(t *testing.T) {
  304. unknownErr := errors.New("unknown error")
  305. boom := errors.New("boom")
  306. testcases := map[string]struct {
  307. result abci.ResponseOfferSnapshot_Result
  308. err error
  309. expectErr error
  310. }{
  311. "accept": {abci.ResponseOfferSnapshot_ACCEPT, nil, nil},
  312. "abort": {abci.ResponseOfferSnapshot_ABORT, nil, errAbort},
  313. "reject": {abci.ResponseOfferSnapshot_REJECT, nil, errRejectSnapshot},
  314. "reject_format": {abci.ResponseOfferSnapshot_REJECT_FORMAT, nil, errRejectFormat},
  315. "reject_sender": {abci.ResponseOfferSnapshot_REJECT_SENDER, nil, errRejectSender},
  316. "unknown": {abci.ResponseOfferSnapshot_UNKNOWN, nil, unknownErr},
  317. "error": {0, boom, boom},
  318. "unknown non-zero": {9, nil, unknownErr},
  319. }
  320. for name, tc := range testcases {
  321. tc := tc
  322. t.Run(name, func(t *testing.T) {
  323. stateProvider := &mocks.StateProvider{}
  324. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  325. rts := setup(t, nil, nil, stateProvider, 2)
  326. s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
  327. rts.conn.On("OfferSnapshotSync", ctx, abci.RequestOfferSnapshot{
  328. Snapshot: toABCI(s),
  329. AppHash: []byte("app_hash"),
  330. }).Return(&abci.ResponseOfferSnapshot{Result: tc.result}, tc.err)
  331. err := rts.syncer.offerSnapshot(ctx, s)
  332. if tc.expectErr == unknownErr {
  333. require.Error(t, err)
  334. } else {
  335. unwrapped := errors.Unwrap(err)
  336. if unwrapped != nil {
  337. err = unwrapped
  338. }
  339. require.Equal(t, tc.expectErr, err)
  340. }
  341. })
  342. }
  343. }
  344. func TestSyncer_applyChunks_Results(t *testing.T) {
  345. unknownErr := errors.New("unknown error")
  346. boom := errors.New("boom")
  347. testcases := map[string]struct {
  348. result abci.ResponseApplySnapshotChunk_Result
  349. err error
  350. expectErr error
  351. }{
  352. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT, nil, nil},
  353. "abort": {abci.ResponseApplySnapshotChunk_ABORT, nil, errAbort},
  354. "retry": {abci.ResponseApplySnapshotChunk_RETRY, nil, nil},
  355. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT, nil, errRetrySnapshot},
  356. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT, nil, errRejectSnapshot},
  357. "unknown": {abci.ResponseApplySnapshotChunk_UNKNOWN, nil, unknownErr},
  358. "error": {0, boom, boom},
  359. "unknown non-zero": {9, nil, unknownErr},
  360. }
  361. for name, tc := range testcases {
  362. tc := tc
  363. t.Run(name, func(t *testing.T) {
  364. stateProvider := &mocks.StateProvider{}
  365. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  366. rts := setup(t, nil, nil, stateProvider, 2)
  367. body := []byte{1, 2, 3}
  368. chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 1}, "")
  369. require.NoError(t, err)
  370. fetchStartTime := time.Now()
  371. _, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: body})
  372. require.NoError(t, err)
  373. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  374. Index: 0, Chunk: body,
  375. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: tc.result}, tc.err)
  376. if tc.result == abci.ResponseApplySnapshotChunk_RETRY {
  377. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  378. Index: 0, Chunk: body,
  379. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  380. Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  381. }
  382. err = rts.syncer.applyChunks(ctx, chunks, fetchStartTime)
  383. if tc.expectErr == unknownErr {
  384. require.Error(t, err)
  385. } else {
  386. unwrapped := errors.Unwrap(err)
  387. if unwrapped != nil {
  388. err = unwrapped
  389. }
  390. require.Equal(t, tc.expectErr, err)
  391. }
  392. rts.conn.AssertExpectations(t)
  393. })
  394. }
  395. }
  396. func TestSyncer_applyChunks_RefetchChunks(t *testing.T) {
  397. // Discarding chunks via refetch_chunks should work the same for all results
  398. testcases := map[string]struct {
  399. result abci.ResponseApplySnapshotChunk_Result
  400. }{
  401. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT},
  402. "abort": {abci.ResponseApplySnapshotChunk_ABORT},
  403. "retry": {abci.ResponseApplySnapshotChunk_RETRY},
  404. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT},
  405. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT},
  406. }
  407. for name, tc := range testcases {
  408. tc := tc
  409. t.Run(name, func(t *testing.T) {
  410. stateProvider := &mocks.StateProvider{}
  411. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  412. rts := setup(t, nil, nil, stateProvider, 2)
  413. chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 3}, "")
  414. require.NoError(t, err)
  415. fetchStartTime := time.Now()
  416. added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}})
  417. require.True(t, added)
  418. require.NoError(t, err)
  419. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}})
  420. require.True(t, added)
  421. require.NoError(t, err)
  422. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}})
  423. require.True(t, added)
  424. require.NoError(t, err)
  425. // The first two chunks are accepted, before the last one asks for 1 to be refetched
  426. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  427. Index: 0, Chunk: []byte{0},
  428. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  429. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  430. Index: 1, Chunk: []byte{1},
  431. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  432. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  433. Index: 2, Chunk: []byte{2},
  434. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  435. Result: tc.result,
  436. RefetchChunks: []uint32{1},
  437. }, nil)
  438. // Since removing the chunk will cause Next() to block, we spawn a goroutine, then
  439. // check the queue contents, and finally close the queue to end the goroutine.
  440. // We don't really care about the result of applyChunks, since it has separate test.
  441. go func() {
  442. rts.syncer.applyChunks(ctx, chunks, fetchStartTime) //nolint:errcheck // purposefully ignore error
  443. }()
  444. time.Sleep(50 * time.Millisecond)
  445. require.True(t, chunks.Has(0))
  446. require.False(t, chunks.Has(1))
  447. require.True(t, chunks.Has(2))
  448. require.NoError(t, chunks.Close())
  449. })
  450. }
  451. }
  452. func TestSyncer_applyChunks_RejectSenders(t *testing.T) {
  453. // Banning chunks senders via ban_chunk_senders should work the same for all results
  454. testcases := map[string]struct {
  455. result abci.ResponseApplySnapshotChunk_Result
  456. }{
  457. "accept": {abci.ResponseApplySnapshotChunk_ACCEPT},
  458. "abort": {abci.ResponseApplySnapshotChunk_ABORT},
  459. "retry": {abci.ResponseApplySnapshotChunk_RETRY},
  460. "retry_snapshot": {abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT},
  461. "reject_snapshot": {abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT},
  462. }
  463. for name, tc := range testcases {
  464. tc := tc
  465. t.Run(name, func(t *testing.T) {
  466. stateProvider := &mocks.StateProvider{}
  467. stateProvider.On("AppHash", mock.Anything, mock.Anything).Return([]byte("app_hash"), nil)
  468. rts := setup(t, nil, nil, stateProvider, 2)
  469. // Set up three peers across two snapshots, and ask for one of them to be banned.
  470. // It should be banned from all snapshots.
  471. peerAID := types.NodeID("aa")
  472. peerBID := types.NodeID("bb")
  473. peerCID := types.NodeID("cc")
  474. s1 := &snapshot{Height: 1, Format: 1, Chunks: 3}
  475. s2 := &snapshot{Height: 2, Format: 1, Chunks: 3}
  476. _, err := rts.syncer.AddSnapshot(peerAID, s1)
  477. require.NoError(t, err)
  478. _, err = rts.syncer.AddSnapshot(peerAID, s2)
  479. require.NoError(t, err)
  480. _, err = rts.syncer.AddSnapshot(peerBID, s1)
  481. require.NoError(t, err)
  482. _, err = rts.syncer.AddSnapshot(peerBID, s2)
  483. require.NoError(t, err)
  484. _, err = rts.syncer.AddSnapshot(peerCID, s1)
  485. require.NoError(t, err)
  486. _, err = rts.syncer.AddSnapshot(peerCID, s2)
  487. require.NoError(t, err)
  488. chunks, err := newChunkQueue(s1, "")
  489. require.NoError(t, err)
  490. fetchStartTime := time.Now()
  491. added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}, Sender: peerAID})
  492. require.True(t, added)
  493. require.NoError(t, err)
  494. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}, Sender: peerBID})
  495. require.True(t, added)
  496. require.NoError(t, err)
  497. added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}, Sender: peerCID})
  498. require.True(t, added)
  499. require.NoError(t, err)
  500. // The first two chunks are accepted, before the last one asks for b sender to be rejected
  501. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  502. Index: 0, Chunk: []byte{0}, Sender: "aa",
  503. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  504. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  505. Index: 1, Chunk: []byte{1}, Sender: "bb",
  506. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  507. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  508. Index: 2, Chunk: []byte{2}, Sender: "cc",
  509. }).Once().Return(&abci.ResponseApplySnapshotChunk{
  510. Result: tc.result,
  511. RejectSenders: []string{string(peerBID)},
  512. }, nil)
  513. // On retry, the last chunk will be tried again, so we just accept it then.
  514. if tc.result == abci.ResponseApplySnapshotChunk_RETRY {
  515. rts.conn.On("ApplySnapshotChunkSync", ctx, abci.RequestApplySnapshotChunk{
  516. Index: 2, Chunk: []byte{2}, Sender: "cc",
  517. }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}, nil)
  518. }
  519. // We don't really care about the result of applyChunks, since it has separate test.
  520. // However, it will block on e.g. retry result, so we spawn a goroutine that will
  521. // be shut down when the chunk queue closes.
  522. go func() {
  523. rts.syncer.applyChunks(ctx, chunks, fetchStartTime) //nolint:errcheck // purposefully ignore error
  524. }()
  525. time.Sleep(50 * time.Millisecond)
  526. s1peers := rts.syncer.snapshots.GetPeers(s1)
  527. require.Len(t, s1peers, 2)
  528. require.EqualValues(t, "aa", s1peers[0])
  529. require.EqualValues(t, "cc", s1peers[1])
  530. rts.syncer.snapshots.GetPeers(s1)
  531. require.Len(t, s1peers, 2)
  532. require.EqualValues(t, "aa", s1peers[0])
  533. require.EqualValues(t, "cc", s1peers[1])
  534. require.NoError(t, chunks.Close())
  535. })
  536. }
  537. }
  538. func TestSyncer_verifyApp(t *testing.T) {
  539. boom := errors.New("boom")
  540. s := &snapshot{Height: 3, Format: 1, Chunks: 5, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
  541. testcases := map[string]struct {
  542. response *abci.ResponseInfo
  543. err error
  544. expectErr error
  545. }{
  546. "verified": {&abci.ResponseInfo{
  547. LastBlockHeight: 3,
  548. LastBlockAppHash: []byte("app_hash"),
  549. AppVersion: 9,
  550. }, nil, nil},
  551. "invalid height": {&abci.ResponseInfo{
  552. LastBlockHeight: 5,
  553. LastBlockAppHash: []byte("app_hash"),
  554. AppVersion: 9,
  555. }, nil, errVerifyFailed},
  556. "invalid hash": {&abci.ResponseInfo{
  557. LastBlockHeight: 3,
  558. LastBlockAppHash: []byte("xxx"),
  559. AppVersion: 9,
  560. }, nil, errVerifyFailed},
  561. "error": {nil, boom, boom},
  562. }
  563. for name, tc := range testcases {
  564. tc := tc
  565. t.Run(name, func(t *testing.T) {
  566. rts := setup(t, nil, nil, nil, 2)
  567. rts.connQuery.On("InfoSync", ctx, proxy.RequestInfo).Return(tc.response, tc.err)
  568. version, err := rts.syncer.verifyApp(s)
  569. unwrapped := errors.Unwrap(err)
  570. if unwrapped != nil {
  571. err = unwrapped
  572. }
  573. require.Equal(t, tc.expectErr, err)
  574. if err == nil {
  575. require.Equal(t, tc.response.AppVersion, version)
  576. }
  577. })
  578. }
  579. }
  580. func toABCI(s *snapshot) *abci.Snapshot {
  581. return &abci.Snapshot{
  582. Height: s.Height,
  583. Format: s.Format,
  584. Chunks: s.Chunks,
  585. Hash: s.Hash,
  586. Metadata: s.Metadata,
  587. }
  588. }