Browse Source

ack messages to broker

pull/7329/head
chrislu 4 weeks ago
parent
commit
7e934d6283
  1. 67
      weed/mq/kafka/integration/broker_client_subscribe.go

67
weed/mq/kafka/integration/broker_client_subscribe.go

@ -3,6 +3,7 @@ package integration
import ( import (
"context" "context"
"fmt" "fmt"
"io"
"time" "time"
"github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/glog"
@ -248,8 +249,11 @@ func (bc *BrokerClient) ReadRecordsFromOffset(ctx context.Context, session *Brok
} }
glog.V(2).Infof("[FETCH] Returning %d cached records for %s at offset %d (cache: %d-%d)", glog.V(2).Infof("[FETCH] Returning %d cached records for %s at offset %d (cache: %d-%d)",
endIdx-startIdx, session.Key(), requestedOffset, cacheStartOffset, cacheEndOffset) endIdx-startIdx, session.Key(), requestedOffset, cacheStartOffset, cacheEndOffset)
// CRITICAL: Capture slice while holding lock to prevent race condition
// If we unlock before slicing, another goroutine could clear consumedRecords
result := session.consumedRecords[startIdx:endIdx]
session.mu.Unlock() session.mu.Unlock()
return session.consumedRecords[startIdx:endIdx], nil
return result, nil
} }
} else { } else {
glog.V(2).Infof("[FETCH] Cache miss for %s: requested=%d, cache=[%d-%d]", glog.V(2).Infof("[FETCH] Cache miss for %s: requested=%d, cache=[%d-%d]",
@ -291,6 +295,11 @@ func (bc *BrokerClient) ReadRecordsFromOffset(ctx context.Context, session *Brok
} }
if err := session.Stream.Send(seekMsg); err != nil { if err := session.Stream.Send(seekMsg); err != nil {
// Handle graceful shutdown: EOF means stream is closing
if err == io.EOF {
glog.V(2).Infof("[FETCH] Stream closing during seek to offset %d, returning empty", requestedOffset)
return []*SeaweedRecord{}, nil
}
return nil, fmt.Errorf("seek to offset %d failed: %v", requestedOffset, err) return nil, fmt.Errorf("seek to offset %d failed: %v", requestedOffset, err)
} }
@ -363,7 +372,9 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
} }
glog.V(2).Infof("[FETCH] Returning %d cached records from index %d to %d", endIdx-startIdx, startIdx, endIdx-1) glog.V(2).Infof("[FETCH] Returning %d cached records from index %d to %d", endIdx-startIdx, startIdx, endIdx-1)
return session.consumedRecords[startIdx:endIdx], nil
// CRITICAL: Capture slice result while holding lock (defer will unlock after return)
result := session.consumedRecords[startIdx:endIdx]
return result, nil
} }
} }
@ -438,6 +449,22 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
currentOffset++ currentOffset++
glog.V(2).Infof("[FETCH] Received record: offset=%d, keyLen=%d, valueLen=%d", glog.V(2).Infof("[FETCH] Received record: offset=%d, keyLen=%d, valueLen=%d",
record.Offset, len(record.Key), len(record.Value)) record.Offset, len(record.Key), len(record.Value))
// CRITICAL: Auto-acknowledge first message immediately for Kafka gateway
// Kafka uses offset commits (not per-message acks) so we must ack to prevent
// broker from blocking on in-flight messages waiting for acks that will never come
ackMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Ack{
Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
Key: dataMsg.Key,
TsNs: dataMsg.TsNs,
},
},
}
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)
// Don't fail the fetch if ack fails - continue reading
}
} }
case <-ctx.Done(): case <-ctx.Done():
@ -518,6 +545,22 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
glog.V(2).Infof("[FETCH] Received record %d: offset=%d, keyLen=%d, valueLen=%d, readTime=%v", glog.V(2).Infof("[FETCH] Received record %d: offset=%d, keyLen=%d, valueLen=%d, readTime=%v",
len(records), record.Offset, len(record.Key), len(record.Value), readDuration) len(records), record.Offset, len(record.Key), len(record.Value), readDuration)
// CRITICAL: Auto-acknowledge message immediately for Kafka gateway
// Kafka uses offset commits (not per-message acks) so we must ack to prevent
// broker from blocking on in-flight messages waiting for acks that will never come
ackMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Ack{
Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
Key: dataMsg.Key,
TsNs: dataMsg.TsNs,
},
},
}
if err := stream.Send(ackMsg); err != nil {
glog.V(2).Infof("[FETCH] Failed to send ack for offset %d: %v (continuing)", record.Offset, err)
// Don't fail the fetch if ack fails - continue reading
}
} }
case <-ctx2.Done(): case <-ctx2.Done():
@ -698,6 +741,11 @@ func (session *BrokerSubscriberSession) SeekToOffset(offset int64) error {
} }
if err := session.Stream.Send(seekMsg); err != nil { 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)
return nil // Not an error during shutdown
}
return fmt.Errorf("seek to offset %d failed: %v", offset, err) return fmt.Errorf("seek to offset %d failed: %v", offset, err)
} }
@ -725,6 +773,11 @@ func (session *BrokerSubscriberSession) SeekToTimestamp(timestampNs int64) error
} }
if err := session.Stream.Send(seekMsg); err != nil { 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)
return nil // Not an error during shutdown
}
return fmt.Errorf("seek to timestamp %d failed: %v", timestampNs, err) return fmt.Errorf("seek to timestamp %d failed: %v", timestampNs, err)
} }
@ -752,6 +805,11 @@ func (session *BrokerSubscriberSession) SeekToEarliest() error {
} }
if err := session.Stream.Send(seekMsg); err != nil { 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)
return nil // Not an error during shutdown
}
return fmt.Errorf("seek to earliest failed: %v", err) return fmt.Errorf("seek to earliest failed: %v", err)
} }
@ -778,6 +836,11 @@ func (session *BrokerSubscriberSession) SeekToLatest() error {
} }
if err := session.Stream.Send(seekMsg); err != nil { 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)
return nil // Not an error during shutdown
}
return fmt.Errorf("seek to latest failed: %v", err) return fmt.Errorf("seek to latest failed: %v", err)
} }

Loading…
Cancel
Save