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.

476 lines
14 KiB

7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
  1. package blockchain
  2. import (
  3. "errors"
  4. "fmt"
  5. "reflect"
  6. "time"
  7. amino "github.com/tendermint/go-amino"
  8. cmn "github.com/tendermint/tendermint/libs/common"
  9. "github.com/tendermint/tendermint/libs/log"
  10. "github.com/tendermint/tendermint/p2p"
  11. sm "github.com/tendermint/tendermint/state"
  12. "github.com/tendermint/tendermint/types"
  13. )
  14. const (
  15. // BlockchainChannel is a channel for blocks and status updates (`BlockStore` height)
  16. BlockchainChannel = byte(0x40)
  17. trySyncIntervalMS = 10
  18. // stop syncing when last block's time is
  19. // within this much of the system time.
  20. // stopSyncingDurationMinutes = 10
  21. // ask for best height every 10s
  22. statusUpdateIntervalSeconds = 10
  23. // check if we should switch to consensus reactor
  24. switchToConsensusIntervalSeconds = 1
  25. // NOTE: keep up to date with bcBlockResponseMessage
  26. bcBlockResponseMessagePrefixSize = 4
  27. bcBlockResponseMessageFieldKeySize = 1
  28. maxMsgSize = types.MaxBlockSizeBytes +
  29. bcBlockResponseMessagePrefixSize +
  30. bcBlockResponseMessageFieldKeySize
  31. )
  32. type consensusReactor interface {
  33. // for when we switch from blockchain reactor and fast sync to
  34. // the consensus machine
  35. SwitchToConsensus(sm.State, int)
  36. }
  37. type peerError struct {
  38. err error
  39. peerID p2p.ID
  40. }
  41. func (e peerError) Error() string {
  42. return fmt.Sprintf("error with peer %v: %s", e.peerID, e.err.Error())
  43. }
  44. // BlockchainReactor handles long-term catchup syncing.
  45. type BlockchainReactor struct {
  46. p2p.BaseReactor
  47. // immutable
  48. initialState sm.State
  49. blockExec *sm.BlockExecutor
  50. store *BlockStore
  51. pool *BlockPool
  52. fastSync bool
  53. requestsCh <-chan BlockRequest
  54. errorsCh <-chan peerError
  55. }
  56. // NewBlockchainReactor returns new reactor instance.
  57. func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *BlockStore,
  58. fastSync bool) *BlockchainReactor {
  59. if state.LastBlockHeight != store.Height() {
  60. panic(fmt.Sprintf("state (%v) and store (%v) height mismatch", state.LastBlockHeight,
  61. store.Height()))
  62. }
  63. requestsCh := make(chan BlockRequest, maxTotalRequesters)
  64. const capacity = 1000 // must be bigger than peers count
  65. errorsCh := make(chan peerError, capacity) // so we don't block in #Receive#pool.AddBlock
  66. pool := NewBlockPool(
  67. store.Height()+1,
  68. requestsCh,
  69. errorsCh,
  70. )
  71. bcR := &BlockchainReactor{
  72. initialState: state,
  73. blockExec: blockExec,
  74. store: store,
  75. pool: pool,
  76. fastSync: fastSync,
  77. requestsCh: requestsCh,
  78. errorsCh: errorsCh,
  79. }
  80. bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
  81. return bcR
  82. }
  83. // SetLogger implements cmn.Service by setting the logger on reactor and pool.
  84. func (bcR *BlockchainReactor) SetLogger(l log.Logger) {
  85. bcR.BaseService.Logger = l
  86. bcR.pool.Logger = l
  87. }
  88. // OnStart implements cmn.Service.
  89. func (bcR *BlockchainReactor) OnStart() error {
  90. if bcR.fastSync {
  91. err := bcR.pool.Start()
  92. if err != nil {
  93. return err
  94. }
  95. go bcR.poolRoutine()
  96. }
  97. return nil
  98. }
  99. // OnStop implements cmn.Service.
  100. func (bcR *BlockchainReactor) OnStop() {
  101. bcR.pool.Stop()
  102. }
  103. // GetChannels implements Reactor
  104. func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor {
  105. return []*p2p.ChannelDescriptor{
  106. {
  107. ID: BlockchainChannel,
  108. Priority: 10,
  109. SendQueueCapacity: 1000,
  110. RecvBufferCapacity: 50 * 4096,
  111. RecvMessageCapacity: maxMsgSize,
  112. },
  113. }
  114. }
  115. // AddPeer implements Reactor by sending our state to peer.
  116. func (bcR *BlockchainReactor) AddPeer(peer p2p.Peer) {
  117. msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
  118. if !peer.Send(BlockchainChannel, msgBytes) {
  119. // doing nothing, will try later in `poolRoutine`
  120. }
  121. // peer is added to the pool once we receive the first
  122. // bcStatusResponseMessage from the peer and call pool.SetPeerHeight
  123. }
  124. // RemovePeer implements Reactor by removing peer from the pool.
  125. func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
  126. bcR.pool.RemovePeer(peer.ID())
  127. }
  128. // respondToPeer loads a block and sends it to the requesting peer,
  129. // if we have it. Otherwise, we'll respond saying we don't have it.
  130. // According to the Tendermint spec, if all nodes are honest,
  131. // no node should be requesting for a block that's non-existent.
  132. func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage,
  133. src p2p.Peer) (queued bool) {
  134. block := bcR.store.LoadBlock(msg.Height)
  135. if block != nil {
  136. msgBytes := cdc.MustMarshalBinaryBare(&bcBlockResponseMessage{Block: block})
  137. return src.TrySend(BlockchainChannel, msgBytes)
  138. }
  139. bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
  140. msgBytes := cdc.MustMarshalBinaryBare(&bcNoBlockResponseMessage{Height: msg.Height})
  141. return src.TrySend(BlockchainChannel, msgBytes)
  142. }
  143. // Receive implements Reactor by handling 4 types of messages (look below).
  144. func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
  145. msg, err := decodeMsg(msgBytes)
  146. if err != nil {
  147. bcR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
  148. bcR.Switch.StopPeerForError(src, err)
  149. return
  150. }
  151. if err = msg.ValidateBasic(); err != nil {
  152. bcR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
  153. bcR.Switch.StopPeerForError(src, err)
  154. return
  155. }
  156. bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
  157. switch msg := msg.(type) {
  158. case *bcBlockRequestMessage:
  159. if queued := bcR.respondToPeer(msg, src); !queued {
  160. // Unfortunately not queued since the queue is full.
  161. }
  162. case *bcBlockResponseMessage:
  163. bcR.pool.AddBlock(src.ID(), msg.Block, len(msgBytes))
  164. case *bcStatusRequestMessage:
  165. // Send peer our state.
  166. msgBytes := cdc.MustMarshalBinaryBare(&bcStatusResponseMessage{bcR.store.Height()})
  167. queued := src.TrySend(BlockchainChannel, msgBytes)
  168. if !queued {
  169. // sorry
  170. }
  171. case *bcStatusResponseMessage:
  172. // Got a peer status. Unverified.
  173. bcR.pool.SetPeerHeight(src.ID(), msg.Height)
  174. default:
  175. bcR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  176. }
  177. }
  178. // Handle messages from the poolReactor telling the reactor what to do.
  179. // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
  180. func (bcR *BlockchainReactor) poolRoutine() {
  181. trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond)
  182. statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second)
  183. switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second)
  184. blocksSynced := 0
  185. chainID := bcR.initialState.ChainID
  186. state := bcR.initialState
  187. lastHundred := time.Now()
  188. lastRate := 0.0
  189. didProcessCh := make(chan struct{}, 1)
  190. FOR_LOOP:
  191. for {
  192. select {
  193. case request := <-bcR.requestsCh:
  194. peer := bcR.Switch.Peers().Get(request.PeerID)
  195. if peer == nil {
  196. continue FOR_LOOP // Peer has since been disconnected.
  197. }
  198. msgBytes := cdc.MustMarshalBinaryBare(&bcBlockRequestMessage{request.Height})
  199. queued := peer.TrySend(BlockchainChannel, msgBytes)
  200. if !queued {
  201. // We couldn't make the request, send-queue full.
  202. // The pool handles timeouts, just let it go.
  203. continue FOR_LOOP
  204. }
  205. case err := <-bcR.errorsCh:
  206. peer := bcR.Switch.Peers().Get(err.peerID)
  207. if peer != nil {
  208. bcR.Switch.StopPeerForError(peer, err)
  209. }
  210. case <-statusUpdateTicker.C:
  211. // ask for status updates
  212. go bcR.BroadcastStatusRequest() // nolint: errcheck
  213. case <-switchToConsensusTicker.C:
  214. height, numPending, lenRequesters := bcR.pool.GetStatus()
  215. outbound, inbound, _ := bcR.Switch.NumPeers()
  216. bcR.Logger.Debug("Consensus ticker", "numPending", numPending, "total", lenRequesters,
  217. "outbound", outbound, "inbound", inbound)
  218. if bcR.pool.IsCaughtUp() {
  219. bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
  220. bcR.pool.Stop()
  221. conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
  222. if ok {
  223. conR.SwitchToConsensus(state, blocksSynced)
  224. } else {
  225. // should only happen during testing
  226. }
  227. break FOR_LOOP
  228. }
  229. case <-trySyncTicker.C: // chan time
  230. select {
  231. case didProcessCh <- struct{}{}:
  232. default:
  233. }
  234. case <-didProcessCh:
  235. // NOTE: It is a subtle mistake to process more than a single block
  236. // at a time (e.g. 10) here, because we only TrySend 1 request per
  237. // loop. The ratio mismatch can result in starving of blocks, a
  238. // sudden burst of requests and responses, and repeat.
  239. // Consequently, it is better to split these routines rather than
  240. // coupling them as it's written here. TODO uncouple from request
  241. // routine.
  242. // See if there are any blocks to sync.
  243. first, second := bcR.pool.PeekTwoBlocks()
  244. //bcR.Logger.Info("TrySync peeked", "first", first, "second", second)
  245. if first == nil || second == nil {
  246. // We need both to sync the first block.
  247. continue FOR_LOOP
  248. } else {
  249. // Try again quickly next loop.
  250. didProcessCh <- struct{}{}
  251. }
  252. firstParts := first.MakePartSet(types.BlockPartSizeBytes)
  253. firstPartsHeader := firstParts.Header()
  254. firstID := types.BlockID{first.Hash(), firstPartsHeader}
  255. // Finally, verify the first block using the second's commit
  256. // NOTE: we can probably make this more efficient, but note that calling
  257. // first.Hash() doesn't verify the tx contents, so MakePartSet() is
  258. // currently necessary.
  259. err := state.Validators.VerifyCommit(
  260. chainID, firstID, first.Height, second.LastCommit)
  261. if err != nil {
  262. bcR.Logger.Error("Error in validation", "err", err)
  263. peerID := bcR.pool.RedoRequest(first.Height)
  264. peer := bcR.Switch.Peers().Get(peerID)
  265. if peer != nil {
  266. // NOTE: we've already removed the peer's request, but we
  267. // still need to clean up the rest.
  268. bcR.Switch.StopPeerForError(peer, fmt.Errorf("BlockchainReactor validation error: %v", err))
  269. }
  270. peerID2 := bcR.pool.RedoRequest(second.Height)
  271. peer2 := bcR.Switch.Peers().Get(peerID2)
  272. if peer2 != nil && peer2 != peer {
  273. // NOTE: we've already removed the peer's request, but we
  274. // still need to clean up the rest.
  275. bcR.Switch.StopPeerForError(peer2, fmt.Errorf("BlockchainReactor validation error: %v", err))
  276. }
  277. continue FOR_LOOP
  278. } else {
  279. bcR.pool.PopRequest()
  280. // TODO: batch saves so we dont persist to disk every block
  281. bcR.store.SaveBlock(first, firstParts, second.LastCommit)
  282. // TODO: same thing for app - but we would need a way to
  283. // get the hash without persisting the state
  284. var err error
  285. state, err = bcR.blockExec.ApplyBlock(state, firstID, first)
  286. if err != nil {
  287. // TODO This is bad, are we zombie?
  288. cmn.PanicQ(fmt.Sprintf("Failed to process committed block (%d:%X): %v",
  289. first.Height, first.Hash(), err))
  290. }
  291. blocksSynced++
  292. if blocksSynced%100 == 0 {
  293. lastRate = 0.9*lastRate + 0.1*(100/time.Since(lastHundred).Seconds())
  294. bcR.Logger.Info("Fast Sync Rate", "height", bcR.pool.height,
  295. "max_peer_height", bcR.pool.MaxPeerHeight(), "blocks/s", lastRate)
  296. lastHundred = time.Now()
  297. }
  298. }
  299. continue FOR_LOOP
  300. case <-bcR.Quit():
  301. break FOR_LOOP
  302. }
  303. }
  304. }
  305. // BroadcastStatusRequest broadcasts `BlockStore` height.
  306. func (bcR *BlockchainReactor) BroadcastStatusRequest() error {
  307. msgBytes := cdc.MustMarshalBinaryBare(&bcStatusRequestMessage{bcR.store.Height()})
  308. bcR.Switch.Broadcast(BlockchainChannel, msgBytes)
  309. return nil
  310. }
  311. //-----------------------------------------------------------------------------
  312. // Messages
  313. // BlockchainMessage is a generic message for this reactor.
  314. type BlockchainMessage interface {
  315. ValidateBasic() error
  316. }
  317. func RegisterBlockchainMessages(cdc *amino.Codec) {
  318. cdc.RegisterInterface((*BlockchainMessage)(nil), nil)
  319. cdc.RegisterConcrete(&bcBlockRequestMessage{}, "tendermint/blockchain/BlockRequest", nil)
  320. cdc.RegisterConcrete(&bcBlockResponseMessage{}, "tendermint/blockchain/BlockResponse", nil)
  321. cdc.RegisterConcrete(&bcNoBlockResponseMessage{}, "tendermint/blockchain/NoBlockResponse", nil)
  322. cdc.RegisterConcrete(&bcStatusResponseMessage{}, "tendermint/blockchain/StatusResponse", nil)
  323. cdc.RegisterConcrete(&bcStatusRequestMessage{}, "tendermint/blockchain/StatusRequest", nil)
  324. }
  325. func decodeMsg(bz []byte) (msg BlockchainMessage, err error) {
  326. if len(bz) > maxMsgSize {
  327. return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
  328. }
  329. err = cdc.UnmarshalBinaryBare(bz, &msg)
  330. return
  331. }
  332. //-------------------------------------
  333. type bcBlockRequestMessage struct {
  334. Height int64
  335. }
  336. // ValidateBasic performs basic validation.
  337. func (m *bcBlockRequestMessage) ValidateBasic() error {
  338. if m.Height < 0 {
  339. return errors.New("Negative Height")
  340. }
  341. return nil
  342. }
  343. func (m *bcBlockRequestMessage) String() string {
  344. return fmt.Sprintf("[bcBlockRequestMessage %v]", m.Height)
  345. }
  346. type bcNoBlockResponseMessage struct {
  347. Height int64
  348. }
  349. // ValidateBasic performs basic validation.
  350. func (m *bcNoBlockResponseMessage) ValidateBasic() error {
  351. if m.Height < 0 {
  352. return errors.New("Negative Height")
  353. }
  354. return nil
  355. }
  356. func (brm *bcNoBlockResponseMessage) String() string {
  357. return fmt.Sprintf("[bcNoBlockResponseMessage %d]", brm.Height)
  358. }
  359. //-------------------------------------
  360. type bcBlockResponseMessage struct {
  361. Block *types.Block
  362. }
  363. // ValidateBasic performs basic validation.
  364. func (m *bcBlockResponseMessage) ValidateBasic() error {
  365. return m.Block.ValidateBasic()
  366. }
  367. func (m *bcBlockResponseMessage) String() string {
  368. return fmt.Sprintf("[bcBlockResponseMessage %v]", m.Block.Height)
  369. }
  370. //-------------------------------------
  371. type bcStatusRequestMessage struct {
  372. Height int64
  373. }
  374. // ValidateBasic performs basic validation.
  375. func (m *bcStatusRequestMessage) ValidateBasic() error {
  376. if m.Height < 0 {
  377. return errors.New("Negative Height")
  378. }
  379. return nil
  380. }
  381. func (m *bcStatusRequestMessage) String() string {
  382. return fmt.Sprintf("[bcStatusRequestMessage %v]", m.Height)
  383. }
  384. //-------------------------------------
  385. type bcStatusResponseMessage struct {
  386. Height int64
  387. }
  388. // ValidateBasic performs basic validation.
  389. func (m *bcStatusResponseMessage) ValidateBasic() error {
  390. if m.Height < 0 {
  391. return errors.New("Negative Height")
  392. }
  393. return nil
  394. }
  395. func (m *bcStatusResponseMessage) String() string {
  396. return fmt.Sprintf("[bcStatusResponseMessage %v]", m.Height)
  397. }