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.
270 lines
9.8 KiB
270 lines
9.8 KiB
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
|
|
correlationID int32 // Added for correlation tracking
|
|
}
|
|
|
|
// 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)
|
|
|
|
// Send result back to client
|
|
select {
|
|
case req.resultChan <- result:
|
|
// Successfully sent
|
|
case <-ctx.Done():
|
|
glog.Warningf("[%s] Context cancelled while sending result for %s[%d]",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID)
|
|
case <-time.After(50 * time.Millisecond):
|
|
glog.Warningf("[%s] Timeout sending result for %s[%d] - CLIENT MAY HAVE DISCONNECTED",
|
|
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.Errorf("[%s] CRITICAL: Failed to get HWM for %s[%d]: %v",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, hwmErr)
|
|
result.recordBatch = []byte{}
|
|
result.highWaterMark = 0
|
|
return
|
|
}
|
|
result.highWaterMark = hwm
|
|
|
|
glog.V(2).Infof("[%s] HWM for %s[%d]: %d (requested: %d)",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, hwm, req.requestedOffset)
|
|
|
|
// 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] Requested offset %d >= HWM %d, returning empty",
|
|
pr.connCtx.ConnectionID, req.requestedOffset, hwm)
|
|
return
|
|
}
|
|
|
|
// Update tracking offset to match requested offset
|
|
pr.bufferMu.Lock()
|
|
if req.requestedOffset != pr.currentOffset {
|
|
glog.V(3).Infof("[%s] Updating currentOffset for %s[%d]: %d -> %d",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, pr.currentOffset, req.requestedOffset)
|
|
pr.currentOffset = req.requestedOffset
|
|
}
|
|
pr.bufferMu.Unlock()
|
|
|
|
// Fetch on-demand - no pre-fetching to avoid overwhelming the broker
|
|
recordBatch, newOffset := pr.readRecords(ctx, req.requestedOffset, req.maxBytes, req.maxWaitMs, hwm)
|
|
|
|
// Log what we got back - DETAILED for diagnostics
|
|
if len(recordBatch) == 0 {
|
|
glog.V(2).Infof("[%s] FETCH %s[%d]: readRecords returned EMPTY (offset=%d, hwm=%d)",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, req.requestedOffset, hwm)
|
|
result.recordBatch = []byte{}
|
|
} else {
|
|
result.recordBatch = recordBatch
|
|
pr.bufferMu.Lock()
|
|
pr.currentOffset = newOffset
|
|
pr.bufferMu.Unlock()
|
|
}
|
|
}
|
|
|
|
// 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) {
|
|
fetchStartTime := time.Now()
|
|
|
|
// 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)
|
|
|
|
// Log slow fetches (potential hangs)
|
|
if fetchDuration > 2*time.Second {
|
|
glog.Warningf("[%s] SLOW FETCH for %s[%d]: offset=%d took %.2fs (maxWait=%dms, HWM=%d)",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, fetchDuration.Seconds(), maxWaitMs, highWaterMark)
|
|
}
|
|
|
|
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
|
|
fallbackStartTime := time.Now()
|
|
smqRecords, err := pr.handler.seaweedMQHandler.GetStoredRecords(ctx, pr.topicName, pr.partitionID, fromOffset, 10)
|
|
fallbackDuration := time.Since(fallbackStartTime)
|
|
|
|
if fallbackDuration > 2*time.Second {
|
|
glog.Warningf("[%s] SLOW FALLBACK for %s[%d]: offset=%d took %.2fs",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, fallbackDuration.Seconds())
|
|
}
|
|
|
|
if err != nil {
|
|
glog.Errorf("[%s] CRITICAL: Both multi-batch AND fallback failed for %s[%d] offset=%d: %v",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, err)
|
|
return []byte{}, fromOffset
|
|
}
|
|
|
|
if len(smqRecords) > 0 {
|
|
recordBatch := pr.handler.constructRecordBatchFromSMQ(pr.topicName, fromOffset, smqRecords)
|
|
nextOffset := fromOffset + int64(len(smqRecords))
|
|
glog.V(3).Infof("[%s] Fallback succeeded: got %d records for %s[%d] offset %d -> %d (total: %v)",
|
|
pr.connCtx.ConnectionID, len(smqRecords), pr.topicName, pr.partitionID, fromOffset, nextOffset, time.Since(fetchStartTime))
|
|
return recordBatch, nextOffset
|
|
}
|
|
|
|
// No records available
|
|
glog.V(3).Infof("[%s] No records available for %s[%d] offset=%d after multi-batch and fallback (total: %v)",
|
|
pr.connCtx.ConnectionID, pr.topicName, pr.partitionID, fromOffset, time.Since(fetchStartTime))
|
|
return []byte{}, fromOffset
|
|
}
|
|
|
|
// close signals the reader to shut down
|
|
func (pr *partitionReader) close() {
|
|
close(pr.closeChan)
|
|
}
|