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.

668 lines
18 KiB

cleanup: Reduce and normalize import path aliasing. (#6975) The code in the Tendermint repository makes heavy use of import aliasing. This is made necessary by our extensive reuse of common base package names, and by repetition of similar names across different subdirectories. Unfortunately we have not been very consistent about which packages we alias in various circumstances, and the aliases we use vary. In the spirit of the advice in the style guide and https://github.com/golang/go/wiki/CodeReviewComments#imports, his change makes an effort to clean up and normalize import aliasing. This change makes no API or behavioral changes. It is a pure cleanup intended o help make the code more readable to developers (including myself) trying to understand what is being imported where. Only unexported names have been modified, and the changes were generated and applied mechanically with gofmt -r and comby, respecting the lexical and syntactic rules of Go. Even so, I did not fix every inconsistency. Where the changes would be too disruptive, I left it alone. The principles I followed in this cleanup are: - Remove aliases that restate the package name. - Remove aliases where the base package name is unambiguous. - Move overly-terse abbreviations from the import to the usage site. - Fix lexical issues (remove underscores, remove capitalization). - Fix import groupings to more closely match the style guide. - Group blank (side-effecting) imports and ensure they are commented. - Add aliases to multiple imports with the same base package name.
3 years ago
cleanup: Reduce and normalize import path aliasing. (#6975) The code in the Tendermint repository makes heavy use of import aliasing. This is made necessary by our extensive reuse of common base package names, and by repetition of similar names across different subdirectories. Unfortunately we have not been very consistent about which packages we alias in various circumstances, and the aliases we use vary. In the spirit of the advice in the style guide and https://github.com/golang/go/wiki/CodeReviewComments#imports, his change makes an effort to clean up and normalize import aliasing. This change makes no API or behavioral changes. It is a pure cleanup intended o help make the code more readable to developers (including myself) trying to understand what is being imported where. Only unexported names have been modified, and the changes were generated and applied mechanically with gofmt -r and comby, respecting the lexical and syntactic rules of Go. Even so, I did not fix every inconsistency. Where the changes would be too disruptive, I left it alone. The principles I followed in this cleanup are: - Remove aliases that restate the package name. - Remove aliases where the base package name is unambiguous. - Move overly-terse abbreviations from the import to the usage site. - Fix lexical issues (remove underscores, remove capitalization). - Fix import groupings to more closely match the style guide. - Group blank (side-effecting) imports and ensure they are commented. - Add aliases to multiple imports with the same base package name.
3 years ago
blocksync: fix shutdown deadlock issue (#7030) When shutting down blocksync, it is observed that the process can hang completely. A dump of running goroutines reveals that this is due to goroutines not listening on the correct shutdown signal. Namely, the `poolRoutine` goroutine does not wait on `pool.Quit`. The `poolRoutine` does not receive any other shutdown signal during `OnStop` becuase it must stop before the `r.closeCh` is closed. Currently the `poolRoutine` listens in the `closeCh` which will not close until the `poolRoutine` stops and calls `poolWG.Done()`. This change also puts the `requestRoutine()` in the `OnStart` method to make it more visible since it does not rely on anything that is spawned in the `poolRoutine`. ``` goroutine 183 [semacquire]: sync.runtime_Semacquire(0xc0000d3bd8) runtime/sema.go:56 +0x45 sync.(*WaitGroup).Wait(0xc0000d3bd0) sync/waitgroup.go:130 +0x65 github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStop(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:193 +0x47 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0000d3a00, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc00052c000) github.com/tendermint/tendermint/node/node.go:758 +0xc62 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00052c000, 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(0xc000df6d20, 0x7f04a68da900, 0xc0004a8930, 0xc0005a72d8) 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 161 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).poolRoutine(0xc0000d3a00, 0x0) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:464 +0x2b3 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:174 +0xf1 goroutine 162 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processBlockSyncCh(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:310 +0x151 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:177 +0x54 goroutine 163 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processPeerUpdates(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:363 +0x12b created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:178 +0x76 ```
3 years ago
blocksync: fix shutdown deadlock issue (#7030) When shutting down blocksync, it is observed that the process can hang completely. A dump of running goroutines reveals that this is due to goroutines not listening on the correct shutdown signal. Namely, the `poolRoutine` goroutine does not wait on `pool.Quit`. The `poolRoutine` does not receive any other shutdown signal during `OnStop` becuase it must stop before the `r.closeCh` is closed. Currently the `poolRoutine` listens in the `closeCh` which will not close until the `poolRoutine` stops and calls `poolWG.Done()`. This change also puts the `requestRoutine()` in the `OnStart` method to make it more visible since it does not rely on anything that is spawned in the `poolRoutine`. ``` goroutine 183 [semacquire]: sync.runtime_Semacquire(0xc0000d3bd8) runtime/sema.go:56 +0x45 sync.(*WaitGroup).Wait(0xc0000d3bd0) sync/waitgroup.go:130 +0x65 github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStop(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:193 +0x47 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0000d3a00, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc00052c000) github.com/tendermint/tendermint/node/node.go:758 +0xc62 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00052c000, 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(0xc000df6d20, 0x7f04a68da900, 0xc0004a8930, 0xc0005a72d8) 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 161 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).poolRoutine(0xc0000d3a00, 0x0) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:464 +0x2b3 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:174 +0xf1 goroutine 162 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processBlockSyncCh(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:310 +0x151 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:177 +0x54 goroutine 163 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processPeerUpdates(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:363 +0x12b created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:178 +0x76 ```
3 years ago
blocksync: fix shutdown deadlock issue (#7030) When shutting down blocksync, it is observed that the process can hang completely. A dump of running goroutines reveals that this is due to goroutines not listening on the correct shutdown signal. Namely, the `poolRoutine` goroutine does not wait on `pool.Quit`. The `poolRoutine` does not receive any other shutdown signal during `OnStop` becuase it must stop before the `r.closeCh` is closed. Currently the `poolRoutine` listens in the `closeCh` which will not close until the `poolRoutine` stops and calls `poolWG.Done()`. This change also puts the `requestRoutine()` in the `OnStart` method to make it more visible since it does not rely on anything that is spawned in the `poolRoutine`. ``` goroutine 183 [semacquire]: sync.runtime_Semacquire(0xc0000d3bd8) runtime/sema.go:56 +0x45 sync.(*WaitGroup).Wait(0xc0000d3bd0) sync/waitgroup.go:130 +0x65 github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStop(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:193 +0x47 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0000d3a00, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc00052c000) github.com/tendermint/tendermint/node/node.go:758 +0xc62 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00052c000, 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(0xc000df6d20, 0x7f04a68da900, 0xc0004a8930, 0xc0005a72d8) 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 161 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).poolRoutine(0xc0000d3a00, 0x0) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:464 +0x2b3 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:174 +0xf1 goroutine 162 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processBlockSyncCh(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:310 +0x151 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:177 +0x54 goroutine 163 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processPeerUpdates(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:363 +0x12b created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:178 +0x76 ```
3 years ago
blocksync: fix shutdown deadlock issue (#7030) When shutting down blocksync, it is observed that the process can hang completely. A dump of running goroutines reveals that this is due to goroutines not listening on the correct shutdown signal. Namely, the `poolRoutine` goroutine does not wait on `pool.Quit`. The `poolRoutine` does not receive any other shutdown signal during `OnStop` becuase it must stop before the `r.closeCh` is closed. Currently the `poolRoutine` listens in the `closeCh` which will not close until the `poolRoutine` stops and calls `poolWG.Done()`. This change also puts the `requestRoutine()` in the `OnStart` method to make it more visible since it does not rely on anything that is spawned in the `poolRoutine`. ``` goroutine 183 [semacquire]: sync.runtime_Semacquire(0xc0000d3bd8) runtime/sema.go:56 +0x45 sync.(*WaitGroup).Wait(0xc0000d3bd0) sync/waitgroup.go:130 +0x65 github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStop(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:193 +0x47 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc0000d3a00, 0x0, 0x0) github.com/tendermint/tendermint/libs/service/service.go:171 +0x323 github.com/tendermint/tendermint/node.(*nodeImpl).OnStop(0xc00052c000) github.com/tendermint/tendermint/node/node.go:758 +0xc62 github.com/tendermint/tendermint/libs/service.(*BaseService).Stop(0xc00052c000, 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(0xc000df6d20, 0x7f04a68da900, 0xc0004a8930, 0xc0005a72d8) 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 161 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).poolRoutine(0xc0000d3a00, 0x0) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:464 +0x2b3 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:174 +0xf1 goroutine 162 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processBlockSyncCh(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:310 +0x151 created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:177 +0x54 goroutine 163 [select]: github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).processPeerUpdates(0xc0000d3a00) github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:363 +0x12b created by github.com/tendermint/tendermint/internal/blocksync/v0.(*Reactor).OnStart github.com/tendermint/tendermint/internal/blocksync/v0/reactor.go:178 +0x76 ```
3 years ago
  1. package blocksync
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "runtime/debug"
  7. "sync"
  8. "sync/atomic"
  9. "time"
  10. "github.com/tendermint/tendermint/internal/consensus"
  11. "github.com/tendermint/tendermint/internal/eventbus"
  12. "github.com/tendermint/tendermint/internal/p2p"
  13. sm "github.com/tendermint/tendermint/internal/state"
  14. "github.com/tendermint/tendermint/internal/store"
  15. "github.com/tendermint/tendermint/libs/log"
  16. "github.com/tendermint/tendermint/libs/service"
  17. bcproto "github.com/tendermint/tendermint/proto/tendermint/blocksync"
  18. "github.com/tendermint/tendermint/types"
  19. )
  20. var _ service.Service = (*Reactor)(nil)
  21. const (
  22. // BlockSyncChannel is a channel for blocks and status updates
  23. BlockSyncChannel = p2p.ChannelID(0x40)
  24. trySyncIntervalMS = 10
  25. // ask for best height every 10s
  26. statusUpdateIntervalSeconds = 10
  27. // check if we should switch to consensus reactor
  28. switchToConsensusIntervalSeconds = 1
  29. // switch to consensus after this duration of inactivity
  30. syncTimeout = 60 * time.Second
  31. )
  32. func GetChannelDescriptor() *p2p.ChannelDescriptor {
  33. return &p2p.ChannelDescriptor{
  34. ID: BlockSyncChannel,
  35. MessageType: new(bcproto.Message),
  36. Priority: 5,
  37. SendQueueCapacity: 1000,
  38. RecvBufferCapacity: 1024,
  39. RecvMessageCapacity: MaxMsgSize,
  40. }
  41. }
  42. type consensusReactor interface {
  43. // For when we switch from block sync reactor to the consensus
  44. // machine.
  45. SwitchToConsensus(ctx context.Context, state sm.State, skipWAL bool)
  46. }
  47. type peerError struct {
  48. err error
  49. peerID types.NodeID
  50. }
  51. func (e peerError) Error() string {
  52. return fmt.Sprintf("error with peer %v: %s", e.peerID, e.err.Error())
  53. }
  54. // Reactor handles long-term catchup syncing.
  55. type Reactor struct {
  56. service.BaseService
  57. logger log.Logger
  58. // immutable
  59. initialState sm.State
  60. blockExec *sm.BlockExecutor
  61. store *store.BlockStore
  62. pool *BlockPool
  63. consReactor consensusReactor
  64. blockSync *atomicBool
  65. blockSyncCh *p2p.Channel
  66. // blockSyncOutBridgeCh defines a channel that acts as a bridge between sending Envelope
  67. // messages that the reactor will consume in processBlockSyncCh and receiving messages
  68. // from the peer updates channel and other goroutines. We do this instead of directly
  69. // sending on blockSyncCh.Out to avoid race conditions in the case where other goroutines
  70. // send Envelopes directly to the to blockSyncCh.Out channel, since processBlockSyncCh
  71. // may close the blockSyncCh.Out channel at the same time that other goroutines send to
  72. // blockSyncCh.Out.
  73. blockSyncOutBridgeCh chan p2p.Envelope
  74. peerUpdates *p2p.PeerUpdates
  75. requestsCh <-chan BlockRequest
  76. errorsCh <-chan peerError
  77. // poolWG is used to synchronize the graceful shutdown of the poolRoutine and
  78. // requestRoutine spawned goroutines when stopping the reactor and before
  79. // stopping the p2p Channel(s).
  80. poolWG sync.WaitGroup
  81. metrics *consensus.Metrics
  82. eventBus *eventbus.EventBus
  83. syncStartTime time.Time
  84. }
  85. // NewReactor returns new reactor instance.
  86. func NewReactor(
  87. ctx context.Context,
  88. logger log.Logger,
  89. state sm.State,
  90. blockExec *sm.BlockExecutor,
  91. store *store.BlockStore,
  92. consReactor consensusReactor,
  93. channelCreator p2p.ChannelCreator,
  94. peerUpdates *p2p.PeerUpdates,
  95. blockSync bool,
  96. metrics *consensus.Metrics,
  97. eventBus *eventbus.EventBus,
  98. ) (*Reactor, error) {
  99. if state.LastBlockHeight != store.Height() {
  100. return nil, fmt.Errorf("state (%v) and store (%v) height mismatch", state.LastBlockHeight, store.Height())
  101. }
  102. startHeight := store.Height() + 1
  103. if startHeight == 1 {
  104. startHeight = state.InitialHeight
  105. }
  106. requestsCh := make(chan BlockRequest, maxTotalRequesters)
  107. errorsCh := make(chan peerError, maxPeerErrBuffer) // NOTE: The capacity should be larger than the peer count.
  108. blockSyncCh, err := channelCreator(ctx, GetChannelDescriptor())
  109. if err != nil {
  110. return nil, err
  111. }
  112. r := &Reactor{
  113. logger: logger,
  114. initialState: state,
  115. blockExec: blockExec,
  116. store: store,
  117. pool: NewBlockPool(logger, startHeight, requestsCh, errorsCh),
  118. consReactor: consReactor,
  119. blockSync: newAtomicBool(blockSync),
  120. requestsCh: requestsCh,
  121. errorsCh: errorsCh,
  122. blockSyncCh: blockSyncCh,
  123. blockSyncOutBridgeCh: make(chan p2p.Envelope),
  124. peerUpdates: peerUpdates,
  125. metrics: metrics,
  126. eventBus: eventBus,
  127. syncStartTime: time.Time{},
  128. }
  129. r.BaseService = *service.NewBaseService(logger, "BlockSync", r)
  130. return r, nil
  131. }
  132. // OnStart starts separate go routines for each p2p Channel and listens for
  133. // envelopes on each. In addition, it also listens for peer updates and handles
  134. // messages on that p2p channel accordingly. The caller must be sure to execute
  135. // OnStop to ensure the outbound p2p Channels are closed.
  136. //
  137. // If blockSync is enabled, we also start the pool and the pool processing
  138. // goroutine. If the pool fails to start, an error is returned.
  139. func (r *Reactor) OnStart(ctx context.Context) error {
  140. if r.blockSync.IsSet() {
  141. if err := r.pool.Start(ctx); err != nil {
  142. return err
  143. }
  144. r.poolWG.Add(1)
  145. go r.requestRoutine(ctx)
  146. r.poolWG.Add(1)
  147. go r.poolRoutine(ctx, false)
  148. }
  149. go r.processBlockSyncCh(ctx)
  150. go r.processBlockSyncBridge(ctx)
  151. go r.processPeerUpdates(ctx)
  152. return nil
  153. }
  154. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  155. // blocking until they all exit.
  156. func (r *Reactor) OnStop() {
  157. if r.blockSync.IsSet() {
  158. r.pool.Stop()
  159. }
  160. // wait for the poolRoutine and requestRoutine goroutines to gracefully exit
  161. r.poolWG.Wait()
  162. }
  163. // respondToPeer loads a block and sends it to the requesting peer, if we have it.
  164. // Otherwise, we'll respond saying we do not have it.
  165. func (r *Reactor) respondToPeer(ctx context.Context, msg *bcproto.BlockRequest, peerID types.NodeID) error {
  166. block := r.store.LoadBlock(msg.Height)
  167. if block != nil {
  168. blockProto, err := block.ToProto()
  169. if err != nil {
  170. r.logger.Error("failed to convert msg to protobuf", "err", err)
  171. return err
  172. }
  173. return r.blockSyncCh.Send(ctx, p2p.Envelope{
  174. To: peerID,
  175. Message: &bcproto.BlockResponse{Block: blockProto},
  176. })
  177. }
  178. r.logger.Info("peer requesting a block we do not have", "peer", peerID, "height", msg.Height)
  179. return r.blockSyncCh.Send(ctx, p2p.Envelope{
  180. To: peerID,
  181. Message: &bcproto.NoBlockResponse{Height: msg.Height},
  182. })
  183. }
  184. // handleBlockSyncMessage handles envelopes sent from peers on the
  185. // BlockSyncChannel. It returns an error only if the Envelope.Message is unknown
  186. // for this channel. This should never be called outside of handleMessage.
  187. func (r *Reactor) handleBlockSyncMessage(ctx context.Context, envelope *p2p.Envelope) error {
  188. logger := r.logger.With("peer", envelope.From)
  189. switch msg := envelope.Message.(type) {
  190. case *bcproto.BlockRequest:
  191. return r.respondToPeer(ctx, msg, envelope.From)
  192. case *bcproto.BlockResponse:
  193. block, err := types.BlockFromProto(msg.Block)
  194. if err != nil {
  195. logger.Error("failed to convert block from proto", "err", err)
  196. return err
  197. }
  198. r.pool.AddBlock(envelope.From, block, block.Size())
  199. case *bcproto.StatusRequest:
  200. return r.blockSyncCh.Send(ctx, p2p.Envelope{
  201. To: envelope.From,
  202. Message: &bcproto.StatusResponse{
  203. Height: r.store.Height(),
  204. Base: r.store.Base(),
  205. },
  206. })
  207. case *bcproto.StatusResponse:
  208. r.pool.SetPeerRange(envelope.From, msg.Base, msg.Height)
  209. case *bcproto.NoBlockResponse:
  210. logger.Debug("peer does not have the requested block", "height", msg.Height)
  211. default:
  212. return fmt.Errorf("received unknown message: %T", msg)
  213. }
  214. return nil
  215. }
  216. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  217. // It will handle errors and any possible panics gracefully. A caller can handle
  218. // any error returned by sending a PeerError on the respective channel.
  219. func (r *Reactor) handleMessage(ctx context.Context, chID p2p.ChannelID, envelope *p2p.Envelope) (err error) {
  220. defer func() {
  221. if e := recover(); e != nil {
  222. err = fmt.Errorf("panic in processing message: %v", e)
  223. r.logger.Error(
  224. "recovering from processing message panic",
  225. "err", err,
  226. "stack", string(debug.Stack()),
  227. )
  228. }
  229. }()
  230. r.logger.Debug("received message", "message", envelope.Message, "peer", envelope.From)
  231. switch chID {
  232. case BlockSyncChannel:
  233. err = r.handleBlockSyncMessage(ctx, envelope)
  234. default:
  235. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  236. }
  237. return err
  238. }
  239. // processBlockSyncCh initiates a blocking process where we listen for and handle
  240. // envelopes on the BlockSyncChannel and blockSyncOutBridgeCh. Any error encountered during
  241. // message execution will result in a PeerError being sent on the BlockSyncChannel.
  242. // When the reactor is stopped, we will catch the signal and close the p2p Channel
  243. // gracefully.
  244. func (r *Reactor) processBlockSyncCh(ctx context.Context) {
  245. iter := r.blockSyncCh.Receive(ctx)
  246. for iter.Next(ctx) {
  247. envelope := iter.Envelope()
  248. if err := r.handleMessage(ctx, r.blockSyncCh.ID, envelope); err != nil {
  249. if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
  250. return
  251. }
  252. r.logger.Error("failed to process message", "ch_id", r.blockSyncCh.ID, "envelope", envelope, "err", err)
  253. if serr := r.blockSyncCh.SendError(ctx, p2p.PeerError{
  254. NodeID: envelope.From,
  255. Err: err,
  256. }); serr != nil {
  257. return
  258. }
  259. }
  260. }
  261. }
  262. func (r *Reactor) processBlockSyncBridge(ctx context.Context) {
  263. for {
  264. select {
  265. case <-ctx.Done():
  266. return
  267. case envelope := <-r.blockSyncOutBridgeCh:
  268. if err := r.blockSyncCh.Send(ctx, envelope); err != nil {
  269. return
  270. }
  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(ctx context.Context) {
  299. for {
  300. select {
  301. case <-ctx.Done():
  302. return
  303. case peerUpdate := <-r.peerUpdates.Updates():
  304. r.processPeerUpdate(peerUpdate)
  305. }
  306. }
  307. }
  308. // SwitchToBlockSync is called by the state sync reactor when switching to fast
  309. // sync.
  310. func (r *Reactor) SwitchToBlockSync(ctx context.Context, state sm.State) error {
  311. r.blockSync.Set()
  312. r.initialState = state
  313. r.pool.height = state.LastBlockHeight + 1
  314. if err := r.pool.Start(ctx); err != nil {
  315. return err
  316. }
  317. r.syncStartTime = time.Now()
  318. r.poolWG.Add(1)
  319. go r.requestRoutine(ctx)
  320. r.poolWG.Add(1)
  321. go r.poolRoutine(ctx, true)
  322. return nil
  323. }
  324. func (r *Reactor) requestRoutine(ctx context.Context) {
  325. statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
  326. defer statusUpdateTicker.Stop()
  327. defer r.poolWG.Done()
  328. for {
  329. select {
  330. case <-ctx.Done():
  331. return
  332. case request := <-r.requestsCh:
  333. r.blockSyncOutBridgeCh <- p2p.Envelope{
  334. To: request.PeerID,
  335. Message: &bcproto.BlockRequest{Height: request.Height},
  336. }
  337. case pErr := <-r.errorsCh:
  338. if err := r.blockSyncCh.SendError(ctx, p2p.PeerError{
  339. NodeID: pErr.peerID,
  340. Err: pErr.err,
  341. }); err != nil {
  342. return
  343. }
  344. case <-statusUpdateTicker.C:
  345. r.poolWG.Add(1)
  346. go func() {
  347. defer r.poolWG.Done()
  348. select {
  349. case r.blockSyncOutBridgeCh <- p2p.Envelope{
  350. Broadcast: true,
  351. Message: &bcproto.StatusRequest{},
  352. }:
  353. case <-ctx.Done():
  354. }
  355. }()
  356. }
  357. }
  358. }
  359. // poolRoutine handles messages from the poolReactor telling the reactor what to
  360. // do.
  361. //
  362. // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
  363. func (r *Reactor) poolRoutine(ctx context.Context, stateSynced bool) {
  364. var (
  365. trySyncTicker = time.NewTicker(trySyncIntervalMS * time.Millisecond)
  366. switchToConsensusTicker = time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
  367. blocksSynced = uint64(0)
  368. chainID = r.initialState.ChainID
  369. state = r.initialState
  370. lastHundred = time.Now()
  371. lastRate = 0.0
  372. didProcessCh = make(chan struct{}, 1)
  373. )
  374. defer trySyncTicker.Stop()
  375. defer switchToConsensusTicker.Stop()
  376. defer r.poolWG.Done()
  377. FOR_LOOP:
  378. for {
  379. select {
  380. case <-switchToConsensusTicker.C:
  381. var (
  382. height, numPending, lenRequesters = r.pool.GetStatus()
  383. lastAdvance = r.pool.LastAdvance()
  384. )
  385. r.logger.Debug(
  386. "consensus ticker",
  387. "num_pending", numPending,
  388. "total", lenRequesters,
  389. "height", height,
  390. )
  391. switch {
  392. case r.pool.IsCaughtUp():
  393. r.logger.Info("switching to consensus reactor", "height", height)
  394. case time.Since(lastAdvance) > syncTimeout:
  395. r.logger.Error("no progress since last advance", "last_advance", lastAdvance)
  396. default:
  397. r.logger.Info(
  398. "not caught up yet",
  399. "height", height,
  400. "max_peer_height", r.pool.MaxPeerHeight(),
  401. "timeout_in", syncTimeout-time.Since(lastAdvance),
  402. )
  403. continue
  404. }
  405. r.pool.Stop()
  406. r.blockSync.UnSet()
  407. if r.consReactor != nil {
  408. r.consReactor.SwitchToConsensus(ctx, state, blocksSynced > 0 || stateSynced)
  409. }
  410. break FOR_LOOP
  411. case <-trySyncTicker.C:
  412. select {
  413. case didProcessCh <- struct{}{}:
  414. default:
  415. }
  416. case <-didProcessCh:
  417. // NOTE: It is a subtle mistake to process more than a single block at a
  418. // time (e.g. 10) here, because we only send one BlockRequest per loop
  419. // iteration. The ratio mismatch can result in starving of blocks, i.e. a
  420. // sudden burst of requests and responses, and repeat. Consequently, it is
  421. // better to split these routines rather than coupling them as it is
  422. // written here.
  423. //
  424. // TODO: Uncouple from request routine.
  425. // see if there are any blocks to sync
  426. first, second := r.pool.PeekTwoBlocks()
  427. if first == nil || second == nil {
  428. // we need both to sync the first block
  429. continue FOR_LOOP
  430. } else {
  431. // try again quickly next loop
  432. didProcessCh <- struct{}{}
  433. }
  434. firstParts, err := first.MakePartSet(types.BlockPartSizeBytes)
  435. if err != nil {
  436. r.logger.Error("failed to make ",
  437. "height", first.Height,
  438. "err", err.Error())
  439. break FOR_LOOP
  440. }
  441. var (
  442. firstPartSetHeader = firstParts.Header()
  443. firstID = types.BlockID{Hash: first.Hash(), PartSetHeader: firstPartSetHeader}
  444. )
  445. // Finally, verify the first block using the second's commit.
  446. //
  447. // NOTE: We can probably make this more efficient, but note that calling
  448. // first.Hash() doesn't verify the tx contents, so MakePartSet() is
  449. // currently necessary.
  450. if err = state.Validators.VerifyCommitLight(chainID, firstID, first.Height, second.LastCommit); err != nil {
  451. err = fmt.Errorf("invalid last commit: %w", err)
  452. r.logger.Error(
  453. err.Error(),
  454. "last_commit", second.LastCommit,
  455. "block_id", firstID,
  456. "height", first.Height,
  457. )
  458. // NOTE: We've already removed the peer's request, but we still need
  459. // to clean up the rest.
  460. peerID := r.pool.RedoRequest(first.Height)
  461. if serr := r.blockSyncCh.SendError(ctx, p2p.PeerError{
  462. NodeID: peerID,
  463. Err: err,
  464. }); serr != nil {
  465. break FOR_LOOP
  466. }
  467. peerID2 := r.pool.RedoRequest(second.Height)
  468. if peerID2 != peerID {
  469. if serr := r.blockSyncCh.SendError(ctx, p2p.PeerError{
  470. NodeID: peerID2,
  471. Err: err,
  472. }); serr != nil {
  473. break FOR_LOOP
  474. }
  475. }
  476. continue FOR_LOOP
  477. } else {
  478. r.pool.PopRequest()
  479. // TODO: batch saves so we do not persist to disk every block
  480. r.store.SaveBlock(first, firstParts, second.LastCommit)
  481. var err error
  482. // TODO: Same thing for app - but we would need a way to get the hash
  483. // without persisting the state.
  484. state, err = r.blockExec.ApplyBlock(ctx, state, firstID, first)
  485. if err != nil {
  486. // TODO: This is bad, are we zombie?
  487. panic(fmt.Sprintf("failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err))
  488. }
  489. r.metrics.RecordConsMetrics(first)
  490. blocksSynced++
  491. if blocksSynced%100 == 0 {
  492. lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
  493. r.logger.Info(
  494. "block sync rate",
  495. "height", r.pool.height,
  496. "max_peer_height", r.pool.MaxPeerHeight(),
  497. "blocks/s", lastRate,
  498. )
  499. lastHundred = time.Now()
  500. }
  501. }
  502. continue FOR_LOOP
  503. case <-ctx.Done():
  504. break FOR_LOOP
  505. case <-r.pool.exitedCh:
  506. break FOR_LOOP
  507. }
  508. }
  509. }
  510. func (r *Reactor) GetMaxPeerBlockHeight() int64 {
  511. return r.pool.MaxPeerHeight()
  512. }
  513. func (r *Reactor) GetTotalSyncedTime() time.Duration {
  514. if !r.blockSync.IsSet() || r.syncStartTime.IsZero() {
  515. return time.Duration(0)
  516. }
  517. return time.Since(r.syncStartTime)
  518. }
  519. func (r *Reactor) GetRemainingSyncTime() time.Duration {
  520. if !r.blockSync.IsSet() {
  521. return time.Duration(0)
  522. }
  523. targetSyncs := r.pool.targetSyncBlocks()
  524. currentSyncs := r.store.Height() - r.pool.startHeight + 1
  525. lastSyncRate := r.pool.getLastSyncRate()
  526. if currentSyncs < 0 || lastSyncRate < 0.001 {
  527. return time.Duration(0)
  528. }
  529. remain := float64(targetSyncs-currentSyncs) / lastSyncRate
  530. return time.Duration(int64(remain * float64(time.Second)))
  531. }
  532. func (r *Reactor) PublishStatus(ctx context.Context, event types.EventDataBlockSyncStatus) error {
  533. if r.eventBus == nil {
  534. return errors.New("event bus is not configured")
  535. }
  536. return r.eventBus.PublishEventBlockSyncStatus(ctx, event)
  537. }
  538. // atomicBool is an atomic Boolean, safe for concurrent use by multiple
  539. // goroutines.
  540. type atomicBool int32
  541. // newAtomicBool creates an atomicBool with given initial value.
  542. func newAtomicBool(ok bool) *atomicBool {
  543. ab := new(atomicBool)
  544. if ok {
  545. ab.Set()
  546. }
  547. return ab
  548. }
  549. // Set sets the Boolean to true.
  550. func (ab *atomicBool) Set() { atomic.StoreInt32((*int32)(ab), 1) }
  551. // UnSet sets the Boolean to false.
  552. func (ab *atomicBool) UnSet() { atomic.StoreInt32((*int32)(ab), 0) }
  553. // IsSet returns whether the Boolean is true.
  554. func (ab *atomicBool) IsSet() bool { return atomic.LoadInt32((*int32)(ab))&1 == 1 }