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.

360 lines
11 KiB

5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
12 months ago
12 months ago
12 months ago
12 months ago
5 years ago
5 years ago
1 year ago
1 year ago
5 years ago
5 years ago
5 years ago
12 months ago
12 months ago
12 months ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
1 year ago
5 years ago
1 year ago
5 years ago
5 years ago
11 months ago
1 year ago
5 years ago
1 year ago
1 year ago
11 months ago
1 year ago
5 years ago
5 years ago
1 year ago
5 years ago
5 years ago
5 years ago
  1. package log_buffer
  2. import (
  3. "bytes"
  4. "sync"
  5. "sync/atomic"
  6. "time"
  7. "google.golang.org/protobuf/proto"
  8. "github.com/seaweedfs/seaweedfs/weed/glog"
  9. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  10. "github.com/seaweedfs/seaweedfs/weed/util"
  11. )
  12. const BufferSize = 4 * 1024 * 1024
  13. const PreviousBufferCount = 3
  14. type dataToFlush struct {
  15. startTime time.Time
  16. stopTime time.Time
  17. data *bytes.Buffer
  18. }
  19. type EachLogEntryFuncType func(logEntry *filer_pb.LogEntry) (isDone bool, err error)
  20. type LogFlushFuncType func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte)
  21. type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error)
  22. type LogBuffer struct {
  23. LastFlushTsNs int64
  24. name string
  25. prevBuffers *SealedBuffers
  26. buf []byte
  27. batchIndex int64
  28. idx []int
  29. pos int
  30. startTime time.Time
  31. stopTime time.Time
  32. lastFlushDataTime time.Time
  33. sizeBuf []byte
  34. flushInterval time.Duration
  35. flushFn LogFlushFuncType
  36. ReadFromDiskFn LogReadFromDiskFuncType
  37. notifyFn func()
  38. isStopping *atomic.Bool
  39. isAllFlushed bool
  40. flushChan chan *dataToFlush
  41. LastTsNs int64
  42. sync.RWMutex
  43. }
  44. func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFuncType,
  45. readFromDiskFn LogReadFromDiskFuncType, notifyFn func()) *LogBuffer {
  46. lb := &LogBuffer{
  47. name: name,
  48. prevBuffers: newSealedBuffers(PreviousBufferCount),
  49. buf: make([]byte, BufferSize),
  50. sizeBuf: make([]byte, 4),
  51. flushInterval: flushInterval,
  52. flushFn: flushFn,
  53. ReadFromDiskFn: readFromDiskFn,
  54. notifyFn: notifyFn,
  55. flushChan: make(chan *dataToFlush, 256),
  56. isStopping: new(atomic.Bool),
  57. }
  58. go lb.loopFlush()
  59. go lb.loopInterval()
  60. return lb
  61. }
  62. func (logBuffer *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64) {
  63. var toFlush *dataToFlush
  64. logBuffer.Lock()
  65. defer func() {
  66. logBuffer.Unlock()
  67. if toFlush != nil {
  68. logBuffer.flushChan <- toFlush
  69. }
  70. if logBuffer.notifyFn != nil {
  71. logBuffer.notifyFn()
  72. }
  73. }()
  74. // need to put the timestamp inside the lock
  75. var ts time.Time
  76. if processingTsNs == 0 {
  77. ts = time.Now()
  78. processingTsNs = ts.UnixNano()
  79. } else {
  80. ts = time.Unix(0, processingTsNs)
  81. }
  82. if logBuffer.LastTsNs >= processingTsNs {
  83. // this is unlikely to happen, but just in case
  84. processingTsNs = logBuffer.LastTsNs + 1
  85. ts = time.Unix(0, processingTsNs)
  86. }
  87. logBuffer.LastTsNs = processingTsNs
  88. logEntry := &filer_pb.LogEntry{
  89. TsNs: processingTsNs,
  90. PartitionKeyHash: util.HashToInt32(partitionKey),
  91. Data: data,
  92. Key: partitionKey,
  93. }
  94. logEntryData, _ := proto.Marshal(logEntry)
  95. size := len(logEntryData)
  96. if logBuffer.pos == 0 {
  97. logBuffer.startTime = ts
  98. }
  99. if logBuffer.startTime.Add(logBuffer.flushInterval).Before(ts) || len(logBuffer.buf)-logBuffer.pos < size+4 {
  100. // glog.V(0).Infof("%s copyToFlush1 batch:%d count:%d start time %v, ts %v, remaining %d bytes", logBuffer.name, logBuffer.batchIndex, len(logBuffer.idx), logBuffer.startTime, ts, len(logBuffer.buf)-logBuffer.pos)
  101. toFlush = logBuffer.copyToFlush()
  102. logBuffer.startTime = ts
  103. if len(logBuffer.buf) < size+4 {
  104. logBuffer.buf = make([]byte, 2*size+4)
  105. }
  106. }
  107. logBuffer.stopTime = ts
  108. logBuffer.idx = append(logBuffer.idx, logBuffer.pos)
  109. util.Uint32toBytes(logBuffer.sizeBuf, uint32(size))
  110. copy(logBuffer.buf[logBuffer.pos:logBuffer.pos+4], logBuffer.sizeBuf)
  111. copy(logBuffer.buf[logBuffer.pos+4:logBuffer.pos+4+size], logEntryData)
  112. logBuffer.pos += size + 4
  113. // fmt.Printf("partitionKey %v entry size %d total %d count %d\n", string(partitionKey), size, m.pos, len(m.idx))
  114. }
  115. func (logBuffer *LogBuffer) IsStopping() bool {
  116. return logBuffer.isStopping.Load()
  117. }
  118. // ShutdownLogBuffer flushes the buffer and stops the log buffer
  119. func (logBuffer *LogBuffer) ShutdownLogBuffer() {
  120. isAlreadyStopped := logBuffer.isStopping.Swap(true)
  121. if isAlreadyStopped {
  122. return
  123. }
  124. toFlush := logBuffer.copyToFlush()
  125. logBuffer.flushChan <- toFlush
  126. close(logBuffer.flushChan)
  127. }
  128. // IsAllFlushed returns true if all data in the buffer has been flushed, after calling ShutdownLogBuffer().
  129. func (logBuffer *LogBuffer) IsAllFlushed() bool {
  130. return logBuffer.isAllFlushed
  131. }
  132. func (logBuffer *LogBuffer) loopFlush() {
  133. for d := range logBuffer.flushChan {
  134. if d != nil {
  135. // glog.V(4).Infof("%s flush [%v, %v] size %d", m.name, d.startTime, d.stopTime, len(d.data.Bytes()))
  136. logBuffer.flushFn(logBuffer, d.startTime, d.stopTime, d.data.Bytes())
  137. d.releaseMemory()
  138. // local logbuffer is different from aggregate logbuffer here
  139. logBuffer.lastFlushDataTime = d.stopTime
  140. }
  141. }
  142. logBuffer.isAllFlushed = true
  143. }
  144. func (logBuffer *LogBuffer) loopInterval() {
  145. for !logBuffer.IsStopping() {
  146. time.Sleep(logBuffer.flushInterval)
  147. if logBuffer.IsStopping() {
  148. return
  149. }
  150. logBuffer.Lock()
  151. toFlush := logBuffer.copyToFlush()
  152. logBuffer.Unlock()
  153. if toFlush != nil {
  154. glog.V(4).Infof("%s flush [%v, %v] size %d", logBuffer.name, toFlush.startTime, toFlush.stopTime, len(toFlush.data.Bytes()))
  155. logBuffer.flushChan <- toFlush
  156. } else {
  157. // glog.V(0).Infof("%s no flush", m.name)
  158. }
  159. }
  160. }
  161. func (logBuffer *LogBuffer) copyToFlush() *dataToFlush {
  162. if logBuffer.pos > 0 {
  163. // fmt.Printf("flush buffer %d pos %d empty space %d\n", len(m.buf), m.pos, len(m.buf)-m.pos)
  164. var d *dataToFlush
  165. if logBuffer.flushFn != nil {
  166. d = &dataToFlush{
  167. startTime: logBuffer.startTime,
  168. stopTime: logBuffer.stopTime,
  169. data: copiedBytes(logBuffer.buf[:logBuffer.pos]),
  170. }
  171. // glog.V(4).Infof("%s flushing [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
  172. } else {
  173. // glog.V(4).Infof("%s removed from memory [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
  174. logBuffer.lastFlushDataTime = logBuffer.stopTime
  175. }
  176. logBuffer.buf = logBuffer.prevBuffers.SealBuffer(logBuffer.startTime, logBuffer.stopTime, logBuffer.buf, logBuffer.pos, logBuffer.batchIndex)
  177. logBuffer.startTime = time.Unix(0, 0)
  178. logBuffer.stopTime = time.Unix(0, 0)
  179. logBuffer.pos = 0
  180. logBuffer.idx = logBuffer.idx[:0]
  181. logBuffer.batchIndex++
  182. return d
  183. }
  184. return nil
  185. }
  186. func (logBuffer *LogBuffer) GetEarliestTime() time.Time {
  187. return logBuffer.startTime
  188. }
  189. func (logBuffer *LogBuffer) GetEarliestPosition() MessagePosition {
  190. return MessagePosition{
  191. Time: logBuffer.startTime,
  192. BatchIndex: logBuffer.batchIndex,
  193. }
  194. }
  195. func (d *dataToFlush) releaseMemory() {
  196. d.data.Reset()
  197. bufferPool.Put(d.data)
  198. }
  199. func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition) (bufferCopy *bytes.Buffer, batchIndex int64, err error) {
  200. logBuffer.RLock()
  201. defer logBuffer.RUnlock()
  202. // Read from disk and memory
  203. // 1. read from disk, last time is = td
  204. // 2. in memory, the earliest time = tm
  205. // if tm <= td, case 2.1
  206. // read from memory
  207. // if tm is empty, case 2.2
  208. // read from memory
  209. // if td < tm, case 2.3
  210. // read from disk again
  211. var tsMemory time.Time
  212. var tsBatchIndex int64
  213. if !logBuffer.startTime.IsZero() {
  214. tsMemory = logBuffer.startTime
  215. tsBatchIndex = logBuffer.batchIndex
  216. }
  217. for _, prevBuf := range logBuffer.prevBuffers.buffers {
  218. if !prevBuf.startTime.IsZero() && prevBuf.startTime.Before(tsMemory) {
  219. tsMemory = prevBuf.startTime
  220. tsBatchIndex = prevBuf.batchIndex
  221. }
  222. }
  223. if tsMemory.IsZero() { // case 2.2
  224. // println("2.2 no data")
  225. return nil, -2, nil
  226. } else if lastReadPosition.Before(tsMemory) && lastReadPosition.BatchIndex+1 < tsBatchIndex { // case 2.3
  227. if !logBuffer.lastFlushDataTime.IsZero() {
  228. glog.V(0).Infof("resume with last flush time: %v", logBuffer.lastFlushDataTime)
  229. return nil, -2, ResumeFromDiskError
  230. }
  231. }
  232. // the following is case 2.1
  233. if lastReadPosition.Equal(logBuffer.stopTime) {
  234. return nil, logBuffer.batchIndex, nil
  235. }
  236. if lastReadPosition.After(logBuffer.stopTime) {
  237. // glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadPosition, m.stopTime)
  238. return nil, logBuffer.batchIndex, nil
  239. }
  240. if lastReadPosition.Before(logBuffer.startTime) {
  241. // println("checking ", lastReadPosition.UnixNano())
  242. for _, buf := range logBuffer.prevBuffers.buffers {
  243. if buf.startTime.After(lastReadPosition.Time) {
  244. // glog.V(4).Infof("%s return the %d sealed buffer %v", m.name, i, buf.startTime)
  245. // println("return the", i, "th in memory", buf.startTime.UnixNano())
  246. return copiedBytes(buf.buf[:buf.size]), buf.batchIndex, nil
  247. }
  248. if !buf.startTime.After(lastReadPosition.Time) && buf.stopTime.After(lastReadPosition.Time) {
  249. pos := buf.locateByTs(lastReadPosition.Time)
  250. // fmt.Printf("locate buffer[%d] pos %d\n", i, pos)
  251. return copiedBytes(buf.buf[pos:buf.size]), buf.batchIndex, nil
  252. }
  253. }
  254. // glog.V(4).Infof("%s return the current buf %v", m.name, lastReadPosition)
  255. return copiedBytes(logBuffer.buf[:logBuffer.pos]), logBuffer.batchIndex, nil
  256. }
  257. lastTs := lastReadPosition.UnixNano()
  258. l, h := 0, len(logBuffer.idx)-1
  259. /*
  260. for i, pos := range m.idx {
  261. logEntry, ts := readTs(m.buf, pos)
  262. event := &filer_pb.SubscribeMetadataResponse{}
  263. proto.Unmarshal(logEntry.Data, event)
  264. entry := event.EventNotification.OldEntry
  265. if entry == nil {
  266. entry = event.EventNotification.NewEntry
  267. }
  268. fmt.Printf("entry %d ts: %v offset:%d dir:%s name:%s\n", i, time.Unix(0, ts), pos, event.Directory, entry.Name)
  269. }
  270. fmt.Printf("l=%d, h=%d\n", l, h)
  271. */
  272. for l <= h {
  273. mid := (l + h) / 2
  274. pos := logBuffer.idx[mid]
  275. _, t := readTs(logBuffer.buf, pos)
  276. if t <= lastTs {
  277. l = mid + 1
  278. } else if lastTs < t {
  279. var prevT int64
  280. if mid > 0 {
  281. _, prevT = readTs(logBuffer.buf, logBuffer.idx[mid-1])
  282. }
  283. if prevT <= lastTs {
  284. // fmt.Printf("found l=%d, m-1=%d(ts=%d), m=%d(ts=%d), h=%d [%d, %d) \n", l, mid-1, prevT, mid, t, h, pos, m.pos)
  285. return copiedBytes(logBuffer.buf[pos:logBuffer.pos]), logBuffer.batchIndex, nil
  286. }
  287. h = mid
  288. }
  289. // fmt.Printf("l=%d, h=%d\n", l, h)
  290. }
  291. // FIXME: this could be that the buffer has been flushed already
  292. println("Not sure why no data", lastReadPosition.BatchIndex, tsBatchIndex)
  293. return nil, -2, nil
  294. }
  295. func (logBuffer *LogBuffer) ReleaseMemory(b *bytes.Buffer) {
  296. bufferPool.Put(b)
  297. }
  298. var bufferPool = sync.Pool{
  299. New: func() interface{} {
  300. return new(bytes.Buffer)
  301. },
  302. }
  303. func copiedBytes(buf []byte) (copied *bytes.Buffer) {
  304. copied = bufferPool.Get().(*bytes.Buffer)
  305. copied.Reset()
  306. copied.Write(buf)
  307. return
  308. }
  309. func readTs(buf []byte, pos int) (size int, ts int64) {
  310. size = int(util.BytesToUint32(buf[pos : pos+4]))
  311. entryData := buf[pos+4 : pos+4+size]
  312. logEntry := &filer_pb.LogEntry{}
  313. err := proto.Unmarshal(entryData, logEntry)
  314. if err != nil {
  315. glog.Fatalf("unexpected unmarshal filer_pb.LogEntry: %v", err)
  316. }
  317. return size, logEntry.TsNs
  318. }