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.

638 lines
17 KiB

blockchain: Reorg reactor (#3561) * go routines in blockchain reactor * Added reference to the go routine diagram * Initial commit * cleanup * Undo testing_logger change, committed by mistake * Fix the test loggers * pulled some fsm code into pool.go * added pool tests * changes to the design added block requests under peer moved the request trigger in the reactor poolRoutine, triggered now by a ticker in general moved everything required for making block requests smarter in the poolRoutine added a simple map of heights to keep track of what will need to be requested next added a few more tests * send errors to FSM in a different channel than blocks send errors (RemovePeer) from switch on a different channel than the one receiving blocks renamed channels added more pool tests * more pool tests * lint errors * more tests * more tests * switch fast sync to new implementation * fixed data race in tests * cleanup * finished fsm tests * address golangci comments :) * address golangci comments :) * Added timeout on next block needed to advance * updating docs and cleanup * fix issue in test from previous cleanup * cleanup * Added termination scenarios, tests and more cleanup * small fixes to adr, comments and cleanup * Fix bug in sendRequest() If we tried to send a request to a peer not present in the switch, a missing continue statement caused the request to be blackholed in a peer that was removed and never retried. While this bug was manifesting, the reactor kept asking for other blocks that would be stored and never consumed. Added the number of unconsumed blocks in the math for requesting blocks ahead of current processing height so eventually there will be no more blocks requested until the already received ones are consumed. * remove bpPeer's didTimeout field * Use distinct err codes for peer timeout and FSM timeouts * Don't allow peers to update with lower height * review comments from Ethan and Zarko * some cleanup, renaming, comments * Move block execution in separate goroutine * Remove pool's numPending * review comments * fix lint, remove old blockchain reactor and duplicates in fsm tests * small reorg around peer after review comments * add the reactor spec * verify block only once * review comments * change to int for max number of pending requests * cleanup and godoc * Add configuration flag fast sync version * golangci fixes * fix config template * move both reactor versions under blockchain * cleanup, golint, renaming stuff * updated documentation, fixed more golint warnings * integrate with behavior package * sync with master * gofmt * add changelog_pending entry * move to improvments * suggestion to changelog entry
5 years ago
blockchain: Reorg reactor (#3561) * go routines in blockchain reactor * Added reference to the go routine diagram * Initial commit * cleanup * Undo testing_logger change, committed by mistake * Fix the test loggers * pulled some fsm code into pool.go * added pool tests * changes to the design added block requests under peer moved the request trigger in the reactor poolRoutine, triggered now by a ticker in general moved everything required for making block requests smarter in the poolRoutine added a simple map of heights to keep track of what will need to be requested next added a few more tests * send errors to FSM in a different channel than blocks send errors (RemovePeer) from switch on a different channel than the one receiving blocks renamed channels added more pool tests * more pool tests * lint errors * more tests * more tests * switch fast sync to new implementation * fixed data race in tests * cleanup * finished fsm tests * address golangci comments :) * address golangci comments :) * Added timeout on next block needed to advance * updating docs and cleanup * fix issue in test from previous cleanup * cleanup * Added termination scenarios, tests and more cleanup * small fixes to adr, comments and cleanup * Fix bug in sendRequest() If we tried to send a request to a peer not present in the switch, a missing continue statement caused the request to be blackholed in a peer that was removed and never retried. While this bug was manifesting, the reactor kept asking for other blocks that would be stored and never consumed. Added the number of unconsumed blocks in the math for requesting blocks ahead of current processing height so eventually there will be no more blocks requested until the already received ones are consumed. * remove bpPeer's didTimeout field * Use distinct err codes for peer timeout and FSM timeouts * Don't allow peers to update with lower height * review comments from Ethan and Zarko * some cleanup, renaming, comments * Move block execution in separate goroutine * Remove pool's numPending * review comments * fix lint, remove old blockchain reactor and duplicates in fsm tests * small reorg around peer after review comments * add the reactor spec * verify block only once * review comments * change to int for max number of pending requests * cleanup and godoc * Add configuration flag fast sync version * golangci fixes * fix config template * move both reactor versions under blockchain * cleanup, golint, renaming stuff * updated documentation, fixed more golint warnings * integrate with behavior package * sync with master * gofmt * add changelog_pending entry * move to improvments * suggestion to changelog entry
5 years ago
  1. package v0
  2. import (
  3. "fmt"
  4. "runtime/debug"
  5. "sync"
  6. "time"
  7. bc "github.com/tendermint/tendermint/internal/blocksync"
  8. cons "github.com/tendermint/tendermint/internal/consensus"
  9. "github.com/tendermint/tendermint/internal/p2p"
  10. "github.com/tendermint/tendermint/libs/log"
  11. "github.com/tendermint/tendermint/libs/service"
  12. tmSync "github.com/tendermint/tendermint/libs/sync"
  13. bcproto "github.com/tendermint/tendermint/proto/tendermint/blocksync"
  14. sm "github.com/tendermint/tendermint/state"
  15. "github.com/tendermint/tendermint/store"
  16. "github.com/tendermint/tendermint/types"
  17. )
  18. var (
  19. _ service.Service = (*Reactor)(nil)
  20. // ChannelShims contains a map of ChannelDescriptorShim objects, where each
  21. // object wraps a reference to a legacy p2p ChannelDescriptor and the corresponding
  22. // p2p proto.Message the new p2p Channel is responsible for handling.
  23. //
  24. //
  25. // TODO: Remove once p2p refactor is complete.
  26. // ref: https://github.com/tendermint/tendermint/issues/5670
  27. ChannelShims = map[p2p.ChannelID]*p2p.ChannelDescriptorShim{
  28. BlockSyncChannel: {
  29. MsgType: new(bcproto.Message),
  30. Descriptor: &p2p.ChannelDescriptor{
  31. ID: byte(BlockSyncChannel),
  32. Priority: 5,
  33. SendQueueCapacity: 1000,
  34. RecvBufferCapacity: 1024,
  35. RecvMessageCapacity: bc.MaxMsgSize,
  36. MaxSendBytes: 100,
  37. },
  38. },
  39. }
  40. )
  41. const (
  42. // BlockSyncChannel is a channel for blocks and status updates
  43. BlockSyncChannel = p2p.ChannelID(0x40)
  44. trySyncIntervalMS = 10
  45. // ask for best height every 10s
  46. statusUpdateIntervalSeconds = 10
  47. // check if we should switch to consensus reactor
  48. switchToConsensusIntervalSeconds = 1
  49. // switch to consensus after this duration of inactivity
  50. syncTimeout = 60 * time.Second
  51. )
  52. type consensusReactor interface {
  53. // For when we switch from block sync reactor to the consensus
  54. // machine.
  55. SwitchToConsensus(state sm.State, skipWAL bool)
  56. }
  57. type peerError struct {
  58. err error
  59. peerID types.NodeID
  60. }
  61. func (e peerError) Error() string {
  62. return fmt.Sprintf("error with peer %v: %s", e.peerID, e.err.Error())
  63. }
  64. // Reactor handles long-term catchup syncing.
  65. type Reactor struct {
  66. service.BaseService
  67. // immutable
  68. initialState sm.State
  69. blockExec *sm.BlockExecutor
  70. store *store.BlockStore
  71. pool *BlockPool
  72. consReactor consensusReactor
  73. blockSync *tmSync.AtomicBool
  74. blockSyncCh *p2p.Channel
  75. // blockSyncOutBridgeCh defines a channel that acts as a bridge between sending Envelope
  76. // messages that the reactor will consume in processBlockSyncCh and receiving messages
  77. // from the peer updates channel and other goroutines. We do this instead of directly
  78. // sending on blockSyncCh.Out to avoid race conditions in the case where other goroutines
  79. // send Envelopes directly to the to blockSyncCh.Out channel, since processBlockSyncCh
  80. // may close the blockSyncCh.Out channel at the same time that other goroutines send to
  81. // blockSyncCh.Out.
  82. blockSyncOutBridgeCh chan p2p.Envelope
  83. peerUpdates *p2p.PeerUpdates
  84. closeCh chan struct{}
  85. requestsCh <-chan BlockRequest
  86. errorsCh <-chan peerError
  87. // poolWG is used to synchronize the graceful shutdown of the poolRoutine and
  88. // requestRoutine spawned goroutines when stopping the reactor and before
  89. // stopping the p2p Channel(s).
  90. poolWG sync.WaitGroup
  91. metrics *cons.Metrics
  92. syncStartTime time.Time
  93. }
  94. // NewReactor returns new reactor instance.
  95. func NewReactor(
  96. logger log.Logger,
  97. state sm.State,
  98. blockExec *sm.BlockExecutor,
  99. store *store.BlockStore,
  100. consReactor consensusReactor,
  101. blockSyncCh *p2p.Channel,
  102. peerUpdates *p2p.PeerUpdates,
  103. blockSync bool,
  104. metrics *cons.Metrics,
  105. ) (*Reactor, error) {
  106. if state.LastBlockHeight != store.Height() {
  107. return nil, fmt.Errorf("state (%v) and store (%v) height mismatch", state.LastBlockHeight, store.Height())
  108. }
  109. startHeight := store.Height() + 1
  110. if startHeight == 1 {
  111. startHeight = state.InitialHeight
  112. }
  113. requestsCh := make(chan BlockRequest, maxTotalRequesters)
  114. errorsCh := make(chan peerError, maxPeerErrBuffer) // NOTE: The capacity should be larger than the peer count.
  115. r := &Reactor{
  116. initialState: state,
  117. blockExec: blockExec,
  118. store: store,
  119. pool: NewBlockPool(startHeight, requestsCh, errorsCh),
  120. consReactor: consReactor,
  121. blockSync: tmSync.NewBool(blockSync),
  122. requestsCh: requestsCh,
  123. errorsCh: errorsCh,
  124. blockSyncCh: blockSyncCh,
  125. blockSyncOutBridgeCh: make(chan p2p.Envelope),
  126. peerUpdates: peerUpdates,
  127. closeCh: make(chan struct{}),
  128. metrics: metrics,
  129. syncStartTime: time.Time{},
  130. }
  131. r.BaseService = *service.NewBaseService(logger, "BlockSync", r)
  132. return r, nil
  133. }
  134. // OnStart starts separate go routines for each p2p Channel and listens for
  135. // envelopes on each. In addition, it also listens for peer updates and handles
  136. // messages on that p2p channel accordingly. The caller must be sure to execute
  137. // OnStop to ensure the outbound p2p Channels are closed.
  138. //
  139. // If blockSync is enabled, we also start the pool and the pool processing
  140. // goroutine. If the pool fails to start, an error is returned.
  141. func (r *Reactor) OnStart() error {
  142. if r.blockSync.IsSet() {
  143. if err := r.pool.Start(); err != nil {
  144. return err
  145. }
  146. r.poolWG.Add(1)
  147. go r.poolRoutine(false)
  148. }
  149. go r.processBlockSyncCh()
  150. go r.processPeerUpdates()
  151. return nil
  152. }
  153. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  154. // blocking until they all exit.
  155. func (r *Reactor) OnStop() {
  156. if r.blockSync.IsSet() {
  157. if err := r.pool.Stop(); err != nil {
  158. r.Logger.Error("failed to stop pool", "err", err)
  159. }
  160. }
  161. // wait for the poolRoutine and requestRoutine goroutines to gracefully exit
  162. r.poolWG.Wait()
  163. // Close closeCh to signal to all spawned goroutines to gracefully exit. All
  164. // p2p Channels should execute Close().
  165. close(r.closeCh)
  166. // Wait for all p2p Channels to be closed before returning. This ensures we
  167. // can easily reason about synchronization of all p2p Channels and ensure no
  168. // panics will occur.
  169. <-r.blockSyncCh.Done()
  170. <-r.peerUpdates.Done()
  171. }
  172. // respondToPeer loads a block and sends it to the requesting peer, if we have it.
  173. // Otherwise, we'll respond saying we do not have it.
  174. func (r *Reactor) respondToPeer(msg *bcproto.BlockRequest, peerID types.NodeID) {
  175. block := r.store.LoadBlock(msg.Height)
  176. if block != nil {
  177. blockProto, err := block.ToProto()
  178. if err != nil {
  179. r.Logger.Error("failed to convert msg to protobuf", "err", err)
  180. return
  181. }
  182. r.blockSyncCh.Out <- p2p.Envelope{
  183. To: peerID,
  184. Message: &bcproto.BlockResponse{Block: blockProto},
  185. }
  186. return
  187. }
  188. r.Logger.Info("peer requesting a block we do not have", "peer", peerID, "height", msg.Height)
  189. r.blockSyncCh.Out <- p2p.Envelope{
  190. To: peerID,
  191. Message: &bcproto.NoBlockResponse{Height: msg.Height},
  192. }
  193. }
  194. // handleBlockSyncMessage handles envelopes sent from peers on the
  195. // BlockSyncChannel. It returns an error only if the Envelope.Message is unknown
  196. // for this channel. This should never be called outside of handleMessage.
  197. func (r *Reactor) handleBlockSyncMessage(envelope p2p.Envelope) error {
  198. logger := r.Logger.With("peer", envelope.From)
  199. switch msg := envelope.Message.(type) {
  200. case *bcproto.BlockRequest:
  201. r.respondToPeer(msg, envelope.From)
  202. case *bcproto.BlockResponse:
  203. block, err := types.BlockFromProto(msg.Block)
  204. if err != nil {
  205. logger.Error("failed to convert block from proto", "err", err)
  206. return err
  207. }
  208. r.pool.AddBlock(envelope.From, block, block.Size())
  209. case *bcproto.StatusRequest:
  210. r.blockSyncCh.Out <- p2p.Envelope{
  211. To: envelope.From,
  212. Message: &bcproto.StatusResponse{
  213. Height: r.store.Height(),
  214. Base: r.store.Base(),
  215. },
  216. }
  217. case *bcproto.StatusResponse:
  218. r.pool.SetPeerRange(envelope.From, msg.Base, msg.Height)
  219. case *bcproto.NoBlockResponse:
  220. logger.Debug("peer does not have the requested block", "height", msg.Height)
  221. default:
  222. return fmt.Errorf("received unknown message: %T", msg)
  223. }
  224. return nil
  225. }
  226. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  227. // It will handle errors and any possible panics gracefully. A caller can handle
  228. // any error returned by sending a PeerError on the respective channel.
  229. func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) {
  230. defer func() {
  231. if e := recover(); e != nil {
  232. err = fmt.Errorf("panic in processing message: %v", e)
  233. r.Logger.Error(
  234. "recovering from processing message panic",
  235. "err", err,
  236. "stack", string(debug.Stack()),
  237. )
  238. }
  239. }()
  240. r.Logger.Debug("received message", "message", envelope.Message, "peer", envelope.From)
  241. switch chID {
  242. case BlockSyncChannel:
  243. err = r.handleBlockSyncMessage(envelope)
  244. default:
  245. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  246. }
  247. return err
  248. }
  249. // processBlockSyncCh initiates a blocking process where we listen for and handle
  250. // envelopes on the BlockSyncChannel and blockSyncOutBridgeCh. Any error encountered during
  251. // message execution will result in a PeerError being sent on the BlockSyncChannel.
  252. // When the reactor is stopped, we will catch the signal and close the p2p Channel
  253. // gracefully.
  254. func (r *Reactor) processBlockSyncCh() {
  255. defer r.blockSyncCh.Close()
  256. for {
  257. select {
  258. case envelope := <-r.blockSyncCh.In:
  259. if err := r.handleMessage(r.blockSyncCh.ID, envelope); err != nil {
  260. r.Logger.Error("failed to process message", "ch_id", r.blockSyncCh.ID, "envelope", envelope, "err", err)
  261. r.blockSyncCh.Error <- p2p.PeerError{
  262. NodeID: envelope.From,
  263. Err: err,
  264. }
  265. }
  266. case envelope := <-r.blockSyncOutBridgeCh:
  267. r.blockSyncCh.Out <- envelope
  268. case <-r.closeCh:
  269. r.Logger.Debug("stopped listening on block sync channel; closing...")
  270. return
  271. }
  272. }
  273. }
  274. // processPeerUpdate processes a PeerUpdate.
  275. func (r *Reactor) processPeerUpdate(peerUpdate p2p.PeerUpdate) {
  276. r.Logger.Debug("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  277. // XXX: Pool#RedoRequest can sometimes give us an empty peer.
  278. if len(peerUpdate.NodeID) == 0 {
  279. return
  280. }
  281. switch peerUpdate.Status {
  282. case p2p.PeerStatusUp:
  283. // send a status update the newly added peer
  284. r.blockSyncOutBridgeCh <- p2p.Envelope{
  285. To: peerUpdate.NodeID,
  286. Message: &bcproto.StatusResponse{
  287. Base: r.store.Base(),
  288. Height: r.store.Height(),
  289. },
  290. }
  291. case p2p.PeerStatusDown:
  292. r.pool.RemovePeer(peerUpdate.NodeID)
  293. }
  294. }
  295. // processPeerUpdates initiates a blocking process where we listen for and handle
  296. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  297. // close the p2p PeerUpdatesCh gracefully.
  298. func (r *Reactor) processPeerUpdates() {
  299. defer r.peerUpdates.Close()
  300. for {
  301. select {
  302. case peerUpdate := <-r.peerUpdates.Updates():
  303. r.processPeerUpdate(peerUpdate)
  304. case <-r.closeCh:
  305. r.Logger.Debug("stopped listening on peer updates channel; closing...")
  306. return
  307. }
  308. }
  309. }
  310. // SwitchToBlockSync is called by the state sync reactor when switching to fast
  311. // sync.
  312. func (r *Reactor) SwitchToBlockSync(state sm.State) error {
  313. r.blockSync.Set()
  314. r.initialState = state
  315. r.pool.height = state.LastBlockHeight + 1
  316. if err := r.pool.Start(); err != nil {
  317. return err
  318. }
  319. r.syncStartTime = time.Now()
  320. r.poolWG.Add(1)
  321. go r.poolRoutine(true)
  322. return nil
  323. }
  324. func (r *Reactor) requestRoutine() {
  325. statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
  326. defer statusUpdateTicker.Stop()
  327. r.poolWG.Add(1)
  328. defer r.poolWG.Done()
  329. for {
  330. select {
  331. case <-r.closeCh:
  332. return
  333. case <-r.pool.Quit():
  334. return
  335. case request := <-r.requestsCh:
  336. r.blockSyncOutBridgeCh <- p2p.Envelope{
  337. To: request.PeerID,
  338. Message: &bcproto.BlockRequest{Height: request.Height},
  339. }
  340. case pErr := <-r.errorsCh:
  341. r.blockSyncCh.Error <- p2p.PeerError{
  342. NodeID: pErr.peerID,
  343. Err: pErr.err,
  344. }
  345. case <-statusUpdateTicker.C:
  346. r.poolWG.Add(1)
  347. go func() {
  348. defer r.poolWG.Done()
  349. r.blockSyncOutBridgeCh <- p2p.Envelope{
  350. Broadcast: true,
  351. Message: &bcproto.StatusRequest{},
  352. }
  353. }()
  354. }
  355. }
  356. }
  357. // poolRoutine handles messages from the poolReactor telling the reactor what to
  358. // do.
  359. //
  360. // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
  361. func (r *Reactor) poolRoutine(stateSynced bool) {
  362. var (
  363. trySyncTicker = time.NewTicker(trySyncIntervalMS * time.Millisecond)
  364. switchToConsensusTicker = time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
  365. blocksSynced = uint64(0)
  366. chainID = r.initialState.ChainID
  367. state = r.initialState
  368. lastHundred = time.Now()
  369. lastRate = 0.0
  370. didProcessCh = make(chan struct{}, 1)
  371. )
  372. defer trySyncTicker.Stop()
  373. defer switchToConsensusTicker.Stop()
  374. go r.requestRoutine()
  375. defer r.poolWG.Done()
  376. FOR_LOOP:
  377. for {
  378. select {
  379. case <-switchToConsensusTicker.C:
  380. var (
  381. height, numPending, lenRequesters = r.pool.GetStatus()
  382. lastAdvance = r.pool.LastAdvance()
  383. )
  384. r.Logger.Debug(
  385. "consensus ticker",
  386. "num_pending", numPending,
  387. "total", lenRequesters,
  388. "height", height,
  389. )
  390. switch {
  391. case r.pool.IsCaughtUp():
  392. r.Logger.Info("switching to consensus reactor", "height", height)
  393. case time.Since(lastAdvance) > syncTimeout:
  394. r.Logger.Error("no progress since last advance", "last_advance", lastAdvance)
  395. default:
  396. r.Logger.Info(
  397. "not caught up yet",
  398. "height", height,
  399. "max_peer_height", r.pool.MaxPeerHeight(),
  400. "timeout_in", syncTimeout-time.Since(lastAdvance),
  401. )
  402. continue
  403. }
  404. if err := r.pool.Stop(); err != nil {
  405. r.Logger.Error("failed to stop pool", "err", err)
  406. }
  407. r.blockSync.UnSet()
  408. if r.consReactor != nil {
  409. r.consReactor.SwitchToConsensus(state, blocksSynced > 0 || stateSynced)
  410. }
  411. break FOR_LOOP
  412. case <-trySyncTicker.C:
  413. select {
  414. case didProcessCh <- struct{}{}:
  415. default:
  416. }
  417. case <-didProcessCh:
  418. // NOTE: It is a subtle mistake to process more than a single block at a
  419. // time (e.g. 10) here, because we only send one BlockRequest per loop
  420. // iteration. The ratio mismatch can result in starving of blocks, i.e. a
  421. // sudden burst of requests and responses, and repeat. Consequently, it is
  422. // better to split these routines rather than coupling them as it is
  423. // written here.
  424. //
  425. // TODO: Uncouple from request routine.
  426. // see if there are any blocks to sync
  427. first, second := r.pool.PeekTwoBlocks()
  428. if first == nil || second == nil {
  429. // we need both to sync the first block
  430. continue FOR_LOOP
  431. } else {
  432. // try again quickly next loop
  433. didProcessCh <- struct{}{}
  434. }
  435. var (
  436. firstParts = first.MakePartSet(types.BlockPartSizeBytes)
  437. firstPartSetHeader = firstParts.Header()
  438. firstID = types.BlockID{Hash: first.Hash(), PartSetHeader: firstPartSetHeader}
  439. )
  440. // Finally, verify the first block using the second's commit.
  441. //
  442. // NOTE: We can probably make this more efficient, but note that calling
  443. // first.Hash() doesn't verify the tx contents, so MakePartSet() is
  444. // currently necessary.
  445. err := state.Validators.VerifyCommitLight(chainID, firstID, first.Height, second.LastCommit)
  446. if err != nil {
  447. err = fmt.Errorf("invalid last commit: %w", err)
  448. r.Logger.Error(
  449. err.Error(),
  450. "last_commit", second.LastCommit,
  451. "block_id", firstID,
  452. "height", first.Height,
  453. )
  454. // NOTE: We've already removed the peer's request, but we still need
  455. // to clean up the rest.
  456. peerID := r.pool.RedoRequest(first.Height)
  457. r.blockSyncCh.Error <- p2p.PeerError{
  458. NodeID: peerID,
  459. Err: err,
  460. }
  461. peerID2 := r.pool.RedoRequest(second.Height)
  462. if peerID2 != peerID {
  463. r.blockSyncCh.Error <- p2p.PeerError{
  464. NodeID: peerID2,
  465. Err: err,
  466. }
  467. }
  468. continue FOR_LOOP
  469. } else {
  470. r.pool.PopRequest()
  471. // TODO: batch saves so we do not persist to disk every block
  472. r.store.SaveBlock(first, firstParts, second.LastCommit)
  473. var err error
  474. // TODO: Same thing for app - but we would need a way to get the hash
  475. // without persisting the state.
  476. state, err = r.blockExec.ApplyBlock(state, firstID, first)
  477. if err != nil {
  478. // TODO: This is bad, are we zombie?
  479. panic(fmt.Sprintf("failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
  480. }
  481. r.metrics.RecordConsMetrics(first)
  482. blocksSynced++
  483. if blocksSynced%100 == 0 {
  484. lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
  485. r.Logger.Info(
  486. "block sync rate",
  487. "height", r.pool.height,
  488. "max_peer_height", r.pool.MaxPeerHeight(),
  489. "blocks/s", lastRate,
  490. )
  491. lastHundred = time.Now()
  492. }
  493. }
  494. continue FOR_LOOP
  495. case <-r.closeCh:
  496. break FOR_LOOP
  497. }
  498. }
  499. }
  500. func (r *Reactor) GetMaxPeerBlockHeight() int64 {
  501. return r.pool.MaxPeerHeight()
  502. }
  503. func (r *Reactor) GetTotalSyncedTime() time.Duration {
  504. if !r.blockSync.IsSet() || r.syncStartTime.IsZero() {
  505. return time.Duration(0)
  506. }
  507. return time.Since(r.syncStartTime)
  508. }
  509. func (r *Reactor) GetRemainingSyncTime() time.Duration {
  510. if !r.blockSync.IsSet() {
  511. return time.Duration(0)
  512. }
  513. targetSyncs := r.pool.targetSyncBlocks()
  514. currentSyncs := r.store.Height() - r.pool.startHeight + 1
  515. lastSyncRate := r.pool.getLastSyncRate()
  516. if currentSyncs < 0 || lastSyncRate < 0.001 {
  517. return time.Duration(0)
  518. }
  519. remain := float64(targetSyncs-currentSyncs) / lastSyncRate
  520. return time.Duration(int64(remain * float64(time.Second)))
  521. }