refactoring
This commit is contained in:
@@ -55,35 +55,36 @@ type CachedDiskChunk struct {
|
||||
}
|
||||
|
||||
type LogBuffer struct {
|
||||
LastFlushTsNs int64
|
||||
name string
|
||||
prevBuffers *SealedBuffers
|
||||
buf []byte
|
||||
offset int64 // Last offset in current buffer (endOffset)
|
||||
bufferStartOffset int64 // First offset in current buffer
|
||||
idx []int
|
||||
pos int
|
||||
// 8-byte aligned fields
|
||||
LastTsNs atomic.Int64
|
||||
lastFlushTsNs atomic.Int64
|
||||
lastFlushedOffset atomic.Int64 // Highest offset that has been flushed to disk (-1 = nothing flushed yet)
|
||||
offset int64
|
||||
bufferStartOffset int64
|
||||
minOffset int64
|
||||
maxOffset int64
|
||||
flushInterval time.Duration
|
||||
startTime time.Time
|
||||
stopTime time.Time
|
||||
lastFlushDataTime time.Time
|
||||
sizeBuf []byte
|
||||
flushInterval time.Duration
|
||||
flushFn LogFlushFuncType
|
||||
ReadFromDiskFn LogReadFromDiskFuncType
|
||||
notifyFn func()
|
||||
|
||||
// Other fields
|
||||
name string
|
||||
prevBuffers *SealedBuffers
|
||||
buf []byte
|
||||
idx []int
|
||||
pos int
|
||||
sizeBuf []byte
|
||||
flushFn LogFlushFuncType
|
||||
ReadFromDiskFn LogReadFromDiskFuncType
|
||||
notifyFn func()
|
||||
// Per-subscriber notification channels for instant wake-up
|
||||
subscribersMu sync.RWMutex
|
||||
subscribers map[string]chan struct{} // subscriberID -> notification channel
|
||||
isStopping *atomic.Bool
|
||||
isAllFlushed bool
|
||||
flushChan chan *dataToFlush
|
||||
LastTsNs atomic.Int64
|
||||
// Offset range tracking for Kafka integration
|
||||
minOffset int64
|
||||
maxOffset int64
|
||||
hasOffsets bool
|
||||
lastFlushedOffset atomic.Int64 // Highest offset that has been flushed to disk (-1 = nothing flushed yet)
|
||||
lastFlushTsNs atomic.Int64 // Latest timestamp that has been flushed to disk (0 = nothing flushed yet)
|
||||
hasOffsets bool
|
||||
// Disk chunk cache for historical data reads
|
||||
diskChunkCache *DiskChunkCache
|
||||
sync.RWMutex
|
||||
@@ -235,7 +236,7 @@ func (logBuffer *LogBuffer) InitializeOffsetFromExistingData(getHighestOffsetFn
|
||||
logBuffer.bufferStartOffset = nextOffset
|
||||
// CRITICAL: Track that data [0...highestOffset] is on disk
|
||||
logBuffer.lastFlushedOffset.Store(highestOffset)
|
||||
// Set lastFlushedTime to current time (we know data up to highestOffset is on disk)
|
||||
// Set lastFlushTsNs to current time (we know data up to highestOffset is on disk)
|
||||
logBuffer.lastFlushTsNs.Store(time.Now().UnixNano())
|
||||
} else {
|
||||
logBuffer.bufferStartOffset = 0 // Start from offset 0
|
||||
@@ -507,10 +508,6 @@ func (logBuffer *LogBuffer) loopFlush() {
|
||||
logBuffer.flushFn(logBuffer, d.startTime, d.stopTime, d.data.Bytes(), d.minOffset, d.maxOffset)
|
||||
d.releaseMemory()
|
||||
// local logbuffer is different from aggregate logbuffer here
|
||||
logBuffer.lastFlushDataTime = d.stopTime
|
||||
|
||||
// CRITICAL: Track what's been flushed to disk for both offset-based and time-based reads
|
||||
// Use >= 0 to include offset 0 (first message in a topic)
|
||||
if d.maxOffset >= 0 {
|
||||
logBuffer.lastFlushedOffset.Store(d.maxOffset)
|
||||
}
|
||||
@@ -567,8 +564,6 @@ func (logBuffer *LogBuffer) copyToFlushInternal(withCallback bool) *dataToFlush
|
||||
if withCallback {
|
||||
d.done = make(chan struct{})
|
||||
}
|
||||
} else {
|
||||
logBuffer.lastFlushDataTime = logBuffer.stopTime
|
||||
}
|
||||
// CRITICAL: logBuffer.offset is the "next offset to assign", so last offset in buffer is offset-1
|
||||
lastOffsetInBuffer := logBuffer.offset - 1
|
||||
@@ -624,6 +619,10 @@ func (logBuffer *LogBuffer) GetLastFlushTsNs() int64 {
|
||||
return logBuffer.lastFlushTsNs.Load()
|
||||
}
|
||||
|
||||
func (logBuffer *LogBuffer) SetLastFlushTsNs(ts int64) {
|
||||
logBuffer.lastFlushTsNs.Store(ts)
|
||||
}
|
||||
|
||||
func (d *dataToFlush) releaseMemory() {
|
||||
d.data.Reset()
|
||||
bufferPool.Put(d.data)
|
||||
|
||||
Reference in New Issue
Block a user