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.

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