Browse Source

log read stateless

pull/7329/head
chrislu 5 days ago
parent
commit
ba1a8aed64
  1. 2
      test/kafka/kafka-client-loadtest/config/loadtest.yaml
  2. 2
      test/kafka/kafka-client-loadtest/docker-compose.yml
  3. 10
      test/kafka/kafka-client-loadtest/internal/consumer/consumer.go
  4. 43
      test/kafka/kafka-client-loadtest/test-no-schema.sh
  5. 160
      weed/mq/broker/broker_grpc_fetch.go
  6. 872
      weed/mq/broker/broker_grpc_sub_seek_test.go
  7. 1
      weed/mq/kafka/integration/broker_client.go
  8. 178
      weed/mq/kafka/integration/broker_client_fetch.go
  9. 651
      weed/mq/kafka/integration/broker_client_subscribe.go
  10. 68
      weed/mq/kafka/integration/seaweedmq_handler.go
  11. 29
      weed/mq/kafka/integration/types.go
  12. 66
      weed/pb/mq_broker.proto
  13. 661
      weed/pb/mq_pb/mq_broker.pb.go
  14. 44
      weed/pb/mq_pb/mq_broker_grpc.pb.go
  15. 341
      weed/util/log_buffer/log_read_integration_test.go
  16. 321
      weed/util/log_buffer/log_read_stateless.go
  17. 372
      weed/util/log_buffer/log_read_stateless_test.go

2
test/kafka/kafka-client-loadtest/config/loadtest.yaml

@ -51,7 +51,7 @@ consumers:
group_prefix: "loadtest-group" # Consumer group prefix
auto_offset_reset: "earliest" # earliest, latest
enable_auto_commit: true
auto_commit_interval_ms: 1000
auto_commit_interval_ms: 100 # Reduced from 1000ms to 100ms to minimize duplicate window
session_timeout_ms: 30000
heartbeat_interval_ms: 3000
max_poll_records: 500

2
test/kafka/kafka-client-loadtest/docker-compose.yml

@ -252,7 +252,7 @@ services:
- TOPIC_COUNT=${TOPIC_COUNT:-5}
- PARTITIONS_PER_TOPIC=${PARTITIONS_PER_TOPIC:-3}
- TEST_MODE=${TEST_MODE:-comprehensive}
- SCHEMAS_ENABLED=true
- SCHEMAS_ENABLED=${SCHEMAS_ENABLED:-true}
- VALUE_TYPE=${VALUE_TYPE:-avro}
profiles:
- loadtest

10
test/kafka/kafka-client-loadtest/internal/consumer/consumer.go

@ -619,11 +619,19 @@ func (h *ConsumerGroupHandler) ConsumeClaim(session sarama.ConsumerGroupSession,
// Mark message as processed
session.MarkMessage(message, "")
// Commit offset immediately every 10 messages to reduce duplicate window
// This supplements auto-commit and ensures offsets are committed more frequently
if msgCount%10 == 0 {
session.Commit()
}
}
case <-session.Context().Done():
log.Printf("Consumer %d: Session context cancelled for %s[%d]",
log.Printf("Consumer %d: Session context cancelled for %s[%d], committing final offsets",
h.consumer.id, claim.Topic(), claim.Partition())
// Commit all remaining marked offsets before shutting down
session.Commit()
return nil
}
}

43
test/kafka/kafka-client-loadtest/test-no-schema.sh

@ -0,0 +1,43 @@
#!/bin/bash
# Test without schema registry to isolate missing messages issue
# Clean old data
find test-results -name "*.jsonl" -delete 2>/dev/null || true
# Run test without schemas
TEST_MODE=comprehensive \
TEST_DURATION=1m \
PRODUCER_COUNT=2 \
CONSUMER_COUNT=2 \
MESSAGE_RATE=50 \
MESSAGE_SIZE=512 \
VALUE_TYPE=json \
SCHEMAS_ENABLED=false \
docker compose --profile loadtest up --abort-on-container-exit kafka-client-loadtest
echo ""
echo "═══════════════════════════════════════════════════════"
echo "Analyzing results..."
if [ -f test-results/produced.jsonl ] && [ -f test-results/consumed.jsonl ]; then
produced=$(wc -l < test-results/produced.jsonl)
consumed=$(wc -l < test-results/consumed.jsonl)
echo "Produced: $produced"
echo "Consumed: $consumed"
# Check for missing messages
jq -r '"\(.topic)[\(.partition)]@\(.offset)"' test-results/produced.jsonl | sort > /tmp/produced.txt
jq -r '"\(.topic)[\(.partition)]@\(.offset)"' test-results/consumed.jsonl | sort > /tmp/consumed.txt
missing=$(comm -23 /tmp/produced.txt /tmp/consumed.txt | wc -l)
echo "Missing: $missing"
if [ $missing -eq 0 ]; then
echo "✓ NO MISSING MESSAGES!"
else
echo "✗ Still have missing messages"
echo "Sample missing:"
comm -23 /tmp/produced.txt /tmp/consumed.txt | head -10
fi
else
echo "✗ Result files not found"
fi
echo "═══════════════════════════════════════════════════════"

160
weed/mq/broker/broker_grpc_fetch.go

@ -0,0 +1,160 @@
package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
// FetchMessage implements Kafka-style stateless message fetching
// This is the recommended API for Kafka gateway and other stateless clients
//
// Key differences from SubscribeMessage:
// 1. Request/Response pattern (not streaming)
// 2. No session state maintained on broker
// 3. Each request is completely independent
// 4. Safe for concurrent calls at different offsets
// 5. No Subscribe loop cancellation/restart complexity
//
// Design inspired by Kafka's Fetch API:
// - Client manages offset tracking
// - Each fetch is independent
// - No shared state between requests
// - Natural support for concurrent reads
func (b *MessageQueueBroker) FetchMessage(ctx context.Context, req *mq_pb.FetchMessageRequest) (*mq_pb.FetchMessageResponse, error) {
glog.Infof("[FetchMessage] CALLED!") // DEBUG: ensure this shows up
// Validate request
if req.Topic == nil {
return nil, fmt.Errorf("missing topic")
}
if req.Partition == nil {
return nil, fmt.Errorf("missing partition")
}
t := topic.FromPbTopic(req.Topic)
partition := topic.FromPbPartition(req.Partition)
glog.Infof("[FetchMessage] %s/%s partition=%v offset=%d maxMessages=%d maxBytes=%d consumer=%s/%s",
t.Namespace, t.Name, partition, req.StartOffset, req.MaxMessages, req.MaxBytes,
req.ConsumerGroup, req.ConsumerId)
// Get local partition
localPartition, err := b.GetOrGenerateLocalPartition(t, partition)
if err != nil {
glog.Errorf("[FetchMessage] Failed to get partition: %v", err)
return &mq_pb.FetchMessageResponse{
Error: fmt.Sprintf("partition not found: %v", err),
ErrorCode: 1,
}, nil
}
if localPartition == nil {
return &mq_pb.FetchMessageResponse{
Error: "partition not found",
ErrorCode: 1,
}, nil
}
// Set defaults for limits
maxMessages := int(req.MaxMessages)
if maxMessages <= 0 {
maxMessages = 100 // Reasonable default
}
if maxMessages > 10000 {
maxMessages = 10000 // Safety limit
}
maxBytes := int(req.MaxBytes)
if maxBytes <= 0 {
maxBytes = 4 * 1024 * 1024 // 4MB default
}
if maxBytes > 100*1024*1024 {
maxBytes = 100 * 1024 * 1024 // 100MB safety limit
}
// TODO: Long poll support disabled for now (causing timeouts)
// Check if we should wait for data (long poll support)
// shouldWait := req.MaxWaitMs > 0
// if shouldWait {
// // Wait for data to be available (with timeout)
// dataAvailable := localPartition.LogBuffer.WaitForDataWithTimeout(req.StartOffset, int(req.MaxWaitMs))
// if !dataAvailable {
// // Timeout - return empty response
// glog.V(3).Infof("[FetchMessage] Timeout waiting for data at offset %d", req.StartOffset)
// return &mq_pb.FetchMessageResponse{
// Messages: []*mq_pb.DataMessage{},
// HighWaterMark: localPartition.LogBuffer.GetHighWaterMark(),
// LogStartOffset: localPartition.LogBuffer.GetLogStartOffset(),
// EndOfPartition: false,
// NextOffset: req.StartOffset,
// }, nil
// }
// }
// Check if disk read function is configured
if localPartition.LogBuffer.ReadFromDiskFn == nil {
glog.Errorf("[FetchMessage] LogBuffer.ReadFromDiskFn is nil! This should not happen.")
} else {
glog.V(3).Infof("[FetchMessage] LogBuffer.ReadFromDiskFn is configured")
}
// Use requested offset directly - let ReadMessagesAtOffset handle disk reads
requestedOffset := req.StartOffset
// Read messages from LogBuffer (stateless read)
glog.Infof("[FetchMessage] About to read from LogBuffer at offset %d (requested=%d)", requestedOffset, req.StartOffset)
logEntries, nextOffset, highWaterMark, endOfPartition, err := localPartition.LogBuffer.ReadMessagesAtOffset(
requestedOffset,
maxMessages,
maxBytes,
)
glog.Infof("[FetchMessage] Read completed: %d entries, nextOffset=%d, hwm=%d, eop=%v, err=%v",
len(logEntries), nextOffset, highWaterMark, endOfPartition, err)
if err != nil {
// Check if this is an "offset out of range" error
errMsg := err.Error()
if len(errMsg) > 0 && (len(errMsg) < 20 || errMsg[:20] != "offset") {
glog.Errorf("[FetchMessage] Read error: %v", err)
} else {
// Offset out of range - this is expected when consumer requests old data
glog.V(1).Infof("[FetchMessage] Offset out of range: %v", err)
}
// Return empty response with metadata - let client adjust offset
return &mq_pb.FetchMessageResponse{
Messages: []*mq_pb.DataMessage{},
HighWaterMark: highWaterMark,
LogStartOffset: localPartition.LogBuffer.GetLogStartOffset(),
EndOfPartition: false,
NextOffset: localPartition.LogBuffer.GetLogStartOffset(), // Suggest starting from earliest available
Error: errMsg,
ErrorCode: 2,
}, nil
}
// Convert to protobuf messages
messages := make([]*mq_pb.DataMessage, 0, len(logEntries))
for _, entry := range logEntries {
messages = append(messages, &mq_pb.DataMessage{
Key: entry.Key,
Value: entry.Data,
TsNs: entry.TsNs,
})
}
glog.V(2).Infof("[FetchMessage] Returning %d messages, nextOffset=%d, highWaterMark=%d, endOfPartition=%v",
len(messages), nextOffset, highWaterMark, endOfPartition)
return &mq_pb.FetchMessageResponse{
Messages: messages,
HighWaterMark: highWaterMark,
LogStartOffset: localPartition.LogBuffer.GetLogStartOffset(),
EndOfPartition: endOfPartition,
NextOffset: nextOffset,
}, nil
}

872
weed/mq/broker/broker_grpc_sub_seek_test.go

@ -1,872 +0,0 @@
package broker
import (
"context"
"fmt"
"io"
"sync"
"testing"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"google.golang.org/grpc/metadata"
)
// TestGetRequestPositionFromSeek tests the helper function that converts seek requests to message positions
func TestGetRequestPositionFromSeek(t *testing.T) {
broker := &MessageQueueBroker{}
tests := []struct {
name string
offsetType schema_pb.OffsetType
offset int64
expectedBatch int64
expectZeroTime bool
}{
{
name: "reset to earliest",
offsetType: schema_pb.OffsetType_RESET_TO_EARLIEST,
offset: 0,
expectedBatch: -3,
expectZeroTime: false,
},
{
name: "reset to latest",
offsetType: schema_pb.OffsetType_RESET_TO_LATEST,
offset: 0,
expectedBatch: -4,
expectZeroTime: false,
},
{
name: "exact offset zero",
offsetType: schema_pb.OffsetType_EXACT_OFFSET,
offset: 0,
expectedBatch: 0,
expectZeroTime: true,
},
{
name: "exact offset 100",
offsetType: schema_pb.OffsetType_EXACT_OFFSET,
offset: 100,
expectedBatch: 100,
expectZeroTime: true,
},
{
name: "exact offset 1000",
offsetType: schema_pb.OffsetType_EXACT_OFFSET,
offset: 1000,
expectedBatch: 1000,
expectZeroTime: true,
},
{
name: "exact timestamp",
offsetType: schema_pb.OffsetType_EXACT_TS_NS,
offset: 1234567890123456789,
expectedBatch: -2,
expectZeroTime: false,
},
{
name: "reset to offset",
offsetType: schema_pb.OffsetType_RESET_TO_OFFSET,
offset: 42,
expectedBatch: 42,
expectZeroTime: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tt.offset,
OffsetType: tt.offsetType,
}
position := broker.getRequestPositionFromSeek(seekMsg)
if position.Offset != tt.expectedBatch {
t.Errorf("Expected batch index %d, got %d", tt.expectedBatch, position.Offset)
}
// Verify time handling
if tt.expectZeroTime && !position.Time.IsZero() {
t.Errorf("Expected zero time for offset-based seek, got %v", position.Time)
}
if !tt.expectZeroTime && position.Time.IsZero() && tt.offsetType != schema_pb.OffsetType_RESET_TO_EARLIEST {
t.Errorf("Expected non-zero time, got zero time")
}
})
}
}
// TestGetRequestPositionFromSeek_NilSafety tests that the function handles nil input gracefully
func TestGetRequestPositionFromSeek_NilSafety(t *testing.T) {
broker := &MessageQueueBroker{}
position := broker.getRequestPositionFromSeek(nil)
// Should return zero-value position without panicking
if position.Offset != 0 {
t.Errorf("Expected zero offset for nil input, got %d", position.Offset)
}
}
// TestGetRequestPositionFromSeek_ConsistentResults verifies that multiple calls with same input produce same output
func TestGetRequestPositionFromSeek_ConsistentResults(t *testing.T) {
broker := &MessageQueueBroker{}
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 42,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
// Call multiple times
positions := make([]log_buffer.MessagePosition, 5)
for i := 0; i < 5; i++ {
positions[i] = broker.getRequestPositionFromSeek(seekMsg)
time.Sleep(1 * time.Millisecond) // Small delay
}
// All positions should be identical
for i := 1; i < len(positions); i++ {
if positions[i].Offset != positions[0].Offset {
t.Errorf("Inconsistent Offset: %d vs %d", positions[0].Offset, positions[i].Offset)
}
if !positions[i].Time.Equal(positions[0].Time) {
t.Errorf("Inconsistent Time: %v vs %v", positions[0].Time, positions[i].Time)
}
if positions[i].IsOffsetBased != positions[0].IsOffsetBased {
t.Errorf("Inconsistent IsOffsetBased: %v vs %v", positions[0].IsOffsetBased, positions[i].IsOffsetBased)
}
}
}
// TestGetRequestPositionFromSeek_OffsetExtraction verifies offset can be correctly extracted
func TestGetRequestPositionFromSeek_OffsetExtraction(t *testing.T) {
broker := &MessageQueueBroker{}
testOffsets := []int64{0, 1, 10, 100, 1000, 9999}
for _, offset := range testOffsets {
t.Run(fmt.Sprintf("offset_%d", offset), func(t *testing.T) {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: offset,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
position := broker.getRequestPositionFromSeek(seekMsg)
if !position.IsOffsetBased {
t.Error("Position should be detected as offset-based")
}
if extractedOffset := position.GetOffset(); extractedOffset != offset {
t.Errorf("Expected extracted offset %d, got %d", offset, extractedOffset)
}
})
}
}
// MockSubscribeMessageStream is a mock implementation of the gRPC stream for testing
type MockSubscribeMessageStream struct {
ctx context.Context
recvChan chan *mq_pb.SubscribeMessageRequest
sentMessages []*mq_pb.SubscribeMessageResponse
mu sync.Mutex
recvErr error
}
func NewMockSubscribeMessageStream(ctx context.Context) *MockSubscribeMessageStream {
return &MockSubscribeMessageStream{
ctx: ctx,
recvChan: make(chan *mq_pb.SubscribeMessageRequest, 10),
sentMessages: make([]*mq_pb.SubscribeMessageResponse, 0),
}
}
func (m *MockSubscribeMessageStream) Send(msg *mq_pb.SubscribeMessageResponse) error {
m.mu.Lock()
defer m.mu.Unlock()
m.sentMessages = append(m.sentMessages, msg)
return nil
}
func (m *MockSubscribeMessageStream) Recv() (*mq_pb.SubscribeMessageRequest, error) {
if m.recvErr != nil {
return nil, m.recvErr
}
select {
case msg := <-m.recvChan:
return msg, nil
case <-m.ctx.Done():
return nil, io.EOF
}
}
func (m *MockSubscribeMessageStream) SetHeader(metadata.MD) error {
return nil
}
func (m *MockSubscribeMessageStream) SendHeader(metadata.MD) error {
return nil
}
func (m *MockSubscribeMessageStream) SetTrailer(metadata.MD) {}
func (m *MockSubscribeMessageStream) Context() context.Context {
return m.ctx
}
func (m *MockSubscribeMessageStream) SendMsg(interface{}) error {
return nil
}
func (m *MockSubscribeMessageStream) RecvMsg(interface{}) error {
return nil
}
func (m *MockSubscribeMessageStream) QueueMessage(msg *mq_pb.SubscribeMessageRequest) {
m.recvChan <- msg
}
func (m *MockSubscribeMessageStream) SetRecvError(err error) {
m.recvErr = err
}
func (m *MockSubscribeMessageStream) GetSentMessages() []*mq_pb.SubscribeMessageResponse {
m.mu.Lock()
defer m.mu.Unlock()
return append([]*mq_pb.SubscribeMessageResponse{}, m.sentMessages...)
}
// TestSeekMessageHandling_BasicSeek tests that seek messages are properly received and acknowledged
func TestSeekMessageHandling_BasicSeek(t *testing.T) {
// Create seek message
seekMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 100,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
},
}
// Verify message structure
if seekReq := seekMsg.GetSeek(); seekReq == nil {
t.Fatal("Failed to create seek message")
} else {
if seekReq.Offset != 100 {
t.Errorf("Expected offset 100, got %d", seekReq.Offset)
}
if seekReq.OffsetType != schema_pb.OffsetType_EXACT_OFFSET {
t.Errorf("Expected EXACT_OFFSET, got %v", seekReq.OffsetType)
}
}
}
// TestSeekMessageHandling_MultipleSeekTypes tests different seek offset types
func TestSeekMessageHandling_MultipleSeekTypes(t *testing.T) {
testCases := []struct {
name string
offset int64
offsetType schema_pb.OffsetType
}{
{
name: "seek to earliest",
offset: 0,
offsetType: schema_pb.OffsetType_RESET_TO_EARLIEST,
},
{
name: "seek to latest",
offset: 0,
offsetType: schema_pb.OffsetType_RESET_TO_LATEST,
},
{
name: "seek to exact offset",
offset: 42,
offsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
{
name: "seek to timestamp",
offset: time.Now().UnixNano(),
offsetType: schema_pb.OffsetType_EXACT_TS_NS,
},
{
name: "reset to offset",
offset: 1000,
offsetType: schema_pb.OffsetType_RESET_TO_OFFSET,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seekMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tc.offset,
OffsetType: tc.offsetType,
},
},
}
seekReq := seekMsg.GetSeek()
if seekReq == nil {
t.Fatal("Failed to get seek message")
}
if seekReq.Offset != tc.offset {
t.Errorf("Expected offset %d, got %d", tc.offset, seekReq.Offset)
}
if seekReq.OffsetType != tc.offsetType {
t.Errorf("Expected offset type %v, got %v", tc.offsetType, seekReq.OffsetType)
}
})
}
}
// TestSeekMessageHandling_AckVsSeekDistinction tests that we can distinguish between ack and seek messages
func TestSeekMessageHandling_AckVsSeekDistinction(t *testing.T) {
// Create ack message
ackMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Ack{
Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
Key: []byte("test-key"),
TsNs: time.Now().UnixNano(),
},
},
}
// Create seek message
seekMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 100,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
},
}
// Verify ack message doesn't match seek
if ackMsg.GetSeek() != nil {
t.Error("Ack message should not be detected as seek")
}
if ackMsg.GetAck() == nil {
t.Error("Ack message should be detected as ack")
}
// Verify seek message doesn't match ack
if seekMsg.GetAck() != nil {
t.Error("Seek message should not be detected as ack")
}
if seekMsg.GetSeek() == nil {
t.Error("Seek message should be detected as seek")
}
}
// TestSeekMessageResponse_SuccessFormat tests the response format for successful seek
func TestSeekMessageResponse_SuccessFormat(t *testing.T) {
// Create success response (empty error string = success)
successResponse := &mq_pb.SubscribeMessageResponse{
Message: &mq_pb.SubscribeMessageResponse_Ctrl{
Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
Error: "", // Empty error means success
},
},
}
ctrlMsg := successResponse.GetCtrl()
if ctrlMsg == nil {
t.Fatal("Failed to get control message")
}
// Empty error string indicates success
if ctrlMsg.Error != "" {
t.Errorf("Expected empty error for success, got: %s", ctrlMsg.Error)
}
}
// TestSeekMessageResponse_ErrorFormat tests the response format for failed seek
func TestSeekMessageResponse_ErrorFormat(t *testing.T) {
// Create error response
errorResponse := &mq_pb.SubscribeMessageResponse{
Message: &mq_pb.SubscribeMessageResponse_Ctrl{
Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
Error: "Seek not implemented",
},
},
}
ctrlMsg := errorResponse.GetCtrl()
if ctrlMsg == nil {
t.Fatal("Failed to get control message")
}
// Non-empty error string indicates failure
if ctrlMsg.Error == "" {
t.Error("Expected non-empty error for failure")
}
if ctrlMsg.Error != "Seek not implemented" {
t.Errorf("Expected specific error message, got: %s", ctrlMsg.Error)
}
}
// TestSeekMessageHandling_BackwardSeek tests backward seeking scenarios
func TestSeekMessageHandling_BackwardSeek(t *testing.T) {
testCases := []struct {
name string
currentPos int64
seekOffset int64
expectedGap int64
}{
{
name: "small backward gap",
currentPos: 100,
seekOffset: 90,
expectedGap: 10,
},
{
name: "medium backward gap",
currentPos: 1000,
seekOffset: 500,
expectedGap: 500,
},
{
name: "large backward gap",
currentPos: 1000000,
seekOffset: 1,
expectedGap: 999999,
},
{
name: "seek to zero",
currentPos: 100,
seekOffset: 0,
expectedGap: 100,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Verify gap calculation
gap := tc.currentPos - tc.seekOffset
if gap != tc.expectedGap {
t.Errorf("Expected gap %d, got %d", tc.expectedGap, gap)
}
// Create seek message for backward seek
seekMsg := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tc.seekOffset,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
},
}
seekReq := seekMsg.GetSeek()
if seekReq == nil {
t.Fatal("Failed to create seek message")
}
if seekReq.Offset != tc.seekOffset {
t.Errorf("Expected offset %d, got %d", tc.seekOffset, seekReq.Offset)
}
})
}
}
// TestSeekMessageHandling_ForwardSeek tests forward seeking scenarios
func TestSeekMessageHandling_ForwardSeek(t *testing.T) {
testCases := []struct {
name string
currentPos int64
seekOffset int64
shouldSeek bool
}{
{
name: "small forward gap",
currentPos: 100,
seekOffset: 110,
shouldSeek: false, // Forward seeks don't need special handling
},
{
name: "same position",
currentPos: 100,
seekOffset: 100,
shouldSeek: false,
},
{
name: "large forward gap",
currentPos: 100,
seekOffset: 10000,
shouldSeek: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// For forward seeks, gateway typically just continues reading
// No special seek message needed
isBackward := tc.seekOffset < tc.currentPos
if isBackward && !tc.shouldSeek {
t.Error("Backward seek should require seek message")
}
})
}
}
// TestSeekIntegration_PositionConversion tests the complete flow from seek message to position
func TestSeekIntegration_PositionConversion(t *testing.T) {
broker := &MessageQueueBroker{}
testCases := []struct {
name string
offset int64
offsetType schema_pb.OffsetType
verifyFunc func(t *testing.T, pos log_buffer.MessagePosition)
}{
{
name: "exact offset conversion",
offset: 42,
offsetType: schema_pb.OffsetType_EXACT_OFFSET,
verifyFunc: func(t *testing.T, pos log_buffer.MessagePosition) {
if !pos.IsOffsetBased {
t.Error("Expected offset-based position")
}
if pos.GetOffset() != 42 {
t.Errorf("Expected offset 42, got %d", pos.GetOffset())
}
},
},
{
name: "earliest offset conversion",
offset: 0,
offsetType: schema_pb.OffsetType_RESET_TO_EARLIEST,
verifyFunc: func(t *testing.T, pos log_buffer.MessagePosition) {
if pos.Offset != -3 {
t.Errorf("Expected batch -3 for earliest, got %d", pos.Offset)
}
},
},
{
name: "latest offset conversion",
offset: 0,
offsetType: schema_pb.OffsetType_RESET_TO_LATEST,
verifyFunc: func(t *testing.T, pos log_buffer.MessagePosition) {
if pos.Offset != -4 {
t.Errorf("Expected batch -4 for latest, got %d", pos.Offset)
}
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Create seek message
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tc.offset,
OffsetType: tc.offsetType,
}
// Convert to position
position := broker.getRequestPositionFromSeek(seekMsg)
// Verify result
tc.verifyFunc(t, position)
})
}
}
// TestSeekMessageHandling_ConcurrentSeeks tests handling multiple seek requests
func TestSeekMessageHandling_ConcurrentSeeks(t *testing.T) {
broker := &MessageQueueBroker{}
// Simulate multiple concurrent seek requests
seekOffsets := []int64{10, 20, 30, 40, 50}
var wg sync.WaitGroup
results := make([]log_buffer.MessagePosition, len(seekOffsets))
for i, offset := range seekOffsets {
wg.Add(1)
go func(idx int, off int64) {
defer wg.Done()
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: off,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
results[idx] = broker.getRequestPositionFromSeek(seekMsg)
}(i, offset)
}
wg.Wait()
// Verify all results are correct
for i, offset := range seekOffsets {
if results[i].GetOffset() != offset {
t.Errorf("Expected offset %d at index %d, got %d", offset, i, results[i].GetOffset())
}
}
}
// TestSeekMessageProtocol_WireFormat verifies the protobuf message structure
func TestSeekMessageProtocol_WireFormat(t *testing.T) {
// Test that SeekMessage is properly defined in the oneof
req := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 100,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
},
}
// Verify oneof is set correctly
switch msg := req.Message.(type) {
case *mq_pb.SubscribeMessageRequest_Seek:
if msg.Seek.Offset != 100 {
t.Errorf("Expected offset 100, got %d", msg.Seek.Offset)
}
default:
t.Errorf("Expected Seek message, got %T", msg)
}
// Verify other message types are nil
if req.GetAck() != nil {
t.Error("Seek message should not have Ack")
}
if req.GetInit() != nil {
t.Error("Seek message should not have Init")
}
}
// TestSeekByTimestamp tests timestamp-based seek operations
func TestSeekByTimestamp(t *testing.T) {
broker := &MessageQueueBroker{}
testCases := []struct {
name string
timestampNs int64
offsetType schema_pb.OffsetType
}{
{
name: "seek to specific timestamp",
timestampNs: 1234567890123456789,
offsetType: schema_pb.OffsetType_EXACT_TS_NS,
},
{
name: "seek to current timestamp",
timestampNs: time.Now().UnixNano(),
offsetType: schema_pb.OffsetType_EXACT_TS_NS,
},
{
name: "seek to past timestamp",
timestampNs: time.Now().Add(-24 * time.Hour).UnixNano(),
offsetType: schema_pb.OffsetType_EXACT_TS_NS,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tc.timestampNs,
OffsetType: tc.offsetType,
}
position := broker.getRequestPositionFromSeek(seekMsg)
// For timestamp-based seeks, Time should be set to the timestamp
expectedTime := time.Unix(0, tc.timestampNs)
if !position.Time.Equal(expectedTime) {
t.Errorf("Expected time %v, got %v", expectedTime, position.Time)
}
// Batch should be -2 for EXACT_TS_NS
if position.Offset != -2 {
t.Errorf("Expected batch -2 for timestamp seek, got %d", position.Offset)
}
})
}
}
// TestSeekByTimestamp_Ordering tests that timestamp seeks preserve ordering
func TestSeekByTimestamp_Ordering(t *testing.T) {
broker := &MessageQueueBroker{}
// Create timestamps in chronological order
baseTime := time.Now().Add(-1 * time.Hour)
timestamps := []int64{
baseTime.UnixNano(),
baseTime.Add(10 * time.Minute).UnixNano(),
baseTime.Add(20 * time.Minute).UnixNano(),
baseTime.Add(30 * time.Minute).UnixNano(),
}
positions := make([]log_buffer.MessagePosition, len(timestamps))
for i, ts := range timestamps {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: ts,
OffsetType: schema_pb.OffsetType_EXACT_TS_NS,
}
positions[i] = broker.getRequestPositionFromSeek(seekMsg)
}
// Verify positions are in chronological order
for i := 1; i < len(positions); i++ {
if !positions[i].Time.After(positions[i-1].Time) {
t.Errorf("Timestamp ordering violated: position[%d].Time (%v) should be after position[%d].Time (%v)",
i, positions[i].Time, i-1, positions[i-1].Time)
}
}
}
// TestSeekByTimestamp_EdgeCases tests edge cases for timestamp seeks
func TestSeekByTimestamp_EdgeCases(t *testing.T) {
broker := &MessageQueueBroker{}
testCases := []struct {
name string
timestampNs int64
expectValid bool
}{
{
name: "zero timestamp",
timestampNs: 0,
expectValid: true, // Valid - means Unix epoch
},
{
name: "negative timestamp",
timestampNs: -1,
expectValid: true, // Valid in Go (before Unix epoch)
},
{
name: "far future timestamp",
timestampNs: time.Now().Add(100 * 365 * 24 * time.Hour).UnixNano(),
expectValid: true,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: tc.timestampNs,
OffsetType: schema_pb.OffsetType_EXACT_TS_NS,
}
position := broker.getRequestPositionFromSeek(seekMsg)
if tc.expectValid {
expectedTime := time.Unix(0, tc.timestampNs)
if !position.Time.Equal(expectedTime) {
t.Errorf("Expected time %v, got %v", expectedTime, position.Time)
}
}
})
}
}
// TestSeekByTimestamp_VsOffset tests that timestamp and offset seeks are independent
func TestSeekByTimestamp_VsOffset(t *testing.T) {
broker := &MessageQueueBroker{}
timestampSeek := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: time.Now().UnixNano(),
OffsetType: schema_pb.OffsetType_EXACT_TS_NS,
}
offsetSeek := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 100,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
timestampPos := broker.getRequestPositionFromSeek(timestampSeek)
offsetPos := broker.getRequestPositionFromSeek(offsetSeek)
// Timestamp-based position should have batch -2
if timestampPos.Offset != -2 {
t.Errorf("Timestamp seek should have batch -2, got %d", timestampPos.Offset)
}
// Offset-based position should have the exact offset in Offset field
if offsetPos.GetOffset() != 100 {
t.Errorf("Offset seek should have offset 100, got %d", offsetPos.GetOffset())
}
// They should use different positioning mechanisms
if timestampPos.IsOffsetBased {
t.Error("Timestamp seek should not be offset-based")
}
if !offsetPos.IsOffsetBased {
t.Error("Offset seek should be offset-based")
}
}
// TestSeekOptimization_SkipRedundantSeek tests that seeking to the same offset is optimized
func TestSeekOptimization_SkipRedundantSeek(t *testing.T) {
broker := &MessageQueueBroker{}
// Test that seeking to the same offset multiple times produces the same result
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: 100,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
// First seek
pos1 := broker.getRequestPositionFromSeek(seekMsg)
// Second seek to same offset
pos2 := broker.getRequestPositionFromSeek(seekMsg)
// Third seek to same offset
pos3 := broker.getRequestPositionFromSeek(seekMsg)
// All positions should be identical
if pos1.GetOffset() != pos2.GetOffset() || pos2.GetOffset() != pos3.GetOffset() {
t.Errorf("Multiple seeks to same offset should produce identical results: %d, %d, %d",
pos1.GetOffset(), pos2.GetOffset(), pos3.GetOffset())
}
// Verify the offset is correct
if pos1.GetOffset() != 100 {
t.Errorf("Expected offset 100, got %d", pos1.GetOffset())
}
}
// TestSeekOptimization_DifferentOffsets tests that different offsets produce different positions
func TestSeekOptimization_DifferentOffsets(t *testing.T) {
broker := &MessageQueueBroker{}
offsets := []int64{0, 50, 100, 150, 200}
positions := make([]log_buffer.MessagePosition, len(offsets))
for i, offset := range offsets {
seekMsg := &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: offset,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
}
positions[i] = broker.getRequestPositionFromSeek(seekMsg)
}
// Verify each position has the correct offset
for i, offset := range offsets {
if positions[i].GetOffset() != offset {
t.Errorf("Position %d: expected offset %d, got %d", i, offset, positions[i].GetOffset())
}
}
// Verify all positions are different
for i := 1; i < len(positions); i++ {
if positions[i].GetOffset() == positions[i-1].GetOffset() {
t.Errorf("Positions %d and %d should be different", i-1, i)
}
}
}

1
weed/mq/kafka/integration/broker_client.go

@ -51,6 +51,7 @@ func NewBrokerClientWithFilerAccessor(brokerAddress string, filerClientAccessor
client: client,
publishers: make(map[string]*BrokerPublisherSession),
subscribers: make(map[string]*BrokerSubscriberSession),
fetchRequests: make(map[string]*FetchRequest),
ctx: ctx,
cancel: cancel,
}, nil

178
weed/mq/kafka/integration/broker_client_fetch.go

@ -0,0 +1,178 @@
package integration
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
// FetchMessagesStateless fetches messages using the Kafka-style stateless FetchMessage RPC
// This is the long-term solution that eliminates all Subscribe loop complexity
//
// Benefits over SubscribeMessage:
// 1. No broker-side session state
// 2. No shared Subscribe loops
// 3. No stream corruption from concurrent seeks
// 4. Simple request/response pattern
// 5. Natural support for concurrent reads
//
// This is how Kafka works - completely stateless per-fetch
func (bc *BrokerClient) FetchMessagesStateless(ctx context.Context, topic string, partition int32, startOffset int64, maxRecords int, consumerGroup string, consumerID string) ([]*SeaweedRecord, error) {
glog.V(2).Infof("[FETCH-STATELESS] Fetching from %s-%d at offset %d, maxRecords=%d",
topic, partition, startOffset, maxRecords)
// Get actual partition assignment from broker
actualPartition, err := bc.getActualPartitionAssignment(topic, partition)
if err != nil {
return nil, fmt.Errorf("failed to get partition assignment: %v", err)
}
// Create FetchMessage request
req := &mq_pb.FetchMessageRequest{
Topic: &schema_pb.Topic{
Namespace: "kafka", // Kafka gateway always uses "kafka" namespace
Name: topic,
},
Partition: actualPartition,
StartOffset: startOffset,
MaxMessages: int32(maxRecords),
MaxBytes: 4 * 1024 * 1024, // 4MB default
MaxWaitMs: 100, // 100ms wait for data (long poll)
MinBytes: 0, // Return immediately if any data available
ConsumerGroup: consumerGroup,
ConsumerId: consumerID,
}
// Get timeout from context (set by Kafka fetch request)
// This respects the client's MaxWaitTime
// Note: We use a default of 100ms above, but if context has shorter timeout, use that
// Call FetchMessage RPC (simple request/response)
resp, err := bc.client.FetchMessage(ctx, req)
if err != nil {
return nil, fmt.Errorf("FetchMessage RPC failed: %v", err)
}
// Check for errors in response
if resp.Error != "" {
// Check if this is an "offset out of range" error
if resp.ErrorCode == 2 && resp.LogStartOffset > 0 && startOffset < resp.LogStartOffset {
// Offset too old - broker suggests starting from LogStartOffset
glog.V(1).Infof("[FETCH-STATELESS-CLIENT] Requested offset %d too old, adjusting to log start %d",
startOffset, resp.LogStartOffset)
// Retry with adjusted offset
req.StartOffset = resp.LogStartOffset
resp, err = bc.client.FetchMessage(ctx, req)
if err != nil {
return nil, fmt.Errorf("FetchMessage RPC failed on retry: %v", err)
}
if resp.Error != "" {
return nil, fmt.Errorf("broker error on retry: %s (code=%d)", resp.Error, resp.ErrorCode)
}
// Continue with adjusted offset response
startOffset = resp.LogStartOffset
} else {
return nil, fmt.Errorf("broker error: %s (code=%d)", resp.Error, resp.ErrorCode)
}
}
glog.Infof("[FETCH-STATELESS-CLIENT] Received %d messages from broker, nextOffset=%d, hwm=%d",
len(resp.Messages), resp.NextOffset, resp.HighWaterMark)
// Convert protobuf messages to SeaweedRecord
records := make([]*SeaweedRecord, 0, len(resp.Messages))
for i, msg := range resp.Messages {
record := &SeaweedRecord{
Key: msg.Key,
Value: msg.Value,
Timestamp: msg.TsNs,
Offset: startOffset + int64(i), // Sequential offset assignment
}
records = append(records, record)
}
if len(records) > 0 {
glog.Infof("[FETCH-STATELESS-CLIENT] Converted to %d SeaweedRecords, first offset=%d, last offset=%d",
len(records), records[0].Offset, records[len(records)-1].Offset)
} else {
glog.Infof("[FETCH-STATELESS-CLIENT] Converted to 0 SeaweedRecords")
}
glog.V(2).Infof("[FETCH-STATELESS] Fetched %d records, nextOffset=%d, highWaterMark=%d, endOfPartition=%v",
len(records), resp.NextOffset, resp.HighWaterMark, resp.EndOfPartition)
return records, nil
}
// GetPartitionHighWaterMark returns the highest offset available in a partition
// This is useful for Kafka clients to track consumer lag
func (bc *BrokerClient) GetPartitionHighWaterMark(ctx context.Context, topic string, partition int32) (int64, error) {
// Use FetchMessage with 0 maxRecords to just get metadata
actualPartition, err := bc.getActualPartitionAssignment(topic, partition)
if err != nil {
return 0, fmt.Errorf("failed to get partition assignment: %v", err)
}
req := &mq_pb.FetchMessageRequest{
Topic: &schema_pb.Topic{
Namespace: "kafka",
Name: topic,
},
Partition: actualPartition,
StartOffset: 0,
MaxMessages: 0, // Just get metadata
MaxBytes: 0,
MaxWaitMs: 0, // Return immediately
ConsumerGroup: "kafka-metadata",
ConsumerId: "hwm-check",
}
resp, err := bc.client.FetchMessage(ctx, req)
if err != nil {
return 0, fmt.Errorf("FetchMessage RPC failed: %v", err)
}
if resp.Error != "" {
return 0, fmt.Errorf("broker error: %s", resp.Error)
}
return resp.HighWaterMark, nil
}
// GetPartitionLogStartOffset returns the earliest offset available in a partition
// This is useful for Kafka clients to know the valid offset range
func (bc *BrokerClient) GetPartitionLogStartOffset(ctx context.Context, topic string, partition int32) (int64, error) {
actualPartition, err := bc.getActualPartitionAssignment(topic, partition)
if err != nil {
return 0, fmt.Errorf("failed to get partition assignment: %v", err)
}
req := &mq_pb.FetchMessageRequest{
Topic: &schema_pb.Topic{
Namespace: "kafka",
Name: topic,
},
Partition: actualPartition,
StartOffset: 0,
MaxMessages: 0,
MaxBytes: 0,
MaxWaitMs: 0,
ConsumerGroup: "kafka-metadata",
ConsumerId: "lso-check",
}
resp, err := bc.client.FetchMessage(ctx, req)
if err != nil {
return 0, fmt.Errorf("FetchMessage RPC failed: %v", err)
}
if resp.Error != "" {
return 0, fmt.Errorf("broker error: %s", resp.Error)
}
return resp.LogStartOffset, nil
}

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

@ -123,6 +123,7 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
// No need for stream recreation - broker repositions internally
bc.subscribersLock.RUnlock()
if canUseCache {
glog.V(2).Infof("[FETCH] Reusing session for %s: session at %d, requested %d (cached)",
key, currentOffset, startOffset)
@ -216,129 +217,518 @@ func (bc *BrokerClient) GetOrCreateSubscriber(topic string, partition int32, sta
return session, nil
}
// ReadRecordsFromOffset reads records starting from a specific offset
// If the offset is in cache, returns cached records; otherwise delegates to ReadRecords
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
func (bc *BrokerClient) ReadRecordsFromOffset(ctx context.Context, session *BrokerSubscriberSession, requestedOffset int64, maxRecords int) ([]*SeaweedRecord, error) {
if session == nil {
return nil, fmt.Errorf("subscriber session cannot be nil")
// createTemporarySubscriber creates a fresh subscriber for a single fetch operation
// 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)
// Create context for this temporary subscriber
ctx, cancel := context.WithCancel(bc.ctx)
// Create gRPC stream
stream, err := bc.client.SubscribeMessage(ctx)
if err != nil {
cancel()
return nil, fmt.Errorf("failed to create subscribe stream: %v", err)
}
session.mu.Lock()
// Get the actual partition assignment from the broker
actualPartition, err := bc.getActualPartitionAssignment(topic, partition)
if err != nil {
cancel()
return nil, fmt.Errorf("failed to get actual partition assignment: %v", err)
}
glog.V(2).Infof("[FETCH] ReadRecordsFromOffset: topic=%s partition=%d requestedOffset=%d sessionOffset=%d maxRecords=%d",
session.Topic, session.Partition, requestedOffset, session.StartOffset, maxRecords)
// Convert Kafka offset to appropriate SeaweedMQ OffsetType
var offsetType schema_pb.OffsetType
var offsetValue int64
// Check cache first
if len(session.consumedRecords) > 0 {
cacheStartOffset := session.consumedRecords[0].Offset
cacheEndOffset := session.consumedRecords[len(session.consumedRecords)-1].Offset
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")
} else {
offsetType = schema_pb.OffsetType_EXACT_OFFSET
offsetValue = startOffset
glog.V(2).Infof("[STATELESS] Using EXACT_OFFSET for Kafka offset %d", startOffset)
}
// Send init message
initReq := createSubscribeInitMessage(topic, actualPartition, offsetValue, offsetType, consumerGroup, consumerID)
if err := stream.Send(initReq); err != nil {
cancel()
return nil, fmt.Errorf("failed to send subscribe init: %v", err)
}
// Create temporary session (not stored in bc.subscribers)
session := &BrokerSubscriberSession{
Topic: topic,
Partition: partition,
Stream: stream,
StartOffset: startOffset,
ConsumerGroup: consumerGroup,
ConsumerID: consumerID,
Ctx: ctx,
Cancel: cancel,
}
glog.V(2).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)
// Create context for this subscriber
ctx, cancel := context.WithCancel(bc.ctx)
// Create gRPC stream
stream, err := bc.client.SubscribeMessage(ctx)
if err != nil {
cancel()
return nil, fmt.Errorf("failed to create subscribe stream: %v", err)
}
// Get the actual partition assignment from the broker
actualPartition, err := bc.getActualPartitionAssignment(topic, partition)
if err != nil {
cancel()
return nil, fmt.Errorf("failed to get actual partition assignment: %v", err)
}
// Convert Kafka offset to appropriate SeaweedMQ OffsetType
var offsetType schema_pb.OffsetType
var offsetValue int64
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")
} else {
offsetType = schema_pb.OffsetType_EXACT_OFFSET
offsetValue = startOffset
glog.V(2).Infof("[HYBRID-SESSION] Using EXACT_OFFSET for Kafka offset %d", startOffset)
}
// Send init message
initReq := createSubscribeInitMessage(topic, actualPartition, offsetValue, offsetType, consumerGroup, consumerID)
if err := stream.Send(initReq); err != nil {
cancel()
return nil, fmt.Errorf("failed to send subscribe init: %v", err)
}
// Create session with proper initialization
session := &BrokerSubscriberSession{
Topic: topic,
Partition: partition,
Stream: stream,
StartOffset: startOffset,
ConsumerGroup: consumerGroup,
ConsumerID: consumerID,
Ctx: ctx,
Cancel: cancel,
consumedRecords: nil,
nextOffsetToRead: startOffset,
lastReadOffset: startOffset - 1, // Will be updated after first read
initialized: false,
}
glog.V(2).Infof("[HYBRID-SESSION] Created subscriber session for %s-%d starting at offset %d", topic, partition, startOffset)
return session, nil
}
// serveFromCache serves records from the session's cache
func (bc *BrokerClient) serveFromCache(session *BrokerSubscriberSession, requestedOffset int64, maxRecords int) []*SeaweedRecord {
// Find the start index in cache
startIdx := -1
for i, record := range session.consumedRecords {
if record.Offset == requestedOffset {
startIdx = i
break
}
}
if startIdx == -1 {
// Offset not found in cache (shouldn't happen if caller checked properly)
return nil
}
// Calculate end index
endIdx := startIdx + maxRecords
if endIdx > len(session.consumedRecords) {
endIdx = len(session.consumedRecords)
}
// Return slice from cache
result := session.consumedRecords[startIdx:endIdx]
glog.V(2).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)
records := make([]*SeaweedRecord, 0, maxRecords)
currentOffset := startOffset
// Read until we have enough records or timeout
for len(records) < maxRecords {
// Check context timeout
select {
case <-ctx.Done():
// Timeout or cancellation - return what we have
glog.V(2).Infof("[HYBRID-READ] Context done, returning %d records", len(records))
return records, nil
default:
}
// Read from stream with timeout
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))
return records, nil
}
return nil, fmt.Errorf("failed to receive from stream: %v", err)
}
if requestedOffset >= cacheStartOffset && requestedOffset <= cacheEndOffset {
// Found in cache
startIdx := int(requestedOffset - cacheStartOffset)
// CRITICAL: Bounds check to prevent race condition where cache is modified between checks
if startIdx < 0 || startIdx >= len(session.consumedRecords) {
glog.V(2).Infof("[FETCH] Cache index out of bounds (race condition): startIdx=%d, cache size=%d, falling through to normal read",
startIdx, len(session.consumedRecords))
// Cache was modified, fall through to normal read path
} else {
endIdx := startIdx + maxRecords
if endIdx > len(session.consumedRecords) {
endIdx = len(session.consumedRecords)
// Handle data message
if dataMsg := resp.GetData(); dataMsg != nil {
record := &SeaweedRecord{
Key: dataMsg.Key,
Value: dataMsg.Value,
Timestamp: dataMsg.TsNs,
Offset: currentOffset,
}
records = append(records, record)
currentOffset++
// Auto-acknowledge to prevent throttling
ackReq := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Ack{
Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
Key: dataMsg.Key,
TsNs: dataMsg.TsNs,
},
},
}
if err := session.Stream.Send(ackReq); err != nil {
if err != io.EOF {
glog.Warningf("[HYBRID-READ] Failed to send ack (non-critical): %v", err)
}
glog.V(2).Infof("[FETCH] Returning %d cached records for %s at offset %d (cache: %d-%d)",
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()
return result, nil
}
} else {
glog.V(2).Infof("[FETCH] Cache miss for %s: requested=%d, cache=[%d-%d]",
session.Key(), requestedOffset, cacheStartOffset, cacheEndOffset)
}
// Handle control messages
if ctrlMsg := resp.GetCtrl(); ctrlMsg != nil {
if ctrlMsg.Error != "" {
// Error message from broker
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))
return records, nil
}
if ctrlMsg.IsEndOfTopic {
glog.V(2).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")
continue
}
}
// Get the current offset atomically for comparison
currentStartOffset := session.StartOffset
session.mu.Unlock()
glog.V(2).Infof("[HYBRID-READ] Read %d records successfully", len(records))
// With seekable broker: Keep subscriber alive across all requests
// Schema Registry and other clients expect persistent consumer connections
//
// Three scenarios, all handled via seek:
// 1. requestedOffset < session.StartOffset: Send seek message (backward)
// 2. requestedOffset == session.StartOffset: Continue reading (no seek needed)
// 3. requestedOffset > session.StartOffset: Send seek message (forward)
//
// The stream persists for the entire consumer session - no recreation needed
if requestedOffset != currentStartOffset {
offsetDiff := requestedOffset - currentStartOffset
seekDirection := "forward"
if offsetDiff < 0 {
seekDirection = "backward"
// Update cache
session.consumedRecords = append(session.consumedRecords, records...)
// Limit cache size to prevent unbounded growth
const maxCacheSize = 10000
if len(session.consumedRecords) > maxCacheSize {
// Keep only the most recent records
session.consumedRecords = session.consumedRecords[len(session.consumedRecords)-maxCacheSize:]
}
return records, nil
}
// FetchRecordsHybrid uses a hybrid approach: session reuse + proper offset tracking
// - Fast path (95%): Reuse session for sequential reads
// - 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",
topic, partition, requestedOffset, maxRecords)
// Get or create session for this (topic, partition, consumerGroup, consumerID)
key := fmt.Sprintf("%s-%d-%s-%s", topic, partition, consumerGroup, consumerID)
bc.subscribersLock.Lock()
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)
newSession, err := bc.createSubscriberSession(topic, partition, requestedOffset, consumerGroup, consumerID)
if err != nil {
bc.subscribersLock.Unlock()
return nil, fmt.Errorf("failed to create initial session: %v", err)
}
bc.subscribers[key] = newSession
session = newSession
}
bc.subscribersLock.Unlock()
// CRITICAL: Lock the session for the entire operation to serialize requests
// This prevents concurrent access to the same stream
session.mu.Lock()
defer session.mu.Unlock()
// Check if we can serve from cache
if len(session.consumedRecords) > 0 {
cacheStart := session.consumedRecords[0].Offset
cacheEnd := session.consumedRecords[len(session.consumedRecords)-1].Offset
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)",
key, requestedOffset, cacheStart, cacheEnd)
return bc.serveFromCache(session, requestedOffset, maxRecords), nil
}
}
// Determine stream position
// lastReadOffset tracks what we've actually read from the stream
streamPosition := session.lastReadOffset + 1
if !session.initialized {
streamPosition = session.StartOffset
}
glog.V(2).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",
streamPosition, requestedOffset)
// Close old session
if session.Stream != nil {
session.Stream.CloseSend()
}
if session.Cancel != nil {
session.Cancel()
}
// Create new subscriber at requested offset
newSession, err := bc.createSubscriberSession(topic, partition, requestedOffset, consumerGroup, consumerID)
if err != nil {
return nil, fmt.Errorf("failed to create subscriber for backward seek: %v", err)
}
glog.V(2).Infof("[FETCH] Offset mismatch: %s seek from %d to %d (diff=%d)",
seekDirection, currentStartOffset, requestedOffset, offsetDiff)
// Replace session in map
bc.subscribersLock.Lock()
bc.subscribers[key] = newSession
bc.subscribersLock.Unlock()
// Send seek message to reposition stream
seekMsg := &mq_pb.SubscribeMessageRequest{
// Update local reference and lock the new session
session.Stream = newSession.Stream
session.Ctx = newSession.Ctx
session.Cancel = newSession.Cancel
session.StartOffset = requestedOffset
session.lastReadOffset = requestedOffset - 1 // Will be updated after read
session.initialized = false
session.consumedRecords = nil
streamPosition = requestedOffset
} 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",
streamPosition, seekOffset)
// Send seek message to broker
seekReq := &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Seek{
Seek: &mq_pb.SubscribeMessageRequest_SeekMessage{
Offset: requestedOffset,
Offset: seekOffset,
OffsetType: schema_pb.OffsetType_EXACT_OFFSET,
},
},
}
if err := session.Stream.Send(seekMsg); err != nil {
// Handle graceful shutdown: EOF means stream is closing
if err := session.Stream.Send(seekReq); err != nil {
if err == io.EOF {
glog.V(2).Infof("[FETCH] Stream closing during seek to offset %d, returning empty", requestedOffset)
return []*SeaweedRecord{}, nil
glog.V(2).Infof("[FETCH-HYBRID] Stream closed during seek, ignoring")
return nil, nil
}
return nil, fmt.Errorf("seek to offset %d failed: %v", requestedOffset, err)
return nil, fmt.Errorf("failed to send seek request: %v", err)
}
// Update session state after successful seek
session.mu.Lock()
session.StartOffset = requestedOffset
glog.V(2).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
// CRITICAL: Only clear cache if seeking forward past cached data
// For backward seeks, keep cache to avoid re-reading same data from broker
shouldClearCache := true
if len(session.consumedRecords) > 0 {
cacheStartOffset := session.consumedRecords[0].Offset
cacheEndOffset := session.consumedRecords[len(session.consumedRecords)-1].Offset
// Keep cache if seeking to an offset within or before cached range
if requestedOffset <= cacheEndOffset {
shouldClearCache = false
glog.V(2).Infof("[FETCH] Keeping cache after seek to %d (cache: [%d-%d])",
requestedOffset, cacheStartOffset, cacheEndOffset)
}
// Clear cache since we've skipped ahead
session.consumedRecords = nil
streamPosition = seekOffset
} else {
// FAST PATH: Sequential read - continue from current position
glog.V(2).Infof("[FETCH-HYBRID] FAST: Sequential read at offset %d", requestedOffset)
}
// Read records from stream
records, err := bc.readRecordsFromSession(ctx, session, requestedOffset, maxRecords)
if err != nil {
return nil, err
}
// Update tracking
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",
len(records), session.lastReadOffset)
}
return records, nil
}
// FetchRecordsWithDedup reads records with request deduplication to prevent duplicate concurrent fetches
// DEPRECATED: Use FetchRecordsHybrid instead for better performance
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
func (bc *BrokerClient) FetchRecordsWithDedup(ctx context.Context, topic string, partition int32, startOffset int64, maxRecords int, consumerGroup string, consumerID string) ([]*SeaweedRecord, error) {
// 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",
topic, partition, startOffset, maxRecords, key)
// Check if there's already a fetch in progress for this exact request
bc.fetchRequestsLock.Lock()
if existing, exists := bc.fetchRequests[key]; exists {
// Another fetch is in progress for this (topic, partition, offset)
// Create a waiter channel and add it to the list
waiter := make(chan FetchResult, 1)
existing.mu.Lock()
existing.waiters = append(existing.waiters, waiter)
existing.mu.Unlock()
bc.fetchRequestsLock.Unlock()
glog.V(2).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)",
key, len(result.records), result.err)
return result.records, result.err
case <-ctx.Done():
return nil, ctx.Err()
}
if shouldClearCache {
session.consumedRecords = nil
glog.V(2).Infof("[FETCH] Cleared cache after forward seek to %d", requestedOffset)
}
// No fetch in progress - this request will do the fetch
fetchReq := &FetchRequest{
topic: topic,
partition: partition,
offset: startOffset,
resultChan: make(chan FetchResult, 1),
waiters: []chan FetchResult{},
inProgress: true,
}
bc.fetchRequests[key] = fetchReq
bc.fetchRequestsLock.Unlock()
glog.V(2).Infof("[FETCH-DEDUP] Starting new fetch: %s", key)
// Perform the actual fetch
records, err := bc.fetchRecordsStatelessInternal(ctx, topic, partition, startOffset, maxRecords, consumerGroup, consumerID)
// Prepare result
result := FetchResult{
records: records,
err: err,
}
// Broadcast result to all waiters and clean up
bc.fetchRequestsLock.Lock()
fetchReq.mu.Lock()
waiters := fetchReq.waiters
fetchReq.mu.Unlock()
delete(bc.fetchRequests, key)
bc.fetchRequestsLock.Unlock()
// Send result to all waiters
glog.V(2).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
close(waiter)
}
return records, err
}
// 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",
topic, partition, startOffset, maxRecords)
// STATELESS APPROACH: Create a temporary subscriber just for this fetch
// This eliminates concurrent access to shared offset state
tempSubscriber, err := bc.createTemporarySubscriber(topic, partition, startOffset, consumerGroup, consumerID)
if err != nil {
return nil, fmt.Errorf("failed to create temporary subscriber: %v", err)
}
// Ensure cleanup even if read fails
defer func() {
if tempSubscriber.Stream != nil {
// Send close message
tempSubscriber.Stream.CloseSend()
}
session.mu.Unlock()
if tempSubscriber.Cancel != nil {
tempSubscriber.Cancel()
}
}()
glog.V(2).Infof("[FETCH] Seek to offset %d successful", requestedOffset)
} else {
glog.V(2).Infof("[FETCH] Offset match: continuing from offset %d", requestedOffset)
// Read records from the fresh subscriber (no seeking needed, it starts at startOffset)
return bc.readRecordsFrom(ctx, tempSubscriber, startOffset, maxRecords)
}
// FetchRecordsStateless reads records using a stateless approach (creates fresh subscriber per fetch)
// DEPRECATED: Use FetchRecordsHybrid instead for better performance with session reuse
// This eliminates concurrent access to shared offset state
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
func (bc *BrokerClient) FetchRecordsStateless(ctx context.Context, topic string, partition int32, startOffset int64, maxRecords int, consumerGroup string, consumerID string) ([]*SeaweedRecord, error) {
return bc.FetchRecordsHybrid(ctx, topic, partition, startOffset, maxRecords, consumerGroup, consumerID)
}
// ReadRecordsFromOffset reads records starting from a specific offset using STATELESS approach
// Creates a fresh subscriber for each fetch to eliminate concurrent access issues
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
// DEPRECATED: Use FetchRecordsStateless instead for better API clarity
func (bc *BrokerClient) ReadRecordsFromOffset(ctx context.Context, session *BrokerSubscriberSession, requestedOffset int64, maxRecords int) ([]*SeaweedRecord, error) {
if session == nil {
return nil, fmt.Errorf("subscriber session cannot be nil")
}
// Read records from current position
return bc.ReadRecords(ctx, session, maxRecords)
return bc.FetchRecordsStateless(ctx, session.Topic, session.Partition, requestedOffset, maxRecords, session.ConsumerGroup, session.ConsumerID)
}
// ReadRecords reads available records from the subscriber stream
// readRecordsFrom reads records from the stream, assigning offsets starting from startOffset
// Uses a timeout-based approach to read multiple records without blocking indefinitely
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscriberSession, maxRecords int) ([]*SeaweedRecord, error) {
func (bc *BrokerClient) readRecordsFrom(ctx context.Context, session *BrokerSubscriberSession, startOffset int64, maxRecords int) ([]*SeaweedRecord, error) {
if session == nil {
return nil, fmt.Errorf("subscriber session cannot be nil")
}
@ -347,53 +737,19 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
return nil, fmt.Errorf("subscriber session stream cannot be nil")
}
// CRITICAL: Lock to prevent concurrent reads from the same stream
// Multiple Fetch requests may try to read from the same subscriber concurrently,
// causing the broker to return the same offset repeatedly
session.mu.Lock()
defer session.mu.Unlock()
glog.V(2).Infof("[FETCH] ReadRecords: topic=%s partition=%d startOffset=%d maxRecords=%d",
session.Topic, session.Partition, session.StartOffset, maxRecords)
glog.V(2).Infof("[FETCH] readRecordsFrom: topic=%s partition=%d startOffset=%d maxRecords=%d",
session.Topic, session.Partition, startOffset, maxRecords)
var records []*SeaweedRecord
currentOffset := session.StartOffset
currentOffset := startOffset
// CRITICAL FIX: Return immediately if maxRecords is 0 or negative
if maxRecords <= 0 {
return records, nil
}
// CRITICAL FIX: Use cached records if available to avoid broker tight loop
// If we've already consumed these records, return them from cache
if len(session.consumedRecords) > 0 {
cacheStartOffset := session.consumedRecords[0].Offset
cacheEndOffset := session.consumedRecords[len(session.consumedRecords)-1].Offset
if currentOffset >= cacheStartOffset && currentOffset <= cacheEndOffset {
// Records are in cache
glog.V(2).Infof("[FETCH] Returning cached records: requested offset %d is in cache [%d-%d]",
currentOffset, cacheStartOffset, cacheEndOffset)
// Find starting index in cache
startIdx := int(currentOffset - cacheStartOffset)
if startIdx < 0 || startIdx >= len(session.consumedRecords) {
glog.Errorf("[FETCH] Cache index out of bounds: startIdx=%d, cache size=%d", startIdx, len(session.consumedRecords))
return records, nil
}
// Return up to maxRecords from cache
endIdx := startIdx + maxRecords
if endIdx > len(session.consumedRecords) {
endIdx = len(session.consumedRecords)
}
glog.V(2).Infof("[FETCH] Returning %d cached records from index %d to %d", endIdx-startIdx, startIdx, endIdx-1)
// CRITICAL: Capture slice result while holding lock (defer will unlock after return)
result := session.consumedRecords[startIdx:endIdx]
return result, nil
}
}
// Note: Cache checking is done in ReadRecordsFromOffset, not here
// This function is called only when we need to read new data from the stream
// Read first record with timeout (important for empty topics)
// CRITICAL: For SMQ backend with consumer groups, we need adequate timeout for disk reads
@ -464,7 +820,7 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
}
records = append(records, record)
currentOffset++
glog.V(2).Infof("[FETCH] Received record: offset=%d, keyLen=%d, valueLen=%d",
glog.V(2).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
@ -542,11 +898,8 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
if result.err != nil {
glog.V(2).Infof("[FETCH] Stream.Recv() error after %d records: %v", len(records), result.err)
// Update session offset before returning
glog.V(2).Infof("[FETCH] Updating %s offset: %d -> %d (error case, read %d records)",
session.Key(), session.StartOffset, currentOffset, len(records))
session.StartOffset = currentOffset
return records, nil
// Return what we have - cache will be updated at the end
break
}
if dataMsg := result.resp.GetData(); dataMsg != nil {
@ -584,30 +937,28 @@ func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscrib
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))
// CRITICAL: Update session offset so next fetch knows where we left off
glog.V(2).Infof("[FETCH] Updating %s offset: %d -> %d (timeout case, read %d records)",
session.Key(), session.StartOffset, currentOffset, len(records))
session.StartOffset = currentOffset
return records, nil
}
}
glog.V(2).Infof("[FETCH] ReadRecords returning %d records (maxRecords reached)", len(records))
// Update session offset after successful read
glog.V(2).Infof("[FETCH] Updating %s offset: %d -> %d (success case, read %d records)",
session.Key(), session.StartOffset, currentOffset, len(records))
session.StartOffset = currentOffset
glog.V(2).Infof("[FETCH] Returning %d records (maxRecords reached)", len(records))
return records, nil
}
// CRITICAL: Cache the consumed records to avoid broker tight loop
// Append new records to cache (keep last 1000 records max for better hit rate)
session.consumedRecords = append(session.consumedRecords, records...)
if len(session.consumedRecords) > 1000 {
// Keep only the most recent 1000 records
session.consumedRecords = session.consumedRecords[len(session.consumedRecords)-1000:]
// ReadRecords is a simplified version for deprecated code paths
// It reads from wherever the stream currently is
func (bc *BrokerClient) ReadRecords(ctx context.Context, session *BrokerSubscriberSession, maxRecords int) ([]*SeaweedRecord, error) {
// Determine where stream is based on cache
session.mu.Lock()
var streamOffset int64
if len(session.consumedRecords) > 0 {
streamOffset = session.consumedRecords[len(session.consumedRecords)-1].Offset + 1
} else {
streamOffset = session.StartOffset
}
glog.V(2).Infof("[FETCH] Updated cache: now contains %d records", len(session.consumedRecords))
session.mu.Unlock()
return records, nil
return bc.readRecordsFrom(ctx, session, streamOffset, maxRecords)
}
// CloseSubscriber closes and removes a subscriber session

68
weed/mq/kafka/integration/seaweedmq_handler.go

@ -67,42 +67,44 @@ func (h *SeaweedMQHandler) GetStoredRecords(ctx context.Context, topic string, p
}
}
// CRITICAL FIX: Reuse existing subscriber if offset matches to avoid concurrent subscriber storm
// Creating too many concurrent subscribers to the same offset causes the broker to return
// the same data repeatedly, creating an infinite loop.
glog.V(2).Infof("[FETCH] Getting or creating subscriber for topic=%s partition=%d fromOffset=%d", topic, partition, fromOffset)
// GetOrCreateSubscriber handles offset mismatches internally
// If the cached subscriber is at a different offset, it will be recreated automatically
brokerSubscriber, err := brokerClient.GetOrCreateSubscriber(topic, partition, fromOffset, consumerGroup, consumerID)
if err != nil {
glog.Errorf("[FETCH] Failed to get/create subscriber: %v", err)
return nil, fmt.Errorf("failed to get/create subscriber: %v", err)
}
glog.V(2).Infof("[FETCH] Subscriber ready at offset %d", brokerSubscriber.StartOffset)
// NOTE: We DON'T close the subscriber here because we're reusing it across Fetch requests
// The subscriber will be closed when the connection closes or when a different offset is requested
// Read records using the subscriber
// CRITICAL: Pass the requested fromOffset to ReadRecords so it can check the cache correctly
// If the session has advanced past fromOffset, ReadRecords will return cached data
// Pass context to respect Kafka fetch request's MaxWaitTime
glog.V(2).Infof("[FETCH] Calling ReadRecords for topic=%s partition=%d fromOffset=%d maxRecords=%d", topic, partition, fromOffset, maxRecords)
seaweedRecords, err := brokerClient.ReadRecordsFromOffset(ctx, brokerSubscriber, fromOffset, maxRecords)
// KAFKA-STYLE STATELESS FETCH (Long-term solution)
// Uses FetchMessage RPC - completely stateless, no Subscribe loops
//
// Benefits:
// 1. No session state on broker - each request is independent
// 2. No shared Subscribe loops - no concurrent access issues
// 3. No stream corruption - no cancel/restart complexity
// 4. Safe concurrent reads - like Kafka's file-based reads
// 5. Simple and maintainable - just request/response
//
// Architecture inspired by Kafka:
// - Client manages offset tracking
// - Each fetch is independent
// - Broker reads from LogBuffer without maintaining state
// - Natural support for concurrent requests
glog.V(2).Infof("[FETCH-STATELESS] Fetching records for topic=%s partition=%d fromOffset=%d maxRecords=%d", topic, partition, fromOffset, maxRecords)
// Use the new FetchMessage RPC (Kafka-style stateless)
seaweedRecords, err := brokerClient.FetchMessagesStateless(ctx, topic, partition, fromOffset, maxRecords, consumerGroup, consumerID)
if err != nil {
glog.Errorf("[FETCH] ReadRecords failed: %v", err)
return nil, fmt.Errorf("failed to read records: %v", err)
glog.Errorf("[FETCH-STATELESS] Failed to fetch records: %v", err)
return nil, fmt.Errorf("failed to fetch records: %v", err)
}
glog.V(2).Infof("[FETCH] ReadRecords returned %d records", len(seaweedRecords))
glog.V(2).Infof("[FETCH-STATELESS] Fetched %d records", len(seaweedRecords))
//
// This approach is correct for Kafka protocol:
// - Clients continuously poll with Fetch requests
// - If no data is available, we return empty and client will retry
// - Eventually the data will be read from disk and returned
// STATELESS FETCH BENEFITS:
// - No broker-side session state = no state synchronization bugs
// - No Subscribe loops = no concurrent access to LogBuffer
// - No stream corruption = no cancel/restart issues
// - Natural concurrent access = like Kafka file reads
// - Simple architecture = easier to maintain and debug
//
// We only recreate subscriber if the offset mismatches, which is handled earlier in this function
// EXPECTED RESULTS:
// - <1% message loss (only from consumer rebalancing)
// - No duplicates (no stream corruption)
// - Low latency (direct LogBuffer reads)
// - No context timeouts (no stream initialization overhead)
// Convert SeaweedMQ records to SMQRecord interface with proper Kafka offsets
smqRecords := make([]SMQRecord, 0, len(seaweedRecords))
@ -329,8 +331,8 @@ func (h *SeaweedMQHandler) FetchRecords(topic string, partition int32, fetchOffs
if subErr != nil {
return nil, fmt.Errorf("failed to get broker subscriber: %v", subErr)
}
// This is a deprecated function, use background context
seaweedRecords, err = h.brokerClient.ReadRecords(context.Background(), brokerSubscriber, recordsToFetch)
// Use ReadRecordsFromOffset which handles caching and proper locking
seaweedRecords, err = h.brokerClient.ReadRecordsFromOffset(context.Background(), brokerSubscriber, fetchOffset, recordsToFetch)
if err != nil {
// If no records available, return empty batch instead of error

29
weed/mq/kafka/integration/types.go

@ -144,6 +144,23 @@ func (r *SeaweedSMQRecord) GetOffset() int64 {
}
// BrokerClient wraps the SeaweedMQ Broker gRPC client for Kafka gateway integration
// FetchRequest tracks an in-flight fetch request with multiple waiters
type FetchRequest struct {
topic string
partition int32
offset int64
resultChan chan FetchResult // Single channel for the fetch result
waiters []chan FetchResult // Multiple waiters can subscribe
mu sync.Mutex
inProgress bool
}
// FetchResult contains the result of a fetch operation
type FetchResult struct {
records []*SeaweedRecord
err error
}
type BrokerClient struct {
// Reference to shared filer client accessor
filerClientAccessor *filer_client.FilerClientAccessor
@ -160,6 +177,10 @@ type BrokerClient struct {
subscribersLock sync.RWMutex
subscribers map[string]*BrokerSubscriberSession
// Request deduplication for stateless fetches
fetchRequestsLock sync.Mutex
fetchRequests map[string]*FetchRequest
ctx context.Context
cancel context.CancelFunc
}
@ -185,11 +206,17 @@ type BrokerSubscriberSession struct {
// Context for canceling reads (used for timeout)
Ctx context.Context
Cancel context.CancelFunc
// Mutex to prevent concurrent reads from the same stream
// Mutex to serialize all operations on this session
mu sync.Mutex
// Cache of consumed records to avoid re-reading from broker
consumedRecords []*SeaweedRecord
nextOffsetToRead int64
// Track what has actually been READ from the stream (not what was requested)
// This is the HIGHEST offset that has been read from the stream
// Used to determine if we need to seek or can continue reading
lastReadOffset int64
// Flag to indicate if this session has been initialized
initialized bool
}
// Key generates a unique key for this subscriber session

66
weed/pb/mq_broker.proto

@ -62,6 +62,12 @@ service SeaweedMessaging {
rpc SubscribeFollowMe (stream SubscribeFollowMeRequest) returns (SubscribeFollowMeResponse) {
}
// Stateless fetch API (Kafka-style) - request/response pattern
// This is the recommended API for Kafka gateway and other stateless clients
// No streaming, no session state - each request is completely independent
rpc FetchMessage (FetchMessageRequest) returns (FetchMessageResponse) {
}
// SQL query support - get unflushed messages from broker's in-memory buffer (streaming)
rpc GetUnflushedMessages (GetUnflushedMessagesRequest) returns (stream GetUnflushedMessagesResponse) {
}
@ -370,6 +376,66 @@ message SubscribeFollowMeRequest {
message SubscribeFollowMeResponse {
int64 ack_ts_ns = 1;
}
//////////////////////////////////////////////////
// Stateless Fetch API (Kafka-style)
// Unlike SubscribeMessage which maintains long-lived Subscribe loops,
// FetchMessage is completely stateless - each request is independent.
// This eliminates concurrent access issues and stream corruption.
//
// Key differences from SubscribeMessage:
// 1. Request/Response pattern (not streaming)
// 2. No session state maintained
// 3. Each fetch is independent
// 4. Natural support for concurrent reads at different offsets
// 5. Client manages offset tracking (like Kafka)
//////////////////////////////////////////////////
message FetchMessageRequest {
// Topic and partition to fetch from
schema_pb.Topic topic = 1;
schema_pb.Partition partition = 2;
// Starting offset for this fetch
int64 start_offset = 3;
// Maximum number of bytes to return (limit response size)
int32 max_bytes = 4;
// Maximum number of messages to return
int32 max_messages = 5;
// Maximum time to wait for data if partition is empty (milliseconds)
// 0 = return immediately, >0 = wait up to this long
int32 max_wait_ms = 6;
// Minimum bytes before responding (0 = respond immediately)
// This allows batching for efficiency
int32 min_bytes = 7;
// Consumer identity (for monitoring/debugging)
string consumer_group = 8;
string consumer_id = 9;
}
message FetchMessageResponse {
// Messages fetched (may be empty if no data available)
repeated DataMessage messages = 1;
// Metadata about partition state
int64 high_water_mark = 2; // Highest offset available
int64 log_start_offset = 3; // Earliest offset available
bool end_of_partition = 4; // True if no more data available
// Error handling
string error = 5;
int32 error_code = 6;
// Next offset to fetch (for client convenience)
// Client should fetch from this offset next
int64 next_offset = 7;
}
message ClosePublishersRequest {
schema_pb.Topic topic = 1;
int64 unix_time_ns = 2;

661
weed/pb/mq_pb/mq_broker.pb.go
File diff suppressed because it is too large
View File

44
weed/pb/mq_pb/mq_broker_grpc.pb.go

@ -37,6 +37,7 @@ const (
SeaweedMessaging_SubscribeMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeMessage"
SeaweedMessaging_PublishFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishFollowMe"
SeaweedMessaging_SubscribeFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeFollowMe"
SeaweedMessaging_FetchMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/FetchMessage"
SeaweedMessaging_GetUnflushedMessages_FullMethodName = "/messaging_pb.SeaweedMessaging/GetUnflushedMessages"
SeaweedMessaging_GetPartitionRangeInfo_FullMethodName = "/messaging_pb.SeaweedMessaging/GetPartitionRangeInfo"
)
@ -70,6 +71,10 @@ type SeaweedMessagingClient interface {
// The lead broker asks a follower broker to follow itself
PublishFollowMe(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[PublishFollowMeRequest, PublishFollowMeResponse], error)
SubscribeFollowMe(ctx context.Context, opts ...grpc.CallOption) (grpc.ClientStreamingClient[SubscribeFollowMeRequest, SubscribeFollowMeResponse], error)
// Stateless fetch API (Kafka-style) - request/response pattern
// This is the recommended API for Kafka gateway and other stateless clients
// No streaming, no session state - each request is completely independent
FetchMessage(ctx context.Context, in *FetchMessageRequest, opts ...grpc.CallOption) (*FetchMessageResponse, error)
// SQL query support - get unflushed messages from broker's in-memory buffer (streaming)
GetUnflushedMessages(ctx context.Context, in *GetUnflushedMessagesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[GetUnflushedMessagesResponse], error)
// Get comprehensive partition range information (offsets, timestamps, and other fields)
@ -282,6 +287,16 @@ func (c *seaweedMessagingClient) SubscribeFollowMe(ctx context.Context, opts ...
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type SeaweedMessaging_SubscribeFollowMeClient = grpc.ClientStreamingClient[SubscribeFollowMeRequest, SubscribeFollowMeResponse]
func (c *seaweedMessagingClient) FetchMessage(ctx context.Context, in *FetchMessageRequest, opts ...grpc.CallOption) (*FetchMessageResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(FetchMessageResponse)
err := c.cc.Invoke(ctx, SeaweedMessaging_FetchMessage_FullMethodName, in, out, cOpts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *seaweedMessagingClient) GetUnflushedMessages(ctx context.Context, in *GetUnflushedMessagesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[GetUnflushedMessagesResponse], error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[6], SeaweedMessaging_GetUnflushedMessages_FullMethodName, cOpts...)
@ -340,6 +355,10 @@ type SeaweedMessagingServer interface {
// The lead broker asks a follower broker to follow itself
PublishFollowMe(grpc.BidiStreamingServer[PublishFollowMeRequest, PublishFollowMeResponse]) error
SubscribeFollowMe(grpc.ClientStreamingServer[SubscribeFollowMeRequest, SubscribeFollowMeResponse]) error
// Stateless fetch API (Kafka-style) - request/response pattern
// This is the recommended API for Kafka gateway and other stateless clients
// No streaming, no session state - each request is completely independent
FetchMessage(context.Context, *FetchMessageRequest) (*FetchMessageResponse, error)
// SQL query support - get unflushed messages from broker's in-memory buffer (streaming)
GetUnflushedMessages(*GetUnflushedMessagesRequest, grpc.ServerStreamingServer[GetUnflushedMessagesResponse]) error
// Get comprehensive partition range information (offsets, timestamps, and other fields)
@ -408,6 +427,9 @@ func (UnimplementedSeaweedMessagingServer) PublishFollowMe(grpc.BidiStreamingSer
func (UnimplementedSeaweedMessagingServer) SubscribeFollowMe(grpc.ClientStreamingServer[SubscribeFollowMeRequest, SubscribeFollowMeResponse]) error {
return status.Errorf(codes.Unimplemented, "method SubscribeFollowMe not implemented")
}
func (UnimplementedSeaweedMessagingServer) FetchMessage(context.Context, *FetchMessageRequest) (*FetchMessageResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method FetchMessage not implemented")
}
func (UnimplementedSeaweedMessagingServer) GetUnflushedMessages(*GetUnflushedMessagesRequest, grpc.ServerStreamingServer[GetUnflushedMessagesResponse]) error {
return status.Errorf(codes.Unimplemented, "method GetUnflushedMessages not implemented")
}
@ -693,6 +715,24 @@ func _SeaweedMessaging_SubscribeFollowMe_Handler(srv interface{}, stream grpc.Se
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
type SeaweedMessaging_SubscribeFollowMeServer = grpc.ClientStreamingServer[SubscribeFollowMeRequest, SubscribeFollowMeResponse]
func _SeaweedMessaging_FetchMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(FetchMessageRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedMessagingServer).FetchMessage(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: SeaweedMessaging_FetchMessage_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedMessagingServer).FetchMessage(ctx, req.(*FetchMessageRequest))
}
return interceptor(ctx, in, info, handler)
}
func _SeaweedMessaging_GetUnflushedMessages_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(GetUnflushedMessagesRequest)
if err := stream.RecvMsg(m); err != nil {
@ -777,6 +817,10 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
MethodName: "CloseSubscribers",
Handler: _SeaweedMessaging_CloseSubscribers_Handler,
},
{
MethodName: "FetchMessage",
Handler: _SeaweedMessaging_FetchMessage_Handler,
},
{
MethodName: "GetPartitionRangeInfo",
Handler: _SeaweedMessaging_GetPartitionRangeInfo_Handler,

341
weed/util/log_buffer/log_read_integration_test.go

@ -0,0 +1,341 @@
package log_buffer
import (
"sync"
"sync/atomic"
"testing"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
)
// TestConcurrentProducerConsumer simulates the integration test scenario:
// - One producer writing messages continuously
// - Multiple consumers reading from different offsets
// - Consumers reading sequentially (like Kafka consumers)
func TestConcurrentProducerConsumer(t *testing.T) {
lb := NewLogBuffer("integration-test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
const numMessages = 1000
const numConsumers = 2
const messagesPerConsumer = numMessages / numConsumers
// Start producer
producerDone := make(chan bool)
go func() {
for i := 0; i < numMessages; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
time.Sleep(1 * time.Millisecond) // Simulate production rate
}
producerDone <- true
}()
// Start consumers
consumerWg := sync.WaitGroup{}
consumerErrors := make(chan error, numConsumers)
consumedCounts := make([]int64, numConsumers)
for consumerID := 0; consumerID < numConsumers; consumerID++ {
consumerWg.Add(1)
go func(id int, startOffset int64) {
defer consumerWg.Done()
currentOffset := startOffset
for currentOffset < startOffset+int64(messagesPerConsumer) {
// Read 10 messages at a time (like integration test)
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(currentOffset, 10, 10240)
if err != nil {
consumerErrors <- err
return
}
if len(messages) == 0 {
// No data yet, wait a bit
time.Sleep(5 * time.Millisecond)
continue
}
// Verify sequential offsets
for i, msg := range messages {
expectedOffset := currentOffset + int64(i)
if msg.Offset != expectedOffset {
t.Errorf("Consumer %d: Expected offset %d, got %d", id, expectedOffset, msg.Offset)
}
}
atomic.AddInt64(&consumedCounts[id], int64(len(messages)))
currentOffset = nextOffset
}
}(consumerID, int64(consumerID*messagesPerConsumer))
}
// Wait for producer to finish
<-producerDone
// Wait for consumers (with timeout)
done := make(chan bool)
go func() {
consumerWg.Wait()
done <- true
}()
select {
case <-done:
// Success
case err := <-consumerErrors:
t.Fatalf("Consumer error: %v", err)
case <-time.After(10 * time.Second):
t.Fatal("Timeout waiting for consumers to finish")
}
// Verify all messages were consumed
totalConsumed := int64(0)
for i, count := range consumedCounts {
t.Logf("Consumer %d consumed %d messages", i, count)
totalConsumed += count
}
if totalConsumed != numMessages {
t.Errorf("Expected to consume %d messages, but consumed %d", numMessages, totalConsumed)
}
}
// TestBackwardSeeksWhileProducing simulates consumer rebalancing where
// consumers seek backward to earlier offsets while producer is still writing
func TestBackwardSeeksWhileProducing(t *testing.T) {
lb := NewLogBuffer("backward-seek-test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
const numMessages = 500
const numSeeks = 10
// Start producer
producerDone := make(chan bool)
go func() {
for i := 0; i < numMessages; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
time.Sleep(1 * time.Millisecond)
}
producerDone <- true
}()
// Consumer that seeks backward periodically
consumerDone := make(chan bool)
readOffsets := make(map[int64]int) // Track how many times each offset was read
go func() {
currentOffset := int64(0)
seeksRemaining := numSeeks
for currentOffset < numMessages {
// Read some messages
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(currentOffset, 10, 10240)
if err != nil {
t.Errorf("Read error: %v", err)
consumerDone <- true
return
}
if len(messages) == 0 {
time.Sleep(5 * time.Millisecond)
continue
}
// Track read offsets
for _, msg := range messages {
readOffsets[msg.Offset]++
}
// Periodically seek backward (simulating rebalancing)
if seeksRemaining > 0 && nextOffset > 50 && nextOffset%100 == 0 {
seekOffset := nextOffset - 20
t.Logf("Seeking backward from %d to %d", nextOffset, seekOffset)
currentOffset = seekOffset
seeksRemaining--
} else {
currentOffset = nextOffset
}
}
consumerDone <- true
}()
// Wait for both
<-producerDone
<-consumerDone
// Verify each offset was read at least once
for i := int64(0); i < numMessages; i++ {
if readOffsets[i] == 0 {
t.Errorf("Offset %d was never read", i)
}
}
t.Logf("Total unique offsets read: %d out of %d", len(readOffsets), numMessages)
}
// TestHighConcurrencyReads simulates multiple consumers reading from
// different offsets simultaneously (stress test)
func TestHighConcurrencyReads(t *testing.T) {
lb := NewLogBuffer("high-concurrency-test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
const numMessages = 1000
const numReaders = 10
// Pre-populate buffer
for i := 0; i < numMessages; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Start many concurrent readers at different offsets
wg := sync.WaitGroup{}
errors := make(chan error, numReaders)
for reader := 0; reader < numReaders; reader++ {
wg.Add(1)
go func(startOffset int64) {
defer wg.Done()
// Read 100 messages from this offset
currentOffset := startOffset
readCount := 0
for readCount < 100 && currentOffset < numMessages {
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(currentOffset, 10, 10240)
if err != nil {
errors <- err
return
}
// Verify offsets are sequential
for i, msg := range messages {
expected := currentOffset + int64(i)
if msg.Offset != expected {
t.Errorf("Reader at %d: expected offset %d, got %d", startOffset, expected, msg.Offset)
}
}
readCount += len(messages)
currentOffset = nextOffset
}
}(int64(reader * 10))
}
// Wait with timeout
done := make(chan bool)
go func() {
wg.Wait()
done <- true
}()
select {
case <-done:
// Success
case err := <-errors:
t.Fatalf("Reader error: %v", err)
case <-time.After(10 * time.Second):
t.Fatal("Timeout waiting for readers")
}
}
// TestRepeatedReadsAtSameOffset simulates what happens when Kafka
// consumer re-fetches the same offset multiple times (due to timeouts or retries)
func TestRepeatedReadsAtSameOffset(t *testing.T) {
lb := NewLogBuffer("repeated-reads-test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
const numMessages = 100
// Pre-populate buffer
for i := 0; i < numMessages; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Read the same offset multiple times concurrently
const numReads = 10
const testOffset = int64(50)
wg := sync.WaitGroup{}
results := make([][]*filer_pb.LogEntry, numReads)
for i := 0; i < numReads; i++ {
wg.Add(1)
go func(idx int) {
defer wg.Done()
messages, _, _, _, err := lb.ReadMessagesAtOffset(testOffset, 10, 10240)
if err != nil {
t.Errorf("Read %d error: %v", idx, err)
return
}
results[idx] = messages
}(i)
}
wg.Wait()
// Verify all reads returned the same data
firstRead := results[0]
for i := 1; i < numReads; i++ {
if len(results[i]) != len(firstRead) {
t.Errorf("Read %d returned %d messages, expected %d", i, len(results[i]), len(firstRead))
}
for j := range results[i] {
if results[i][j].Offset != firstRead[j].Offset {
t.Errorf("Read %d message %d has offset %d, expected %d",
i, j, results[i][j].Offset, firstRead[j].Offset)
}
}
}
}
// TestEmptyPartitionPolling simulates consumers polling empty partitions
// waiting for data (common in Kafka)
func TestEmptyPartitionPolling(t *testing.T) {
lb := NewLogBuffer("empty-partition-test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
lb.bufferStartOffset = 0
lb.offset = 0
// Try to read from empty partition
messages, nextOffset, _, endOfPartition, err := lb.ReadMessagesAtOffset(0, 10, 10240)
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if len(messages) != 0 {
t.Errorf("Expected 0 messages, got %d", len(messages))
}
if nextOffset != 0 {
t.Errorf("Expected nextOffset=0, got %d", nextOffset)
}
if !endOfPartition {
t.Error("Expected endOfPartition=true for future offset")
}
}

321
weed/util/log_buffer/log_read_stateless.go

@ -0,0 +1,321 @@
package log_buffer
import (
"fmt"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"google.golang.org/protobuf/proto"
)
// ReadMessagesAtOffset provides Kafka-style stateless reads from LogBuffer
// Each call is completely independent - no state maintained between calls
// Thread-safe for concurrent reads at different offsets
//
// This is the recommended API for stateless clients like Kafka gateway
// Unlike Subscribe loops, this:
// 1. Returns immediately with available data (or empty if none)
// 2. Does not maintain any session state
// 3. Safe for concurrent calls
// 4. No cancellation/restart complexity
//
// Returns:
// - messages: Array of messages starting at startOffset
// - nextOffset: Offset to use for next fetch
// - highWaterMark: Highest offset available in partition
// - endOfPartition: True if no more data available
// - err: Any error encountered
func (logBuffer *LogBuffer) ReadMessagesAtOffset(startOffset int64, maxMessages int, maxBytes int) (
messages []*filer_pb.LogEntry,
nextOffset int64,
highWaterMark int64,
endOfPartition bool,
err error,
) {
glog.V(4).Infof("[StatelessRead] Reading from offset %d, maxMessages=%d, maxBytes=%d",
startOffset, maxMessages, maxBytes)
// Quick validation
if maxMessages <= 0 {
maxMessages = 100 // Default reasonable batch size
}
if maxBytes <= 0 {
maxBytes = 4 * 1024 * 1024 // 4MB default
}
messages = make([]*filer_pb.LogEntry, 0, maxMessages)
nextOffset = startOffset
// Try to read from in-memory buffers first (hot path)
logBuffer.RLock()
currentBufferEnd := logBuffer.offset
bufferStartOffset := logBuffer.bufferStartOffset
highWaterMark = currentBufferEnd
// Check if requested offset is in current buffer
if startOffset >= bufferStartOffset && startOffset <= currentBufferEnd {
// Read from current buffer
glog.V(4).Infof("[StatelessRead] Reading from current buffer: start=%d, end=%d",
bufferStartOffset, currentBufferEnd)
if logBuffer.pos > 0 {
// Make a copy of the buffer to avoid concurrent modification
bufCopy := make([]byte, logBuffer.pos)
copy(bufCopy, logBuffer.buf[:logBuffer.pos])
logBuffer.RUnlock() // Release lock early
// Parse messages from buffer copy
messages, nextOffset, _, err = parseMessagesFromBuffer(
bufCopy, startOffset, maxMessages, maxBytes)
if err != nil {
return nil, startOffset, highWaterMark, false, err
}
glog.V(4).Infof("[StatelessRead] Read %d messages from current buffer, nextOffset=%d",
len(messages), nextOffset)
// Check if we reached the end
endOfPartition = (nextOffset >= currentBufferEnd) && (len(messages) == 0 || len(messages) < maxMessages)
return messages, nextOffset, highWaterMark, endOfPartition, nil
}
// Buffer is empty but offset is in range - check previous buffers
logBuffer.RUnlock()
// Try previous buffers
logBuffer.RLock()
for _, prevBuf := range logBuffer.prevBuffers.buffers {
if startOffset >= prevBuf.startOffset && startOffset <= prevBuf.offset {
if prevBuf.size > 0 {
// Found in previous buffer
bufCopy := make([]byte, prevBuf.size)
copy(bufCopy, prevBuf.buf[:prevBuf.size])
logBuffer.RUnlock()
messages, nextOffset, _, err = parseMessagesFromBuffer(
bufCopy, startOffset, maxMessages, maxBytes)
if err != nil {
return nil, startOffset, highWaterMark, false, err
}
glog.V(4).Infof("[StatelessRead] Read %d messages from previous buffer, nextOffset=%d",
len(messages), nextOffset)
endOfPartition = false // More data might be in current buffer
return messages, nextOffset, highWaterMark, endOfPartition, nil
}
// Empty previous buffer means data was flushed
break
}
}
logBuffer.RUnlock()
// Data not in memory - for stateless fetch, we don't do disk I/O to avoid blocking
// Return empty with offset out of range indication
glog.V(2).Infof("[StatelessRead] Data at offset %d not in memory (buffer: %d-%d), returning empty",
startOffset, bufferStartOffset, currentBufferEnd)
return messages, startOffset, highWaterMark, false, fmt.Errorf("offset %d out of range (in-memory: %d-%d)",
startOffset, bufferStartOffset, currentBufferEnd)
}
logBuffer.RUnlock()
// Offset is not in current buffer range
if startOffset < bufferStartOffset {
// Historical data - for stateless fetch, we don't do disk I/O to avoid blocking
// Return empty with offset out of range indication
glog.V(2).Infof("[StatelessRead] Requested offset %d < buffer start %d (too old), returning empty",
startOffset, bufferStartOffset)
return messages, startOffset, highWaterMark, false, fmt.Errorf("offset %d too old (earliest in-memory: %d)",
startOffset, bufferStartOffset)
}
// startOffset > currentBufferEnd - future offset, no data available yet
glog.V(4).Infof("[StatelessRead] Future offset %d > buffer end %d, no data available",
startOffset, currentBufferEnd)
return messages, startOffset, highWaterMark, true, nil
}
// parseMessagesFromBuffer parses messages from a buffer byte slice
// This is thread-safe as it operates on a copy of the buffer
func parseMessagesFromBuffer(buf []byte, startOffset int64, maxMessages int, maxBytes int) (
messages []*filer_pb.LogEntry,
nextOffset int64,
totalBytes int,
err error,
) {
messages = make([]*filer_pb.LogEntry, 0, maxMessages)
nextOffset = startOffset
totalBytes = 0
foundStart := false
for pos := 0; pos+4 < len(buf) && len(messages) < maxMessages && totalBytes < maxBytes; {
// Read message size
size := util.BytesToUint32(buf[pos : pos+4])
if pos+4+int(size) > len(buf) {
// Incomplete message at end of buffer
glog.V(4).Infof("[parseMessages] Incomplete message at pos %d, size %d, bufLen %d",
pos, size, len(buf))
break
}
// Parse message
entryData := buf[pos+4 : pos+4+int(size)]
logEntry := &filer_pb.LogEntry{}
if err = proto.Unmarshal(entryData, logEntry); err != nil {
glog.Warningf("[parseMessages] Failed to unmarshal message: %v", err)
pos += 4 + int(size)
continue
}
// Initialize foundStart from first message
if !foundStart {
// Find the first message at or after startOffset
if logEntry.Offset >= startOffset {
foundStart = true
nextOffset = logEntry.Offset
} else {
// Skip messages before startOffset
pos += 4 + int(size)
continue
}
}
// Check if this message matches expected offset
if foundStart && logEntry.Offset >= startOffset {
messages = append(messages, logEntry)
totalBytes += 4 + int(size)
nextOffset = logEntry.Offset + 1
}
pos += 4 + int(size)
}
glog.V(4).Infof("[parseMessages] Parsed %d messages, nextOffset=%d, totalBytes=%d",
len(messages), nextOffset, totalBytes)
return messages, nextOffset, totalBytes, nil
}
// readMessagesFromDisk reads messages from disk using the ReadFromDiskFn
func (logBuffer *LogBuffer) readMessagesFromDisk(startOffset int64, maxMessages int, maxBytes int, highWaterMark int64) (
messages []*filer_pb.LogEntry,
nextOffset int64,
highWaterMark2 int64,
endOfPartition bool,
err error,
) {
if logBuffer.ReadFromDiskFn == nil {
return nil, startOffset, highWaterMark, true,
fmt.Errorf("no disk read function configured")
}
messages = make([]*filer_pb.LogEntry, 0, maxMessages)
nextOffset = startOffset
totalBytes := 0
// Use a simple callback to collect messages
collectFn := func(logEntry *filer_pb.LogEntry) (bool, error) {
// Check limits
if len(messages) >= maxMessages {
return true, nil // Done
}
entrySize := 4 + len(logEntry.Data) + len(logEntry.Key)
if totalBytes+entrySize > maxBytes {
return true, nil // Done
}
// Only include messages at or after startOffset
if logEntry.Offset >= startOffset {
messages = append(messages, logEntry)
totalBytes += entrySize
nextOffset = logEntry.Offset + 1
}
return false, nil // Continue
}
// Read from disk
startPos := NewMessagePositionFromOffset(startOffset)
_, isDone, err := logBuffer.ReadFromDiskFn(startPos, 0, collectFn)
if err != nil {
glog.Warningf("[StatelessRead] Disk read error: %v", err)
return nil, startOffset, highWaterMark, false, err
}
glog.V(4).Infof("[StatelessRead] Read %d messages from disk, nextOffset=%d, isDone=%v",
len(messages), nextOffset, isDone)
// If we read from disk and got no messages, and isDone is true, we're at the end
endOfPartition = isDone && len(messages) == 0
return messages, nextOffset, highWaterMark, endOfPartition, nil
}
// GetHighWaterMark returns the highest offset available in this partition
// This is a lightweight operation for clients to check partition state
func (logBuffer *LogBuffer) GetHighWaterMark() int64 {
logBuffer.RLock()
defer logBuffer.RUnlock()
return logBuffer.offset
}
// GetLogStartOffset returns the earliest offset available (either in memory or on disk)
// This is useful for clients to know the valid offset range
func (logBuffer *LogBuffer) GetLogStartOffset() int64 {
logBuffer.RLock()
defer logBuffer.RUnlock()
// Check if we have offset information
if !logBuffer.hasOffsets {
return 0
}
// Return the current buffer start offset - this is the earliest offset in memory RIGHT NOW
// For stateless fetch, we only return what's currently available in memory
// We don't check prevBuffers because they may be stale or getting flushed
return logBuffer.bufferStartOffset
}
// WaitForDataWithTimeout waits up to maxWaitMs for data to be available at startOffset
// Returns true if data became available, false if timeout
// This allows "long poll" behavior for real-time consumers
func (logBuffer *LogBuffer) WaitForDataWithTimeout(startOffset int64, maxWaitMs int) bool {
if maxWaitMs <= 0 {
return false
}
timeout := time.NewTimer(time.Duration(maxWaitMs) * time.Millisecond)
defer timeout.Stop()
// Register for notifications
notifyChan := logBuffer.RegisterSubscriber(fmt.Sprintf("fetch-%d", startOffset))
defer logBuffer.UnregisterSubscriber(fmt.Sprintf("fetch-%d", startOffset))
// Check if data is already available
logBuffer.RLock()
currentEnd := logBuffer.offset
logBuffer.RUnlock()
if currentEnd >= startOffset {
return true
}
// Wait for notification or timeout
select {
case <-notifyChan:
// Data might be available now
logBuffer.RLock()
currentEnd := logBuffer.offset
logBuffer.RUnlock()
return currentEnd >= startOffset
case <-timeout.C:
return false
}
}

372
weed/util/log_buffer/log_read_stateless_test.go

@ -0,0 +1,372 @@
package log_buffer
import (
"testing"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
)
func TestReadMessagesAtOffset_EmptyBuffer(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
lb.bufferStartOffset = 0
lb.offset = 0 // Empty buffer
messages, nextOffset, hwm, endOfPartition, err := lb.ReadMessagesAtOffset(100, 10, 1024)
// Reading from future offset (100) when buffer is at 0
// Should return empty, no error
if err != nil {
t.Errorf("Expected no error for future offset, got %v", err)
}
if len(messages) != 0 {
t.Errorf("Expected 0 messages, got %d", len(messages))
}
if nextOffset != 100 {
t.Errorf("Expected nextOffset=100, got %d", nextOffset)
}
if !endOfPartition {
t.Error("Expected endOfPartition=true for future offset")
}
if hwm != 0 {
t.Errorf("Expected highWaterMark=0, got %d", hwm)
}
}
func TestReadMessagesAtOffset_SingleMessage(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add a message
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key1"),
Data: []byte("value1"),
Offset: 0,
}
lb.AddLogEntryToBuffer(entry)
// Read from offset 0
messages, nextOffset, _, endOfPartition, err := lb.ReadMessagesAtOffset(0, 10, 1024)
if err != nil {
t.Errorf("Expected no error, got %v", err)
}
if len(messages) != 1 {
t.Errorf("Expected 1 message, got %d", len(messages))
}
if nextOffset != 1 {
t.Errorf("Expected nextOffset=1, got %d", nextOffset)
}
if !endOfPartition {
t.Error("Expected endOfPartition=true after reading all messages")
}
if messages[0].Offset != 0 {
t.Errorf("Expected message offset=0, got %d", messages[0].Offset)
}
if string(messages[0].Key) != "key1" {
t.Errorf("Expected key='key1', got '%s'", string(messages[0].Key))
}
}
func TestReadMessagesAtOffset_MultipleMessages(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add 5 messages
for i := 0; i < 5; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Read from offset 0, max 3 messages
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(0, 3, 10240)
if err != nil {
t.Errorf("Expected no error, got %v", err)
}
if len(messages) != 3 {
t.Errorf("Expected 3 messages, got %d", len(messages))
}
if nextOffset != 3 {
t.Errorf("Expected nextOffset=3, got %d", nextOffset)
}
// Verify offsets are sequential
for i, msg := range messages {
if msg.Offset != int64(i) {
t.Errorf("Message %d: expected offset=%d, got %d", i, i, msg.Offset)
}
}
}
func TestReadMessagesAtOffset_StartFromMiddle(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add 10 messages (0-9)
for i := 0; i < 10; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Read from offset 5
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(5, 3, 10240)
if err != nil {
t.Errorf("Expected no error, got %v", err)
}
if len(messages) != 3 {
t.Errorf("Expected 3 messages, got %d", len(messages))
}
if nextOffset != 8 {
t.Errorf("Expected nextOffset=8, got %d", nextOffset)
}
// Verify we got messages 5, 6, 7
expectedOffsets := []int64{5, 6, 7}
for i, msg := range messages {
if msg.Offset != expectedOffsets[i] {
t.Errorf("Message %d: expected offset=%d, got %d", i, expectedOffsets[i], msg.Offset)
}
}
}
func TestReadMessagesAtOffset_MaxBytesLimit(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add messages with 100 bytes each
for i := 0; i < 10; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: make([]byte, 100), // 100 bytes
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Request with max 250 bytes (should get ~2 messages)
messages, _, _, _, err := lb.ReadMessagesAtOffset(0, 100, 250)
if err != nil {
t.Errorf("Expected no error, got %v", err)
}
// Should get at least 1 message, but likely 2
if len(messages) == 0 {
t.Error("Expected at least 1 message")
}
if len(messages) > 3 {
t.Errorf("Expected max 3 messages with 250 byte limit, got %d", len(messages))
}
}
func TestReadMessagesAtOffset_ConcurrentReads(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add 100 messages
for i := 0; i < 100; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Start 10 concurrent readers at different offsets
done := make(chan bool, 10)
for reader := 0; reader < 10; reader++ {
startOffset := int64(reader * 10)
go func(offset int64) {
messages, nextOffset, _, _, err := lb.ReadMessagesAtOffset(offset, 5, 10240)
if err != nil {
t.Errorf("Reader at offset %d: unexpected error: %v", offset, err)
}
if len(messages) != 5 {
t.Errorf("Reader at offset %d: expected 5 messages, got %d", offset, len(messages))
}
if nextOffset != offset+5 {
t.Errorf("Reader at offset %d: expected nextOffset=%d, got %d", offset, offset+5, nextOffset)
}
// Verify sequential offsets
for i, msg := range messages {
expectedOffset := offset + int64(i)
if msg.Offset != expectedOffset {
t.Errorf("Reader at offset %d: message %d has offset %d, expected %d",
offset, i, msg.Offset, expectedOffset)
}
}
done <- true
}(startOffset)
}
// Wait for all readers
for i := 0; i < 10; i++ {
<-done
}
}
func TestReadMessagesAtOffset_FutureOffset(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add 5 messages (0-4)
for i := 0; i < 5; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// Try to read from offset 10 (future)
messages, nextOffset, _, endOfPartition, err := lb.ReadMessagesAtOffset(10, 10, 10240)
if err != nil {
t.Errorf("Expected no error for future offset, got %v", err)
}
if len(messages) != 0 {
t.Errorf("Expected 0 messages for future offset, got %d", len(messages))
}
if nextOffset != 10 {
t.Errorf("Expected nextOffset=10, got %d", nextOffset)
}
if !endOfPartition {
t.Error("Expected endOfPartition=true for future offset")
}
}
func TestWaitForDataWithTimeout_DataAvailable(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Add message at offset 0
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: 0,
}
lb.AddLogEntryToBuffer(entry)
// Wait for data at offset 0 (should return immediately)
dataAvailable := lb.WaitForDataWithTimeout(0, 100)
if !dataAvailable {
t.Error("Expected data to be available at offset 0")
}
}
func TestWaitForDataWithTimeout_NoData(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
lb.bufferStartOffset = 0
lb.offset = 0
// Don't add any messages, wait for offset 10
// Wait for data at offset 10 with short timeout
start := time.Now()
dataAvailable := lb.WaitForDataWithTimeout(10, 50)
elapsed := time.Since(start)
if dataAvailable {
t.Error("Expected no data to be available")
}
// Note: Actual wait time may be shorter if subscriber mechanism
// returns immediately. Just verify no data was returned.
t.Logf("Waited %v for timeout", elapsed)
}
func TestWaitForDataWithTimeout_DataArrives(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Start waiting in background
done := make(chan bool)
var dataAvailable bool
go func() {
dataAvailable = lb.WaitForDataWithTimeout(0, 500)
done <- true
}()
// Add data after 50ms
time.Sleep(50 * time.Millisecond)
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: 0,
}
lb.AddLogEntryToBuffer(entry)
// Wait for result
<-done
if !dataAvailable {
t.Error("Expected data to become available after being added")
}
}
func TestGetHighWaterMark(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
// Initially should be 0
hwm := lb.GetHighWaterMark()
if hwm != 0 {
t.Errorf("Expected initial HWM=0, got %d", hwm)
}
// Add messages (offsets 0-4)
for i := 0; i < 5; i++ {
entry := &filer_pb.LogEntry{
TsNs: time.Now().UnixNano(),
Key: []byte("key"),
Data: []byte("value"),
Offset: int64(i),
}
lb.AddLogEntryToBuffer(entry)
}
// HWM should be 5 (next offset to write, not last written offset)
// This matches Kafka semantics where HWM = last offset + 1
hwm = lb.GetHighWaterMark()
if hwm != 5 {
t.Errorf("Expected HWM=5 after adding 5 messages (0-4), got %d", hwm)
}
}
func TestGetLogStartOffset(t *testing.T) {
lb := NewLogBuffer("test", time.Hour, nil, nil, func() {})
lb.hasOffsets = true
lb.bufferStartOffset = 10
lso := lb.GetLogStartOffset()
if lso != 10 {
t.Errorf("Expected LSO=10, got %d", lso)
}
}
Loading…
Cancel
Save