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.

413 lines
13 KiB

8 years ago
8 years ago
8 years ago
  1. package consensus
  2. import (
  3. "bytes"
  4. "errors"
  5. "fmt"
  6. "io"
  7. "reflect"
  8. "strconv"
  9. "strings"
  10. "time"
  11. abci "github.com/tendermint/abci/types"
  12. "github.com/tendermint/go-wire"
  13. auto "github.com/tendermint/tmlibs/autofile"
  14. cmn "github.com/tendermint/tmlibs/common"
  15. "github.com/tendermint/tendermint/proxy"
  16. sm "github.com/tendermint/tendermint/state"
  17. "github.com/tendermint/tendermint/types"
  18. )
  19. // Functionality to replay blocks and messages on recovery from a crash.
  20. // There are two general failure scenarios: failure during consensus, and failure while applying the block.
  21. // The former is handled by the WAL, the latter by the proxyApp Handshake on restart,
  22. // which ultimately hands off the work to the WAL.
  23. //-----------------------------------------
  24. // recover from failure during consensus
  25. // by replaying messages from the WAL
  26. // Unmarshal and apply a single message to the consensus state
  27. // as if it were received in receiveRoutine
  28. // Lines that start with "#" are ignored.
  29. // NOTE: receiveRoutine should not be running
  30. func (cs *ConsensusState) readReplayMessage(msgBytes []byte, newStepCh chan interface{}) error {
  31. // Skip over empty and meta lines
  32. if len(msgBytes) == 0 || msgBytes[0] == '#' {
  33. return nil
  34. }
  35. var err error
  36. var msg TimedWALMessage
  37. wire.ReadJSON(&msg, msgBytes, &err)
  38. if err != nil {
  39. fmt.Println("MsgBytes:", msgBytes, string(msgBytes))
  40. return fmt.Errorf("Error reading json data: %v", err)
  41. }
  42. // for logging
  43. switch m := msg.Msg.(type) {
  44. case types.EventDataRoundState:
  45. log.Notice("Replay: New Step", "height", m.Height, "round", m.Round, "step", m.Step)
  46. // these are playback checks
  47. ticker := time.After(time.Second * 2)
  48. if newStepCh != nil {
  49. select {
  50. case mi := <-newStepCh:
  51. m2 := mi.(types.EventDataRoundState)
  52. if m.Height != m2.Height || m.Round != m2.Round || m.Step != m2.Step {
  53. return fmt.Errorf("RoundState mismatch. Got %v; Expected %v", m2, m)
  54. }
  55. case <-ticker:
  56. return fmt.Errorf("Failed to read off newStepCh")
  57. }
  58. }
  59. case msgInfo:
  60. peerKey := m.PeerKey
  61. if peerKey == "" {
  62. peerKey = "local"
  63. }
  64. switch msg := m.Msg.(type) {
  65. case *ProposalMessage:
  66. p := msg.Proposal
  67. log.Notice("Replay: Proposal", "height", p.Height, "round", p.Round, "header",
  68. p.BlockPartsHeader, "pol", p.POLRound, "peer", peerKey)
  69. case *BlockPartMessage:
  70. log.Notice("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerKey)
  71. case *VoteMessage:
  72. v := msg.Vote
  73. log.Notice("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type,
  74. "blockID", v.BlockID, "peer", peerKey)
  75. }
  76. cs.handleMsg(m, cs.RoundState)
  77. case timeoutInfo:
  78. log.Notice("Replay: Timeout", "height", m.Height, "round", m.Round, "step", m.Step, "dur", m.Duration)
  79. cs.handleTimeout(m, cs.RoundState)
  80. default:
  81. return fmt.Errorf("Replay: Unknown TimedWALMessage type: %v", reflect.TypeOf(msg.Msg))
  82. }
  83. return nil
  84. }
  85. // replay only those messages since the last block.
  86. // timeoutRoutine should run concurrently to read off tickChan
  87. func (cs *ConsensusState) catchupReplay(csHeight int) error {
  88. // set replayMode
  89. cs.replayMode = true
  90. defer func() { cs.replayMode = false }()
  91. // Ensure that ENDHEIGHT for this height doesn't exist
  92. // NOTE: This is just a sanity check. As far as we know things work fine without it,
  93. // and Handshake could reuse ConsensusState if it weren't for this check (since we can crash after writing ENDHEIGHT).
  94. gr, found, err := cs.wal.group.Search("#ENDHEIGHT: ", makeHeightSearchFunc(csHeight))
  95. if gr != nil {
  96. gr.Close()
  97. }
  98. if found {
  99. return errors.New(cmn.Fmt("WAL should not contain #ENDHEIGHT %d.", csHeight))
  100. }
  101. // Search for last height marker
  102. gr, found, err = cs.wal.group.Search("#ENDHEIGHT: ", makeHeightSearchFunc(csHeight-1))
  103. if err == io.EOF {
  104. log.Warn("Replay: wal.group.Search returned EOF", "#ENDHEIGHT", csHeight-1)
  105. // if we upgraded from 0.9 to 0.9.1, we may have #HEIGHT instead
  106. // TODO (0.10.0): remove this
  107. gr, found, err = cs.wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(csHeight))
  108. if err == io.EOF {
  109. log.Warn("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
  110. return nil
  111. } else if err != nil {
  112. return err
  113. }
  114. } else if err != nil {
  115. return err
  116. } else {
  117. defer gr.Close()
  118. }
  119. if !found {
  120. // if we upgraded from 0.9 to 0.9.1, we may have #HEIGHT instead
  121. // TODO (0.10.0): remove this
  122. gr, found, err = cs.wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(csHeight))
  123. if err == io.EOF {
  124. log.Warn("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
  125. return nil
  126. } else if err != nil {
  127. return err
  128. } else {
  129. defer gr.Close()
  130. }
  131. // TODO (0.10.0): uncomment
  132. // return errors.New(cmn.Fmt("Cannot replay height %d. WAL does not contain #ENDHEIGHT for %d.", csHeight, csHeight-1))
  133. }
  134. log.Notice("Catchup by replaying consensus messages", "height", csHeight)
  135. for {
  136. line, err := gr.ReadLine()
  137. if err != nil {
  138. if err == io.EOF {
  139. break
  140. } else {
  141. return err
  142. }
  143. }
  144. // NOTE: since the priv key is set when the msgs are received
  145. // it will attempt to eg double sign but we can just ignore it
  146. // since the votes will be replayed and we'll get to the next step
  147. if err := cs.readReplayMessage([]byte(line), nil); err != nil {
  148. return err
  149. }
  150. }
  151. log.Notice("Replay: Done")
  152. return nil
  153. }
  154. //--------------------------------------------------------------------------------
  155. // Parses marker lines of the form:
  156. // #ENDHEIGHT: 12345
  157. func makeHeightSearchFunc(height int) auto.SearchFunc {
  158. return func(line string) (int, error) {
  159. line = strings.TrimRight(line, "\n")
  160. parts := strings.Split(line, " ")
  161. if len(parts) != 2 {
  162. return -1, errors.New("Line did not have 2 parts")
  163. }
  164. i, err := strconv.Atoi(parts[1])
  165. if err != nil {
  166. return -1, errors.New("Failed to parse INFO: " + err.Error())
  167. }
  168. if height < i {
  169. return 1, nil
  170. } else if height == i {
  171. return 0, nil
  172. } else {
  173. return -1, nil
  174. }
  175. }
  176. }
  177. //----------------------------------------------
  178. // Recover from failure during block processing
  179. // by handshaking with the app to figure out where
  180. // we were last and using the WAL to recover there
  181. type Handshaker struct {
  182. state *sm.State
  183. store types.BlockStore
  184. nBlocks int // number of blocks applied to the state
  185. }
  186. func NewHandshaker(state *sm.State, store types.BlockStore) *Handshaker {
  187. return &Handshaker{state, store, 0}
  188. }
  189. func (h *Handshaker) NBlocks() int {
  190. return h.nBlocks
  191. }
  192. // TODO: retry the handshake/replay if it fails ?
  193. func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
  194. // handshake is done via info request on the query conn
  195. res, err := proxyApp.Query().InfoSync()
  196. if err != nil {
  197. return errors.New(cmn.Fmt("Error calling Info: %v", err))
  198. }
  199. blockHeight := int(res.LastBlockHeight) // XXX: beware overflow
  200. appHash := res.LastBlockAppHash
  201. log.Notice("ABCI Handshake", "appHeight", blockHeight, "appHash", appHash)
  202. // TODO: check version
  203. // replay blocks up to the latest in the blockstore
  204. _, err = h.ReplayBlocks(appHash, blockHeight, proxyApp)
  205. if err != nil {
  206. return errors.New(cmn.Fmt("Error on replay: %v", err))
  207. }
  208. log.Notice("Completed ABCI Handshake - Tendermint and App are synced", "appHeight", blockHeight, "appHash", appHash)
  209. // TODO: (on restart) replay mempool
  210. return nil
  211. }
  212. // Replay all blocks since appBlockHeight and ensure the result matches the current state.
  213. // Returns the final AppHash or an error
  214. func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) {
  215. storeBlockHeight := h.store.Height()
  216. stateBlockHeight := h.state.LastBlockHeight
  217. log.Notice("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight)
  218. // If appBlockHeight == 0 it means that we are at genesis and hence should send InitChain
  219. if appBlockHeight == 0 {
  220. validators := types.TM2PB.Validators(h.state.Validators)
  221. proxyApp.Consensus().InitChainSync(validators)
  222. }
  223. // First handle edge cases and constraints on the storeBlockHeight
  224. if storeBlockHeight == 0 {
  225. return appHash, h.checkAppHash(appHash)
  226. } else if storeBlockHeight < appBlockHeight {
  227. // the app should never be ahead of the store (but this is under app's control)
  228. return appHash, sm.ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight}
  229. } else if storeBlockHeight < stateBlockHeight {
  230. // the state should never be ahead of the store (this is under tendermint's control)
  231. cmn.PanicSanity(cmn.Fmt("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight))
  232. } else if storeBlockHeight > stateBlockHeight+1 {
  233. // store should be at most one ahead of the state (this is under tendermint's control)
  234. cmn.PanicSanity(cmn.Fmt("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1))
  235. }
  236. // Now either store is equal to state, or one ahead.
  237. // For each, consider all cases of where the app could be, given app <= store
  238. if storeBlockHeight == stateBlockHeight {
  239. // Tendermint ran Commit and saved the state.
  240. // Either the app is asking for replay, or we're all synced up.
  241. if appBlockHeight < storeBlockHeight {
  242. // the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store)
  243. return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, false)
  244. } else if appBlockHeight == storeBlockHeight {
  245. // We're good!
  246. return appHash, h.checkAppHash(appHash)
  247. }
  248. } else if storeBlockHeight == stateBlockHeight+1 {
  249. // We saved the block in the store but haven't updated the state,
  250. // so we'll need to replay a block using the WAL.
  251. if appBlockHeight < stateBlockHeight {
  252. // the app is further behind than it should be, so replay blocks
  253. // but leave the last block to go through the WAL
  254. return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, true)
  255. } else if appBlockHeight == stateBlockHeight {
  256. // We haven't run Commit (both the state and app are one block behind),
  257. // so replayBlock with the real app.
  258. // NOTE: We could instead use the cs.WAL on cs.Start,
  259. // but we'd have to allow the WAL to replay a block that wrote it's ENDHEIGHT
  260. log.Info("Replay last block using real app")
  261. return h.replayBlock(storeBlockHeight, proxyApp.Consensus())
  262. } else if appBlockHeight == storeBlockHeight {
  263. // We ran Commit, but didn't save the state, so replayBlock with mock app
  264. abciResponses := h.state.LoadABCIResponses()
  265. mockApp := newMockProxyApp(appHash, abciResponses)
  266. log.Info("Replay last block using mock app")
  267. return h.replayBlock(storeBlockHeight, mockApp)
  268. }
  269. }
  270. cmn.PanicSanity("Should never happen")
  271. return nil, nil
  272. }
  273. func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, mutateState bool) ([]byte, error) {
  274. // App is further behind than it should be, so we need to replay blocks.
  275. // We replay all blocks from appBlockHeight+1.
  276. // Note that we don't have an old version of the state,
  277. // so we by-pass state validation/mutation using sm.ExecCommitBlock.
  278. // If mutateState == true, the final block is replayed with h.replayBlock()
  279. var appHash []byte
  280. var err error
  281. finalBlock := storeBlockHeight
  282. if mutateState {
  283. finalBlock -= 1
  284. }
  285. for i := appBlockHeight + 1; i <= finalBlock; i++ {
  286. log.Info("Applying block", "height", i)
  287. block := h.store.LoadBlock(i)
  288. appHash, err = sm.ExecCommitBlock(proxyApp.Consensus(), block)
  289. if err != nil {
  290. return nil, err
  291. }
  292. h.nBlocks += 1
  293. }
  294. if mutateState {
  295. // sync the final block
  296. return h.replayBlock(storeBlockHeight, proxyApp.Consensus())
  297. }
  298. return appHash, h.checkAppHash(appHash)
  299. }
  300. // ApplyBlock on the proxyApp with the last block.
  301. func (h *Handshaker) replayBlock(height int, proxyApp proxy.AppConnConsensus) ([]byte, error) {
  302. mempool := types.MockMempool{}
  303. var eventCache types.Fireable // nil
  304. block := h.store.LoadBlock(height)
  305. meta := h.store.LoadBlockMeta(height)
  306. if err := h.state.ApplyBlock(eventCache, proxyApp, block, meta.BlockID.PartsHeader, mempool); err != nil {
  307. return nil, err
  308. }
  309. h.nBlocks += 1
  310. return h.state.AppHash, nil
  311. }
  312. func (h *Handshaker) checkAppHash(appHash []byte) error {
  313. if !bytes.Equal(h.state.AppHash, appHash) {
  314. panic(errors.New(cmn.Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)).Error())
  315. return nil
  316. }
  317. return nil
  318. }
  319. //--------------------------------------------------------------------------------
  320. // mockProxyApp uses ABCIResponses to give the right results
  321. // Useful because we don't want to call Commit() twice for the same block on the real app.
  322. func newMockProxyApp(appHash []byte, abciResponses *sm.ABCIResponses) proxy.AppConnConsensus {
  323. clientCreator := proxy.NewLocalClientCreator(&mockProxyApp{
  324. appHash: appHash,
  325. abciResponses: abciResponses,
  326. })
  327. cli, _ := clientCreator.NewABCIClient()
  328. return proxy.NewAppConnConsensus(cli)
  329. }
  330. type mockProxyApp struct {
  331. abci.BaseApplication
  332. appHash []byte
  333. txCount int
  334. abciResponses *sm.ABCIResponses
  335. }
  336. func (mock *mockProxyApp) DeliverTx(tx []byte) abci.Result {
  337. r := mock.abciResponses.DeliverTx[mock.txCount]
  338. mock.txCount += 1
  339. return abci.Result{
  340. r.Code,
  341. r.Data,
  342. r.Log,
  343. }
  344. }
  345. func (mock *mockProxyApp) EndBlock(height uint64) abci.ResponseEndBlock {
  346. mock.txCount = 0
  347. return mock.abciResponses.EndBlock
  348. }
  349. func (mock *mockProxyApp) Commit() abci.Result {
  350. return abci.NewResultOK(mock.appHash, "")
  351. }