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.

591 lines
18 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. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "time"
  8. abci "github.com/tendermint/tendermint/abci/types"
  9. "github.com/tendermint/tendermint/config"
  10. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  11. "github.com/tendermint/tendermint/internal/p2p"
  12. "github.com/tendermint/tendermint/internal/proxy"
  13. sm "github.com/tendermint/tendermint/internal/state"
  14. "github.com/tendermint/tendermint/libs/log"
  15. "github.com/tendermint/tendermint/light"
  16. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  17. "github.com/tendermint/tendermint/types"
  18. )
  19. const (
  20. // chunkTimeout is the timeout while waiting for the next chunk from the chunk queue.
  21. chunkTimeout = 2 * time.Minute
  22. // minimumDiscoveryTime is the lowest allowable time for a
  23. // SyncAny discovery time.
  24. minimumDiscoveryTime = 5 * time.Second
  25. )
  26. var (
  27. // errAbort is returned by Sync() when snapshot restoration is aborted.
  28. errAbort = errors.New("state sync aborted")
  29. // errRetrySnapshot is returned by Sync() when the snapshot should be retried.
  30. errRetrySnapshot = errors.New("retry snapshot")
  31. // errRejectSnapshot is returned by Sync() when the snapshot is rejected.
  32. errRejectSnapshot = errors.New("snapshot was rejected")
  33. // errRejectFormat is returned by Sync() when the snapshot format is rejected.
  34. errRejectFormat = errors.New("snapshot format was rejected")
  35. // errRejectSender is returned by Sync() when the snapshot sender is rejected.
  36. errRejectSender = errors.New("snapshot sender was rejected")
  37. // errVerifyFailed is returned by Sync() when app hash or last height
  38. // verification fails.
  39. errVerifyFailed = errors.New("verification with app failed")
  40. // errTimeout is returned by Sync() when we've waited too long to receive a chunk.
  41. errTimeout = errors.New("timed out waiting for chunk")
  42. // errNoSnapshots is returned by SyncAny() if no snapshots are found and discovery is disabled.
  43. errNoSnapshots = errors.New("no suitable snapshots found")
  44. )
  45. // syncer runs a state sync against an ABCI app. Use either SyncAny() to automatically attempt to
  46. // sync all snapshots in the pool (pausing to discover new ones), or Sync() to sync a specific
  47. // snapshot. Snapshots and chunks are fed via AddSnapshot() and AddChunk() as appropriate.
  48. type syncer struct {
  49. logger log.Logger
  50. stateProvider StateProvider
  51. conn proxy.AppConnSnapshot
  52. connQuery proxy.AppConnQuery
  53. snapshots *snapshotPool
  54. snapshotCh chan<- p2p.Envelope
  55. chunkCh chan<- p2p.Envelope
  56. tempDir string
  57. fetchers int32
  58. retryTimeout time.Duration
  59. mtx tmsync.RWMutex
  60. chunks *chunkQueue
  61. metrics *Metrics
  62. avgChunkTime int64
  63. lastSyncedSnapshotHeight int64
  64. processingSnapshot *snapshot
  65. closeCh <-chan struct{}
  66. }
  67. // newSyncer creates a new syncer.
  68. func newSyncer(
  69. cfg config.StateSyncConfig,
  70. logger log.Logger,
  71. conn proxy.AppConnSnapshot,
  72. connQuery proxy.AppConnQuery,
  73. stateProvider StateProvider,
  74. snapshotCh chan<- p2p.Envelope,
  75. chunkCh chan<- p2p.Envelope,
  76. closeCh <-chan struct{},
  77. tempDir string,
  78. metrics *Metrics,
  79. ) *syncer {
  80. return &syncer{
  81. logger: logger,
  82. stateProvider: stateProvider,
  83. conn: conn,
  84. connQuery: connQuery,
  85. snapshots: newSnapshotPool(),
  86. snapshotCh: snapshotCh,
  87. chunkCh: chunkCh,
  88. tempDir: tempDir,
  89. fetchers: cfg.Fetchers,
  90. retryTimeout: cfg.ChunkRequestTimeout,
  91. metrics: metrics,
  92. closeCh: closeCh,
  93. }
  94. }
  95. // AddChunk adds a chunk to the chunk queue, if any. It returns false if the chunk has already
  96. // been added to the queue, or an error if there's no sync in progress.
  97. func (s *syncer) AddChunk(chunk *chunk) (bool, error) {
  98. s.mtx.RLock()
  99. defer s.mtx.RUnlock()
  100. if s.chunks == nil {
  101. return false, errors.New("no state sync in progress")
  102. }
  103. added, err := s.chunks.Add(chunk)
  104. if err != nil {
  105. return false, err
  106. }
  107. if added {
  108. s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format,
  109. "chunk", chunk.Index)
  110. } else {
  111. s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format,
  112. "chunk", chunk.Index)
  113. }
  114. return added, nil
  115. }
  116. // AddSnapshot adds a snapshot to the snapshot pool. It returns true if a new, previously unseen
  117. // snapshot was accepted and added.
  118. func (s *syncer) AddSnapshot(peerID types.NodeID, snapshot *snapshot) (bool, error) {
  119. added, err := s.snapshots.Add(peerID, snapshot)
  120. if err != nil {
  121. return false, err
  122. }
  123. if added {
  124. s.metrics.TotalSnapshots.Add(1)
  125. s.logger.Info("Discovered new snapshot", "height", snapshot.Height, "format", snapshot.Format,
  126. "hash", snapshot.Hash)
  127. }
  128. return added, nil
  129. }
  130. // AddPeer adds a peer to the pool. For now we just keep it simple and send a
  131. // single request to discover snapshots, later we may want to do retries and stuff.
  132. func (s *syncer) AddPeer(peerID types.NodeID) (err error) {
  133. defer func() {
  134. // TODO: remove panic recover once AddPeer can no longer accientally send on
  135. // closed channel.
  136. // This recover was added to protect against the p2p message being sent
  137. // to the snapshot channel after the snapshot channel was closed.
  138. if r := recover(); r != nil {
  139. err = fmt.Errorf("panic sending peer snapshot request: %v", r)
  140. }
  141. }()
  142. s.logger.Debug("Requesting snapshots from peer", "peer", peerID)
  143. msg := p2p.Envelope{
  144. To: peerID,
  145. Message: &ssproto.SnapshotsRequest{},
  146. }
  147. select {
  148. case <-s.closeCh:
  149. case s.snapshotCh <- msg:
  150. }
  151. return err
  152. }
  153. // RemovePeer removes a peer from the pool.
  154. func (s *syncer) RemovePeer(peerID types.NodeID) {
  155. s.logger.Debug("Removing peer from sync", "peer", peerID)
  156. s.snapshots.RemovePeer(peerID)
  157. }
  158. // SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further
  159. // snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit
  160. // which the caller must use to bootstrap the node.
  161. func (s *syncer) SyncAny(
  162. ctx context.Context,
  163. discoveryTime time.Duration,
  164. requestSnapshots func(),
  165. ) (sm.State, *types.Commit, error) {
  166. if discoveryTime != 0 && discoveryTime < minimumDiscoveryTime {
  167. discoveryTime = minimumDiscoveryTime
  168. }
  169. if discoveryTime > 0 {
  170. requestSnapshots()
  171. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  172. time.Sleep(discoveryTime)
  173. }
  174. // The app may ask us to retry a snapshot restoration, in which case we need to reuse
  175. // the snapshot and chunk queue from the previous loop iteration.
  176. var (
  177. snapshot *snapshot
  178. chunks *chunkQueue
  179. err error
  180. )
  181. for {
  182. // If not nil, we're going to retry restoration of the same snapshot.
  183. if snapshot == nil {
  184. snapshot = s.snapshots.Best()
  185. chunks = nil
  186. }
  187. if snapshot == nil {
  188. if discoveryTime == 0 {
  189. return sm.State{}, nil, errNoSnapshots
  190. }
  191. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  192. time.Sleep(discoveryTime)
  193. continue
  194. }
  195. if chunks == nil {
  196. chunks, err = newChunkQueue(snapshot, s.tempDir)
  197. if err != nil {
  198. return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err)
  199. }
  200. defer chunks.Close() // in case we forget to close it elsewhere
  201. }
  202. s.processingSnapshot = snapshot
  203. s.metrics.SnapshotChunkTotal.Set(float64(snapshot.Chunks))
  204. newState, commit, err := s.Sync(ctx, snapshot, chunks)
  205. switch {
  206. case err == nil:
  207. s.metrics.SnapshotHeight.Set(float64(snapshot.Height))
  208. s.lastSyncedSnapshotHeight = int64(snapshot.Height)
  209. return newState, commit, nil
  210. case errors.Is(err, errAbort):
  211. return sm.State{}, nil, err
  212. case errors.Is(err, errRetrySnapshot):
  213. chunks.RetryAll()
  214. s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format,
  215. "hash", snapshot.Hash)
  216. continue
  217. case errors.Is(err, errTimeout):
  218. s.snapshots.Reject(snapshot)
  219. s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot",
  220. "height", snapshot.Height, "format", snapshot.Format, "hash", snapshot.Hash)
  221. case errors.Is(err, errRejectSnapshot):
  222. s.snapshots.Reject(snapshot)
  223. s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format,
  224. "hash", snapshot.Hash)
  225. case errors.Is(err, errRejectFormat):
  226. s.snapshots.RejectFormat(snapshot.Format)
  227. s.logger.Info("Snapshot format rejected", "format", snapshot.Format)
  228. case errors.Is(err, errRejectSender):
  229. s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format,
  230. "hash", snapshot.Hash)
  231. for _, peer := range s.snapshots.GetPeers(snapshot) {
  232. s.snapshots.RejectPeer(peer)
  233. s.logger.Info("Snapshot sender rejected", "peer", peer)
  234. }
  235. default:
  236. return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
  237. }
  238. // Discard snapshot and chunks for next iteration
  239. err = chunks.Close()
  240. if err != nil {
  241. s.logger.Error("Failed to clean up chunk queue", "err", err)
  242. }
  243. snapshot = nil
  244. chunks = nil
  245. s.processingSnapshot = nil
  246. }
  247. }
  248. // Sync executes a sync for a specific snapshot, returning the latest state and block commit which
  249. // the caller must use to bootstrap the node.
  250. func (s *syncer) Sync(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) {
  251. s.mtx.Lock()
  252. if s.chunks != nil {
  253. s.mtx.Unlock()
  254. return sm.State{}, nil, errors.New("a state sync is already in progress")
  255. }
  256. s.chunks = chunks
  257. s.mtx.Unlock()
  258. defer func() {
  259. s.mtx.Lock()
  260. s.chunks = nil
  261. s.mtx.Unlock()
  262. }()
  263. hctx, hcancel := context.WithTimeout(ctx, 30*time.Second)
  264. defer hcancel()
  265. // Fetch the app hash corresponding to the snapshot
  266. appHash, err := s.stateProvider.AppHash(hctx, snapshot.Height)
  267. if err != nil {
  268. // check if the main context was triggered
  269. if ctx.Err() != nil {
  270. return sm.State{}, nil, ctx.Err()
  271. }
  272. // catch the case where all the light client providers have been exhausted
  273. if err == light.ErrNoWitnesses {
  274. return sm.State{}, nil,
  275. fmt.Errorf("failed to get app hash at height %d. No witnesses remaining", snapshot.Height)
  276. }
  277. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  278. "err", err, "height", snapshot.Height)
  279. return sm.State{}, nil, errRejectSnapshot
  280. }
  281. snapshot.trustedAppHash = appHash
  282. // Offer snapshot to ABCI app.
  283. err = s.offerSnapshot(ctx, snapshot)
  284. if err != nil {
  285. return sm.State{}, nil, err
  286. }
  287. // Spawn chunk fetchers. They will terminate when the chunk queue is closed or context canceled.
  288. fetchCtx, cancel := context.WithCancel(ctx)
  289. defer cancel()
  290. fetchStartTime := time.Now()
  291. for i := int32(0); i < s.fetchers; i++ {
  292. go s.fetchChunks(fetchCtx, snapshot, chunks)
  293. }
  294. pctx, pcancel := context.WithTimeout(ctx, 1*time.Minute)
  295. defer pcancel()
  296. // Optimistically build new state, so we don't discover any light client failures at the end.
  297. state, err := s.stateProvider.State(pctx, snapshot.Height)
  298. if err != nil {
  299. // check if the main context was triggered
  300. if ctx.Err() != nil {
  301. return sm.State{}, nil, ctx.Err()
  302. }
  303. if err == light.ErrNoWitnesses {
  304. return sm.State{}, nil,
  305. fmt.Errorf("failed to get tendermint state at height %d. No witnesses remaining", snapshot.Height)
  306. }
  307. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  308. "err", err, "height", snapshot.Height)
  309. return sm.State{}, nil, errRejectSnapshot
  310. }
  311. commit, err := s.stateProvider.Commit(pctx, snapshot.Height)
  312. if err != nil {
  313. // check if the provider context exceeded the 10 second deadline
  314. if ctx.Err() != nil {
  315. return sm.State{}, nil, ctx.Err()
  316. }
  317. if err == light.ErrNoWitnesses {
  318. return sm.State{}, nil,
  319. fmt.Errorf("failed to get commit at height %d. No witnesses remaining", snapshot.Height)
  320. }
  321. s.logger.Info("failed to get and verify commit. Dropping snapshot and trying again",
  322. "err", err, "height", snapshot.Height)
  323. return sm.State{}, nil, errRejectSnapshot
  324. }
  325. // Restore snapshot
  326. err = s.applyChunks(ctx, chunks, fetchStartTime)
  327. if err != nil {
  328. return sm.State{}, nil, err
  329. }
  330. // Verify app and update app version
  331. appVersion, err := s.verifyApp(snapshot)
  332. if err != nil {
  333. return sm.State{}, nil, err
  334. }
  335. state.Version.Consensus.App = appVersion
  336. // Done! 🎉
  337. s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format,
  338. "hash", snapshot.Hash)
  339. return state, commit, nil
  340. }
  341. // offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's
  342. // response, or nil if the snapshot was accepted.
  343. func (s *syncer) offerSnapshot(ctx context.Context, snapshot *snapshot) error {
  344. s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height,
  345. "format", snapshot.Format, "hash", snapshot.Hash)
  346. resp, err := s.conn.OfferSnapshotSync(ctx, abci.RequestOfferSnapshot{
  347. Snapshot: &abci.Snapshot{
  348. Height: snapshot.Height,
  349. Format: snapshot.Format,
  350. Chunks: snapshot.Chunks,
  351. Hash: snapshot.Hash,
  352. Metadata: snapshot.Metadata,
  353. },
  354. AppHash: snapshot.trustedAppHash,
  355. })
  356. if err != nil {
  357. return fmt.Errorf("failed to offer snapshot: %w", err)
  358. }
  359. switch resp.Result {
  360. case abci.ResponseOfferSnapshot_ACCEPT:
  361. s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height,
  362. "format", snapshot.Format, "hash", snapshot.Hash)
  363. return nil
  364. case abci.ResponseOfferSnapshot_ABORT:
  365. return errAbort
  366. case abci.ResponseOfferSnapshot_REJECT:
  367. return errRejectSnapshot
  368. case abci.ResponseOfferSnapshot_REJECT_FORMAT:
  369. return errRejectFormat
  370. case abci.ResponseOfferSnapshot_REJECT_SENDER:
  371. return errRejectSender
  372. default:
  373. return fmt.Errorf("unknown ResponseOfferSnapshot result %v", resp.Result)
  374. }
  375. }
  376. // applyChunks applies chunks to the app. It returns various errors depending on the app's
  377. // response, or nil once the snapshot is fully restored.
  378. func (s *syncer) applyChunks(ctx context.Context, chunks *chunkQueue, start time.Time) error {
  379. for {
  380. chunk, err := chunks.Next()
  381. if err == errDone {
  382. return nil
  383. } else if err != nil {
  384. return fmt.Errorf("failed to fetch chunk: %w", err)
  385. }
  386. resp, err := s.conn.ApplySnapshotChunkSync(ctx, abci.RequestApplySnapshotChunk{
  387. Index: chunk.Index,
  388. Chunk: chunk.Chunk,
  389. Sender: string(chunk.Sender),
  390. })
  391. if err != nil {
  392. return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err)
  393. }
  394. s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height,
  395. "format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size())
  396. // Discard and refetch any chunks as requested by the app
  397. for _, index := range resp.RefetchChunks {
  398. err := chunks.Discard(index)
  399. if err != nil {
  400. return fmt.Errorf("failed to discard chunk %v: %w", index, err)
  401. }
  402. }
  403. // Reject any senders as requested by the app
  404. for _, sender := range resp.RejectSenders {
  405. if sender != "" {
  406. peerID := types.NodeID(sender)
  407. s.snapshots.RejectPeer(peerID)
  408. if err := chunks.DiscardSender(peerID); err != nil {
  409. return fmt.Errorf("failed to reject sender: %w", err)
  410. }
  411. }
  412. }
  413. switch resp.Result {
  414. case abci.ResponseApplySnapshotChunk_ACCEPT:
  415. s.metrics.SnapshotChunk.Add(1)
  416. s.avgChunkTime = time.Since(start).Nanoseconds() / int64(chunks.numChunksReturned())
  417. s.metrics.ChunkProcessAvgTime.Set(float64(s.avgChunkTime))
  418. case abci.ResponseApplySnapshotChunk_ABORT:
  419. return errAbort
  420. case abci.ResponseApplySnapshotChunk_RETRY:
  421. chunks.Retry(chunk.Index)
  422. case abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT:
  423. return errRetrySnapshot
  424. case abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT:
  425. return errRejectSnapshot
  426. default:
  427. return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result)
  428. }
  429. }
  430. }
  431. // fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks
  432. // will be received from the reactor via syncer.AddChunks() to chunkQueue.Add().
  433. func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) {
  434. var (
  435. next = true
  436. index uint32
  437. err error
  438. )
  439. for {
  440. if next {
  441. index, err = chunks.Allocate()
  442. if errors.Is(err, errDone) {
  443. // Keep checking until the context is canceled (restore is done), in case any
  444. // chunks need to be refetched.
  445. select {
  446. case <-ctx.Done():
  447. return
  448. case <-s.closeCh:
  449. return
  450. case <-time.After(2 * time.Second):
  451. continue
  452. }
  453. }
  454. if err != nil {
  455. s.logger.Error("Failed to allocate chunk from queue", "err", err)
  456. return
  457. }
  458. }
  459. s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height,
  460. "format", snapshot.Format, "chunk", index, "total", chunks.Size())
  461. ticker := time.NewTicker(s.retryTimeout)
  462. defer ticker.Stop()
  463. s.requestChunk(snapshot, index)
  464. select {
  465. case <-chunks.WaitFor(index):
  466. next = true
  467. case <-ticker.C:
  468. next = false
  469. case <-ctx.Done():
  470. return
  471. case <-s.closeCh:
  472. return
  473. }
  474. ticker.Stop()
  475. }
  476. }
  477. // requestChunk requests a chunk from a peer.
  478. func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
  479. peer := s.snapshots.GetPeer(snapshot)
  480. if peer == "" {
  481. s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height,
  482. "format", snapshot.Format, "hash", snapshot.Hash)
  483. return
  484. }
  485. s.logger.Debug(
  486. "Requesting snapshot chunk",
  487. "height", snapshot.Height,
  488. "format", snapshot.Format,
  489. "chunk", chunk,
  490. "peer", peer,
  491. )
  492. msg := p2p.Envelope{
  493. To: peer,
  494. Message: &ssproto.ChunkRequest{
  495. Height: snapshot.Height,
  496. Format: snapshot.Format,
  497. Index: chunk,
  498. },
  499. }
  500. select {
  501. case s.chunkCh <- msg:
  502. case <-s.closeCh:
  503. }
  504. }
  505. // verifyApp verifies the sync, checking the app hash and last block height. It returns the
  506. // app version, which should be returned as part of the initial state.
  507. func (s *syncer) verifyApp(snapshot *snapshot) (uint64, error) {
  508. resp, err := s.connQuery.InfoSync(context.TODO(), proxy.RequestInfo)
  509. if err != nil {
  510. return 0, fmt.Errorf("failed to query ABCI app for appHash: %w", err)
  511. }
  512. if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) {
  513. s.logger.Error("appHash verification failed",
  514. "expected", snapshot.trustedAppHash,
  515. "actual", resp.LastBlockAppHash)
  516. return 0, errVerifyFailed
  517. }
  518. if uint64(resp.LastBlockHeight) != snapshot.Height {
  519. s.logger.Error(
  520. "ABCI app reported unexpected last block height",
  521. "expected", snapshot.Height,
  522. "actual", resp.LastBlockHeight,
  523. )
  524. return 0, errVerifyFailed
  525. }
  526. s.logger.Info("Verified ABCI app", "height", snapshot.Height, "appHash", snapshot.trustedAppHash)
  527. return resp.AppVersion, nil
  528. }