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.

580 lines
18 KiB

  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) {
  133. s.logger.Debug("Requesting snapshots from peer", "peer", peerID)
  134. msg := p2p.Envelope{
  135. To: peerID,
  136. Message: &ssproto.SnapshotsRequest{},
  137. }
  138. select {
  139. case <-s.closeCh:
  140. case s.snapshotCh <- msg:
  141. }
  142. }
  143. // RemovePeer removes a peer from the pool.
  144. func (s *syncer) RemovePeer(peerID types.NodeID) {
  145. s.logger.Debug("Removing peer from sync", "peer", peerID)
  146. s.snapshots.RemovePeer(peerID)
  147. }
  148. // SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further
  149. // snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit
  150. // which the caller must use to bootstrap the node.
  151. func (s *syncer) SyncAny(
  152. ctx context.Context,
  153. discoveryTime time.Duration,
  154. requestSnapshots func(),
  155. ) (sm.State, *types.Commit, error) {
  156. if discoveryTime != 0 && discoveryTime < minimumDiscoveryTime {
  157. discoveryTime = minimumDiscoveryTime
  158. }
  159. if discoveryTime > 0 {
  160. requestSnapshots()
  161. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  162. time.Sleep(discoveryTime)
  163. }
  164. // The app may ask us to retry a snapshot restoration, in which case we need to reuse
  165. // the snapshot and chunk queue from the previous loop iteration.
  166. var (
  167. snapshot *snapshot
  168. chunks *chunkQueue
  169. err error
  170. )
  171. for {
  172. // If not nil, we're going to retry restoration of the same snapshot.
  173. if snapshot == nil {
  174. snapshot = s.snapshots.Best()
  175. chunks = nil
  176. }
  177. if snapshot == nil {
  178. if discoveryTime == 0 {
  179. return sm.State{}, nil, errNoSnapshots
  180. }
  181. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  182. time.Sleep(discoveryTime)
  183. continue
  184. }
  185. if chunks == nil {
  186. chunks, err = newChunkQueue(snapshot, s.tempDir)
  187. if err != nil {
  188. return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err)
  189. }
  190. defer chunks.Close() // in case we forget to close it elsewhere
  191. }
  192. s.processingSnapshot = snapshot
  193. s.metrics.SnapshotChunkTotal.Set(float64(snapshot.Chunks))
  194. newState, commit, err := s.Sync(ctx, snapshot, chunks)
  195. switch {
  196. case err == nil:
  197. s.metrics.SnapshotHeight.Set(float64(snapshot.Height))
  198. s.lastSyncedSnapshotHeight = int64(snapshot.Height)
  199. return newState, commit, nil
  200. case errors.Is(err, errAbort):
  201. return sm.State{}, nil, err
  202. case errors.Is(err, errRetrySnapshot):
  203. chunks.RetryAll()
  204. s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format,
  205. "hash", snapshot.Hash)
  206. continue
  207. case errors.Is(err, errTimeout):
  208. s.snapshots.Reject(snapshot)
  209. s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot",
  210. "height", snapshot.Height, "format", snapshot.Format, "hash", snapshot.Hash)
  211. case errors.Is(err, errRejectSnapshot):
  212. s.snapshots.Reject(snapshot)
  213. s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format,
  214. "hash", snapshot.Hash)
  215. case errors.Is(err, errRejectFormat):
  216. s.snapshots.RejectFormat(snapshot.Format)
  217. s.logger.Info("Snapshot format rejected", "format", snapshot.Format)
  218. case errors.Is(err, errRejectSender):
  219. s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format,
  220. "hash", snapshot.Hash)
  221. for _, peer := range s.snapshots.GetPeers(snapshot) {
  222. s.snapshots.RejectPeer(peer)
  223. s.logger.Info("Snapshot sender rejected", "peer", peer)
  224. }
  225. default:
  226. return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
  227. }
  228. // Discard snapshot and chunks for next iteration
  229. err = chunks.Close()
  230. if err != nil {
  231. s.logger.Error("Failed to clean up chunk queue", "err", err)
  232. }
  233. snapshot = nil
  234. chunks = nil
  235. s.processingSnapshot = nil
  236. }
  237. }
  238. // Sync executes a sync for a specific snapshot, returning the latest state and block commit which
  239. // the caller must use to bootstrap the node.
  240. func (s *syncer) Sync(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) {
  241. s.mtx.Lock()
  242. if s.chunks != nil {
  243. s.mtx.Unlock()
  244. return sm.State{}, nil, errors.New("a state sync is already in progress")
  245. }
  246. s.chunks = chunks
  247. s.mtx.Unlock()
  248. defer func() {
  249. s.mtx.Lock()
  250. s.chunks = nil
  251. s.mtx.Unlock()
  252. }()
  253. hctx, hcancel := context.WithTimeout(ctx, 30*time.Second)
  254. defer hcancel()
  255. // Fetch the app hash corresponding to the snapshot
  256. appHash, err := s.stateProvider.AppHash(hctx, snapshot.Height)
  257. if err != nil {
  258. // check if the main context was triggered
  259. if ctx.Err() != nil {
  260. return sm.State{}, nil, ctx.Err()
  261. }
  262. // catch the case where all the light client providers have been exhausted
  263. if err == light.ErrNoWitnesses {
  264. return sm.State{}, nil,
  265. fmt.Errorf("failed to get app hash at height %d. No witnesses remaining", snapshot.Height)
  266. }
  267. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  268. "err", err, "height", snapshot.Height)
  269. return sm.State{}, nil, errRejectSnapshot
  270. }
  271. snapshot.trustedAppHash = appHash
  272. // Offer snapshot to ABCI app.
  273. err = s.offerSnapshot(ctx, snapshot)
  274. if err != nil {
  275. return sm.State{}, nil, err
  276. }
  277. // Spawn chunk fetchers. They will terminate when the chunk queue is closed or context canceled.
  278. fetchCtx, cancel := context.WithCancel(ctx)
  279. defer cancel()
  280. fetchStartTime := time.Now()
  281. for i := int32(0); i < s.fetchers; i++ {
  282. go s.fetchChunks(fetchCtx, snapshot, chunks)
  283. }
  284. pctx, pcancel := context.WithTimeout(ctx, 1*time.Minute)
  285. defer pcancel()
  286. // Optimistically build new state, so we don't discover any light client failures at the end.
  287. state, err := s.stateProvider.State(pctx, snapshot.Height)
  288. if err != nil {
  289. // check if the main context was triggered
  290. if ctx.Err() != nil {
  291. return sm.State{}, nil, ctx.Err()
  292. }
  293. if err == light.ErrNoWitnesses {
  294. return sm.State{}, nil,
  295. fmt.Errorf("failed to get tendermint state at height %d. No witnesses remaining", snapshot.Height)
  296. }
  297. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  298. "err", err, "height", snapshot.Height)
  299. return sm.State{}, nil, errRejectSnapshot
  300. }
  301. commit, err := s.stateProvider.Commit(pctx, snapshot.Height)
  302. if err != nil {
  303. // check if the provider context exceeded the 10 second deadline
  304. if ctx.Err() != nil {
  305. return sm.State{}, nil, ctx.Err()
  306. }
  307. if err == light.ErrNoWitnesses {
  308. return sm.State{}, nil,
  309. fmt.Errorf("failed to get commit at height %d. No witnesses remaining", snapshot.Height)
  310. }
  311. s.logger.Info("failed to get and verify commit. Dropping snapshot and trying again",
  312. "err", err, "height", snapshot.Height)
  313. return sm.State{}, nil, errRejectSnapshot
  314. }
  315. // Restore snapshot
  316. err = s.applyChunks(ctx, chunks, fetchStartTime)
  317. if err != nil {
  318. return sm.State{}, nil, err
  319. }
  320. // Verify app and update app version
  321. appVersion, err := s.verifyApp(snapshot)
  322. if err != nil {
  323. return sm.State{}, nil, err
  324. }
  325. state.Version.Consensus.App = appVersion
  326. // Done! 🎉
  327. s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format,
  328. "hash", snapshot.Hash)
  329. return state, commit, nil
  330. }
  331. // offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's
  332. // response, or nil if the snapshot was accepted.
  333. func (s *syncer) offerSnapshot(ctx context.Context, snapshot *snapshot) error {
  334. s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height,
  335. "format", snapshot.Format, "hash", snapshot.Hash)
  336. resp, err := s.conn.OfferSnapshotSync(ctx, abci.RequestOfferSnapshot{
  337. Snapshot: &abci.Snapshot{
  338. Height: snapshot.Height,
  339. Format: snapshot.Format,
  340. Chunks: snapshot.Chunks,
  341. Hash: snapshot.Hash,
  342. Metadata: snapshot.Metadata,
  343. },
  344. AppHash: snapshot.trustedAppHash,
  345. })
  346. if err != nil {
  347. return fmt.Errorf("failed to offer snapshot: %w", err)
  348. }
  349. switch resp.Result {
  350. case abci.ResponseOfferSnapshot_ACCEPT:
  351. s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height,
  352. "format", snapshot.Format, "hash", snapshot.Hash)
  353. return nil
  354. case abci.ResponseOfferSnapshot_ABORT:
  355. return errAbort
  356. case abci.ResponseOfferSnapshot_REJECT:
  357. return errRejectSnapshot
  358. case abci.ResponseOfferSnapshot_REJECT_FORMAT:
  359. return errRejectFormat
  360. case abci.ResponseOfferSnapshot_REJECT_SENDER:
  361. return errRejectSender
  362. default:
  363. return fmt.Errorf("unknown ResponseOfferSnapshot result %v", resp.Result)
  364. }
  365. }
  366. // applyChunks applies chunks to the app. It returns various errors depending on the app's
  367. // response, or nil once the snapshot is fully restored.
  368. func (s *syncer) applyChunks(ctx context.Context, chunks *chunkQueue, start time.Time) error {
  369. for {
  370. chunk, err := chunks.Next()
  371. if err == errDone {
  372. return nil
  373. } else if err != nil {
  374. return fmt.Errorf("failed to fetch chunk: %w", err)
  375. }
  376. resp, err := s.conn.ApplySnapshotChunkSync(ctx, abci.RequestApplySnapshotChunk{
  377. Index: chunk.Index,
  378. Chunk: chunk.Chunk,
  379. Sender: string(chunk.Sender),
  380. })
  381. if err != nil {
  382. return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err)
  383. }
  384. s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height,
  385. "format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size())
  386. // Discard and refetch any chunks as requested by the app
  387. for _, index := range resp.RefetchChunks {
  388. err := chunks.Discard(index)
  389. if err != nil {
  390. return fmt.Errorf("failed to discard chunk %v: %w", index, err)
  391. }
  392. }
  393. // Reject any senders as requested by the app
  394. for _, sender := range resp.RejectSenders {
  395. if sender != "" {
  396. peerID := types.NodeID(sender)
  397. s.snapshots.RejectPeer(peerID)
  398. if err := chunks.DiscardSender(peerID); err != nil {
  399. return fmt.Errorf("failed to reject sender: %w", err)
  400. }
  401. }
  402. }
  403. switch resp.Result {
  404. case abci.ResponseApplySnapshotChunk_ACCEPT:
  405. s.metrics.SnapshotChunk.Add(1)
  406. s.avgChunkTime = time.Since(start).Nanoseconds() / int64(chunks.numChunksReturned())
  407. s.metrics.ChunkProcessAvgTime.Set(float64(s.avgChunkTime))
  408. case abci.ResponseApplySnapshotChunk_ABORT:
  409. return errAbort
  410. case abci.ResponseApplySnapshotChunk_RETRY:
  411. chunks.Retry(chunk.Index)
  412. case abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT:
  413. return errRetrySnapshot
  414. case abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT:
  415. return errRejectSnapshot
  416. default:
  417. return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result)
  418. }
  419. }
  420. }
  421. // fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks
  422. // will be received from the reactor via syncer.AddChunks() to chunkQueue.Add().
  423. func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) {
  424. var (
  425. next = true
  426. index uint32
  427. err error
  428. )
  429. for {
  430. if next {
  431. index, err = chunks.Allocate()
  432. if errors.Is(err, errDone) {
  433. // Keep checking until the context is canceled (restore is done), in case any
  434. // chunks need to be refetched.
  435. select {
  436. case <-ctx.Done():
  437. return
  438. case <-s.closeCh:
  439. return
  440. case <-time.After(2 * time.Second):
  441. continue
  442. }
  443. }
  444. if err != nil {
  445. s.logger.Error("Failed to allocate chunk from queue", "err", err)
  446. return
  447. }
  448. }
  449. s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height,
  450. "format", snapshot.Format, "chunk", index, "total", chunks.Size())
  451. ticker := time.NewTicker(s.retryTimeout)
  452. defer ticker.Stop()
  453. s.requestChunk(snapshot, index)
  454. select {
  455. case <-chunks.WaitFor(index):
  456. next = true
  457. case <-ticker.C:
  458. next = false
  459. case <-ctx.Done():
  460. return
  461. case <-s.closeCh:
  462. return
  463. }
  464. ticker.Stop()
  465. }
  466. }
  467. // requestChunk requests a chunk from a peer.
  468. func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
  469. peer := s.snapshots.GetPeer(snapshot)
  470. if peer == "" {
  471. s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height,
  472. "format", snapshot.Format, "hash", snapshot.Hash)
  473. return
  474. }
  475. s.logger.Debug(
  476. "Requesting snapshot chunk",
  477. "height", snapshot.Height,
  478. "format", snapshot.Format,
  479. "chunk", chunk,
  480. "peer", peer,
  481. )
  482. msg := p2p.Envelope{
  483. To: peer,
  484. Message: &ssproto.ChunkRequest{
  485. Height: snapshot.Height,
  486. Format: snapshot.Format,
  487. Index: chunk,
  488. },
  489. }
  490. select {
  491. case s.chunkCh <- msg:
  492. case <-s.closeCh:
  493. }
  494. }
  495. // verifyApp verifies the sync, checking the app hash and last block height. It returns the
  496. // app version, which should be returned as part of the initial state.
  497. func (s *syncer) verifyApp(snapshot *snapshot) (uint64, error) {
  498. resp, err := s.connQuery.InfoSync(context.Background(), proxy.RequestInfo)
  499. if err != nil {
  500. return 0, fmt.Errorf("failed to query ABCI app for appHash: %w", err)
  501. }
  502. if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) {
  503. s.logger.Error("appHash verification failed",
  504. "expected", snapshot.trustedAppHash,
  505. "actual", resp.LastBlockAppHash)
  506. return 0, errVerifyFailed
  507. }
  508. if uint64(resp.LastBlockHeight) != snapshot.Height {
  509. s.logger.Error(
  510. "ABCI app reported unexpected last block height",
  511. "expected", snapshot.Height,
  512. "actual", resp.LastBlockHeight,
  513. )
  514. return 0, errVerifyFailed
  515. }
  516. s.logger.Info("Verified ABCI app", "height", snapshot.Height, "appHash", snapshot.trustedAppHash)
  517. return resp.AppVersion, nil
  518. }