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.

212 lines
6.7 KiB

mempool: move interface into mempool package (#3524) ## Description Refs #2659 Breaking changes in the mempool package: [mempool] #2659 Mempool now an interface old Mempool renamed to CListMempool NewMempool renamed to NewCListMempool Option renamed to CListOption MempoolReactor renamed to Reactor NewMempoolReactor renamed to NewReactor unexpose TxID method TxInfo.PeerID renamed to SenderID unexpose MempoolReactor.Mempool Breaking changes in the state package: [state] #2659 Mempool interface moved to mempool package MockMempool moved to top-level mock package and renamed to Mempool Non Breaking changes in the node package: [node] #2659 Add Mempool method, which allows you to access mempool ## Commits * move Mempool interface into mempool package Refs #2659 Breaking changes in the mempool package: - Mempool now an interface - old Mempool renamed to CListMempool Breaking changes to state package: - MockMempool moved to mempool/mock package and renamed to Mempool - Mempool interface moved to mempool package * assert CListMempool impl Mempool * gofmt code * rename MempoolReactor to Reactor - combine everything into one interface - rename TxInfo.PeerID to TxInfo.SenderID - unexpose MempoolReactor.Mempool * move mempool mock into top-level mock package * add a fixme TxsFront should not be a part of the Mempool interface because it leaks implementation details. Instead, we need to come up with general interface for querying the mempool so the MempoolReactor can fetch and broadcast txs to peers. * change node#Mempool to return interface * save commit = new reactor arch * Revert "save commit = new reactor arch" This reverts commit 1bfceacd9d65a720574683a7f22771e69af9af4d. * require CListMempool in mempool.Reactor * add two changelog entries * fixes after my own review * quote interfaces, structs and functions * fixes after Ismail's review * make node's mempool an interface * make InitWAL/CloseWAL methods a part of Mempool interface * fix merge conflicts * make node's mempool an interface
5 years ago
mempool: move interface into mempool package (#3524) ## Description Refs #2659 Breaking changes in the mempool package: [mempool] #2659 Mempool now an interface old Mempool renamed to CListMempool NewMempool renamed to NewCListMempool Option renamed to CListOption MempoolReactor renamed to Reactor NewMempoolReactor renamed to NewReactor unexpose TxID method TxInfo.PeerID renamed to SenderID unexpose MempoolReactor.Mempool Breaking changes in the state package: [state] #2659 Mempool interface moved to mempool package MockMempool moved to top-level mock package and renamed to Mempool Non Breaking changes in the node package: [node] #2659 Add Mempool method, which allows you to access mempool ## Commits * move Mempool interface into mempool package Refs #2659 Breaking changes in the mempool package: - Mempool now an interface - old Mempool renamed to CListMempool Breaking changes to state package: - MockMempool moved to mempool/mock package and renamed to Mempool - Mempool interface moved to mempool package * assert CListMempool impl Mempool * gofmt code * rename MempoolReactor to Reactor - combine everything into one interface - rename TxInfo.PeerID to TxInfo.SenderID - unexpose MempoolReactor.Mempool * move mempool mock into top-level mock package * add a fixme TxsFront should not be a part of the Mempool interface because it leaks implementation details. Instead, we need to come up with general interface for querying the mempool so the MempoolReactor can fetch and broadcast txs to peers. * change node#Mempool to return interface * save commit = new reactor arch * Revert "save commit = new reactor arch" This reverts commit 1bfceacd9d65a720574683a7f22771e69af9af4d. * require CListMempool in mempool.Reactor * add two changelog entries * fixes after my own review * quote interfaces, structs and functions * fixes after Ismail's review * make node's mempool an interface * make InitWAL/CloseWAL methods a part of Mempool interface * fix merge conflicts * make node's mempool an interface
5 years ago
blockchain: Reorg reactor (#3561) * go routines in blockchain reactor * Added reference to the go routine diagram * Initial commit * cleanup * Undo testing_logger change, committed by mistake * Fix the test loggers * pulled some fsm code into pool.go * added pool tests * changes to the design added block requests under peer moved the request trigger in the reactor poolRoutine, triggered now by a ticker in general moved everything required for making block requests smarter in the poolRoutine added a simple map of heights to keep track of what will need to be requested next added a few more tests * send errors to FSM in a different channel than blocks send errors (RemovePeer) from switch on a different channel than the one receiving blocks renamed channels added more pool tests * more pool tests * lint errors * more tests * more tests * switch fast sync to new implementation * fixed data race in tests * cleanup * finished fsm tests * address golangci comments :) * address golangci comments :) * Added timeout on next block needed to advance * updating docs and cleanup * fix issue in test from previous cleanup * cleanup * Added termination scenarios, tests and more cleanup * small fixes to adr, comments and cleanup * Fix bug in sendRequest() If we tried to send a request to a peer not present in the switch, a missing continue statement caused the request to be blackholed in a peer that was removed and never retried. While this bug was manifesting, the reactor kept asking for other blocks that would be stored and never consumed. Added the number of unconsumed blocks in the math for requesting blocks ahead of current processing height so eventually there will be no more blocks requested until the already received ones are consumed. * remove bpPeer's didTimeout field * Use distinct err codes for peer timeout and FSM timeouts * Don't allow peers to update with lower height * review comments from Ethan and Zarko * some cleanup, renaming, comments * Move block execution in separate goroutine * Remove pool's numPending * review comments * fix lint, remove old blockchain reactor and duplicates in fsm tests * small reorg around peer after review comments * add the reactor spec * verify block only once * review comments * change to int for max number of pending requests * cleanup and godoc * Add configuration flag fast sync version * golangci fixes * fix config template * move both reactor versions under blockchain * cleanup, golint, renaming stuff * updated documentation, fixed more golint warnings * integrate with behavior package * sync with master * gofmt * add changelog_pending entry * move to improvments * suggestion to changelog entry
5 years ago
cs: sync WAL more frequently (#3300) As per #3043, this adds a ticker to sync the WAL every 2s while the WAL is running. * Flush WAL every 2s This adds a ticker that flushes the WAL every 2s while the WAL is running. This is related to #3043. * Fix spelling * Increase timeout to 2mins for slower build environments * Make WAL sync interval configurable * Add TODO to replace testChan with more comprehensive testBus * Remove extraneous debug statement * Remove testChan in favour of using system time As per https://github.com/tendermint/tendermint/pull/3300#discussion_r255886586, this removes the `testChan` WAL member and replaces the approach with a system time-oriented one. In this new approach, we keep track of the system time at which each flush and periodic flush successfully occurred. The naming of the various functions is also updated here to be more consistent with "flushing" as opposed to "sync'ing". * Update naming convention and ensure lock for timestamp update * Add Flush method as part of WAL interface Adds a `Flush` method as part of the WAL interface to enforce the idea that we can manually trigger a WAL flush from outside of the WAL. This is employed in the consensus state management to flush the WAL prior to signing votes/proposals, as per https://github.com/tendermint/tendermint/issues/3043#issuecomment-453853630 * Update CHANGELOG_PENDING * Remove mutex approach and replace with DI The dependency injection approach to dealing with testing concerns could allow similar effects to some kind of "testing bus"-based approach. This commit introduces an example of this, where instead of relying on (potentially fragile) timing of things between the code and the test, we inject code into the function under test that can signal the test through a channel. This allows us to avoid the `time.Sleep()`-based approach previously employed. * Update comment on WAL flushing during vote signing Co-Authored-By: thanethomson <connect@thanethomson.com> * Simplify flush interval definition Co-Authored-By: thanethomson <connect@thanethomson.com> * Expand commentary on WAL disk flushing Co-Authored-By: thanethomson <connect@thanethomson.com> * Add broken test to illustrate WAL sync test problem Removes test-related state (dependency injection code) from the WAL data structure and adds test code to illustrate the problem with using `WALGenerateNBlocks` and `wal.SearchForEndHeight` to test periodic sync'ing. * Fix test error messages * Use WAL group buffer size to check for flush A function is added to `libs/autofile/group.go#Group` in order to return the size of the buffered data (i.e. data that has not yet been flushed to disk). The test now checks that, prior to a `time.Sleep`, the group buffer has data in it. After the `time.Sleep` (during which time the periodic flush should have been called), the buffer should be empty. * Remove config root dir removal from #3291 * Add godoc for NewWAL mentioning periodic sync
5 years ago
cs/replay: execCommitBlock should not read from state.lastValidators (#3067) * execCommitBlock should not read from state.lastValidators * fix height 1 * fix blockchain/reactor_test * fix consensus/mempool_test * fix consensus/reactor_test * fix consensus/replay_test * add CHANGELOG * fix consensus/reactor_test * fix consensus/replay_test * add a test for replay validators change * fix mem_pool test * fix byzantine test * remove a redundant code * reduce validator change blocks to 6 * fix * return peer0 config * seperate testName * seperate testName 1 * seperate testName 2 * seperate app db path * seperate app db path 1 * add a lock before startNet * move the lock to reactor_test * simulate just once * try to find problem * handshake only saveState when app version changed * update gometalinter to 3.0.0 (#3233) in the attempt to fix https://circleci.com/gh/tendermint/tendermint/43165 also code is simplified by running gofmt -s . remove unused vars enable linters we're currently passing remove deprecated linters (cherry picked from commit d47094550315c094512a242445e0dde24b5a03f5) * gofmt code * goimport code * change the bool name to testValidatorsChange * adjust receive kvstore.ProtocolVersion * adjust receive kvstore.ProtocolVersion 1 * adjust receive kvstore.ProtocolVersion 3 * fix merge execution.go * fix merge develop * fix merge develop 1 * fix run cleanupFunc * adjust code according to reviewers' opinion * modify the func name match the convention * simplify simulate a chain containing some validator change txs 1 * test CI error * Merge remote-tracking branch 'upstream/develop' into fixReplay 1 * fix pubsub_test * subscribeUnbuffered vote channel
5 years ago
cs/replay: execCommitBlock should not read from state.lastValidators (#3067) * execCommitBlock should not read from state.lastValidators * fix height 1 * fix blockchain/reactor_test * fix consensus/mempool_test * fix consensus/reactor_test * fix consensus/replay_test * add CHANGELOG * fix consensus/reactor_test * fix consensus/replay_test * add a test for replay validators change * fix mem_pool test * fix byzantine test * remove a redundant code * reduce validator change blocks to 6 * fix * return peer0 config * seperate testName * seperate testName 1 * seperate testName 2 * seperate app db path * seperate app db path 1 * add a lock before startNet * move the lock to reactor_test * simulate just once * try to find problem * handshake only saveState when app version changed * update gometalinter to 3.0.0 (#3233) in the attempt to fix https://circleci.com/gh/tendermint/tendermint/43165 also code is simplified by running gofmt -s . remove unused vars enable linters we're currently passing remove deprecated linters (cherry picked from commit d47094550315c094512a242445e0dde24b5a03f5) * gofmt code * goimport code * change the bool name to testValidatorsChange * adjust receive kvstore.ProtocolVersion * adjust receive kvstore.ProtocolVersion 1 * adjust receive kvstore.ProtocolVersion 3 * fix merge execution.go * fix merge develop * fix merge develop 1 * fix run cleanupFunc * adjust code according to reviewers' opinion * modify the func name match the convention * simplify simulate a chain containing some validator change txs 1 * test CI error * Merge remote-tracking branch 'upstream/develop' into fixReplay 1 * fix pubsub_test * subscribeUnbuffered vote channel
5 years ago
blockchain: Reorg reactor (#3561) * go routines in blockchain reactor * Added reference to the go routine diagram * Initial commit * cleanup * Undo testing_logger change, committed by mistake * Fix the test loggers * pulled some fsm code into pool.go * added pool tests * changes to the design added block requests under peer moved the request trigger in the reactor poolRoutine, triggered now by a ticker in general moved everything required for making block requests smarter in the poolRoutine added a simple map of heights to keep track of what will need to be requested next added a few more tests * send errors to FSM in a different channel than blocks send errors (RemovePeer) from switch on a different channel than the one receiving blocks renamed channels added more pool tests * more pool tests * lint errors * more tests * more tests * switch fast sync to new implementation * fixed data race in tests * cleanup * finished fsm tests * address golangci comments :) * address golangci comments :) * Added timeout on next block needed to advance * updating docs and cleanup * fix issue in test from previous cleanup * cleanup * Added termination scenarios, tests and more cleanup * small fixes to adr, comments and cleanup * Fix bug in sendRequest() If we tried to send a request to a peer not present in the switch, a missing continue statement caused the request to be blackholed in a peer that was removed and never retried. While this bug was manifesting, the reactor kept asking for other blocks that would be stored and never consumed. Added the number of unconsumed blocks in the math for requesting blocks ahead of current processing height so eventually there will be no more blocks requested until the already received ones are consumed. * remove bpPeer's didTimeout field * Use distinct err codes for peer timeout and FSM timeouts * Don't allow peers to update with lower height * review comments from Ethan and Zarko * some cleanup, renaming, comments * Move block execution in separate goroutine * Remove pool's numPending * review comments * fix lint, remove old blockchain reactor and duplicates in fsm tests * small reorg around peer after review comments * add the reactor spec * verify block only once * review comments * change to int for max number of pending requests * cleanup and godoc * Add configuration flag fast sync version * golangci fixes * fix config template * move both reactor versions under blockchain * cleanup, golint, renaming stuff * updated documentation, fixed more golint warnings * integrate with behavior package * sync with master * gofmt * add changelog_pending entry * move to improvments * suggestion to changelog entry
5 years ago
mempool: move interface into mempool package (#3524) ## Description Refs #2659 Breaking changes in the mempool package: [mempool] #2659 Mempool now an interface old Mempool renamed to CListMempool NewMempool renamed to NewCListMempool Option renamed to CListOption MempoolReactor renamed to Reactor NewMempoolReactor renamed to NewReactor unexpose TxID method TxInfo.PeerID renamed to SenderID unexpose MempoolReactor.Mempool Breaking changes in the state package: [state] #2659 Mempool interface moved to mempool package MockMempool moved to top-level mock package and renamed to Mempool Non Breaking changes in the node package: [node] #2659 Add Mempool method, which allows you to access mempool ## Commits * move Mempool interface into mempool package Refs #2659 Breaking changes in the mempool package: - Mempool now an interface - old Mempool renamed to CListMempool Breaking changes to state package: - MockMempool moved to mempool/mock package and renamed to Mempool - Mempool interface moved to mempool package * assert CListMempool impl Mempool * gofmt code * rename MempoolReactor to Reactor - combine everything into one interface - rename TxInfo.PeerID to TxInfo.SenderID - unexpose MempoolReactor.Mempool * move mempool mock into top-level mock package * add a fixme TxsFront should not be a part of the Mempool interface because it leaks implementation details. Instead, we need to come up with general interface for querying the mempool so the MempoolReactor can fetch and broadcast txs to peers. * change node#Mempool to return interface * save commit = new reactor arch * Revert "save commit = new reactor arch" This reverts commit 1bfceacd9d65a720574683a7f22771e69af9af4d. * require CListMempool in mempool.Reactor * add two changelog entries * fixes after my own review * quote interfaces, structs and functions * fixes after Ismail's review * make node's mempool an interface * make InitWAL/CloseWAL methods a part of Mempool interface * fix merge conflicts * make node's mempool an interface
5 years ago
lint: Enable Golint (#4212) * Fix many golint errors * Fix golint errors in the 'lite' package * Don't export Pool.store * Fix typo * Revert unwanted changes * Fix errors in counter package * Fix linter errors in kvstore package * Fix linter error in example package * Fix error in tests package * Fix linter errors in v2 package * Fix linter errors in consensus package * Fix linter errors in evidence package * Fix linter error in fail package * Fix linter errors in query package * Fix linter errors in core package * Fix linter errors in node package * Fix linter errors in mempool package * Fix linter error in conn package * Fix linter errors in pex package * Rename PEXReactor export to Reactor * Fix linter errors in trust package * Fix linter errors in upnp package * Fix linter errors in p2p package * Fix linter errors in proxy package * Fix linter errors in mock_test package * Fix linter error in client_test package * Fix linter errors in coretypes package * Fix linter errors in coregrpc package * Fix linter errors in rpcserver package * Fix linter errors in rpctypes package * Fix linter errors in rpctest package * Fix linter error in json2wal script * Fix linter error in wal2json script * Fix linter errors in kv package * Fix linter error in state package * Fix linter error in grpc_client * Fix linter errors in types package * Fix linter error in version package * Fix remaining errors * Address review comments * Fix broken tests * Reconcile package coregrpc * Fix golangci bot error * Fix new golint errors * Fix broken reference * Enable golint linter * minor changes to bring golint into line * fix failing test * fix pex reactor naming * address PR comments
5 years ago
  1. package consensus
  2. import (
  3. "bufio"
  4. "bytes"
  5. "fmt"
  6. "io"
  7. "path/filepath"
  8. "testing"
  9. "time"
  10. "github.com/pkg/errors"
  11. db "github.com/tendermint/tm-db"
  12. "github.com/tendermint/tendermint/abci/example/kvstore"
  13. cfg "github.com/tendermint/tendermint/config"
  14. "github.com/tendermint/tendermint/libs/log"
  15. tmrand "github.com/tendermint/tendermint/libs/rand"
  16. "github.com/tendermint/tendermint/mock"
  17. "github.com/tendermint/tendermint/privval"
  18. "github.com/tendermint/tendermint/proxy"
  19. sm "github.com/tendermint/tendermint/state"
  20. "github.com/tendermint/tendermint/store"
  21. "github.com/tendermint/tendermint/types"
  22. )
  23. // WALGenerateNBlocks generates a consensus WAL. It does this by spinning up a
  24. // stripped down version of node (proxy app, event bus, consensus state) with a
  25. // persistent kvstore application and special consensus wal instance
  26. // (byteBufferWAL) and waits until numBlocks are created.
  27. // If the node fails to produce given numBlocks, it returns an error.
  28. func WALGenerateNBlocks(t *testing.T, wr io.Writer, numBlocks int) (err error) {
  29. config := getConfig(t)
  30. app := kvstore.NewPersistentKVStoreApplication(filepath.Join(config.DBDir(), "wal_generator"))
  31. logger := log.TestingLogger().With("wal_generator", "wal_generator")
  32. logger.Info("generating WAL (last height msg excluded)", "numBlocks", numBlocks)
  33. /////////////////////////////////////////////////////////////////////////////
  34. // COPY PASTE FROM node.go WITH A FEW MODIFICATIONS
  35. // NOTE: we can't import node package because of circular dependency.
  36. // NOTE: we don't do handshake so need to set state.Version.Consensus.App directly.
  37. privValidatorKeyFile := config.PrivValidatorKeyFile()
  38. privValidatorStateFile := config.PrivValidatorStateFile()
  39. privValidator := privval.LoadOrGenFilePV(privValidatorKeyFile, privValidatorStateFile)
  40. genDoc, err := types.GenesisDocFromFile(config.GenesisFile())
  41. if err != nil {
  42. return errors.Wrap(err, "failed to read genesis file")
  43. }
  44. blockStoreDB := db.NewMemDB()
  45. stateDB := blockStoreDB
  46. state, err := sm.MakeGenesisState(genDoc)
  47. if err != nil {
  48. return errors.Wrap(err, "failed to make genesis state")
  49. }
  50. state.Version.Consensus.App = kvstore.ProtocolVersion
  51. sm.SaveState(stateDB, state)
  52. blockStore := store.NewBlockStore(blockStoreDB)
  53. proxyApp := proxy.NewAppConns(proxy.NewLocalClientCreator(app))
  54. proxyApp.SetLogger(logger.With("module", "proxy"))
  55. if err := proxyApp.Start(); err != nil {
  56. return errors.Wrap(err, "failed to start proxy app connections")
  57. }
  58. defer proxyApp.Stop()
  59. eventBus := types.NewEventBus()
  60. eventBus.SetLogger(logger.With("module", "events"))
  61. if err := eventBus.Start(); err != nil {
  62. return errors.Wrap(err, "failed to start event bus")
  63. }
  64. defer eventBus.Stop()
  65. mempool := mock.Mempool{}
  66. evpool := sm.MockEvidencePool{}
  67. blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
  68. consensusState := NewState(config.Consensus, state.Copy(), blockExec, blockStore, mempool, evpool)
  69. consensusState.SetLogger(logger)
  70. consensusState.SetEventBus(eventBus)
  71. if privValidator != nil {
  72. consensusState.SetPrivValidator(privValidator)
  73. }
  74. // END OF COPY PASTE
  75. /////////////////////////////////////////////////////////////////////////////
  76. // set consensus wal to buffered WAL, which will write all incoming msgs to buffer
  77. numBlocksWritten := make(chan struct{})
  78. wal := newByteBufferWAL(logger, NewWALEncoder(wr), int64(numBlocks), numBlocksWritten)
  79. // see wal.go#103
  80. wal.Write(EndHeightMessage{0})
  81. consensusState.wal = wal
  82. if err := consensusState.Start(); err != nil {
  83. return errors.Wrap(err, "failed to start consensus state")
  84. }
  85. select {
  86. case <-numBlocksWritten:
  87. consensusState.Stop()
  88. return nil
  89. case <-time.After(1 * time.Minute):
  90. consensusState.Stop()
  91. return fmt.Errorf("waited too long for tendermint to produce %d blocks (grep logs for `wal_generator`)", numBlocks)
  92. }
  93. }
  94. //WALWithNBlocks returns a WAL content with numBlocks.
  95. func WALWithNBlocks(t *testing.T, numBlocks int) (data []byte, err error) {
  96. var b bytes.Buffer
  97. wr := bufio.NewWriter(&b)
  98. if err := WALGenerateNBlocks(t, wr, numBlocks); err != nil {
  99. return []byte{}, err
  100. }
  101. wr.Flush()
  102. return b.Bytes(), nil
  103. }
  104. func randPort() int {
  105. // returns between base and base + spread
  106. base, spread := 20000, 20000
  107. return base + tmrand.Intn(spread)
  108. }
  109. func makeAddrs() (string, string, string) {
  110. start := randPort()
  111. return fmt.Sprintf("tcp://127.0.0.1:%d", start),
  112. fmt.Sprintf("tcp://127.0.0.1:%d", start+1),
  113. fmt.Sprintf("tcp://127.0.0.1:%d", start+2)
  114. }
  115. // getConfig returns a config for test cases
  116. func getConfig(t *testing.T) *cfg.Config {
  117. c := cfg.ResetTestRoot(t.Name())
  118. // and we use random ports to run in parallel
  119. tm, rpc, grpc := makeAddrs()
  120. c.P2P.ListenAddress = tm
  121. c.RPC.ListenAddress = rpc
  122. c.RPC.GRPCListenAddress = grpc
  123. return c
  124. }
  125. // byteBufferWAL is a WAL which writes all msgs to a byte buffer. Writing stops
  126. // when the heightToStop is reached. Client will be notified via
  127. // signalWhenStopsTo channel.
  128. type byteBufferWAL struct {
  129. enc *WALEncoder
  130. stopped bool
  131. heightToStop int64
  132. signalWhenStopsTo chan<- struct{}
  133. logger log.Logger
  134. }
  135. // needed for determinism
  136. var fixedTime, _ = time.Parse(time.RFC3339, "2017-01-02T15:04:05Z")
  137. func newByteBufferWAL(logger log.Logger, enc *WALEncoder, nBlocks int64, signalStop chan<- struct{}) *byteBufferWAL {
  138. return &byteBufferWAL{
  139. enc: enc,
  140. heightToStop: nBlocks,
  141. signalWhenStopsTo: signalStop,
  142. logger: logger,
  143. }
  144. }
  145. // Save writes message to the internal buffer except when heightToStop is
  146. // reached, in which case it will signal the caller via signalWhenStopsTo and
  147. // skip writing.
  148. func (w *byteBufferWAL) Write(m WALMessage) error {
  149. if w.stopped {
  150. w.logger.Debug("WAL already stopped. Not writing message", "msg", m)
  151. return nil
  152. }
  153. if endMsg, ok := m.(EndHeightMessage); ok {
  154. w.logger.Debug("WAL write end height message", "height", endMsg.Height, "stopHeight", w.heightToStop)
  155. if endMsg.Height == w.heightToStop {
  156. w.logger.Debug("Stopping WAL at height", "height", endMsg.Height)
  157. w.signalWhenStopsTo <- struct{}{}
  158. w.stopped = true
  159. return nil
  160. }
  161. }
  162. w.logger.Debug("WAL Write Message", "msg", m)
  163. err := w.enc.Encode(&TimedWALMessage{fixedTime, m})
  164. if err != nil {
  165. panic(fmt.Sprintf("failed to encode the msg %v", m))
  166. }
  167. return nil
  168. }
  169. func (w *byteBufferWAL) WriteSync(m WALMessage) error {
  170. return w.Write(m)
  171. }
  172. func (w *byteBufferWAL) FlushAndSync() error { return nil }
  173. func (w *byteBufferWAL) SearchForEndHeight(
  174. height int64,
  175. options *WALSearchOptions) (rd io.ReadCloser, found bool, err error) {
  176. return nil, false, nil
  177. }
  178. func (w *byteBufferWAL) Start() error { return nil }
  179. func (w *byteBufferWAL) Stop() error { return nil }
  180. func (w *byteBufferWAL) Wait() {}