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.

1114 lines
31 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
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. "reflect"
  8. "runtime/debug"
  9. "sort"
  10. "sync"
  11. "time"
  12. abciclient "github.com/tendermint/tendermint/abci/client"
  13. abci "github.com/tendermint/tendermint/abci/types"
  14. "github.com/tendermint/tendermint/config"
  15. "github.com/tendermint/tendermint/internal/eventbus"
  16. "github.com/tendermint/tendermint/internal/p2p"
  17. sm "github.com/tendermint/tendermint/internal/state"
  18. "github.com/tendermint/tendermint/internal/store"
  19. "github.com/tendermint/tendermint/libs/log"
  20. "github.com/tendermint/tendermint/libs/service"
  21. "github.com/tendermint/tendermint/light"
  22. "github.com/tendermint/tendermint/light/provider"
  23. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  24. "github.com/tendermint/tendermint/types"
  25. )
  26. var (
  27. _ service.Service = (*Reactor)(nil)
  28. _ p2p.Wrapper = (*ssproto.Message)(nil)
  29. )
  30. const (
  31. // SnapshotChannel exchanges snapshot metadata
  32. SnapshotChannel = p2p.ChannelID(0x60)
  33. // ChunkChannel exchanges chunk contents
  34. ChunkChannel = p2p.ChannelID(0x61)
  35. // LightBlockChannel exchanges light blocks
  36. LightBlockChannel = p2p.ChannelID(0x62)
  37. // ParamsChannel exchanges consensus params
  38. ParamsChannel = p2p.ChannelID(0x63)
  39. // recentSnapshots is the number of recent snapshots to send and receive per peer.
  40. recentSnapshots = 10
  41. // snapshotMsgSize is the maximum size of a snapshotResponseMessage
  42. snapshotMsgSize = int(4e6) // ~4MB
  43. // chunkMsgSize is the maximum size of a chunkResponseMessage
  44. chunkMsgSize = int(16e6) // ~16MB
  45. // lightBlockMsgSize is the maximum size of a lightBlockResponseMessage
  46. lightBlockMsgSize = int(1e7) // ~1MB
  47. // paramMsgSize is the maximum size of a paramsResponseMessage
  48. paramMsgSize = int(1e5) // ~100kb
  49. // lightBlockResponseTimeout is how long the dispatcher waits for a peer to
  50. // return a light block
  51. lightBlockResponseTimeout = 10 * time.Second
  52. // consensusParamsResponseTimeout is the time the p2p state provider waits
  53. // before performing a secondary call
  54. consensusParamsResponseTimeout = 5 * time.Second
  55. // maxLightBlockRequestRetries is the amount of retries acceptable before
  56. // the backfill process aborts
  57. maxLightBlockRequestRetries = 20
  58. )
  59. func getChannelDescriptors() map[p2p.ChannelID]*p2p.ChannelDescriptor {
  60. return map[p2p.ChannelID]*p2p.ChannelDescriptor{
  61. SnapshotChannel: {
  62. ID: SnapshotChannel,
  63. MessageType: new(ssproto.Message),
  64. Priority: 6,
  65. SendQueueCapacity: 10,
  66. RecvMessageCapacity: snapshotMsgSize,
  67. RecvBufferCapacity: 128,
  68. },
  69. ChunkChannel: {
  70. ID: ChunkChannel,
  71. Priority: 3,
  72. MessageType: new(ssproto.Message),
  73. SendQueueCapacity: 4,
  74. RecvMessageCapacity: chunkMsgSize,
  75. RecvBufferCapacity: 128,
  76. },
  77. LightBlockChannel: {
  78. ID: LightBlockChannel,
  79. MessageType: new(ssproto.Message),
  80. Priority: 5,
  81. SendQueueCapacity: 10,
  82. RecvMessageCapacity: lightBlockMsgSize,
  83. RecvBufferCapacity: 128,
  84. },
  85. ParamsChannel: {
  86. ID: ParamsChannel,
  87. MessageType: new(ssproto.Message),
  88. Priority: 2,
  89. SendQueueCapacity: 10,
  90. RecvMessageCapacity: paramMsgSize,
  91. RecvBufferCapacity: 128,
  92. },
  93. }
  94. }
  95. // Metricer defines an interface used for the rpc sync info query, please see statesync.metrics
  96. // for the details.
  97. type Metricer interface {
  98. TotalSnapshots() int64
  99. ChunkProcessAvgTime() time.Duration
  100. SnapshotHeight() int64
  101. SnapshotChunksCount() int64
  102. SnapshotChunksTotal() int64
  103. BackFilledBlocks() int64
  104. BackFillBlocksTotal() int64
  105. }
  106. // Reactor handles state sync, both restoring snapshots for the local node and
  107. // serving snapshots for other nodes.
  108. type Reactor struct {
  109. service.BaseService
  110. logger log.Logger
  111. chainID string
  112. initialHeight int64
  113. cfg config.StateSyncConfig
  114. stateStore sm.Store
  115. blockStore *store.BlockStore
  116. conn abciclient.Client
  117. tempDir string
  118. snapshotCh *p2p.Channel
  119. chunkCh *p2p.Channel
  120. blockCh *p2p.Channel
  121. paramsCh *p2p.Channel
  122. peerUpdates *p2p.PeerUpdates
  123. // Dispatcher is used to multiplex light block requests and responses over multiple
  124. // peers used by the p2p state provider and in reverse sync.
  125. dispatcher *Dispatcher
  126. peers *peerList
  127. // These will only be set when a state sync is in progress. It is used to feed
  128. // received snapshots and chunks into the syncer and manage incoming and outgoing
  129. // providers.
  130. mtx sync.RWMutex
  131. syncer *syncer
  132. providers map[types.NodeID]*BlockProvider
  133. stateProvider StateProvider
  134. eventBus *eventbus.EventBus
  135. metrics *Metrics
  136. backfillBlockTotal int64
  137. backfilledBlocks int64
  138. }
  139. // NewReactor returns a reference to a new state sync reactor, which implements
  140. // the service.Service interface. It accepts a logger, connections for snapshots
  141. // and querying, references to p2p Channels and a channel to listen for peer
  142. // updates on. Note, the reactor will close all p2p Channels when stopping.
  143. func NewReactor(
  144. ctx context.Context,
  145. chainID string,
  146. initialHeight int64,
  147. cfg config.StateSyncConfig,
  148. logger log.Logger,
  149. conn abciclient.Client,
  150. channelCreator p2p.ChannelCreator,
  151. peerUpdates *p2p.PeerUpdates,
  152. stateStore sm.Store,
  153. blockStore *store.BlockStore,
  154. tempDir string,
  155. ssMetrics *Metrics,
  156. eventBus *eventbus.EventBus,
  157. ) (*Reactor, error) {
  158. chDesc := getChannelDescriptors()
  159. snapshotCh, err := channelCreator(ctx, chDesc[SnapshotChannel])
  160. if err != nil {
  161. return nil, err
  162. }
  163. chunkCh, err := channelCreator(ctx, chDesc[ChunkChannel])
  164. if err != nil {
  165. return nil, err
  166. }
  167. blockCh, err := channelCreator(ctx, chDesc[LightBlockChannel])
  168. if err != nil {
  169. return nil, err
  170. }
  171. paramsCh, err := channelCreator(ctx, chDesc[ParamsChannel])
  172. if err != nil {
  173. return nil, err
  174. }
  175. r := &Reactor{
  176. logger: logger,
  177. chainID: chainID,
  178. initialHeight: initialHeight,
  179. cfg: cfg,
  180. conn: conn,
  181. snapshotCh: snapshotCh,
  182. chunkCh: chunkCh,
  183. blockCh: blockCh,
  184. paramsCh: paramsCh,
  185. peerUpdates: peerUpdates,
  186. tempDir: tempDir,
  187. stateStore: stateStore,
  188. blockStore: blockStore,
  189. peers: newPeerList(),
  190. dispatcher: NewDispatcher(blockCh),
  191. providers: make(map[types.NodeID]*BlockProvider),
  192. metrics: ssMetrics,
  193. eventBus: eventBus,
  194. }
  195. r.BaseService = *service.NewBaseService(logger, "StateSync", r)
  196. return r, nil
  197. }
  198. // OnStart starts separate go routines for each p2p Channel and listens for
  199. // envelopes on each. In addition, it also listens for peer updates and handles
  200. // messages on that p2p channel accordingly. Note, we do not launch a go-routine to
  201. // handle individual envelopes as to not have to deal with bounding workers or pools.
  202. // The caller must be sure to execute OnStop to ensure the outbound p2p Channels are
  203. // closed. No error is returned.
  204. func (r *Reactor) OnStart(ctx context.Context) error {
  205. go r.processCh(ctx, r.snapshotCh, "snapshot")
  206. go r.processCh(ctx, r.chunkCh, "chunk")
  207. go r.processCh(ctx, r.blockCh, "light block")
  208. go r.processCh(ctx, r.paramsCh, "consensus params")
  209. go r.processPeerUpdates(ctx)
  210. return nil
  211. }
  212. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  213. // blocking until they all exit.
  214. func (r *Reactor) OnStop() {
  215. // tell the dispatcher to stop sending any more requests
  216. r.dispatcher.Close()
  217. }
  218. func (r *Reactor) PublishStatus(ctx context.Context, event types.EventDataStateSyncStatus) error {
  219. if r.eventBus == nil {
  220. return errors.New("event system is not configured")
  221. }
  222. return r.eventBus.PublishEventStateSyncStatus(ctx, event)
  223. }
  224. // Sync runs a state sync, fetching snapshots and providing chunks to the
  225. // application. At the close of the operation, Sync will bootstrap the state
  226. // store and persist the commit at that height so that either consensus or
  227. // blocksync can commence. It will then proceed to backfill the necessary amount
  228. // of historical blocks before participating in consensus
  229. func (r *Reactor) Sync(ctx context.Context) (sm.State, error) {
  230. // We need at least two peers (for cross-referencing of light blocks) before we can
  231. // begin state sync
  232. if err := r.waitForEnoughPeers(ctx, 2); err != nil {
  233. return sm.State{}, err
  234. }
  235. r.mtx.Lock()
  236. if r.syncer != nil {
  237. r.mtx.Unlock()
  238. return sm.State{}, errors.New("a state sync is already in progress")
  239. }
  240. if err := r.initStateProvider(ctx, r.chainID, r.initialHeight); err != nil {
  241. r.mtx.Unlock()
  242. return sm.State{}, err
  243. }
  244. r.syncer = newSyncer(
  245. r.cfg,
  246. r.logger,
  247. r.conn,
  248. r.stateProvider,
  249. r.snapshotCh,
  250. r.chunkCh,
  251. r.tempDir,
  252. r.metrics,
  253. )
  254. r.mtx.Unlock()
  255. defer func() {
  256. r.mtx.Lock()
  257. // reset syncing objects at the close of Sync
  258. r.syncer = nil
  259. r.stateProvider = nil
  260. r.mtx.Unlock()
  261. }()
  262. requestSnapshotsHook := func() error {
  263. // request snapshots from all currently connected peers
  264. return r.snapshotCh.Send(ctx, p2p.Envelope{
  265. Broadcast: true,
  266. Message: &ssproto.SnapshotsRequest{},
  267. })
  268. }
  269. state, commit, err := r.syncer.SyncAny(ctx, r.cfg.DiscoveryTime, requestSnapshotsHook)
  270. if err != nil {
  271. return sm.State{}, err
  272. }
  273. err = r.stateStore.Bootstrap(state)
  274. if err != nil {
  275. return sm.State{}, fmt.Errorf("failed to bootstrap node with new state: %w", err)
  276. }
  277. err = r.blockStore.SaveSeenCommit(state.LastBlockHeight, commit)
  278. if err != nil {
  279. return sm.State{}, fmt.Errorf("failed to store last seen commit: %w", err)
  280. }
  281. err = r.Backfill(ctx, state)
  282. if err != nil {
  283. r.logger.Error("backfill failed. Proceeding optimistically...", "err", err)
  284. }
  285. return state, nil
  286. }
  287. // Backfill sequentially fetches, verifies and stores light blocks in reverse
  288. // order. It does not stop verifying blocks until reaching a block with a height
  289. // and time that is less or equal to the stopHeight and stopTime. The
  290. // trustedBlockID should be of the header at startHeight.
  291. func (r *Reactor) Backfill(ctx context.Context, state sm.State) error {
  292. params := state.ConsensusParams.Evidence
  293. stopHeight := state.LastBlockHeight - params.MaxAgeNumBlocks
  294. stopTime := state.LastBlockTime.Add(-params.MaxAgeDuration)
  295. // ensure that stop height doesn't go below the initial height
  296. if stopHeight < state.InitialHeight {
  297. stopHeight = state.InitialHeight
  298. // this essentially makes stop time a void criteria for termination
  299. stopTime = state.LastBlockTime
  300. }
  301. return r.backfill(
  302. ctx,
  303. state.ChainID,
  304. state.LastBlockHeight,
  305. stopHeight,
  306. state.InitialHeight,
  307. state.LastBlockID,
  308. stopTime,
  309. )
  310. }
  311. func (r *Reactor) backfill(
  312. ctx context.Context,
  313. chainID string,
  314. startHeight, stopHeight, initialHeight int64,
  315. trustedBlockID types.BlockID,
  316. stopTime time.Time,
  317. ) error {
  318. r.logger.Info("starting backfill process...", "startHeight", startHeight,
  319. "stopHeight", stopHeight, "stopTime", stopTime, "trustedBlockID", trustedBlockID)
  320. r.backfillBlockTotal = startHeight - stopHeight + 1
  321. r.metrics.BackFillBlocksTotal.Set(float64(r.backfillBlockTotal))
  322. const sleepTime = 1 * time.Second
  323. var (
  324. lastValidatorSet *types.ValidatorSet
  325. lastChangeHeight = startHeight
  326. )
  327. queue := newBlockQueue(startHeight, stopHeight, initialHeight, stopTime, maxLightBlockRequestRetries)
  328. // fetch light blocks across four workers. The aim with deploying concurrent
  329. // workers is to equate the network messaging time with the verification
  330. // time. Ideally we want the verification process to never have to be
  331. // waiting on blocks. If it takes 4s to retrieve a block and 1s to verify
  332. // it, then steady state involves four workers.
  333. for i := 0; i < int(r.cfg.Fetchers); i++ {
  334. ctxWithCancel, cancel := context.WithCancel(ctx)
  335. defer cancel()
  336. go func() {
  337. for {
  338. select {
  339. case <-ctx.Done():
  340. return
  341. case height := <-queue.nextHeight():
  342. // pop the next peer of the list to send a request to
  343. peer := r.peers.Pop(ctx)
  344. r.logger.Debug("fetching next block", "height", height, "peer", peer)
  345. subCtx, cancel := context.WithTimeout(ctxWithCancel, lightBlockResponseTimeout)
  346. defer cancel()
  347. lb, err := func() (*types.LightBlock, error) {
  348. defer cancel()
  349. // request the light block with a timeout
  350. return r.dispatcher.LightBlock(subCtx, height, peer)
  351. }()
  352. // once the peer has returned a value, add it back to the peer list to be used again
  353. r.peers.Append(peer)
  354. if errors.Is(err, context.Canceled) {
  355. return
  356. }
  357. if err != nil {
  358. queue.retry(height)
  359. if errors.Is(err, errNoConnectedPeers) {
  360. r.logger.Info("backfill: no connected peers to fetch light blocks from; sleeping...",
  361. "sleepTime", sleepTime)
  362. time.Sleep(sleepTime)
  363. } else {
  364. // we don't punish the peer as it might just have not responded in time
  365. r.logger.Info("backfill: error with fetching light block",
  366. "height", height, "err", err)
  367. }
  368. continue
  369. }
  370. if lb == nil {
  371. r.logger.Info("backfill: peer didn't have block, fetching from another peer", "height", height)
  372. queue.retry(height)
  373. // As we are fetching blocks backwards, if this node doesn't have the block it likely doesn't
  374. // have any prior ones, thus we remove it from the peer list.
  375. r.peers.Remove(peer)
  376. continue
  377. }
  378. // run a validate basic. This checks the validator set and commit
  379. // hashes line up
  380. err = lb.ValidateBasic(chainID)
  381. if err != nil || lb.Height != height {
  382. r.logger.Info("backfill: fetched light block failed validate basic, removing peer...",
  383. "err", err, "height", height)
  384. queue.retry(height)
  385. if serr := r.blockCh.SendError(ctx, p2p.PeerError{
  386. NodeID: peer,
  387. Err: fmt.Errorf("received invalid light block: %w", err),
  388. }); serr != nil {
  389. return
  390. }
  391. continue
  392. }
  393. // add block to queue to be verified
  394. queue.add(lightBlockResponse{
  395. block: lb,
  396. peer: peer,
  397. })
  398. r.logger.Debug("backfill: added light block to processing queue", "height", height)
  399. case <-queue.done():
  400. return
  401. }
  402. }
  403. }()
  404. }
  405. // verify all light blocks
  406. for {
  407. select {
  408. case <-ctx.Done():
  409. queue.close()
  410. return nil
  411. case resp := <-queue.verifyNext():
  412. // validate the header hash. We take the last block id of the
  413. // previous header (i.e. one height above) as the trusted hash which
  414. // we equate to. ValidatorsHash and CommitHash have already been
  415. // checked in the `ValidateBasic`
  416. if w, g := trustedBlockID.Hash, resp.block.Hash(); !bytes.Equal(w, g) {
  417. r.logger.Info("received invalid light block. header hash doesn't match trusted LastBlockID",
  418. "trustedHash", w, "receivedHash", g, "height", resp.block.Height)
  419. if err := r.blockCh.SendError(ctx, p2p.PeerError{
  420. NodeID: resp.peer,
  421. Err: fmt.Errorf("received invalid light block. Expected hash %v, got: %v", w, g),
  422. }); err != nil {
  423. return nil
  424. }
  425. queue.retry(resp.block.Height)
  426. continue
  427. }
  428. // save the signed headers
  429. if err := r.blockStore.SaveSignedHeader(resp.block.SignedHeader, trustedBlockID); err != nil {
  430. return err
  431. }
  432. // check if there has been a change in the validator set
  433. if lastValidatorSet != nil && !bytes.Equal(resp.block.Header.ValidatorsHash, resp.block.Header.NextValidatorsHash) {
  434. // save all the heights that the last validator set was the same
  435. if err := r.stateStore.SaveValidatorSets(resp.block.Height+1, lastChangeHeight, lastValidatorSet); err != nil {
  436. return err
  437. }
  438. // update the lastChangeHeight
  439. lastChangeHeight = resp.block.Height
  440. }
  441. trustedBlockID = resp.block.LastBlockID
  442. queue.success()
  443. r.logger.Info("backfill: verified and stored light block", "height", resp.block.Height)
  444. lastValidatorSet = resp.block.ValidatorSet
  445. r.backfilledBlocks++
  446. r.metrics.BackFilledBlocks.Add(1)
  447. // The block height might be less than the stopHeight because of the stopTime condition
  448. // hasn't been fulfilled.
  449. if resp.block.Height < stopHeight {
  450. r.backfillBlockTotal++
  451. r.metrics.BackFillBlocksTotal.Set(float64(r.backfillBlockTotal))
  452. }
  453. case <-queue.done():
  454. if err := queue.error(); err != nil {
  455. return err
  456. }
  457. // save the final batch of validators
  458. if err := r.stateStore.SaveValidatorSets(queue.terminal.Height, lastChangeHeight, lastValidatorSet); err != nil {
  459. return err
  460. }
  461. r.logger.Info("successfully completed backfill process", "endHeight", queue.terminal.Height)
  462. return nil
  463. }
  464. }
  465. }
  466. // handleSnapshotMessage handles envelopes sent from peers on the
  467. // SnapshotChannel. It returns an error only if the Envelope.Message is unknown
  468. // for this channel. This should never be called outside of handleMessage.
  469. func (r *Reactor) handleSnapshotMessage(ctx context.Context, envelope *p2p.Envelope) error {
  470. logger := r.logger.With("peer", envelope.From)
  471. switch msg := envelope.Message.(type) {
  472. case *ssproto.SnapshotsRequest:
  473. snapshots, err := r.recentSnapshots(ctx, recentSnapshots)
  474. if err != nil {
  475. logger.Error("failed to fetch snapshots", "err", err)
  476. return nil
  477. }
  478. for _, snapshot := range snapshots {
  479. logger.Info(
  480. "advertising snapshot",
  481. "height", snapshot.Height,
  482. "format", snapshot.Format,
  483. "peer", envelope.From,
  484. )
  485. if err := r.snapshotCh.Send(ctx, p2p.Envelope{
  486. To: envelope.From,
  487. Message: &ssproto.SnapshotsResponse{
  488. Height: snapshot.Height,
  489. Format: snapshot.Format,
  490. Chunks: snapshot.Chunks,
  491. Hash: snapshot.Hash,
  492. Metadata: snapshot.Metadata,
  493. },
  494. }); err != nil {
  495. return err
  496. }
  497. }
  498. case *ssproto.SnapshotsResponse:
  499. r.mtx.RLock()
  500. defer r.mtx.RUnlock()
  501. if r.syncer == nil {
  502. logger.Debug("received unexpected snapshot; no state sync in progress")
  503. return nil
  504. }
  505. logger.Info("received snapshot", "height", msg.Height, "format", msg.Format)
  506. _, err := r.syncer.AddSnapshot(envelope.From, &snapshot{
  507. Height: msg.Height,
  508. Format: msg.Format,
  509. Chunks: msg.Chunks,
  510. Hash: msg.Hash,
  511. Metadata: msg.Metadata,
  512. })
  513. if err != nil {
  514. logger.Error(
  515. "failed to add snapshot",
  516. "height", msg.Height,
  517. "format", msg.Format,
  518. "err", err,
  519. "channel", r.snapshotCh.ID,
  520. )
  521. return nil
  522. }
  523. logger.Info("added snapshot", "height", msg.Height, "format", msg.Format)
  524. default:
  525. return fmt.Errorf("received unknown message: %T", msg)
  526. }
  527. return nil
  528. }
  529. // handleChunkMessage handles envelopes sent from peers on the ChunkChannel.
  530. // It returns an error only if the Envelope.Message is unknown for this channel.
  531. // This should never be called outside of handleMessage.
  532. func (r *Reactor) handleChunkMessage(ctx context.Context, envelope *p2p.Envelope) error {
  533. switch msg := envelope.Message.(type) {
  534. case *ssproto.ChunkRequest:
  535. r.logger.Debug(
  536. "received chunk request",
  537. "height", msg.Height,
  538. "format", msg.Format,
  539. "chunk", msg.Index,
  540. "peer", envelope.From,
  541. )
  542. resp, err := r.conn.LoadSnapshotChunk(ctx, abci.RequestLoadSnapshotChunk{
  543. Height: msg.Height,
  544. Format: msg.Format,
  545. Chunk: msg.Index,
  546. })
  547. if err != nil {
  548. r.logger.Error(
  549. "failed to load chunk",
  550. "height", msg.Height,
  551. "format", msg.Format,
  552. "chunk", msg.Index,
  553. "err", err,
  554. "peer", envelope.From,
  555. )
  556. return nil
  557. }
  558. r.logger.Debug(
  559. "sending chunk",
  560. "height", msg.Height,
  561. "format", msg.Format,
  562. "chunk", msg.Index,
  563. "peer", envelope.From,
  564. )
  565. if err := r.chunkCh.Send(ctx, p2p.Envelope{
  566. To: envelope.From,
  567. Message: &ssproto.ChunkResponse{
  568. Height: msg.Height,
  569. Format: msg.Format,
  570. Index: msg.Index,
  571. Chunk: resp.Chunk,
  572. Missing: resp.Chunk == nil,
  573. },
  574. }); err != nil {
  575. return err
  576. }
  577. case *ssproto.ChunkResponse:
  578. r.mtx.RLock()
  579. defer r.mtx.RUnlock()
  580. if r.syncer == nil {
  581. r.logger.Debug("received unexpected chunk; no state sync in progress", "peer", envelope.From)
  582. return nil
  583. }
  584. r.logger.Debug(
  585. "received chunk; adding to sync",
  586. "height", msg.Height,
  587. "format", msg.Format,
  588. "chunk", msg.Index,
  589. "peer", envelope.From,
  590. )
  591. _, err := r.syncer.AddChunk(&chunk{
  592. Height: msg.Height,
  593. Format: msg.Format,
  594. Index: msg.Index,
  595. Chunk: msg.Chunk,
  596. Sender: envelope.From,
  597. })
  598. if err != nil {
  599. r.logger.Error(
  600. "failed to add chunk",
  601. "height", msg.Height,
  602. "format", msg.Format,
  603. "chunk", msg.Index,
  604. "err", err,
  605. "peer", envelope.From,
  606. )
  607. return nil
  608. }
  609. default:
  610. return fmt.Errorf("received unknown message: %T", msg)
  611. }
  612. return nil
  613. }
  614. func (r *Reactor) handleLightBlockMessage(ctx context.Context, envelope *p2p.Envelope) error {
  615. switch msg := envelope.Message.(type) {
  616. case *ssproto.LightBlockRequest:
  617. r.logger.Info("received light block request", "height", msg.Height)
  618. lb, err := r.fetchLightBlock(msg.Height)
  619. if err != nil {
  620. r.logger.Error("failed to retrieve light block", "err", err, "height", msg.Height)
  621. return err
  622. }
  623. if lb == nil {
  624. if err := r.blockCh.Send(ctx, p2p.Envelope{
  625. To: envelope.From,
  626. Message: &ssproto.LightBlockResponse{
  627. LightBlock: nil,
  628. },
  629. }); err != nil {
  630. return err
  631. }
  632. return nil
  633. }
  634. lbproto, err := lb.ToProto()
  635. if err != nil {
  636. r.logger.Error("marshaling light block to proto", "err", err)
  637. return nil
  638. }
  639. // NOTE: If we don't have the light block we will send a nil light block
  640. // back to the requested node, indicating that we don't have it.
  641. if err := r.blockCh.Send(ctx, p2p.Envelope{
  642. To: envelope.From,
  643. Message: &ssproto.LightBlockResponse{
  644. LightBlock: lbproto,
  645. },
  646. }); err != nil {
  647. return err
  648. }
  649. case *ssproto.LightBlockResponse:
  650. var height int64
  651. if msg.LightBlock != nil {
  652. height = msg.LightBlock.SignedHeader.Header.Height
  653. }
  654. r.logger.Info("received light block response", "peer", envelope.From, "height", height)
  655. if err := r.dispatcher.Respond(ctx, msg.LightBlock, envelope.From); err != nil {
  656. if errors.Is(err, context.Canceled) {
  657. return err
  658. }
  659. r.logger.Error("error processing light block response", "err", err, "height", height)
  660. }
  661. default:
  662. return fmt.Errorf("received unknown message: %T", msg)
  663. }
  664. return nil
  665. }
  666. func (r *Reactor) handleParamsMessage(ctx context.Context, envelope *p2p.Envelope) error {
  667. switch msg := envelope.Message.(type) {
  668. case *ssproto.ParamsRequest:
  669. r.logger.Debug("received consensus params request", "height", msg.Height)
  670. cp, err := r.stateStore.LoadConsensusParams(int64(msg.Height))
  671. if err != nil {
  672. r.logger.Error("failed to fetch requested consensus params", "err", err, "height", msg.Height)
  673. return nil
  674. }
  675. cpproto := cp.ToProto()
  676. if err := r.paramsCh.Send(ctx, p2p.Envelope{
  677. To: envelope.From,
  678. Message: &ssproto.ParamsResponse{
  679. Height: msg.Height,
  680. ConsensusParams: cpproto,
  681. },
  682. }); err != nil {
  683. return err
  684. }
  685. case *ssproto.ParamsResponse:
  686. r.mtx.RLock()
  687. defer r.mtx.RUnlock()
  688. r.logger.Debug("received consensus params response", "height", msg.Height)
  689. cp := types.ConsensusParamsFromProto(msg.ConsensusParams)
  690. if sp, ok := r.stateProvider.(*stateProviderP2P); ok {
  691. select {
  692. case sp.paramsRecvCh <- cp:
  693. case <-time.After(time.Second):
  694. return errors.New("failed to send consensus params, stateprovider not ready for response")
  695. }
  696. } else {
  697. r.logger.Debug("received unexpected params response; using RPC state provider", "peer", envelope.From)
  698. }
  699. default:
  700. return fmt.Errorf("received unknown message: %T", msg)
  701. }
  702. return nil
  703. }
  704. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  705. // It will handle errors and any possible panics gracefully. A caller can handle
  706. // any error returned by sending a PeerError on the respective channel.
  707. func (r *Reactor) handleMessage(ctx context.Context, chID p2p.ChannelID, envelope *p2p.Envelope) (err error) {
  708. defer func() {
  709. if e := recover(); e != nil {
  710. err = fmt.Errorf("panic in processing message: %v", e)
  711. r.logger.Error(
  712. "recovering from processing message panic",
  713. "err", err,
  714. "stack", string(debug.Stack()),
  715. )
  716. }
  717. }()
  718. r.logger.Debug("received message", "message", reflect.TypeOf(envelope.Message), "peer", envelope.From)
  719. switch chID {
  720. case SnapshotChannel:
  721. err = r.handleSnapshotMessage(ctx, envelope)
  722. case ChunkChannel:
  723. err = r.handleChunkMessage(ctx, envelope)
  724. case LightBlockChannel:
  725. err = r.handleLightBlockMessage(ctx, envelope)
  726. case ParamsChannel:
  727. err = r.handleParamsMessage(ctx, envelope)
  728. default:
  729. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  730. }
  731. return err
  732. }
  733. // processCh routes state sync messages to their respective handlers. Any error
  734. // encountered during message execution will result in a PeerError being sent on
  735. // the respective channel. When the reactor is stopped, we will catch the signal
  736. // and close the p2p Channel gracefully.
  737. func (r *Reactor) processCh(ctx context.Context, ch *p2p.Channel, chName string) {
  738. iter := ch.Receive(ctx)
  739. for iter.Next(ctx) {
  740. envelope := iter.Envelope()
  741. if err := r.handleMessage(ctx, ch.ID, envelope); err != nil {
  742. r.logger.Error("failed to process message",
  743. "err", err,
  744. "channel", chName,
  745. "ch_id", ch.ID,
  746. "envelope", envelope)
  747. if serr := ch.SendError(ctx, p2p.PeerError{
  748. NodeID: envelope.From,
  749. Err: err,
  750. }); serr != nil {
  751. return
  752. }
  753. }
  754. }
  755. }
  756. // processPeerUpdate processes a PeerUpdate, returning an error upon failing to
  757. // handle the PeerUpdate or if a panic is recovered.
  758. func (r *Reactor) processPeerUpdate(ctx context.Context, peerUpdate p2p.PeerUpdate) {
  759. r.logger.Info("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  760. switch peerUpdate.Status {
  761. case p2p.PeerStatusUp:
  762. if peerUpdate.Channels.Contains(SnapshotChannel) &&
  763. peerUpdate.Channels.Contains(ChunkChannel) &&
  764. peerUpdate.Channels.Contains(LightBlockChannel) &&
  765. peerUpdate.Channels.Contains(ParamsChannel) {
  766. r.peers.Append(peerUpdate.NodeID)
  767. } else {
  768. r.logger.Error("could not use peer for statesync", "peer", peerUpdate.NodeID)
  769. }
  770. case p2p.PeerStatusDown:
  771. r.peers.Remove(peerUpdate.NodeID)
  772. }
  773. r.mtx.Lock()
  774. defer r.mtx.Unlock()
  775. if r.syncer == nil {
  776. return
  777. }
  778. switch peerUpdate.Status {
  779. case p2p.PeerStatusUp:
  780. newProvider := NewBlockProvider(peerUpdate.NodeID, r.chainID, r.dispatcher)
  781. r.providers[peerUpdate.NodeID] = newProvider
  782. err := r.syncer.AddPeer(ctx, peerUpdate.NodeID)
  783. if err != nil {
  784. r.logger.Error("error adding peer to syncer", "error", err)
  785. return
  786. }
  787. if sp, ok := r.stateProvider.(*stateProviderP2P); ok {
  788. // we do this in a separate routine to not block whilst waiting for the light client to finish
  789. // whatever call it's currently executing
  790. go sp.addProvider(newProvider)
  791. }
  792. case p2p.PeerStatusDown:
  793. delete(r.providers, peerUpdate.NodeID)
  794. r.syncer.RemovePeer(peerUpdate.NodeID)
  795. }
  796. r.logger.Info("processed peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  797. }
  798. // processPeerUpdates initiates a blocking process where we listen for and handle
  799. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  800. // close the p2p PeerUpdatesCh gracefully.
  801. func (r *Reactor) processPeerUpdates(ctx context.Context) {
  802. for {
  803. select {
  804. case <-ctx.Done():
  805. return
  806. case peerUpdate := <-r.peerUpdates.Updates():
  807. r.processPeerUpdate(ctx, peerUpdate)
  808. }
  809. }
  810. }
  811. // recentSnapshots fetches the n most recent snapshots from the app
  812. func (r *Reactor) recentSnapshots(ctx context.Context, n uint32) ([]*snapshot, error) {
  813. resp, err := r.conn.ListSnapshots(ctx, abci.RequestListSnapshots{})
  814. if err != nil {
  815. return nil, err
  816. }
  817. sort.Slice(resp.Snapshots, func(i, j int) bool {
  818. a := resp.Snapshots[i]
  819. b := resp.Snapshots[j]
  820. switch {
  821. case a.Height > b.Height:
  822. return true
  823. case a.Height == b.Height && a.Format > b.Format:
  824. return true
  825. default:
  826. return false
  827. }
  828. })
  829. snapshots := make([]*snapshot, 0, n)
  830. for i, s := range resp.Snapshots {
  831. if i >= recentSnapshots {
  832. break
  833. }
  834. snapshots = append(snapshots, &snapshot{
  835. Height: s.Height,
  836. Format: s.Format,
  837. Chunks: s.Chunks,
  838. Hash: s.Hash,
  839. Metadata: s.Metadata,
  840. })
  841. }
  842. return snapshots, nil
  843. }
  844. // fetchLightBlock works out whether the node has a light block at a particular
  845. // height and if so returns it so it can be gossiped to peers
  846. func (r *Reactor) fetchLightBlock(height uint64) (*types.LightBlock, error) {
  847. h := int64(height)
  848. blockMeta := r.blockStore.LoadBlockMeta(h)
  849. if blockMeta == nil {
  850. return nil, nil
  851. }
  852. commit := r.blockStore.LoadBlockCommit(h)
  853. if commit == nil {
  854. return nil, nil
  855. }
  856. vals, err := r.stateStore.LoadValidators(h)
  857. if err != nil {
  858. return nil, err
  859. }
  860. if vals == nil {
  861. return nil, nil
  862. }
  863. return &types.LightBlock{
  864. SignedHeader: &types.SignedHeader{
  865. Header: &blockMeta.Header,
  866. Commit: commit,
  867. },
  868. ValidatorSet: vals,
  869. }, nil
  870. }
  871. func (r *Reactor) waitForEnoughPeers(ctx context.Context, numPeers int) error {
  872. startAt := time.Now()
  873. t := time.NewTicker(100 * time.Millisecond)
  874. defer t.Stop()
  875. logT := time.NewTicker(time.Minute)
  876. defer logT.Stop()
  877. var iter int
  878. for r.peers.Len() < numPeers {
  879. iter++
  880. select {
  881. case <-ctx.Done():
  882. return fmt.Errorf("operation canceled while waiting for peers after %.2fs [%d/%d]",
  883. time.Since(startAt).Seconds(), r.peers.Len(), numPeers)
  884. case <-t.C:
  885. continue
  886. case <-logT.C:
  887. r.logger.Info("waiting for sufficient peers to start statesync",
  888. "duration", time.Since(startAt).String(),
  889. "target", numPeers,
  890. "peers", r.peers.Len(),
  891. "iters", iter,
  892. )
  893. continue
  894. }
  895. }
  896. return nil
  897. }
  898. func (r *Reactor) initStateProvider(ctx context.Context, chainID string, initialHeight int64) error {
  899. var err error
  900. to := light.TrustOptions{
  901. Period: r.cfg.TrustPeriod,
  902. Height: r.cfg.TrustHeight,
  903. Hash: r.cfg.TrustHashBytes(),
  904. }
  905. spLogger := r.logger.With("module", "stateprovider")
  906. spLogger.Info("initializing state provider", "trustPeriod", to.Period,
  907. "trustHeight", to.Height, "useP2P", r.cfg.UseP2P)
  908. if r.cfg.UseP2P {
  909. if err := r.waitForEnoughPeers(ctx, 2); err != nil {
  910. return err
  911. }
  912. peers := r.peers.All()
  913. providers := make([]provider.Provider, len(peers))
  914. for idx, p := range peers {
  915. providers[idx] = NewBlockProvider(p, chainID, r.dispatcher)
  916. }
  917. r.stateProvider, err = NewP2PStateProvider(ctx, chainID, initialHeight, providers, to, r.paramsCh, spLogger)
  918. if err != nil {
  919. return fmt.Errorf("failed to initialize P2P state provider: %w", err)
  920. }
  921. } else {
  922. r.stateProvider, err = NewRPCStateProvider(ctx, chainID, initialHeight, r.cfg.RPCServers, to, spLogger)
  923. if err != nil {
  924. return fmt.Errorf("failed to initialize RPC state provider: %w", err)
  925. }
  926. }
  927. return nil
  928. }
  929. func (r *Reactor) TotalSnapshots() int64 {
  930. r.mtx.RLock()
  931. defer r.mtx.RUnlock()
  932. if r.syncer != nil && r.syncer.snapshots != nil {
  933. return int64(len(r.syncer.snapshots.snapshots))
  934. }
  935. return 0
  936. }
  937. func (r *Reactor) ChunkProcessAvgTime() time.Duration {
  938. r.mtx.RLock()
  939. defer r.mtx.RUnlock()
  940. if r.syncer != nil {
  941. return time.Duration(r.syncer.avgChunkTime)
  942. }
  943. return time.Duration(0)
  944. }
  945. func (r *Reactor) SnapshotHeight() int64 {
  946. r.mtx.RLock()
  947. defer r.mtx.RUnlock()
  948. if r.syncer != nil {
  949. return r.syncer.lastSyncedSnapshotHeight
  950. }
  951. return 0
  952. }
  953. func (r *Reactor) SnapshotChunksCount() int64 {
  954. r.mtx.RLock()
  955. defer r.mtx.RUnlock()
  956. if r.syncer != nil && r.syncer.chunks != nil {
  957. return int64(r.syncer.chunks.numChunksReturned())
  958. }
  959. return 0
  960. }
  961. func (r *Reactor) SnapshotChunksTotal() int64 {
  962. r.mtx.RLock()
  963. defer r.mtx.RUnlock()
  964. if r.syncer != nil && r.syncer.processingSnapshot != nil {
  965. return int64(r.syncer.processingSnapshot.Chunks)
  966. }
  967. return 0
  968. }
  969. func (r *Reactor) BackFilledBlocks() int64 {
  970. r.mtx.RLock()
  971. defer r.mtx.RUnlock()
  972. return r.backfilledBlocks
  973. }
  974. func (r *Reactor) BackFillBlocksTotal() int64 {
  975. r.mtx.RLock()
  976. defer r.mtx.RUnlock()
  977. return r.backfillBlockTotal
  978. }