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.

395 lines
13 KiB

8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
7 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
8 years ago
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
7 years ago
  1. package consensus
  2. import (
  3. "bytes"
  4. "errors"
  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. "github.com/tendermint/tmlibs/log"
  16. "github.com/tendermint/tendermint/proxy"
  17. sm "github.com/tendermint/tendermint/state"
  18. "github.com/tendermint/tendermint/types"
  19. "github.com/tendermint/tendermint/version"
  20. )
  21. var crc32c = crc32.MakeTable(crc32.Castagnoli)
  22. // Functionality to replay blocks and messages on recovery from a crash.
  23. // There are two general failure scenarios: failure during consensus, and failure while applying the block.
  24. // The former is handled by the WAL, the latter by the proxyApp Handshake on restart,
  25. // which ultimately hands off the work to the WAL.
  26. //-----------------------------------------
  27. // recover from failure during consensus
  28. // by replaying messages from the WAL
  29. // Unmarshal and apply a single message to the consensus state
  30. // as if it were received in receiveRoutine
  31. // Lines that start with "#" are ignored.
  32. // NOTE: receiveRoutine should not be running
  33. func (cs *ConsensusState) readReplayMessage(msg *TimedWALMessage, newStepCh chan interface{}) error {
  34. // skip meta messages
  35. if _, ok := msg.Msg.(EndHeightMessage); ok {
  36. return nil
  37. }
  38. // for logging
  39. switch m := msg.Msg.(type) {
  40. case types.EventDataRoundState:
  41. cs.Logger.Info("Replay: New Step", "height", m.Height, "round", m.Round, "step", m.Step)
  42. // these are playback checks
  43. ticker := time.After(time.Second * 2)
  44. if newStepCh != nil {
  45. select {
  46. case mi := <-newStepCh:
  47. m2 := mi.(types.EventDataRoundState)
  48. if m.Height != m2.Height || m.Round != m2.Round || m.Step != m2.Step {
  49. return fmt.Errorf("RoundState mismatch. Got %v; Expected %v", m2, m)
  50. }
  51. case <-ticker:
  52. return fmt.Errorf("Failed to read off newStepCh")
  53. }
  54. }
  55. case msgInfo:
  56. peerKey := m.PeerKey
  57. if peerKey == "" {
  58. peerKey = "local"
  59. }
  60. switch msg := m.Msg.(type) {
  61. case *ProposalMessage:
  62. p := msg.Proposal
  63. cs.Logger.Info("Replay: Proposal", "height", p.Height, "round", p.Round, "header",
  64. p.BlockPartsHeader, "pol", p.POLRound, "peer", peerKey)
  65. case *BlockPartMessage:
  66. cs.Logger.Info("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerKey)
  67. case *VoteMessage:
  68. v := msg.Vote
  69. cs.Logger.Info("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type,
  70. "blockID", v.BlockID, "peer", peerKey)
  71. }
  72. cs.handleMsg(m)
  73. case timeoutInfo:
  74. cs.Logger.Info("Replay: Timeout", "height", m.Height, "round", m.Round, "step", m.Step, "dur", m.Duration)
  75. cs.handleTimeout(m, cs.RoundState)
  76. default:
  77. return fmt.Errorf("Replay: Unknown TimedWALMessage type: %v", reflect.TypeOf(msg.Msg))
  78. }
  79. return nil
  80. }
  81. // replay only those messages since the last block.
  82. // timeoutRoutine should run concurrently to read off tickChan
  83. func (cs *ConsensusState) catchupReplay(csHeight int) error {
  84. // set replayMode
  85. cs.replayMode = true
  86. defer func() { cs.replayMode = false }()
  87. // Ensure that ENDHEIGHT for this height doesn't exist
  88. // NOTE: This is just a sanity check. As far as we know things work fine without it,
  89. // and Handshake could reuse ConsensusState if it weren't for this check (since we can crash after writing ENDHEIGHT).
  90. gr, found, err := cs.wal.SearchForEndHeight(uint64(csHeight))
  91. if gr != nil {
  92. gr.Close()
  93. }
  94. if found {
  95. return fmt.Errorf("WAL should not contain #ENDHEIGHT %d.", csHeight)
  96. }
  97. // Search for last height marker
  98. gr, found, err = cs.wal.SearchForEndHeight(uint64(csHeight - 1))
  99. if err == io.EOF {
  100. cs.Logger.Error("Replay: wal.group.Search returned EOF", "#ENDHEIGHT", csHeight-1)
  101. } else if err != nil {
  102. return err
  103. }
  104. if !found {
  105. return errors.New(cmn.Fmt("Cannot replay height %d. WAL does not contain #ENDHEIGHT for %d.", csHeight, csHeight-1))
  106. }
  107. defer gr.Close()
  108. cs.Logger.Info("Catchup by replaying consensus messages", "height", csHeight)
  109. var msg *TimedWALMessage
  110. dec := WALDecoder{gr}
  111. for {
  112. msg, err = dec.Decode()
  113. if err == io.EOF {
  114. break
  115. } else if err != nil {
  116. return err
  117. }
  118. // NOTE: since the priv key is set when the msgs are received
  119. // it will attempt to eg double sign but we can just ignore it
  120. // since the votes will be replayed and we'll get to the next step
  121. if err := cs.readReplayMessage(msg, nil); err != nil {
  122. return err
  123. }
  124. }
  125. cs.Logger.Info("Replay: Done")
  126. return nil
  127. }
  128. //--------------------------------------------------------------------------------
  129. // Parses marker lines of the form:
  130. // #ENDHEIGHT: 12345
  131. func makeHeightSearchFunc(height int) auto.SearchFunc {
  132. return func(line string) (int, error) {
  133. line = strings.TrimRight(line, "\n")
  134. parts := strings.Split(line, " ")
  135. if len(parts) != 2 {
  136. return -1, errors.New("Line did not have 2 parts")
  137. }
  138. i, err := strconv.Atoi(parts[1])
  139. if err != nil {
  140. return -1, errors.New("Failed to parse INFO: " + err.Error())
  141. }
  142. if height < i {
  143. return 1, nil
  144. } else if height == i {
  145. return 0, nil
  146. } else {
  147. return -1, nil
  148. }
  149. }
  150. }
  151. //----------------------------------------------
  152. // Recover from failure during block processing
  153. // by handshaking with the app to figure out where
  154. // we were last and using the WAL to recover there
  155. type Handshaker struct {
  156. state *sm.State
  157. store types.BlockStore
  158. logger log.Logger
  159. nBlocks int // number of blocks applied to the state
  160. }
  161. func NewHandshaker(state *sm.State, store types.BlockStore) *Handshaker {
  162. return &Handshaker{state, store, log.NewNopLogger(), 0}
  163. }
  164. func (h *Handshaker) SetLogger(l log.Logger) {
  165. h.logger = l
  166. }
  167. func (h *Handshaker) NBlocks() int {
  168. return h.nBlocks
  169. }
  170. // TODO: retry the handshake/replay if it fails ?
  171. func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
  172. // handshake is done via info request on the query conn
  173. res, err := proxyApp.Query().InfoSync(abci.RequestInfo{version.Version})
  174. if err != nil {
  175. return errors.New(cmn.Fmt("Error calling Info: %v", err))
  176. }
  177. blockHeight := int(res.LastBlockHeight) // XXX: beware overflow
  178. appHash := res.LastBlockAppHash
  179. h.logger.Info("ABCI Handshake", "appHeight", blockHeight, "appHash", fmt.Sprintf("%X", appHash))
  180. // TODO: check version
  181. // replay blocks up to the latest in the blockstore
  182. _, err = h.ReplayBlocks(appHash, blockHeight, proxyApp)
  183. if err != nil {
  184. return errors.New(cmn.Fmt("Error on replay: %v", err))
  185. }
  186. h.logger.Info("Completed ABCI Handshake - Tendermint and App are synced", "appHeight", blockHeight, "appHash", fmt.Sprintf("%X", appHash))
  187. // TODO: (on restart) replay mempool
  188. return nil
  189. }
  190. // Replay all blocks since appBlockHeight and ensure the result matches the current state.
  191. // Returns the final AppHash or an error
  192. func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) {
  193. storeBlockHeight := h.store.Height()
  194. stateBlockHeight := h.state.LastBlockHeight
  195. h.logger.Info("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight)
  196. // If appBlockHeight == 0 it means that we are at genesis and hence should send InitChain
  197. if appBlockHeight == 0 {
  198. validators := types.TM2PB.Validators(h.state.Validators)
  199. proxyApp.Consensus().InitChainSync(abci.RequestInitChain{validators})
  200. }
  201. // First handle edge cases and constraints on the storeBlockHeight
  202. if storeBlockHeight == 0 {
  203. return appHash, h.checkAppHash(appHash)
  204. } else if storeBlockHeight < appBlockHeight {
  205. // the app should never be ahead of the store (but this is under app's control)
  206. return appHash, sm.ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight}
  207. } else if storeBlockHeight < stateBlockHeight {
  208. // the state should never be ahead of the store (this is under tendermint's control)
  209. cmn.PanicSanity(cmn.Fmt("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight))
  210. } else if storeBlockHeight > stateBlockHeight+1 {
  211. // store should be at most one ahead of the state (this is under tendermint's control)
  212. cmn.PanicSanity(cmn.Fmt("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1))
  213. }
  214. // Now either store is equal to state, or one ahead.
  215. // For each, consider all cases of where the app could be, given app <= store
  216. if storeBlockHeight == stateBlockHeight {
  217. // Tendermint ran Commit and saved the state.
  218. // Either the app is asking for replay, or we're all synced up.
  219. if appBlockHeight < storeBlockHeight {
  220. // the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store)
  221. return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, false)
  222. } else if appBlockHeight == storeBlockHeight {
  223. // We're good!
  224. return appHash, h.checkAppHash(appHash)
  225. }
  226. } else if storeBlockHeight == stateBlockHeight+1 {
  227. // We saved the block in the store but haven't updated the state,
  228. // so we'll need to replay a block using the WAL.
  229. if appBlockHeight < stateBlockHeight {
  230. // the app is further behind than it should be, so replay blocks
  231. // but leave the last block to go through the WAL
  232. return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, true)
  233. } else if appBlockHeight == stateBlockHeight {
  234. // We haven't run Commit (both the state and app are one block behind),
  235. // so replayBlock with the real app.
  236. // NOTE: We could instead use the cs.WAL on cs.Start,
  237. // but we'd have to allow the WAL to replay a block that wrote it's ENDHEIGHT
  238. h.logger.Info("Replay last block using real app")
  239. return h.replayBlock(storeBlockHeight, proxyApp.Consensus())
  240. } else if appBlockHeight == storeBlockHeight {
  241. // We ran Commit, but didn't save the state, so replayBlock with mock app
  242. abciResponses := h.state.LoadABCIResponses()
  243. mockApp := newMockProxyApp(appHash, abciResponses)
  244. h.logger.Info("Replay last block using mock app")
  245. return h.replayBlock(storeBlockHeight, mockApp)
  246. }
  247. }
  248. cmn.PanicSanity("Should never happen")
  249. return nil, nil
  250. }
  251. func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, mutateState bool) ([]byte, error) {
  252. // App is further behind than it should be, so we need to replay blocks.
  253. // We replay all blocks from appBlockHeight+1.
  254. //
  255. // Note that we don't have an old version of the state,
  256. // so we by-pass state validation/mutation using sm.ExecCommitBlock.
  257. // This also means we won't be saving validator sets if they change during this period.
  258. //
  259. // If mutateState == true, the final block is replayed with h.replayBlock()
  260. var appHash []byte
  261. var err error
  262. finalBlock := storeBlockHeight
  263. if mutateState {
  264. finalBlock -= 1
  265. }
  266. for i := appBlockHeight + 1; i <= finalBlock; i++ {
  267. h.logger.Info("Applying block", "height", i)
  268. block := h.store.LoadBlock(i)
  269. appHash, err = sm.ExecCommitBlock(proxyApp.Consensus(), block, h.logger)
  270. if err != nil {
  271. return nil, err
  272. }
  273. h.nBlocks += 1
  274. }
  275. if mutateState {
  276. // sync the final block
  277. return h.replayBlock(storeBlockHeight, proxyApp.Consensus())
  278. }
  279. return appHash, h.checkAppHash(appHash)
  280. }
  281. // ApplyBlock on the proxyApp with the last block.
  282. func (h *Handshaker) replayBlock(height int, proxyApp proxy.AppConnConsensus) ([]byte, error) {
  283. mempool := types.MockMempool{}
  284. block := h.store.LoadBlock(height)
  285. meta := h.store.LoadBlockMeta(height)
  286. if err := h.state.ApplyBlock(types.NopEventBus{}, proxyApp, block, meta.BlockID.PartsHeader, mempool); err != nil {
  287. return nil, err
  288. }
  289. h.nBlocks += 1
  290. return h.state.AppHash, nil
  291. }
  292. func (h *Handshaker) checkAppHash(appHash []byte) error {
  293. if !bytes.Equal(h.state.AppHash, appHash) {
  294. panic(errors.New(cmn.Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)).Error())
  295. return nil
  296. }
  297. return nil
  298. }
  299. //--------------------------------------------------------------------------------
  300. // mockProxyApp uses ABCIResponses to give the right results
  301. // Useful because we don't want to call Commit() twice for the same block on the real app.
  302. func newMockProxyApp(appHash []byte, abciResponses *sm.ABCIResponses) proxy.AppConnConsensus {
  303. clientCreator := proxy.NewLocalClientCreator(&mockProxyApp{
  304. appHash: appHash,
  305. abciResponses: abciResponses,
  306. })
  307. cli, _ := clientCreator.NewABCIClient()
  308. cli.Start()
  309. return proxy.NewAppConnConsensus(cli)
  310. }
  311. type mockProxyApp struct {
  312. abci.BaseApplication
  313. appHash []byte
  314. txCount int
  315. abciResponses *sm.ABCIResponses
  316. }
  317. func (mock *mockProxyApp) DeliverTx(tx []byte) abci.Result {
  318. r := mock.abciResponses.DeliverTx[mock.txCount]
  319. mock.txCount += 1
  320. return abci.Result{
  321. r.Code,
  322. r.Data,
  323. r.Log,
  324. }
  325. }
  326. func (mock *mockProxyApp) EndBlock(height uint64) abci.ResponseEndBlock {
  327. mock.txCount = 0
  328. return mock.abciResponses.EndBlock
  329. }
  330. func (mock *mockProxyApp) Commit() abci.Result {
  331. return abci.NewResultOK(mock.appHash, "")
  332. }