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.

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