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.
401 lines
13 KiB
401 lines
13 KiB
package log_buffer
|
|
|
|
import (
|
|
"bytes"
|
|
"fmt"
|
|
"time"
|
|
|
|
"google.golang.org/protobuf/proto"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
|
)
|
|
|
|
var (
|
|
ResumeError = fmt.Errorf("resume")
|
|
ResumeFromDiskError = fmt.Errorf("resumeFromDisk")
|
|
)
|
|
|
|
type MessagePosition struct {
|
|
Time time.Time // timestamp of the message
|
|
Offset int64 // Kafka offset for offset-based positioning, or batch index for timestamp-based
|
|
IsOffsetBased bool // true if this position is offset-based, false if timestamp-based
|
|
}
|
|
|
|
func NewMessagePosition(tsNs int64, offset int64) MessagePosition {
|
|
return MessagePosition{
|
|
Time: time.Unix(0, tsNs).UTC(),
|
|
Offset: offset,
|
|
IsOffsetBased: false, // timestamp-based by default
|
|
}
|
|
}
|
|
|
|
// NewMessagePositionFromOffset creates a MessagePosition that represents a specific offset
|
|
func NewMessagePositionFromOffset(offset int64) MessagePosition {
|
|
return MessagePosition{
|
|
Time: time.Time{}, // Zero time for offset-based positions
|
|
Offset: offset,
|
|
IsOffsetBased: true,
|
|
}
|
|
}
|
|
|
|
// GetOffset extracts the offset from an offset-based MessagePosition
|
|
func (mp MessagePosition) GetOffset() int64 {
|
|
if !mp.IsOffsetBased {
|
|
return -1 // Not an offset-based position
|
|
}
|
|
return mp.Offset // Offset is stored directly
|
|
}
|
|
|
|
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition MessagePosition, stopTsNs int64,
|
|
waitForDataFn func() bool, eachLogDataFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error) {
|
|
|
|
// Register for instant notifications (<1ms latency)
|
|
notifyChan := logBuffer.RegisterSubscriber(readerName)
|
|
defer logBuffer.UnregisterSubscriber(readerName)
|
|
|
|
// loop through all messages
|
|
var bytesBuf *bytes.Buffer
|
|
var batchIndex int64
|
|
lastReadPosition = startPosition
|
|
var entryCounter int64
|
|
defer func() {
|
|
if bytesBuf != nil {
|
|
logBuffer.ReleaseMemory(bytesBuf)
|
|
}
|
|
// println("LoopProcessLogData", readerName, "sent messages total", entryCounter)
|
|
}()
|
|
|
|
for {
|
|
|
|
if bytesBuf != nil {
|
|
logBuffer.ReleaseMemory(bytesBuf)
|
|
}
|
|
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition)
|
|
if err == ResumeFromDiskError {
|
|
time.Sleep(1127 * time.Millisecond)
|
|
return lastReadPosition, isDone, ResumeFromDiskError
|
|
}
|
|
readSize := 0
|
|
if bytesBuf != nil {
|
|
readSize = bytesBuf.Len()
|
|
}
|
|
glog.V(4).Infof("%s ReadFromBuffer at %v offset %d. Read bytes %v batchIndex %d", readerName, lastReadPosition, lastReadPosition.Offset, readSize, batchIndex)
|
|
if bytesBuf == nil {
|
|
if batchIndex >= 0 {
|
|
lastReadPosition = NewMessagePosition(lastReadPosition.Time.UnixNano(), batchIndex)
|
|
}
|
|
if stopTsNs != 0 {
|
|
isDone = true
|
|
return
|
|
}
|
|
lastTsNs := logBuffer.LastTsNs.Load()
|
|
|
|
for lastTsNs == logBuffer.LastTsNs.Load() {
|
|
if !waitForDataFn() {
|
|
isDone = true
|
|
return
|
|
}
|
|
// Wait for notification or timeout (instant wake-up when data arrives)
|
|
select {
|
|
case <-notifyChan:
|
|
// New data available, break and retry read
|
|
glog.V(3).Infof("%s: Woke up from notification (LoopProcessLogData)", readerName)
|
|
break
|
|
case <-time.After(10 * time.Millisecond):
|
|
// Timeout, check if timestamp changed
|
|
if lastTsNs != logBuffer.LastTsNs.Load() {
|
|
break
|
|
}
|
|
glog.V(4).Infof("%s: Notification timeout (LoopProcessLogData), polling", readerName)
|
|
}
|
|
}
|
|
if logBuffer.IsStopping() {
|
|
isDone = true
|
|
return
|
|
}
|
|
continue
|
|
}
|
|
|
|
buf := bytesBuf.Bytes()
|
|
// fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadPosition, len(buf))
|
|
|
|
batchSize := 0
|
|
|
|
for pos := 0; pos+4 < len(buf); {
|
|
|
|
size := util.BytesToUint32(buf[pos : pos+4])
|
|
if pos+4+int(size) > len(buf) {
|
|
err = ResumeError
|
|
glog.Errorf("LoopProcessLogData: %s read buffer %v read %d entries [%d,%d) from [0,%d)", readerName, lastReadPosition, batchSize, pos, pos+int(size)+4, len(buf))
|
|
return
|
|
}
|
|
entryData := buf[pos+4 : pos+4+int(size)]
|
|
|
|
logEntry := &filer_pb.LogEntry{}
|
|
if err = proto.Unmarshal(entryData, logEntry); err != nil {
|
|
glog.Errorf("unexpected unmarshal mq_pb.Message: %v", err)
|
|
pos += 4 + int(size)
|
|
continue
|
|
}
|
|
|
|
// Handle offset-based filtering for offset-based start positions
|
|
if startPosition.IsOffsetBased {
|
|
startOffset := startPosition.GetOffset()
|
|
if logEntry.Offset < startOffset {
|
|
// Skip entries before the starting offset
|
|
pos += 4 + int(size)
|
|
batchSize++
|
|
continue
|
|
}
|
|
}
|
|
|
|
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
|
|
isDone = true
|
|
// println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
|
|
return
|
|
}
|
|
lastReadPosition = NewMessagePosition(logEntry.TsNs, batchIndex)
|
|
|
|
if isDone, err = eachLogDataFn(logEntry); err != nil {
|
|
glog.Errorf("LoopProcessLogData: %s process log entry %d %v: %v", readerName, batchSize+1, logEntry, err)
|
|
return
|
|
}
|
|
if isDone {
|
|
glog.V(0).Infof("LoopProcessLogData2: %s process log entry %d", readerName, batchSize+1)
|
|
return
|
|
}
|
|
|
|
pos += 4 + int(size)
|
|
batchSize++
|
|
entryCounter++
|
|
|
|
}
|
|
|
|
glog.V(4).Infof("%s sent messages ts[%+v,%+v] size %d\n", readerName, startPosition, lastReadPosition, batchSize)
|
|
}
|
|
|
|
}
|
|
|
|
// LoopProcessLogDataWithOffset is similar to LoopProcessLogData but provides offset to the callback
|
|
func (logBuffer *LogBuffer) LoopProcessLogDataWithOffset(readerName string, startPosition MessagePosition, stopTsNs int64,
|
|
waitForDataFn func() bool, eachLogDataFn EachLogEntryWithOffsetFuncType) (lastReadPosition MessagePosition, isDone bool, err error) {
|
|
glog.V(4).Infof("LoopProcessLogDataWithOffset started for %s, startPosition=%v", readerName, startPosition)
|
|
|
|
// Register for instant notifications (<1ms latency)
|
|
notifyChan := logBuffer.RegisterSubscriber(readerName)
|
|
defer logBuffer.UnregisterSubscriber(readerName)
|
|
|
|
// loop through all messages
|
|
var bytesBuf *bytes.Buffer
|
|
var offset int64
|
|
lastReadPosition = startPosition
|
|
var entryCounter int64
|
|
defer func() {
|
|
if bytesBuf != nil {
|
|
logBuffer.ReleaseMemory(bytesBuf)
|
|
}
|
|
// println("LoopProcessLogDataWithOffset", readerName, "sent messages total", entryCounter)
|
|
}()
|
|
|
|
for {
|
|
// Check stopTsNs at the beginning of each iteration
|
|
// This ensures we exit immediately if the stop time is in the past
|
|
if stopTsNs != 0 && time.Now().UnixNano() > stopTsNs {
|
|
isDone = true
|
|
return
|
|
}
|
|
|
|
if bytesBuf != nil {
|
|
logBuffer.ReleaseMemory(bytesBuf)
|
|
}
|
|
bytesBuf, offset, err = logBuffer.ReadFromBuffer(lastReadPosition)
|
|
glog.V(4).Infof("ReadFromBuffer for %s returned bytesBuf=%v, offset=%d, err=%v", readerName, bytesBuf != nil, offset, err)
|
|
if err == ResumeFromDiskError {
|
|
// Try to read from disk if readFromDiskFn is available
|
|
if logBuffer.ReadFromDiskFn != nil {
|
|
// Wrap eachLogDataFn to match the expected signature
|
|
diskReadFn := func(logEntry *filer_pb.LogEntry) (bool, error) {
|
|
return eachLogDataFn(logEntry, logEntry.Offset)
|
|
}
|
|
lastReadPosition, isDone, err = logBuffer.ReadFromDiskFn(lastReadPosition, stopTsNs, diskReadFn)
|
|
if err != nil {
|
|
return lastReadPosition, isDone, err
|
|
}
|
|
if isDone {
|
|
return lastReadPosition, isDone, nil
|
|
}
|
|
// Continue to next iteration after disk read
|
|
}
|
|
|
|
// CRITICAL: Check if client is still connected after disk read
|
|
if !waitForDataFn() {
|
|
// Client disconnected - exit cleanly
|
|
glog.V(4).Infof("%s: Client disconnected after disk read", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
|
|
// Wait for notification or timeout (instant wake-up when data arrives)
|
|
select {
|
|
case <-notifyChan:
|
|
// New data available, retry immediately
|
|
glog.V(3).Infof("%s: Woke up from notification after disk read", readerName)
|
|
case <-time.After(10 * time.Millisecond):
|
|
// Timeout, retry anyway (fallback for edge cases)
|
|
glog.V(4).Infof("%s: Notification timeout, polling", readerName)
|
|
}
|
|
|
|
// Continue to next iteration (don't return ResumeFromDiskError)
|
|
continue
|
|
}
|
|
readSize := 0
|
|
if bytesBuf != nil {
|
|
readSize = bytesBuf.Len()
|
|
}
|
|
glog.V(4).Infof("%s ReadFromBuffer at %v posOffset %d. Read bytes %v bufferOffset %d", readerName, lastReadPosition, lastReadPosition.Offset, readSize, offset)
|
|
if bytesBuf == nil {
|
|
// CRITICAL: Check if subscription is still active BEFORE waiting
|
|
// This prevents infinite loops when client has disconnected
|
|
if !waitForDataFn() {
|
|
glog.V(4).Infof("%s: waitForDataFn returned false, subscription ending", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
|
|
if offset >= 0 {
|
|
lastReadPosition = NewMessagePosition(lastReadPosition.Time.UnixNano(), offset)
|
|
}
|
|
if stopTsNs != 0 {
|
|
isDone = true
|
|
return
|
|
}
|
|
|
|
// CRITICAL FIX: If we're reading offset-based and there's no data in LogBuffer,
|
|
// return ResumeFromDiskError to let Subscribe try reading from disk again.
|
|
// This prevents infinite blocking when all data is on disk (e.g., after restart).
|
|
if startPosition.IsOffsetBased {
|
|
glog.V(4).Infof("%s: No data in LogBuffer for offset-based read at %v, checking if client still connected", readerName, lastReadPosition)
|
|
// Check if client is still connected before busy-looping
|
|
if !waitForDataFn() {
|
|
glog.V(4).Infof("%s: Client disconnected, stopping offset-based read", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
// Wait for notification or timeout (instant wake-up when data arrives)
|
|
select {
|
|
case <-notifyChan:
|
|
// New data available, retry immediately
|
|
glog.V(3).Infof("%s: Woke up from notification for offset-based read", readerName)
|
|
case <-time.After(10 * time.Millisecond):
|
|
// Timeout, retry anyway (fallback for edge cases)
|
|
glog.V(4).Infof("%s: Notification timeout for offset-based, polling", readerName)
|
|
}
|
|
return lastReadPosition, isDone, ResumeFromDiskError
|
|
}
|
|
|
|
lastTsNs := logBuffer.LastTsNs.Load()
|
|
|
|
for lastTsNs == logBuffer.LastTsNs.Load() {
|
|
if !waitForDataFn() {
|
|
glog.V(4).Infof("%s: Client disconnected during timestamp wait", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
// Wait for notification or timeout (instant wake-up when data arrives)
|
|
select {
|
|
case <-notifyChan:
|
|
// New data available, break and retry read
|
|
glog.V(3).Infof("%s: Woke up from notification (main loop)", readerName)
|
|
break
|
|
case <-time.After(10 * time.Millisecond):
|
|
// Timeout, check if timestamp changed
|
|
if lastTsNs != logBuffer.LastTsNs.Load() {
|
|
break
|
|
}
|
|
glog.V(4).Infof("%s: Notification timeout (main loop), polling", readerName)
|
|
}
|
|
}
|
|
if logBuffer.IsStopping() {
|
|
glog.V(4).Infof("%s: LogBuffer is stopping", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
continue
|
|
}
|
|
|
|
buf := bytesBuf.Bytes()
|
|
// fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadPosition, len(buf))
|
|
glog.V(4).Infof("Processing buffer with %d bytes for %s", len(buf), readerName)
|
|
|
|
// If buffer is empty, check if client is still connected before looping
|
|
if len(buf) == 0 {
|
|
glog.V(4).Infof("Empty buffer for %s, checking if client still connected", readerName)
|
|
if !waitForDataFn() {
|
|
glog.V(4).Infof("%s: Client disconnected on empty buffer", readerName)
|
|
return lastReadPosition, true, nil
|
|
}
|
|
// Sleep to avoid busy-wait on empty buffer
|
|
time.Sleep(10 * time.Millisecond)
|
|
continue
|
|
}
|
|
|
|
batchSize := 0
|
|
|
|
for pos := 0; pos+4 < len(buf); {
|
|
|
|
size := util.BytesToUint32(buf[pos : pos+4])
|
|
if pos+4+int(size) > len(buf) {
|
|
err = ResumeError
|
|
glog.Errorf("LoopProcessLogDataWithOffset: %s read buffer %v read %d entries [%d,%d) from [0,%d)", readerName, lastReadPosition, batchSize, pos, pos+int(size)+4, len(buf))
|
|
return
|
|
}
|
|
entryData := buf[pos+4 : pos+4+int(size)]
|
|
|
|
logEntry := &filer_pb.LogEntry{}
|
|
if err = proto.Unmarshal(entryData, logEntry); err != nil {
|
|
glog.Errorf("unexpected unmarshal mq_pb.Message: %v", err)
|
|
pos += 4 + int(size)
|
|
continue
|
|
}
|
|
|
|
glog.V(4).Infof("Unmarshaled log entry %d: TsNs=%d, Offset=%d, Key=%s", batchSize+1, logEntry.TsNs, logEntry.Offset, string(logEntry.Key))
|
|
|
|
// Handle offset-based filtering for offset-based start positions
|
|
if startPosition.IsOffsetBased {
|
|
startOffset := startPosition.GetOffset()
|
|
glog.V(4).Infof("Offset-based filtering: logEntry.Offset=%d, startOffset=%d", logEntry.Offset, startOffset)
|
|
if logEntry.Offset < startOffset {
|
|
// Skip entries before the starting offset
|
|
glog.V(4).Infof("Skipping entry due to offset filter")
|
|
pos += 4 + int(size)
|
|
batchSize++
|
|
continue
|
|
}
|
|
}
|
|
|
|
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
|
|
glog.V(4).Infof("Stopping due to stopTsNs")
|
|
isDone = true
|
|
// println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
|
|
return
|
|
}
|
|
// CRITICAL FIX: Use logEntry.Offset + 1 to move PAST the current entry
|
|
// This prevents infinite loops where we keep requesting the same offset
|
|
lastReadPosition = NewMessagePosition(logEntry.TsNs, logEntry.Offset+1)
|
|
|
|
glog.V(4).Infof("Calling eachLogDataFn for entry at offset %d, next position will be %d", logEntry.Offset, logEntry.Offset+1)
|
|
if isDone, err = eachLogDataFn(logEntry, logEntry.Offset); err != nil {
|
|
glog.Errorf("LoopProcessLogDataWithOffset: %s process log entry %d %v: %v", readerName, batchSize+1, logEntry, err)
|
|
return
|
|
}
|
|
if isDone {
|
|
glog.V(0).Infof("LoopProcessLogDataWithOffset: %s process log entry %d", readerName, batchSize+1)
|
|
return
|
|
}
|
|
|
|
pos += 4 + int(size)
|
|
batchSize++
|
|
entryCounter++
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|