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.

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