@ -59,7 +59,7 @@ func (bc *BrokerClient) CreateFreshSubscriber(topic string, partition int32, sta
// Send init message to start subscription with Kafka client's consumer group and ID
initReq := createSubscribeInitMessage ( topic , actualPartition , startOffset , offsetType , consumerGroup , consumerID )
glog . V ( 2 ) . Infof ( "[SUBSCRIBE-INIT] CreateFreshSubscriber sending init: topic=%s partition=%d startOffset=%d offsetType=%v consumerGroup=%s consumerID=%s" ,
glog . V ( 4 ) . Infof ( "[SUBSCRIBE-INIT] CreateFreshSubscriber sending init: topic=%s partition=%d startOffset=%d offsetType=%v consumerGroup=%s consumerID=%s" ,
topic , partition , startOffset , offsetType , consumerGroup , consumerID )
if err := stream . Send ( initReq ) ; err != nil {
@ -125,13 +125,13 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
bc . subscribersLock . RUnlock ( )
if canUseCache {
glog . V ( 2 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (cached)" ,
glog . V ( 4 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (cached)" ,
key , currentOffset , startOffset )
} else if startOffset >= currentOffset {
glog . V ( 2 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (forward read)" ,
glog . V ( 4 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (forward read)" ,
key , currentOffset , startOffset )
} else {
glog . V ( 2 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (will seek backward)" ,
glog . V ( 4 ) . Infof ( "[FETCH] Reusing session for %s: session at %d, requested %d (will seek backward)" ,
key , currentOffset , startOffset )
}
return session , nil
@ -153,7 +153,7 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
existingOffset := session . StartOffset
session . mu . Unlock ( )
glog . V ( 1 ) . Infof ( "[FETCH] Session created concurrently at offset %d (requested %d), reusing" , existingOffset , startOffset )
glog . V ( 3 ) . Infof ( "[FETCH] Session created concurrently at offset %d (requested %d), reusing" , existingOffset , startOffset )
return session , nil
}
@ -192,7 +192,7 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
glog . V ( 2 ) . Infof ( "Creating subscriber for topic=%s partition=%d: Kafka offset %d -> SeaweedMQ %s" ,
topic , partition , startOffset , offsetType )
glog . V ( 2 ) . Infof ( "[SUBSCRIBE-INIT] GetOrCreateSubscriber sending init: topic=%s partition=%d startOffset=%d offsetType=%v consumerGroup=%s consumerID=%s" ,
glog . V ( 4 ) . Infof ( "[SUBSCRIBE-INIT] GetOrCreateSubscriber sending init: topic=%s partition=%d startOffset=%d offsetType=%v consumerGroup=%s consumerID=%s" ,
topic , partition , offsetValue , offsetType , consumerGroup , consumerID )
// Send init message using the actual partition structure that the broker allocated
@ -221,7 +221,7 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
// This is used by the stateless fetch approach to eliminate concurrent access issues
// The subscriber is NOT stored in bc.subscribers and must be cleaned up by the caller
func ( bc * BrokerClient ) createTemporarySubscriber ( topic string , partition int32 , startOffset int64 , consumerGroup string , consumerID string ) ( * BrokerSubscriberSession , error ) {
glog . V ( 2 ) . Infof ( "[STATELESS] Creating temporary subscriber for %s-%d at offset %d" , topic , partition , startOffset )
glog . V ( 4 ) . Infof ( "[STATELESS] Creating temporary subscriber for %s-%d at offset %d" , topic , partition , startOffset )
// Create context for this temporary subscriber
ctx , cancel := context . WithCancel ( bc . ctx )
@ -247,11 +247,11 @@ func (bc *BrokerClient) createTemporarySubscriber(topic string, partition int32,
if startOffset == - 1 {
offsetType = schema_pb . OffsetType_RESET_TO_LATEST
offsetValue = 0
glog . V ( 2 ) . Infof ( "[STATELESS] Using RESET_TO_LATEST for Kafka offset -1" )
glog . V ( 4 ) . Infof ( "[STATELESS] Using RESET_TO_LATEST for Kafka offset -1" )
} else {
offsetType = schema_pb . OffsetType_EXACT_OFFSET
offsetValue = startOffset
glog . V ( 2 ) . Infof ( "[STATELESS] Using EXACT_OFFSET for Kafka offset %d" , startOffset )
glog . V ( 4 ) . Infof ( "[STATELESS] Using EXACT_OFFSET for Kafka offset %d" , startOffset )
}
// Send init message
@ -273,14 +273,14 @@ func (bc *BrokerClient) createTemporarySubscriber(topic string, partition int32,
Cancel : cancel ,
}
glog . V ( 2 ) . Infof ( "[STATELESS] Created temporary subscriber for %s-%d starting at offset %d" , topic , partition , startOffset )
glog . V ( 4 ) . Infof ( "[STATELESS] Created temporary subscriber for %s-%d starting at offset %d" , topic , partition , startOffset )
return session , nil
}
// createSubscriberSession creates a new subscriber session with proper initialization
// This is used by the hybrid approach for initial connections and backward seeks
func ( bc * BrokerClient ) createSubscriberSession ( topic string , partition int32 , startOffset int64 , consumerGroup string , consumerID string ) ( * BrokerSubscriberSession , error ) {
glog . V ( 2 ) . Infof ( "[HYBRID-SESSION] Creating subscriber session for %s-%d at offset %d" , topic , partition , startOffset )
glog . V ( 4 ) . Infof ( "[HYBRID-SESSION] Creating subscriber session for %s-%d at offset %d" , topic , partition , startOffset )
// Create context for this subscriber
ctx , cancel := context . WithCancel ( bc . ctx )
@ -306,11 +306,11 @@ func (bc *BrokerClient) createSubscriberSession(topic string, partition int32, s
if startOffset == - 1 {
offsetType = schema_pb . OffsetType_RESET_TO_LATEST
offsetValue = 0
glog . V ( 2 ) . Infof ( "[HYBRID-SESSION] Using RESET_TO_LATEST for Kafka offset -1" )
glog . V ( 4 ) . Infof ( "[HYBRID-SESSION] Using RESET_TO_LATEST for Kafka offset -1" )
} else {
offsetType = schema_pb . OffsetType_EXACT_OFFSET
offsetValue = startOffset
glog . V ( 2 ) . Infof ( "[HYBRID-SESSION] Using EXACT_OFFSET for Kafka offset %d" , startOffset )
glog . V ( 4 ) . Infof ( "[HYBRID-SESSION] Using EXACT_OFFSET for Kafka offset %d" , startOffset )
}
// Send init message
@ -336,7 +336,7 @@ func (bc *BrokerClient) createSubscriberSession(topic string, partition int32, s
initialized : false ,
}
glog . V ( 2 ) . Infof ( "[HYBRID-SESSION] Created subscriber session for %s-%d starting at offset %d" , topic , partition , startOffset )
glog . V ( 4 ) . Infof ( "[HYBRID-SESSION] Created subscriber session for %s-%d starting at offset %d" , topic , partition , startOffset )
return session , nil
}
@ -364,14 +364,14 @@ func (bc *BrokerClient) serveFromCache(session *BrokerSubscriberSession, request
// Return slice from cache
result := session . consumedRecords [ startIdx : endIdx ]
glog . V ( 2 ) . Infof ( "[HYBRID-CACHE] Served %d records from cache (requested %d, offset %d)" ,
glog . V ( 4 ) . Infof ( "[HYBRID-CACHE] Served %d records from cache (requested %d, offset %d)" ,
len ( result ) , maxRecords , requestedOffset )
return result
}
// readRecordsFromSession reads records from the session's stream
func ( bc * BrokerClient ) readRecordsFromSession ( ctx context . Context , session * BrokerSubscriberSession , startOffset int64 , maxRecords int ) ( [ ] * SeaweedRecord , error ) {
glog . V ( 2 ) . Infof ( "[HYBRID-READ] Reading from stream: offset=%d maxRecords=%d" , startOffset , maxRecords )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] Reading from stream: offset=%d maxRecords=%d" , startOffset , maxRecords )
records := make ( [ ] * SeaweedRecord , 0 , maxRecords )
currentOffset := startOffset
@ -382,7 +382,7 @@ func (bc *BrokerClient) readRecordsFromSession(ctx context.Context, session *Bro
select {
case <- ctx . Done ( ) :
// Timeout or cancellation - return what we have
glog . V ( 2 ) . Infof ( "[HYBRID-READ] Context done, returning %d records" , len ( records ) )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] Context done, returning %d records" , len ( records ) )
return records , nil
default :
}
@ -391,7 +391,7 @@ func (bc *BrokerClient) readRecordsFromSession(ctx context.Context, session *Bro
resp , err := session . Stream . Recv ( )
if err != nil {
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[HYBRID-READ] Stream closed (EOF), returning %d records" , len ( records ) )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] Stream closed (EOF), returning %d records" , len ( records ) )
return records , nil
}
return nil , fmt . Errorf ( "failed to receive from stream: %v" , err )
@ -431,20 +431,20 @@ func (bc *BrokerClient) readRecordsFromSession(ctx context.Context, session *Bro
return nil , fmt . Errorf ( "broker error: %s" , ctrlMsg . Error )
}
if ctrlMsg . IsEndOfStream {
glog . V ( 2 ) . Infof ( "[HYBRID-READ] End of stream, returning %d records" , len ( records ) )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] End of stream, returning %d records" , len ( records ) )
return records , nil
}
if ctrlMsg . IsEndOfTopic {
glog . V ( 2 ) . Infof ( "[HYBRID-READ] End of topic, returning %d records" , len ( records ) )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] End of topic, returning %d records" , len ( records ) )
return records , nil
}
// Empty control message (e.g., seek ack) - continue reading
glog . V ( 2 ) . Infof ( "[HYBRID-READ] Received control message (seek ack?), continuing" )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] Received control message (seek ack?), continuing" )
continue
}
}
glog . V ( 2 ) . Infof ( "[HYBRID-READ] Read %d records successfully" , len ( records ) )
glog . V ( 4 ) . Infof ( "[HYBRID-READ] Read %d records successfully" , len ( records ) )
// Update cache
session . consumedRecords = append ( session . consumedRecords , records ... )
@ -463,7 +463,7 @@ func (bc *BrokerClient) readRecordsFromSession(ctx context.Context, session *Bro
// - Slow path (5%): Create new subscriber for backward seeks
// This combines performance (connection reuse) with correctness (proper tracking)
func ( bc * BrokerClient ) FetchRecordsHybrid ( ctx context . Context , topic string , partition int32 , requestedOffset int64 , maxRecords int , consumerGroup string , consumerID string ) ( [ ] * SeaweedRecord , error ) {
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] topic=%s partition=%d requestedOffset=%d maxRecords=%d" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] topic=%s partition=%d requestedOffset=%d maxRecords=%d" ,
topic , partition , requestedOffset , maxRecords )
// Get or create session for this (topic, partition, consumerGroup, consumerID)
@ -473,7 +473,7 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
session , exists := bc . subscribers [ key ]
if ! exists {
// No session - create one (this is initial fetch)
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] Creating initial session for %s at offset %d" , key , requestedOffset )
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] Creating initial session for %s at offset %d" , key , requestedOffset )
newSession , err := bc . createSubscriberSession ( topic , partition , requestedOffset , consumerGroup , consumerID )
if err != nil {
bc . subscribersLock . Unlock ( )
@ -496,7 +496,7 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
if requestedOffset >= cacheStart && requestedOffset <= cacheEnd {
// Serve from cache
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] FAST: Serving from cache for %s offset %d (cache: %d-%d)" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] FAST: Serving from cache for %s offset %d (cache: %d-%d)" ,
key , requestedOffset , cacheStart , cacheEnd )
return bc . serveFromCache ( session , requestedOffset , maxRecords ) , nil
}
@ -509,13 +509,13 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
streamPosition = session . StartOffset
}
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] requestedOffset=%d streamPosition=%d lastReadOffset=%d" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] requestedOffset=%d streamPosition=%d lastReadOffset=%d" ,
requestedOffset , streamPosition , session . lastReadOffset )
// Decision: Fast path or slow path?
if requestedOffset < streamPosition {
// SLOW PATH: Backward seek - need new subscriber
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] SLOW: Backward seek from %d to %d, creating new subscriber" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] SLOW: Backward seek from %d to %d, creating new subscriber" ,
streamPosition , requestedOffset )
// Close old session
@ -550,7 +550,7 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
} else if requestedOffset > streamPosition {
// FAST PATH: Forward seek - use server-side seek
seekOffset := requestedOffset
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] FAST: Forward seek from %d to %d using server-side seek" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] FAST: Forward seek from %d to %d using server-side seek" ,
streamPosition , seekOffset )
// Send seek message to broker
@ -565,13 +565,13 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
if err := session . Stream . Send ( seekReq ) ; err != nil {
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] Stream closed during seek, ignoring" )
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] Stream closed during seek, ignoring" )
return nil , nil
}
return nil , fmt . Errorf ( "failed to send seek request: %v" , err )
}
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] Seek request sent, broker will reposition stream to offset %d" , seekOffset )
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] Seek request sent, broker will reposition stream to offset %d" , seekOffset )
// NOTE: Don't wait for ack - the broker will restart Subscribe loop and send data
// The ack will be handled inline with data messages in readRecordsFromSession
@ -580,7 +580,7 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
streamPosition = seekOffset
} else {
// FAST PATH: Sequential read - continue from current position
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] FAST: Sequential read at offset %d" , requestedOffset )
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] FAST: Sequential read at offset %d" , requestedOffset )
}
// Read records from stream
@ -593,7 +593,7 @@ func (bc *BrokerClient) FetchRecordsHybrid(ctx context.Context, topic string, pa
if len ( records ) > 0 {
session . lastReadOffset = records [ len ( records ) - 1 ] . Offset
session . initialized = true
glog . V ( 2 ) . Infof ( "[FETCH-HYBRID] Read %d records, lastReadOffset now %d" ,
glog . V ( 4 ) . Infof ( "[FETCH-HYBRID] Read %d records, lastReadOffset now %d" ,
len ( records ) , session . lastReadOffset )
}
@ -607,7 +607,7 @@ func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string,
// Create key for this fetch request
key := fmt . Sprintf ( "%s-%d-%d" , topic , partition , startOffset )
glog . V ( 2 ) . Infof ( "[FETCH-DEDUP] topic=%s partition=%d offset=%d maxRecords=%d key=%s" ,
glog . V ( 4 ) . Infof ( "[FETCH-DEDUP] topic=%s partition=%d offset=%d maxRecords=%d key=%s" ,
topic , partition , startOffset , maxRecords , key )
// Check if there's already a fetch in progress for this exact request
@ -622,12 +622,12 @@ func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string,
existing . mu . Unlock ( )
bc . fetchRequestsLock . Unlock ( )
glog . V ( 2 ) . Infof ( "[FETCH-DEDUP] Waiting for in-progress fetch: %s" , key )
glog . V ( 4 ) . Infof ( "[FETCH-DEDUP] Waiting for in-progress fetch: %s" , key )
// Wait for the result from the in-progress fetch
select {
case result := <- waiter :
glog . V ( 2 ) . Infof ( "[FETCH-DEDUP] Received result from in-progress fetch: %s (records=%d, err=%v)" ,
glog . V ( 4 ) . Infof ( "[FETCH-DEDUP] Received result from in-progress fetch: %s (records=%d, err=%v)" ,
key , len ( result . records ) , result . err )
return result . records , result . err
case <- ctx . Done ( ) :
@ -647,7 +647,7 @@ func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string,
bc . fetchRequests [ key ] = fetchReq
bc . fetchRequestsLock . Unlock ( )
glog . V ( 2 ) . Infof ( "[FETCH-DEDUP] Starting new fetch: %s" , key )
glog . V ( 4 ) . Infof ( "[FETCH-DEDUP] Starting new fetch: %s" , key )
// Perform the actual fetch
records , err := bc . fetchRecordsStatelessInternal ( ctx , topic , partition , startOffset , maxRecords , consumerGroup , consumerID )
@ -667,7 +667,7 @@ func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string,
bc . fetchRequestsLock . Unlock ( )
// Send result to all waiters
glog . V ( 2 ) . Infof ( "[FETCH-DEDUP] Broadcasting result to %d waiters: %s (records=%d, err=%v)" ,
glog . V ( 4 ) . Infof ( "[FETCH-DEDUP] Broadcasting result to %d waiters: %s (records=%d, err=%v)" ,
len ( waiters ) , key , len ( records ) , err )
for _ , waiter := range waiters {
waiter <- result
@ -680,7 +680,7 @@ func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string,
// fetchRecordsStatelessInternal is the internal implementation of stateless fetch
// This is called by FetchRecordsWithDedup and should not be called directly
func ( bc * BrokerClient ) fetchRecordsStatelessInternal ( ctx context . Context , topic string , partition int32 , startOffset int64 , maxRecords int , consumerGroup string , consumerID string ) ( [ ] * SeaweedRecord , error ) {
glog . V ( 2 ) . Infof ( "[FETCH-STATELESS] topic=%s partition=%d offset=%d maxRecords=%d" ,
glog . V ( 4 ) . Infof ( "[FETCH-STATELESS] topic=%s partition=%d offset=%d maxRecords=%d" ,
topic , partition , startOffset , maxRecords )
// STATELESS APPROACH: Create a temporary subscriber just for this fetch
@ -737,7 +737,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
return nil , fmt . Errorf ( "subscriber session stream cannot be nil" )
}
glog . V ( 2 ) . Infof ( "[FETCH] readRecordsFrom: topic=%s partition=%d startOffset=%d maxRecords=%d" ,
glog . V ( 4 ) . Infof ( "[FETCH] readRecordsFrom: topic=%s partition=%d startOffset=%d maxRecords=%d" ,
session . Topic , session . Partition , startOffset , maxRecords )
var records [ ] * SeaweedRecord
@ -775,7 +775,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
// If we access session.Stream in the goroutine, it could become nil between check and use
stream := session . Stream
if stream == nil {
glog . V ( 2 ) . Infof ( "[FETCH] Stream is nil, cannot read" )
glog . V ( 4 ) . Infof ( "[FETCH] Stream is nil, cannot read" )
return records , nil
}
@ -807,7 +807,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
select {
case result := <- recvChan :
if result . err != nil {
glog . V ( 2 ) . Infof ( "[FETCH] Stream.Recv() error on first record: %v" , result . err )
glog . V ( 4 ) . Infof ( "[FETCH] Stream.Recv() error on first record: %v" , result . err )
return records , nil // Return empty - no error for empty topic
}
@ -820,7 +820,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
}
records = append ( records , record )
currentOffset ++
glog . V ( 2 ) . Infof ( "[FETCH] Received first record: offset=%d, keyLen=%d, valueLen=%d" ,
glog . V ( 4 ) . Infof ( "[FETCH] Received first record: offset=%d, keyLen=%d, valueLen=%d" ,
record . Offset , len ( record . Key ) , len ( record . Value ) )
// CRITICAL: Auto-acknowledge first message immediately for Kafka gateway
@ -835,14 +835,14 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
} ,
}
if err := stream . Send ( ackMsg ) ; err != nil {
glog . V ( 2 ) . Infof ( "[FETCH] Failed to send ack for first record offset %d: %v (continuing)" , record . Offset , err )
glog . V ( 4 ) . Infof ( "[FETCH] Failed to send ack for first record offset %d: %v (continuing)" , record . Offset , err )
// Don't fail the fetch if ack fails - continue reading
}
}
case <- ctx . Done ( ) :
// Timeout on first record - topic is empty or no data available
glog . V ( 2 ) . Infof ( "[FETCH] No data available (timeout on first record)" )
glog . V ( 4 ) . Infof ( "[FETCH] No data available (timeout on first record)" )
return records , nil
}
@ -897,7 +897,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
readDuration := time . Since ( readStart )
if result . err != nil {
glog . V ( 2 ) . Infof ( "[FETCH] Stream.Recv() error after %d records: %v" , len ( records ) , result . err )
glog . V ( 4 ) . Infof ( "[FETCH] Stream.Recv() error after %d records: %v" , len ( records ) , result . err )
// Return what we have - cache will be updated at the end
break
}
@ -913,7 +913,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
currentOffset ++
consecutiveReads ++ // Track number of successful reads for adaptive timeout
glog . V ( 2 ) . Infof ( "[FETCH] Received record %d: offset=%d, keyLen=%d, valueLen=%d, readTime=%v" ,
glog . V ( 4 ) . Infof ( "[FETCH] Received record %d: offset=%d, keyLen=%d, valueLen=%d, readTime=%v" ,
len ( records ) , record . Offset , len ( record . Key ) , len ( record . Value ) , readDuration )
// CRITICAL: Auto-acknowledge message immediately for Kafka gateway
@ -928,7 +928,7 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
} ,
}
if err := stream . Send ( ackMsg ) ; err != nil {
glog . V ( 2 ) . Infof ( "[FETCH] Failed to send ack for offset %d: %v (continuing)" , record . Offset , err )
glog . V ( 4 ) . Infof ( "[FETCH] Failed to send ack for offset %d: %v (continuing)" , record . Offset , err )
// Don't fail the fetch if ack fails - continue reading
}
}
@ -936,12 +936,12 @@ func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubs
case <- ctx2 . Done ( ) :
cancel2 ( )
// Timeout - return what we have
glog . V ( 2 ) . Infof ( "[FETCH] Read timeout after %d records (waited %v), returning batch" , len ( records ) , time . Since ( readStart ) )
glog . V ( 4 ) . Infof ( "[FETCH] Read timeout after %d records (waited %v), returning batch" , len ( records ) , time . Since ( readStart ) )
return records , nil
}
}
glog . V ( 2 ) . Infof ( "[FETCH] Returning %d records (maxRecords reached)" , len ( records ) )
glog . V ( 4 ) . Infof ( "[FETCH] Returning %d records (maxRecords reached)" , len ( records ) )
return records , nil
}
@ -985,7 +985,7 @@ func (bc *BrokerClient) CloseSubscriber(topic string, partition int32, consumerG
}
session . mu . Unlock ( )
delete ( bc . subscribers , key )
glog . V ( 2 ) . Infof ( "[FETCH] Closed subscriber for %s" , key )
glog . V ( 4 ) . Infof ( "[FETCH] Closed subscriber for %s" , key )
}
}
@ -1032,7 +1032,7 @@ func (bc *BrokerClient) RestartSubscriber(session *BrokerSubscriberSession, newO
session . mu . Lock ( )
defer session . mu . Unlock ( )
glog . V ( 2 ) . Infof ( "[FETCH] Restarting subscriber for %s[%d]: from offset %d to %d" ,
glog . V ( 4 ) . Infof ( "[FETCH] Restarting subscriber for %s[%d]: from offset %d to %d" ,
session . Topic , session . Partition , session . StartOffset , newOffset )
// Close existing stream
@ -1079,7 +1079,7 @@ func (bc *BrokerClient) RestartSubscriber(session *BrokerSubscriberSession, newO
session . Ctx = subscriberCtx
session . StartOffset = newOffset
glog . V ( 2 ) . Infof ( "[FETCH] Successfully restarted subscriber for %s[%d] at offset %d" ,
glog . V ( 4 ) . Infof ( "[FETCH] Successfully restarted subscriber for %s[%d] at offset %d" ,
session . Topic , session . Partition , newOffset )
return nil
@ -1095,7 +1095,7 @@ func (session *BrokerSubscriberSession) SeekToOffset(offset int64) error {
session . mu . Unlock ( )
if currentOffset == offset {
glog . V ( 2 ) . Infof ( "[SEEK] Already at offset %d for %s[%d], skipping seek" , offset , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Already at offset %d for %s[%d], skipping seek" , offset , session . Topic , session . Partition )
return nil
}
@ -1111,7 +1111,7 @@ func (session *BrokerSubscriberSession) SeekToOffset(offset int64) error {
if err := session . Stream . Send ( seekMsg ) ; err != nil {
// Handle graceful shutdown
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[SEEK] Stream closing during seek to offset %d for %s[%d]" , offset , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Stream closing during seek to offset %d for %s[%d]" , offset , session . Topic , session . Partition )
return nil // Not an error during shutdown
}
return fmt . Errorf ( "seek to offset %d failed: %v" , offset , err )
@ -1132,7 +1132,7 @@ func (session *BrokerSubscriberSession) SeekToOffset(offset int64) error {
}
session . mu . Unlock ( )
glog . V ( 2 ) . Infof ( "[SEEK] Seeked to offset %d for %s[%d]" , offset , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Seeked to offset %d for %s[%d]" , offset , session . Topic , session . Partition )
return nil
}
@ -1153,7 +1153,7 @@ func (session *BrokerSubscriberSession) SeekToTimestamp(timestampNs int64) error
if err := session . Stream . Send ( seekMsg ) ; err != nil {
// Handle graceful shutdown
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[SEEK] Stream closing during seek to timestamp %d for %s[%d]" , timestampNs , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Stream closing during seek to timestamp %d for %s[%d]" , timestampNs , session . Topic , session . Partition )
return nil // Not an error during shutdown
}
return fmt . Errorf ( "seek to timestamp %d failed: %v" , timestampNs , err )
@ -1165,7 +1165,7 @@ func (session *BrokerSubscriberSession) SeekToTimestamp(timestampNs int64) error
session . consumedRecords = nil
session . mu . Unlock ( )
glog . V ( 2 ) . Infof ( "[SEEK] Seeked to timestamp %d for %s[%d]" , timestampNs , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Seeked to timestamp %d for %s[%d]" , timestampNs , session . Topic , session . Partition )
return nil
}
@ -1185,7 +1185,7 @@ func (session *BrokerSubscriberSession) SeekToEarliest() error {
if err := session . Stream . Send ( seekMsg ) ; err != nil {
// Handle graceful shutdown
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[SEEK] Stream closing during seek to earliest for %s[%d]" , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Stream closing during seek to earliest for %s[%d]" , session . Topic , session . Partition )
return nil // Not an error during shutdown
}
return fmt . Errorf ( "seek to earliest failed: %v" , err )
@ -1196,7 +1196,7 @@ func (session *BrokerSubscriberSession) SeekToEarliest() error {
session . consumedRecords = nil
session . mu . Unlock ( )
glog . V ( 2 ) . Infof ( "[SEEK] Seeked to earliest for %s[%d]" , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Seeked to earliest for %s[%d]" , session . Topic , session . Partition )
return nil
}
@ -1216,7 +1216,7 @@ func (session *BrokerSubscriberSession) SeekToLatest() error {
if err := session . Stream . Send ( seekMsg ) ; err != nil {
// Handle graceful shutdown
if err == io . EOF {
glog . V ( 2 ) . Infof ( "[SEEK] Stream closing during seek to latest for %s[%d]" , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Stream closing during seek to latest for %s[%d]" , session . Topic , session . Partition )
return nil // Not an error during shutdown
}
return fmt . Errorf ( "seek to latest failed: %v" , err )
@ -1227,6 +1227,6 @@ func (session *BrokerSubscriberSession) SeekToLatest() error {
session . consumedRecords = nil
session . mu . Unlock ( )
glog . V ( 2 ) . Infof ( "[SEEK] Seeked to latest for %s[%d]" , session . Topic , session . Partition )
glog . V ( 4 ) . Infof ( "[SEEK] Seeked to latest for %s[%d]" , session . Topic , session . Partition )
return nil
}