Files
seaweedfs/weed/mq/kafka/protocol/fetch_partition_reader.go
chrislu c6807de7cc fix: Load persisted offsets into memory cache immediately on fetch
This fixes the root cause of message loss: offset resets to auto.offset.reset.

ROOT CAUSE:
When OffsetFetch is called during rebalancing:
1. Offset not found in memory → returns -1
2. Consumer gets -1 → triggers auto.offset.reset=earliest
3. Consumer restarts from offset 0
4. Previously consumed messages 39-786 are never fetched again

ANALYSIS:
Test shows missing messages are contiguous ranges:
- loadtest-topic-2[0]: Missing offsets 39-786 (748 messages)
- loadtest-topic-0[1]: Missing 675 messages from offset ~117
- Pattern: Initial messages 0-38 consumed, then restart, then 39+ never fetched

FIX:
When OffsetFetch finds offset in SMQ storage:
1. Return the offset to client
2. IMMEDIATELY cache in in-memory map via h.commitOffset()
3. Next fetch will find it in memory (no reset)
4. Consumer continues from correct offset

This prevents the offset reset loop that causes the 21% message loss.

Revert "fix: Load persisted offsets into memory cache immediately on fetch"

This reverts commit d9809eabb9206759b9eb4ffb8bf98b4c5c2f4c64.

fix: Increase fetch timeout and add logging for timeout failures

ROOT CAUSE:
Consumer fetches messages 0-30 successfully, then ALL subsequent fetches
fail silently. Partition reader stops responding after ~3-4 batches.

ANALYSIS:
The fetch request timeout is set to client's MaxWaitTime (100ms-500ms).
When GetStoredRecords takes longer than this (disk I/O, broker latency),
context times out. The multi-batch fetcher returns error/empty, fallback
single-batch also times out, and function returns empty bytes silently.

Consumer never retries - it just gets empty response and gives up.

Result: Messages from offset 31+ are never fetched (3,956 missing = 32%).

FIX:
1. Increase internal timeout to 1.5x client timeout (min 5 seconds)
   This allows batch fetchers to complete even if slightly delayed

2. Add comprehensive logging at WARNING level for timeout failures
   So we can diagnose these issues in the field

3. Better error messages with duration info
   Helps distinguish between timeout vs no-data situations

This ensures the fetch path doesn't silently fail just because a batch
took slightly longer than expected to fetch from disk.

fix: Use fresh context for fallback fetch to avoid cascading timeouts

PROBLEM IDENTIFIED:
After previous fix, missing messages reduced 32%→16% BUT duplicates
increased 18.5%→56.6%. Root cause: When multi-batch fetch times out,
the fallback single-batch ALSO uses the expired context.

Result:
1. Multi-batch fetch times out (context expired)
2. Fallback single-batch uses SAME expired context → also times out
3. Both return empty bytes
4. Consumer gets empty response, offset resets to memory cache
5. Consumer re-fetches from earlier offset
6. DUPLICATES result from re-fetching old messages

FIX:
Use ORIGINAL context for fallback fetch, not the timed-out fetchCtx.
This gives the fallback a fresh chance to fetch data even if multi-batch
timed out.

IMPROVEMENTS:
1. Fallback now uses fresh context (not expired from multi-batch)
2. Add WARNING logs for ALL multi-batch failures (not just errors)
3. Distinguish between 'failed' (timed out) and 'no data available'
4. Log total duration for diagnostics

Expected Result:
- Duplicates should decrease significantly (56.6% → 5-10%)
- Missing messages should stay low (~16%) or improve further
- Warnings in logs will show which fetches are timing out

fmt
2025-10-16 22:17:47 -07:00

245 lines
8.9 KiB
Go

package protocol
import (
"context"
"sync"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
)
// partitionReader maintains a persistent connection to a single topic-partition
// and streams records forward, eliminating repeated offset lookups
// Pre-fetches and buffers records for instant serving
type partitionReader struct {
topicName string
partitionID int32
currentOffset int64
fetchChan chan *partitionFetchRequest
closeChan chan struct{}
// Pre-fetch buffer support
recordBuffer chan *bufferedRecords // Buffered pre-fetched records
bufferMu sync.Mutex // Protects offset access
handler *Handler
connCtx *ConnectionContext
}
// bufferedRecords represents a batch of pre-fetched records
type bufferedRecords struct {
recordBatch []byte
startOffset int64
endOffset int64
highWaterMark int64
}
// partitionFetchRequest represents a request to fetch data from this partition
type partitionFetchRequest struct {
requestedOffset int64
maxBytes int32
maxWaitMs int32 // MaxWaitTime from Kafka fetch request
resultChan chan *partitionFetchResult
isSchematized bool
apiVersion uint16
}
// newPartitionReader creates and starts a new partition reader with pre-fetch buffering
func newPartitionReader(ctx context.Context, handler *Handler, connCtx *ConnectionContext, topicName string, partitionID int32, startOffset int64) *partitionReader {
pr := &partitionReader{
topicName: topicName,
partitionID: partitionID,
currentOffset: startOffset,
fetchChan: make(chan *partitionFetchRequest, 200), // Buffer 200 requests to handle Schema Registry's rapid polling in slow CI environments
closeChan: make(chan struct{}),
recordBuffer: make(chan *bufferedRecords, 5), // Buffer 5 batches of records
handler: handler,
connCtx: connCtx,
}
// Start the pre-fetch goroutine that continuously fetches ahead
go pr.preFetchLoop(ctx)
// Start the request handler goroutine
go pr.handleRequests(ctx)
glog.V(4).Infof("[%s] Created partition reader for %s[%d] starting at offset %d (sequential with ch=200)",
connCtx.ConnectionID, topicName, partitionID, startOffset)
return pr
}
// preFetchLoop is disabled for SMQ backend to prevent subscriber storms
// SMQ reads from disk and creating multiple concurrent subscribers causes
// broker overload and partition shutdowns. Fetch requests are handled
// on-demand in serveFetchRequest instead.
func (pr *partitionReader) preFetchLoop(ctx context.Context) {
defer func() {
glog.V(4).Infof("[%s] Pre-fetch loop exiting for %s[%d]",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID)
close(pr.recordBuffer)
}()
// Wait for shutdown - no continuous pre-fetching to avoid overwhelming the broker
select {
case <-ctx.Done():
return
case <-pr.closeChan:
return
}
}
// handleRequests serves fetch requests SEQUENTIALLY to prevent subscriber storm
// Sequential processing is essential for SMQ backend because:
// 1. GetStoredRecords may create a new subscriber on each call
// 2. Concurrent calls create multiple subscribers for the same partition
// 3. This overwhelms the broker and causes partition shutdowns
func (pr *partitionReader) handleRequests(ctx context.Context) {
defer func() {
glog.V(4).Infof("[%s] Request handler exiting for %s[%d]",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID)
}()
for {
select {
case <-ctx.Done():
return
case <-pr.closeChan:
return
case req := <-pr.fetchChan:
// Process sequentially to prevent subscriber storm
pr.serveFetchRequest(ctx, req)
}
}
}
// serveFetchRequest fetches data on-demand (no pre-fetching)
func (pr *partitionReader) serveFetchRequest(ctx context.Context, req *partitionFetchRequest) {
startTime := time.Now()
result := &partitionFetchResult{}
defer func() {
result.fetchDuration = time.Since(startTime)
select {
case req.resultChan <- result:
case <-ctx.Done():
case <-time.After(50 * time.Millisecond):
glog.Warningf("[%s] Timeout sending result for %s[%d]",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID)
}
}()
// Get high water mark
hwm, hwmErr := pr.handler.seaweedMQHandler.GetLatestOffset(pr.topicName, pr.partitionID)
if hwmErr != nil {
glog.Warningf("[%s] Failed to get high water mark for %s[%d]: %v",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, hwmErr)
result.recordBatch = []byte{}
return
}
result.highWaterMark = hwm
// If requested offset >= HWM, return immediately with empty result
// This prevents overwhelming the broker with futile read attempts when no data is available
if req.requestedOffset >= hwm {
result.recordBatch = []byte{}
glog.V(3).Infof("[%s] No data available for %s[%d]: offset=%d >= hwm=%d",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, req.requestedOffset, hwm)
return
}
// Update tracking offset to match requested offset
pr.bufferMu.Lock()
if req.requestedOffset != pr.currentOffset {
glog.V(4).Infof("[%s] Offset seek for %s[%d]: requested=%d current=%d",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, req.requestedOffset, pr.currentOffset)
pr.currentOffset = req.requestedOffset
}
pr.bufferMu.Unlock()
// Fetch on-demand - no pre-fetching to avoid overwhelming the broker
// Pass the requested offset and maxWaitMs directly to avoid race conditions
recordBatch, newOffset := pr.readRecords(ctx, req.requestedOffset, req.maxBytes, req.maxWaitMs, hwm)
if len(recordBatch) > 0 && newOffset > pr.currentOffset {
result.recordBatch = recordBatch
pr.bufferMu.Lock()
pr.currentOffset = newOffset
pr.bufferMu.Unlock()
glog.V(4).Infof("[%s] On-demand fetch for %s[%d]: offset %d->%d, %d bytes",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID,
req.requestedOffset, newOffset, len(recordBatch))
} else {
result.recordBatch = []byte{}
}
}
// readRecords reads records forward using the multi-batch fetcher
func (pr *partitionReader) readRecords(ctx context.Context, fromOffset int64, maxBytes int32, maxWaitMs int32, highWaterMark int64) ([]byte, int64) {
// Create context with timeout based on Kafka fetch request's MaxWaitTime
// This ensures we wait exactly as long as the client requested
fetchCtx := ctx
if maxWaitMs > 0 {
var cancel context.CancelFunc
// Use 1.5x the client timeout to account for internal processing overhead
// This prevents legitimate slow reads from being killed by client timeout
internalTimeoutMs := int32(float64(maxWaitMs) * 1.5)
if internalTimeoutMs > 5000 {
internalTimeoutMs = 5000 // Cap at 5 seconds
}
fetchCtx, cancel = context.WithTimeout(ctx, time.Duration(internalTimeoutMs)*time.Millisecond)
defer cancel()
}
// Use multi-batch fetcher for better MaxBytes compliance
multiFetcher := NewMultiBatchFetcher(pr.handler)
startTime := time.Now()
fetchResult, err := multiFetcher.FetchMultipleBatches(
fetchCtx,
pr.topicName,
pr.partitionID,
fromOffset,
highWaterMark,
maxBytes,
)
fetchDuration := time.Since(startTime)
if err == nil && fetchResult.TotalSize > 0 {
glog.V(4).Infof("[%s] Multi-batch fetch for %s[%d]: %d batches, %d bytes, offset %d -> %d (duration: %v)",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID,
fetchResult.BatchCount, fetchResult.TotalSize, fromOffset, fetchResult.NextOffset, fetchDuration)
return fetchResult.RecordBatches, fetchResult.NextOffset
}
// Multi-batch failed - try single batch WITHOUT the timeout constraint
// to ensure we get at least some data even if multi-batch timed out
glog.Warningf("[%s] Multi-batch fetch failed for %s[%d] offset=%d after %v, falling back to single-batch (err: %v)",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, fetchDuration, err)
// Use original context for fallback, NOT the timed-out fetchCtx
// This ensures the fallback has a fresh chance to fetch data
smqRecords, err := pr.handler.seaweedMQHandler.GetStoredRecords(ctx, pr.topicName, pr.partitionID, fromOffset, 10)
if err != nil {
glog.Warningf("[%s] Single-batch fetch also failed for %s[%d] offset=%d: %v (total duration: %v)",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, err, time.Since(startTime))
return []byte{}, fromOffset
}
if len(smqRecords) > 0 {
recordBatch := pr.handler.constructRecordBatchFromSMQ(pr.topicName, fromOffset, smqRecords)
nextOffset := fromOffset + int64(len(smqRecords))
glog.V(4).Infof("[%s] Single-batch fallback for %s[%d]: %d records, %d bytes, offset %d -> %d (total duration: %v)",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID,
len(smqRecords), len(recordBatch), fromOffset, nextOffset, time.Since(startTime))
return recordBatch, nextOffset
}
// No records available
glog.V(3).Infof("[%s] No records available for %s[%d] offset=%d (total duration: %v)",
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, time.Since(startTime))
return []byte{}, fromOffset
}
// close signals the reader to shut down
func (pr *partitionReader) close() {
close(pr.closeChan)
}