17 changed files with 2553 additions and 1268 deletions
-
2test/kafka/kafka-client-loadtest/config/loadtest.yaml
-
2test/kafka/kafka-client-loadtest/docker-compose.yml
-
10test/kafka/kafka-client-loadtest/internal/consumer/consumer.go
-
43test/kafka/kafka-client-loadtest/test-no-schema.sh
-
160weed/mq/broker/broker_grpc_fetch.go
-
872weed/mq/broker/broker_grpc_sub_seek_test.go
-
1weed/mq/kafka/integration/broker_client.go
-
178weed/mq/kafka/integration/broker_client_fetch.go
-
651weed/mq/kafka/integration/broker_client_subscribe.go
-
68weed/mq/kafka/integration/seaweedmq_handler.go
-
29weed/mq/kafka/integration/types.go
-
66weed/pb/mq_broker.proto
-
661weed/pb/mq_pb/mq_broker.pb.go
-
44weed/pb/mq_pb/mq_broker_grpc.pb.go
-
341weed/util/log_buffer/log_read_integration_test.go
-
321weed/util/log_buffer/log_read_stateless.go
-
372weed/util/log_buffer/log_read_stateless_test.go
@ -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 "═══════════════════════════════════════════════════════" |
@ -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 |
|||
} |
@ -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) |
|||
} |
|||
} |
|||
} |
@ -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 |
|||
} |
661
weed/pb/mq_pb/mq_broker.pb.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -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") |
|||
} |
|||
} |
@ -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 |
|||
} |
|||
} |
@ -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) |
|||
} |
|||
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue