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.

407 lines
11 KiB

  1. package consensus
  2. import (
  3. "encoding/binary"
  4. "errors"
  5. "fmt"
  6. "hash/crc32"
  7. "io"
  8. "path/filepath"
  9. "time"
  10. "github.com/gogo/protobuf/proto"
  11. // tmjson "github.com/tendermint/tendermint/libs/json"
  12. tmcon "github.com/tendermint/tendermint/consensus"
  13. auto "github.com/tendermint/tendermint/libs/autofile"
  14. "github.com/tendermint/tendermint/libs/log"
  15. tmos "github.com/tendermint/tendermint/libs/os"
  16. "github.com/tendermint/tendermint/libs/service"
  17. tmcons "github.com/tendermint/tendermint/proto/tendermint/consensus"
  18. tmtime "github.com/tendermint/tendermint/types/time"
  19. )
  20. const (
  21. // time.Time + max consensus msg size
  22. maxMsgSizeBytes = maxMsgSize + 24
  23. // how often the WAL should be sync'd during period sync'ing
  24. walDefaultFlushInterval = 2 * time.Second
  25. )
  26. //--------------------------------------------------------
  27. // types and functions for savings consensus messages
  28. // func init() {
  29. // tmjson.RegisterType(msgInfo{}, "tendermint/wal/MsgInfo")
  30. // tmjson.RegisterType(timeoutInfo{}, "tendermint/wal/TimeoutInfo")
  31. // tmjson.RegisterType(tmcon.EndHeightMessage {}, "tendermint/wal/EndHeightMessage ")
  32. // }
  33. // Write ahead logger writes msgs to disk before they are processed.
  34. // Can be used for crash-recovery and deterministic replay.
  35. // TODO: currently the wal is overwritten during replay catchup, give it a mode
  36. // so it's either reading or appending - must read to end to start appending
  37. // again.
  38. type BaseWAL struct {
  39. service.BaseService
  40. group *auto.Group
  41. enc *WALEncoder
  42. flushTicker *time.Ticker
  43. flushInterval time.Duration
  44. }
  45. var _ tmcon.WAL = &BaseWAL{}
  46. // NewWAL returns a new write-ahead logger based on `baseWAL`, which implements
  47. // WAL. It's flushed and synced to disk every 2s and once when stopped.
  48. func NewWAL(walFile string, groupOptions ...func(*auto.Group)) (*BaseWAL, error) {
  49. err := tmos.EnsureDir(filepath.Dir(walFile), 0700)
  50. if err != nil {
  51. return nil, fmt.Errorf("failed to ensure WAL directory is in place: %w", err)
  52. }
  53. group, err := auto.OpenGroup(walFile, groupOptions...)
  54. if err != nil {
  55. return nil, err
  56. }
  57. wal := &BaseWAL{
  58. group: group,
  59. enc: NewWALEncoder(group),
  60. flushInterval: walDefaultFlushInterval,
  61. }
  62. wal.BaseService = *service.NewBaseService(nil, "baseWAL", wal)
  63. return wal, nil
  64. }
  65. // SetFlushInterval allows us to override the periodic flush interval for the WAL.
  66. func (wal *BaseWAL) SetFlushInterval(i time.Duration) {
  67. wal.flushInterval = i
  68. }
  69. func (wal *BaseWAL) Group() *auto.Group {
  70. return wal.group
  71. }
  72. func (wal *BaseWAL) SetLogger(l log.Logger) {
  73. wal.BaseService.Logger = l
  74. wal.group.SetLogger(l)
  75. }
  76. func (wal *BaseWAL) OnStart() error {
  77. size, err := wal.group.Head.Size()
  78. if err != nil {
  79. return err
  80. } else if size == 0 {
  81. if err := wal.WriteSync(tmcon.EndHeightMessage{Height: 0}); err != nil {
  82. return err
  83. }
  84. }
  85. err = wal.group.Start()
  86. if err != nil {
  87. return err
  88. }
  89. wal.flushTicker = time.NewTicker(wal.flushInterval)
  90. go wal.processFlushTicks()
  91. return nil
  92. }
  93. func (wal *BaseWAL) processFlushTicks() {
  94. for {
  95. select {
  96. case <-wal.flushTicker.C:
  97. if err := wal.FlushAndSync(); err != nil {
  98. wal.Logger.Error("Periodic WAL flush failed", "err", err)
  99. }
  100. case <-wal.Quit():
  101. return
  102. }
  103. }
  104. }
  105. // FlushAndSync flushes and fsync's the underlying group's data to disk.
  106. // See auto#FlushAndSync
  107. func (wal *BaseWAL) FlushAndSync() error {
  108. return wal.group.FlushAndSync()
  109. }
  110. // Stop the underlying autofile group.
  111. // Use Wait() to ensure it's finished shutting down
  112. // before cleaning up files.
  113. func (wal *BaseWAL) OnStop() {
  114. wal.flushTicker.Stop()
  115. if err := wal.FlushAndSync(); err != nil {
  116. wal.Logger.Error("error on flush data to disk", "error", err)
  117. }
  118. if err := wal.group.Stop(); err != nil {
  119. wal.Logger.Error("error trying to stop wal", "error", err)
  120. }
  121. wal.group.Close()
  122. }
  123. // Wait for the underlying autofile group to finish shutting down
  124. // so it's safe to cleanup files.
  125. func (wal *BaseWAL) Wait() {
  126. wal.group.Wait()
  127. }
  128. // Write is called in newStep and for each receive on the
  129. // peerMsgQueue and the timeoutTicker.
  130. // NOTE: does not call fsync()
  131. func (wal *BaseWAL) Write(msg tmcon.WALMessage) error {
  132. if wal == nil {
  133. return nil
  134. }
  135. if err := wal.enc.Encode(&tmcon.TimedWALMessage{Time: tmtime.Now(), Msg: msg}); err != nil {
  136. wal.Logger.Error("Error writing msg to consensus wal. WARNING: recover may not be possible for the current height",
  137. "err", err, "msg", msg)
  138. return err
  139. }
  140. return nil
  141. }
  142. // WriteSync is called when we receive a msg from ourselves
  143. // so that we write to disk before sending signed messages.
  144. // NOTE: calls fsync()
  145. func (wal *BaseWAL) WriteSync(msg tmcon.WALMessage) error {
  146. if wal == nil {
  147. return nil
  148. }
  149. if err := wal.Write(msg); err != nil {
  150. return err
  151. }
  152. if err := wal.FlushAndSync(); err != nil {
  153. wal.Logger.Error(`WriteSync failed to flush consensus wal.
  154. WARNING: may result in creating alternative proposals / votes for the current height iff the node restarted`,
  155. "err", err)
  156. return err
  157. }
  158. return nil
  159. }
  160. // WALSearchOptions are optional arguments to SearchForEndHeight.
  161. type WALSearchOptions struct {
  162. // IgnoreDataCorruptionErrors set to true will result in skipping data corruption errors.
  163. IgnoreDataCorruptionErrors bool
  164. }
  165. // SearchForEndHeight searches for the EndHeightMessage with the given height
  166. // and returns an auto.GroupReader, whenever it was found or not and an error.
  167. // Group reader will be nil if found equals false.
  168. //
  169. // CONTRACT: caller must close group reader.
  170. func (wal *BaseWAL) SearchForEndHeight(
  171. height int64,
  172. options *tmcon.WALSearchOptions) (rd io.ReadCloser, found bool, err error) {
  173. var (
  174. msg *tmcon.TimedWALMessage
  175. gr *auto.GroupReader
  176. )
  177. lastHeightFound := int64(-1)
  178. // NOTE: starting from the last file in the group because we're usually
  179. // searching for the last height. See replay.go
  180. min, max := wal.group.MinIndex(), wal.group.MaxIndex()
  181. wal.Logger.Info("Searching for height", "height", height, "min", min, "max", max)
  182. for index := max; index >= min; index-- {
  183. gr, err = wal.group.NewReader(index)
  184. if err != nil {
  185. return nil, false, err
  186. }
  187. dec := NewWALDecoder(gr)
  188. for {
  189. msg, err = dec.Decode()
  190. if err == io.EOF {
  191. // OPTIMISATION: no need to look for height in older files if we've seen h < height
  192. if lastHeightFound > 0 && lastHeightFound < height {
  193. gr.Close()
  194. return nil, false, nil
  195. }
  196. // check next file
  197. break
  198. }
  199. if options.IgnoreDataCorruptionErrors && IsDataCorruptionError(err) {
  200. wal.Logger.Error("Corrupted entry. Skipping...", "err", err)
  201. // do nothing
  202. continue
  203. } else if err != nil {
  204. gr.Close()
  205. return nil, false, err
  206. }
  207. if m, ok := msg.Msg.(tmcon.EndHeightMessage); ok {
  208. lastHeightFound = m.Height
  209. if m.Height == height { // found
  210. wal.Logger.Info("Found", "height", height, "index", index)
  211. return gr, true, nil
  212. }
  213. }
  214. }
  215. gr.Close()
  216. }
  217. return nil, false, nil
  218. }
  219. // /////////////////////////////////////////////////////////////////////////////
  220. // A WALEncoder writes custom-encoded WAL messages to an output stream.
  221. //
  222. // Format: 4 bytes CRC sum + 4 bytes length + arbitrary-length value
  223. type WALEncoder struct {
  224. wr io.Writer
  225. }
  226. // NewWALEncoder returns a new encoder that writes to wr.
  227. func NewWALEncoder(wr io.Writer) *WALEncoder {
  228. return &WALEncoder{wr}
  229. }
  230. // Encode writes the custom encoding of v to the stream. It returns an error if
  231. // the encoded size of v is greater than 1MB. Any error encountered
  232. // during the write is also returned.
  233. func (enc *WALEncoder) Encode(v *tmcon.TimedWALMessage) error {
  234. pbMsg, err := WALToProto(v.Msg)
  235. if err != nil {
  236. return err
  237. }
  238. pv := tmcons.TimedWALMessage{
  239. Time: v.Time,
  240. Msg: pbMsg,
  241. }
  242. data, err := proto.Marshal(&pv)
  243. if err != nil {
  244. panic(fmt.Errorf("encode timed wall message failure: %w", err))
  245. }
  246. crc := crc32.Checksum(data, crc32c)
  247. length := uint32(len(data))
  248. if length > maxMsgSizeBytes {
  249. return fmt.Errorf("msg is too big: %d bytes, max: %d bytes", length, maxMsgSizeBytes)
  250. }
  251. totalLength := 8 + int(length)
  252. msg := make([]byte, totalLength)
  253. binary.BigEndian.PutUint32(msg[0:4], crc)
  254. binary.BigEndian.PutUint32(msg[4:8], length)
  255. copy(msg[8:], data)
  256. _, err = enc.wr.Write(msg)
  257. return err
  258. }
  259. // /////////////////////////////////////////////////////////////////////////////
  260. // IsDataCorruptionError returns true if data has been corrupted inside WAL.
  261. func IsDataCorruptionError(err error) bool {
  262. _, ok := err.(DataCorruptionError)
  263. return ok
  264. }
  265. // DataCorruptionError is an error that occures if data on disk was corrupted.
  266. type DataCorruptionError struct {
  267. cause error
  268. }
  269. func (e DataCorruptionError) Error() string {
  270. return fmt.Sprintf("DataCorruptionError[%v]", e.cause)
  271. }
  272. func (e DataCorruptionError) Cause() error {
  273. return e.cause
  274. }
  275. // A WALDecoder reads and decodes custom-encoded WAL messages from an input
  276. // stream. See WALEncoder for the format used.
  277. //
  278. // It will also compare the checksums and make sure data size is equal to the
  279. // length from the header. If that is not the case, error will be returned.
  280. type WALDecoder struct {
  281. rd io.Reader
  282. }
  283. // NewWALDecoder returns a new decoder that reads from rd.
  284. func NewWALDecoder(rd io.Reader) *WALDecoder {
  285. return &WALDecoder{rd}
  286. }
  287. // Decode reads the next custom-encoded value from its reader and returns it.
  288. func (dec *WALDecoder) Decode() (*tmcon.TimedWALMessage, error) {
  289. b := make([]byte, 4)
  290. _, err := dec.rd.Read(b)
  291. if errors.Is(err, io.EOF) {
  292. return nil, err
  293. }
  294. if err != nil {
  295. return nil, DataCorruptionError{fmt.Errorf("failed to read checksum: %v", err)}
  296. }
  297. crc := binary.BigEndian.Uint32(b)
  298. b = make([]byte, 4)
  299. _, err = dec.rd.Read(b)
  300. if err != nil {
  301. return nil, DataCorruptionError{fmt.Errorf("failed to read length: %v", err)}
  302. }
  303. length := binary.BigEndian.Uint32(b)
  304. if length > maxMsgSizeBytes {
  305. return nil, DataCorruptionError{fmt.Errorf(
  306. "length %d exceeded maximum possible value of %d bytes",
  307. length,
  308. maxMsgSizeBytes)}
  309. }
  310. data := make([]byte, length)
  311. n, err := dec.rd.Read(data)
  312. if err != nil {
  313. return nil, DataCorruptionError{fmt.Errorf("failed to read data: %v (read: %d, wanted: %d)", err, n, length)}
  314. }
  315. // check checksum before decoding data
  316. actualCRC := crc32.Checksum(data, crc32c)
  317. if actualCRC != crc {
  318. return nil, DataCorruptionError{fmt.Errorf("checksums do not match: read: %v, actual: %v", crc, actualCRC)}
  319. }
  320. var res = new(tmcons.TimedWALMessage)
  321. err = proto.Unmarshal(data, res)
  322. if err != nil {
  323. return nil, DataCorruptionError{fmt.Errorf("failed to decode data: %v", err)}
  324. }
  325. walMsg, err := WALFromProto(res.Msg)
  326. if err != nil {
  327. return nil, DataCorruptionError{fmt.Errorf("failed to convert from proto: %w", err)}
  328. }
  329. tMsgWal := &tmcon.TimedWALMessage{
  330. Time: res.Time,
  331. Msg: walMsg,
  332. }
  333. return tMsgWal, err
  334. }
  335. type nilWAL struct{}
  336. var _ tmcon.WAL = nilWAL{}
  337. func (nilWAL) Write(m tmcon.WALMessage) error { return nil }
  338. func (nilWAL) WriteSync(m tmcon.WALMessage) error { return nil }
  339. func (nilWAL) FlushAndSync() error { return nil }
  340. func (nilWAL) SearchForEndHeight(height int64,
  341. options *tmcon.WALSearchOptions) (rd io.ReadCloser, found bool, err error) {
  342. return nil, false, nil
  343. }
  344. func (nilWAL) Start() error { return nil }
  345. func (nilWAL) Stop() error { return nil }
  346. func (nilWAL) Wait() {}