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.

443 lines
14 KiB

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