From ba1a8aed64826b2eed4e3e72b3d08f553fdb3c93 Mon Sep 17 00:00:00 2001 From: chrislu Date: Wed, 15 Oct 2025 16:08:21 -0700 Subject: [PATCH] log read stateless --- .../config/loadtest.yaml | 2 +- .../kafka-client-loadtest/docker-compose.yml | 2 +- .../internal/consumer/consumer.go | 10 +- .../kafka-client-loadtest/test-no-schema.sh | 43 + weed/mq/broker/broker_grpc_fetch.go | 160 ++++ weed/mq/broker/broker_grpc_sub_seek_test.go | 872 ------------------ weed/mq/kafka/integration/broker_client.go | 1 + .../kafka/integration/broker_client_fetch.go | 178 ++++ .../integration/broker_client_subscribe.go | 651 ++++++++++--- .../mq/kafka/integration/seaweedmq_handler.go | 68 +- weed/mq/kafka/integration/types.go | 29 +- weed/pb/mq_broker.proto | 66 ++ weed/pb/mq_pb/mq_broker.pb.go | 661 ++++++++----- weed/pb/mq_pb/mq_broker_grpc.pb.go | 44 + .../log_buffer/log_read_integration_test.go | 341 +++++++ weed/util/log_buffer/log_read_stateless.go | 321 +++++++ .../log_buffer/log_read_stateless_test.go | 372 ++++++++ 17 files changed, 2553 insertions(+), 1268 deletions(-) create mode 100755 test/kafka/kafka-client-loadtest/test-no-schema.sh create mode 100644 weed/mq/broker/broker_grpc_fetch.go delete mode 100644 weed/mq/broker/broker_grpc_sub_seek_test.go create mode 100644 weed/mq/kafka/integration/broker_client_fetch.go create mode 100644 weed/util/log_buffer/log_read_integration_test.go create mode 100644 weed/util/log_buffer/log_read_stateless.go create mode 100644 weed/util/log_buffer/log_read_stateless_test.go diff --git a/test/kafka/kafka-client-loadtest/config/loadtest.yaml b/test/kafka/kafka-client-loadtest/config/loadtest.yaml index 6a453aab9..35c6ef399 100644 --- a/test/kafka/kafka-client-loadtest/config/loadtest.yaml +++ b/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 diff --git a/test/kafka/kafka-client-loadtest/docker-compose.yml b/test/kafka/kafka-client-loadtest/docker-compose.yml index 54b49ecd2..26dca83f7 100644 --- a/test/kafka/kafka-client-loadtest/docker-compose.yml +++ b/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 diff --git a/test/kafka/kafka-client-loadtest/internal/consumer/consumer.go b/test/kafka/kafka-client-loadtest/internal/consumer/consumer.go index 6c03288b9..273dc9c77 100644 --- a/test/kafka/kafka-client-loadtest/internal/consumer/consumer.go +++ b/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 } } diff --git a/test/kafka/kafka-client-loadtest/test-no-schema.sh b/test/kafka/kafka-client-loadtest/test-no-schema.sh new file mode 100755 index 000000000..6c852cf8d --- /dev/null +++ b/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 "═══════════════════════════════════════════════════════" diff --git a/weed/mq/broker/broker_grpc_fetch.go b/weed/mq/broker/broker_grpc_fetch.go new file mode 100644 index 000000000..d06cc160f --- /dev/null +++ b/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 +} diff --git a/weed/mq/broker/broker_grpc_sub_seek_test.go b/weed/mq/broker/broker_grpc_sub_seek_test.go deleted file mode 100644 index adf3df68c..000000000 --- a/weed/mq/broker/broker_grpc_sub_seek_test.go +++ /dev/null @@ -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) - } - } -} diff --git a/weed/mq/kafka/integration/broker_client.go b/weed/mq/kafka/integration/broker_client.go index f4db2a7c6..4030ab0e2 100644 --- a/weed/mq/kafka/integration/broker_client.go +++ b/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 diff --git a/weed/mq/kafka/integration/broker_client_fetch.go b/weed/mq/kafka/integration/broker_client_fetch.go new file mode 100644 index 000000000..933db1492 --- /dev/null +++ b/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 +} diff --git a/weed/mq/kafka/integration/broker_client_subscribe.go b/weed/mq/kafka/integration/broker_client_subscribe.go index 4b0abe7a0..d9eaec1f0 100644 --- a/weed/mq/kafka/integration/broker_client_subscribe.go +++ b/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 diff --git a/weed/mq/kafka/integration/seaweedmq_handler.go b/weed/mq/kafka/integration/seaweedmq_handler.go index 554acb9f3..0cb2e47bc 100644 --- a/weed/mq/kafka/integration/seaweedmq_handler.go +++ b/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 diff --git a/weed/mq/kafka/integration/types.go b/weed/mq/kafka/integration/types.go index 764006e9d..ac4ae428f 100644 --- a/weed/mq/kafka/integration/types.go +++ b/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 diff --git a/weed/pb/mq_broker.proto b/weed/pb/mq_broker.proto index 205e17aba..47e4aaa8c 100644 --- a/weed/pb/mq_broker.proto +++ b/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; diff --git a/weed/pb/mq_pb/mq_broker.pb.go b/weed/pb/mq_pb/mq_broker.pb.go index 49be07f50..7e7f706cb 100644 --- a/weed/pb/mq_pb/mq_broker.pb.go +++ b/weed/pb/mq_pb/mq_broker.pb.go @@ -2566,6 +2566,220 @@ func (x *SubscribeFollowMeResponse) GetAckTsNs() int64 { return 0 } +type FetchMessageRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Topic and partition to fetch from + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + // Starting offset for this fetch + StartOffset int64 `protobuf:"varint,3,opt,name=start_offset,json=startOffset,proto3" json:"start_offset,omitempty"` + // Maximum number of bytes to return (limit response size) + MaxBytes int32 `protobuf:"varint,4,opt,name=max_bytes,json=maxBytes,proto3" json:"max_bytes,omitempty"` + // Maximum number of messages to return + MaxMessages int32 `protobuf:"varint,5,opt,name=max_messages,json=maxMessages,proto3" json:"max_messages,omitempty"` + // Maximum time to wait for data if partition is empty (milliseconds) + // 0 = return immediately, >0 = wait up to this long + MaxWaitMs int32 `protobuf:"varint,6,opt,name=max_wait_ms,json=maxWaitMs,proto3" json:"max_wait_ms,omitempty"` + // Minimum bytes before responding (0 = respond immediately) + // This allows batching for efficiency + MinBytes int32 `protobuf:"varint,7,opt,name=min_bytes,json=minBytes,proto3" json:"min_bytes,omitempty"` + // Consumer identity (for monitoring/debugging) + ConsumerGroup string `protobuf:"bytes,8,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerId string `protobuf:"bytes,9,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FetchMessageRequest) Reset() { + *x = FetchMessageRequest{} + mi := &file_mq_broker_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FetchMessageRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FetchMessageRequest) ProtoMessage() {} + +func (x *FetchMessageRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_broker_proto_msgTypes[40] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FetchMessageRequest.ProtoReflect.Descriptor instead. +func (*FetchMessageRequest) Descriptor() ([]byte, []int) { + return file_mq_broker_proto_rawDescGZIP(), []int{40} +} + +func (x *FetchMessageRequest) GetTopic() *schema_pb.Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *FetchMessageRequest) GetPartition() *schema_pb.Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *FetchMessageRequest) GetStartOffset() int64 { + if x != nil { + return x.StartOffset + } + return 0 +} + +func (x *FetchMessageRequest) GetMaxBytes() int32 { + if x != nil { + return x.MaxBytes + } + return 0 +} + +func (x *FetchMessageRequest) GetMaxMessages() int32 { + if x != nil { + return x.MaxMessages + } + return 0 +} + +func (x *FetchMessageRequest) GetMaxWaitMs() int32 { + if x != nil { + return x.MaxWaitMs + } + return 0 +} + +func (x *FetchMessageRequest) GetMinBytes() int32 { + if x != nil { + return x.MinBytes + } + return 0 +} + +func (x *FetchMessageRequest) GetConsumerGroup() string { + if x != nil { + return x.ConsumerGroup + } + return "" +} + +func (x *FetchMessageRequest) GetConsumerId() string { + if x != nil { + return x.ConsumerId + } + return "" +} + +type FetchMessageResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Messages fetched (may be empty if no data available) + Messages []*DataMessage `protobuf:"bytes,1,rep,name=messages,proto3" json:"messages,omitempty"` + // Metadata about partition state + HighWaterMark int64 `protobuf:"varint,2,opt,name=high_water_mark,json=highWaterMark,proto3" json:"high_water_mark,omitempty"` // Highest offset available + LogStartOffset int64 `protobuf:"varint,3,opt,name=log_start_offset,json=logStartOffset,proto3" json:"log_start_offset,omitempty"` // Earliest offset available + EndOfPartition bool `protobuf:"varint,4,opt,name=end_of_partition,json=endOfPartition,proto3" json:"end_of_partition,omitempty"` // True if no more data available + // Error handling + Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` + ErrorCode int32 `protobuf:"varint,6,opt,name=error_code,json=errorCode,proto3" json:"error_code,omitempty"` + // Next offset to fetch (for client convenience) + // Client should fetch from this offset next + NextOffset int64 `protobuf:"varint,7,opt,name=next_offset,json=nextOffset,proto3" json:"next_offset,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FetchMessageResponse) Reset() { + *x = FetchMessageResponse{} + mi := &file_mq_broker_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FetchMessageResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FetchMessageResponse) ProtoMessage() {} + +func (x *FetchMessageResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_broker_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FetchMessageResponse.ProtoReflect.Descriptor instead. +func (*FetchMessageResponse) Descriptor() ([]byte, []int) { + return file_mq_broker_proto_rawDescGZIP(), []int{41} +} + +func (x *FetchMessageResponse) GetMessages() []*DataMessage { + if x != nil { + return x.Messages + } + return nil +} + +func (x *FetchMessageResponse) GetHighWaterMark() int64 { + if x != nil { + return x.HighWaterMark + } + return 0 +} + +func (x *FetchMessageResponse) GetLogStartOffset() int64 { + if x != nil { + return x.LogStartOffset + } + return 0 +} + +func (x *FetchMessageResponse) GetEndOfPartition() bool { + if x != nil { + return x.EndOfPartition + } + return false +} + +func (x *FetchMessageResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +func (x *FetchMessageResponse) GetErrorCode() int32 { + if x != nil { + return x.ErrorCode + } + return 0 +} + +func (x *FetchMessageResponse) GetNextOffset() int64 { + if x != nil { + return x.NextOffset + } + return 0 +} + type ClosePublishersRequest struct { state protoimpl.MessageState `protogen:"open.v1"` Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` @@ -2576,7 +2790,7 @@ type ClosePublishersRequest struct { func (x *ClosePublishersRequest) Reset() { *x = ClosePublishersRequest{} - mi := &file_mq_broker_proto_msgTypes[40] + mi := &file_mq_broker_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2588,7 +2802,7 @@ func (x *ClosePublishersRequest) String() string { func (*ClosePublishersRequest) ProtoMessage() {} func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[40] + mi := &file_mq_broker_proto_msgTypes[42] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2601,7 +2815,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{40} + return file_mq_broker_proto_rawDescGZIP(), []int{42} } func (x *ClosePublishersRequest) GetTopic() *schema_pb.Topic { @@ -2626,7 +2840,7 @@ type ClosePublishersResponse struct { func (x *ClosePublishersResponse) Reset() { *x = ClosePublishersResponse{} - mi := &file_mq_broker_proto_msgTypes[41] + mi := &file_mq_broker_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2638,7 +2852,7 @@ func (x *ClosePublishersResponse) String() string { func (*ClosePublishersResponse) ProtoMessage() {} func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[41] + mi := &file_mq_broker_proto_msgTypes[43] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2651,7 +2865,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{41} + return file_mq_broker_proto_rawDescGZIP(), []int{43} } type CloseSubscribersRequest struct { @@ -2664,7 +2878,7 @@ type CloseSubscribersRequest struct { func (x *CloseSubscribersRequest) Reset() { *x = CloseSubscribersRequest{} - mi := &file_mq_broker_proto_msgTypes[42] + mi := &file_mq_broker_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2676,7 +2890,7 @@ func (x *CloseSubscribersRequest) String() string { func (*CloseSubscribersRequest) ProtoMessage() {} func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[42] + mi := &file_mq_broker_proto_msgTypes[44] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2689,7 +2903,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{42} + return file_mq_broker_proto_rawDescGZIP(), []int{44} } func (x *CloseSubscribersRequest) GetTopic() *schema_pb.Topic { @@ -2714,7 +2928,7 @@ type CloseSubscribersResponse struct { func (x *CloseSubscribersResponse) Reset() { *x = CloseSubscribersResponse{} - mi := &file_mq_broker_proto_msgTypes[43] + mi := &file_mq_broker_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2726,7 +2940,7 @@ func (x *CloseSubscribersResponse) String() string { func (*CloseSubscribersResponse) ProtoMessage() {} func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[43] + mi := &file_mq_broker_proto_msgTypes[45] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2739,7 +2953,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{43} + return file_mq_broker_proto_rawDescGZIP(), []int{45} } type GetUnflushedMessagesRequest struct { @@ -2753,7 +2967,7 @@ type GetUnflushedMessagesRequest struct { func (x *GetUnflushedMessagesRequest) Reset() { *x = GetUnflushedMessagesRequest{} - mi := &file_mq_broker_proto_msgTypes[44] + mi := &file_mq_broker_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2765,7 +2979,7 @@ func (x *GetUnflushedMessagesRequest) String() string { func (*GetUnflushedMessagesRequest) ProtoMessage() {} func (x *GetUnflushedMessagesRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[44] + mi := &file_mq_broker_proto_msgTypes[46] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2778,7 +2992,7 @@ func (x *GetUnflushedMessagesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetUnflushedMessagesRequest.ProtoReflect.Descriptor instead. func (*GetUnflushedMessagesRequest) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{44} + return file_mq_broker_proto_rawDescGZIP(), []int{46} } func (x *GetUnflushedMessagesRequest) GetTopic() *schema_pb.Topic { @@ -2813,7 +3027,7 @@ type GetUnflushedMessagesResponse struct { func (x *GetUnflushedMessagesResponse) Reset() { *x = GetUnflushedMessagesResponse{} - mi := &file_mq_broker_proto_msgTypes[45] + mi := &file_mq_broker_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2825,7 +3039,7 @@ func (x *GetUnflushedMessagesResponse) String() string { func (*GetUnflushedMessagesResponse) ProtoMessage() {} func (x *GetUnflushedMessagesResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[45] + mi := &file_mq_broker_proto_msgTypes[47] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2838,7 +3052,7 @@ func (x *GetUnflushedMessagesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetUnflushedMessagesResponse.ProtoReflect.Descriptor instead. func (*GetUnflushedMessagesResponse) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{45} + return file_mq_broker_proto_rawDescGZIP(), []int{47} } func (x *GetUnflushedMessagesResponse) GetMessage() *filer_pb.LogEntry { @@ -2872,7 +3086,7 @@ type GetPartitionRangeInfoRequest struct { func (x *GetPartitionRangeInfoRequest) Reset() { *x = GetPartitionRangeInfoRequest{} - mi := &file_mq_broker_proto_msgTypes[46] + mi := &file_mq_broker_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2884,7 +3098,7 @@ func (x *GetPartitionRangeInfoRequest) String() string { func (*GetPartitionRangeInfoRequest) ProtoMessage() {} func (x *GetPartitionRangeInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[46] + mi := &file_mq_broker_proto_msgTypes[48] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2897,7 +3111,7 @@ func (x *GetPartitionRangeInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetPartitionRangeInfoRequest.ProtoReflect.Descriptor instead. func (*GetPartitionRangeInfoRequest) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{46} + return file_mq_broker_proto_rawDescGZIP(), []int{48} } func (x *GetPartitionRangeInfoRequest) GetTopic() *schema_pb.Topic { @@ -2930,7 +3144,7 @@ type GetPartitionRangeInfoResponse struct { func (x *GetPartitionRangeInfoResponse) Reset() { *x = GetPartitionRangeInfoResponse{} - mi := &file_mq_broker_proto_msgTypes[47] + mi := &file_mq_broker_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2942,7 +3156,7 @@ func (x *GetPartitionRangeInfoResponse) String() string { func (*GetPartitionRangeInfoResponse) ProtoMessage() {} func (x *GetPartitionRangeInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[47] + mi := &file_mq_broker_proto_msgTypes[49] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2955,7 +3169,7 @@ func (x *GetPartitionRangeInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetPartitionRangeInfoResponse.ProtoReflect.Descriptor instead. func (*GetPartitionRangeInfoResponse) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{47} + return file_mq_broker_proto_rawDescGZIP(), []int{49} } func (x *GetPartitionRangeInfoResponse) GetOffsetRange() *OffsetRangeInfo { @@ -3004,7 +3218,7 @@ type OffsetRangeInfo struct { func (x *OffsetRangeInfo) Reset() { *x = OffsetRangeInfo{} - mi := &file_mq_broker_proto_msgTypes[48] + mi := &file_mq_broker_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3016,7 +3230,7 @@ func (x *OffsetRangeInfo) String() string { func (*OffsetRangeInfo) ProtoMessage() {} func (x *OffsetRangeInfo) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[48] + mi := &file_mq_broker_proto_msgTypes[50] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3029,7 +3243,7 @@ func (x *OffsetRangeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use OffsetRangeInfo.ProtoReflect.Descriptor instead. func (*OffsetRangeInfo) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{48} + return file_mq_broker_proto_rawDescGZIP(), []int{50} } func (x *OffsetRangeInfo) GetEarliestOffset() int64 { @@ -3063,7 +3277,7 @@ type TimestampRangeInfo struct { func (x *TimestampRangeInfo) Reset() { *x = TimestampRangeInfo{} - mi := &file_mq_broker_proto_msgTypes[49] + mi := &file_mq_broker_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3075,7 +3289,7 @@ func (x *TimestampRangeInfo) String() string { func (*TimestampRangeInfo) ProtoMessage() {} func (x *TimestampRangeInfo) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[49] + mi := &file_mq_broker_proto_msgTypes[51] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3088,7 +3302,7 @@ func (x *TimestampRangeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use TimestampRangeInfo.ProtoReflect.Descriptor instead. func (*TimestampRangeInfo) Descriptor() ([]byte, []int) { - return file_mq_broker_proto_rawDescGZIP(), []int{49} + return file_mq_broker_proto_rawDescGZIP(), []int{51} } func (x *TimestampRangeInfo) GetEarliestTimestampNs() int64 { @@ -3114,7 +3328,7 @@ type PublisherToPubBalancerRequest_InitMessage struct { func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { *x = PublisherToPubBalancerRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[51] + mi := &file_mq_broker_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3126,7 +3340,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string { func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[51] + mi := &file_mq_broker_proto_msgTypes[53] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3170,7 +3384,7 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[52] + mi := &file_mq_broker_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3182,7 +3396,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[52] + mi := &file_mq_broker_proto_msgTypes[54] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3242,7 +3456,7 @@ type SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage struct { func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{} - mi := &file_mq_broker_proto_msgTypes[53] + mi := &file_mq_broker_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3254,7 +3468,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) String() stri func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[53] + mi := &file_mq_broker_proto_msgTypes[55] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3286,7 +3500,7 @@ type SubscriberToSubCoordinatorRequest_AckAssignmentMessage struct { func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckAssignmentMessage{} - mi := &file_mq_broker_proto_msgTypes[54] + mi := &file_mq_broker_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3298,7 +3512,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) String() string func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[54] + mi := &file_mq_broker_proto_msgTypes[56] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3330,7 +3544,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct { func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} - mi := &file_mq_broker_proto_msgTypes[55] + mi := &file_mq_broker_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3342,7 +3556,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[55] + mi := &file_mq_broker_proto_msgTypes[57] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3374,7 +3588,7 @@ type SubscriberToSubCoordinatorResponse_UnAssignment struct { func (x *SubscriberToSubCoordinatorResponse_UnAssignment) Reset() { *x = SubscriberToSubCoordinatorResponse_UnAssignment{} - mi := &file_mq_broker_proto_msgTypes[56] + mi := &file_mq_broker_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3386,7 +3600,7 @@ func (x *SubscriberToSubCoordinatorResponse_UnAssignment) String() string { func (*SubscriberToSubCoordinatorResponse_UnAssignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[56] + mi := &file_mq_broker_proto_msgTypes[58] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3422,7 +3636,7 @@ type PublishMessageRequest_InitMessage struct { func (x *PublishMessageRequest_InitMessage) Reset() { *x = PublishMessageRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[57] + mi := &file_mq_broker_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3434,7 +3648,7 @@ func (x *PublishMessageRequest_InitMessage) String() string { func (*PublishMessageRequest_InitMessage) ProtoMessage() {} func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[57] + mi := &file_mq_broker_proto_msgTypes[59] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3495,7 +3709,7 @@ type PublishFollowMeRequest_InitMessage struct { func (x *PublishFollowMeRequest_InitMessage) Reset() { *x = PublishFollowMeRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[58] + mi := &file_mq_broker_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3507,7 +3721,7 @@ func (x *PublishFollowMeRequest_InitMessage) String() string { func (*PublishFollowMeRequest_InitMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[58] + mi := &file_mq_broker_proto_msgTypes[60] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3546,7 +3760,7 @@ type PublishFollowMeRequest_FlushMessage struct { func (x *PublishFollowMeRequest_FlushMessage) Reset() { *x = PublishFollowMeRequest_FlushMessage{} - mi := &file_mq_broker_proto_msgTypes[59] + mi := &file_mq_broker_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3558,7 +3772,7 @@ func (x *PublishFollowMeRequest_FlushMessage) String() string { func (*PublishFollowMeRequest_FlushMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[59] + mi := &file_mq_broker_proto_msgTypes[61] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3589,7 +3803,7 @@ type PublishFollowMeRequest_CloseMessage struct { func (x *PublishFollowMeRequest_CloseMessage) Reset() { *x = PublishFollowMeRequest_CloseMessage{} - mi := &file_mq_broker_proto_msgTypes[60] + mi := &file_mq_broker_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3601,7 +3815,7 @@ func (x *PublishFollowMeRequest_CloseMessage) String() string { func (*PublishFollowMeRequest_CloseMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[60] + mi := &file_mq_broker_proto_msgTypes[62] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3634,7 +3848,7 @@ type SubscribeMessageRequest_InitMessage struct { func (x *SubscribeMessageRequest_InitMessage) Reset() { *x = SubscribeMessageRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[61] + mi := &file_mq_broker_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3646,7 +3860,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string { func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[61] + mi := &file_mq_broker_proto_msgTypes[63] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3735,7 +3949,7 @@ type SubscribeMessageRequest_AckMessage struct { func (x *SubscribeMessageRequest_AckMessage) Reset() { *x = SubscribeMessageRequest_AckMessage{} - mi := &file_mq_broker_proto_msgTypes[62] + mi := &file_mq_broker_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3747,7 +3961,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string { func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[62] + mi := &file_mq_broker_proto_msgTypes[64] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3787,7 +4001,7 @@ type SubscribeMessageRequest_SeekMessage struct { func (x *SubscribeMessageRequest_SeekMessage) Reset() { *x = SubscribeMessageRequest_SeekMessage{} - mi := &file_mq_broker_proto_msgTypes[63] + mi := &file_mq_broker_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3799,7 +4013,7 @@ func (x *SubscribeMessageRequest_SeekMessage) String() string { func (*SubscribeMessageRequest_SeekMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_SeekMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[63] + mi := &file_mq_broker_proto_msgTypes[65] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3840,7 +4054,7 @@ type SubscribeMessageResponse_SubscribeCtrlMessage struct { func (x *SubscribeMessageResponse_SubscribeCtrlMessage) Reset() { *x = SubscribeMessageResponse_SubscribeCtrlMessage{} - mi := &file_mq_broker_proto_msgTypes[64] + mi := &file_mq_broker_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3852,7 +4066,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) String() string { func (*SubscribeMessageResponse_SubscribeCtrlMessage) ProtoMessage() {} func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[64] + mi := &file_mq_broker_proto_msgTypes[66] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3900,7 +4114,7 @@ type SubscribeFollowMeRequest_InitMessage struct { func (x *SubscribeFollowMeRequest_InitMessage) Reset() { *x = SubscribeFollowMeRequest_InitMessage{} - mi := &file_mq_broker_proto_msgTypes[65] + mi := &file_mq_broker_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3912,7 +4126,7 @@ func (x *SubscribeFollowMeRequest_InitMessage) String() string { func (*SubscribeFollowMeRequest_InitMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[65] + mi := &file_mq_broker_proto_msgTypes[67] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3958,7 +4172,7 @@ type SubscribeFollowMeRequest_AckMessage struct { func (x *SubscribeFollowMeRequest_AckMessage) Reset() { *x = SubscribeFollowMeRequest_AckMessage{} - mi := &file_mq_broker_proto_msgTypes[66] + mi := &file_mq_broker_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3970,7 +4184,7 @@ func (x *SubscribeFollowMeRequest_AckMessage) String() string { func (*SubscribeFollowMeRequest_AckMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[66] + mi := &file_mq_broker_proto_msgTypes[68] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4001,7 +4215,7 @@ type SubscribeFollowMeRequest_CloseMessage struct { func (x *SubscribeFollowMeRequest_CloseMessage) Reset() { *x = SubscribeFollowMeRequest_CloseMessage{} - mi := &file_mq_broker_proto_msgTypes[67] + mi := &file_mq_broker_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4013,7 +4227,7 @@ func (x *SubscribeFollowMeRequest_CloseMessage) String() string { func (*SubscribeFollowMeRequest_CloseMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_broker_proto_msgTypes[67] + mi := &file_mq_broker_proto_msgTypes[69] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4261,7 +4475,28 @@ const file_mq_broker_proto_rawDesc = "" + "\fCloseMessageB\t\n" + "\amessage\"7\n" + "\x19SubscribeFollowMeResponse\x12\x1a\n" + - "\tack_ts_ns\x18\x01 \x01(\x03R\aackTsNs\"b\n" + + "\tack_ts_ns\x18\x01 \x01(\x03R\aackTsNs\"\xd9\x02\n" + + "\x13FetchMessageRequest\x12&\n" + + "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x122\n" + + "\tpartition\x18\x02 \x01(\v2\x14.schema_pb.PartitionR\tpartition\x12!\n" + + "\fstart_offset\x18\x03 \x01(\x03R\vstartOffset\x12\x1b\n" + + "\tmax_bytes\x18\x04 \x01(\x05R\bmaxBytes\x12!\n" + + "\fmax_messages\x18\x05 \x01(\x05R\vmaxMessages\x12\x1e\n" + + "\vmax_wait_ms\x18\x06 \x01(\x05R\tmaxWaitMs\x12\x1b\n" + + "\tmin_bytes\x18\a \x01(\x05R\bminBytes\x12%\n" + + "\x0econsumer_group\x18\b \x01(\tR\rconsumerGroup\x12\x1f\n" + + "\vconsumer_id\x18\t \x01(\tR\n" + + "consumerId\"\x9f\x02\n" + + "\x14FetchMessageResponse\x125\n" + + "\bmessages\x18\x01 \x03(\v2\x19.messaging_pb.DataMessageR\bmessages\x12&\n" + + "\x0fhigh_water_mark\x18\x02 \x01(\x03R\rhighWaterMark\x12(\n" + + "\x10log_start_offset\x18\x03 \x01(\x03R\x0elogStartOffset\x12(\n" + + "\x10end_of_partition\x18\x04 \x01(\bR\x0eendOfPartition\x12\x14\n" + + "\x05error\x18\x05 \x01(\tR\x05error\x12\x1d\n" + + "\n" + + "error_code\x18\x06 \x01(\x05R\terrorCode\x12\x1f\n" + + "\vnext_offset\x18\a \x01(\x03R\n" + + "nextOffset\"b\n" + "\x16ClosePublishersRequest\x12&\n" + "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x12 \n" + "\funix_time_ns\x18\x02 \x01(\x03R\n" + @@ -4296,7 +4531,7 @@ const file_mq_broker_proto_rawDesc = "" + "\x0fhigh_water_mark\x18\x03 \x01(\x03R\rhighWaterMark\"x\n" + "\x12TimestampRangeInfo\x122\n" + "\x15earliest_timestamp_ns\x18\x01 \x01(\x03R\x13earliestTimestampNs\x12.\n" + - "\x13latest_timestamp_ns\x18\x02 \x01(\x03R\x11latestTimestampNs2\xd4\x10\n" + + "\x13latest_timestamp_ns\x18\x02 \x01(\x03R\x11latestTimestampNs2\xad\x11\n" + "\x10SeaweedMessaging\x12c\n" + "\x10FindBrokerLeader\x12%.messaging_pb.FindBrokerLeaderRequest\x1a&.messaging_pb.FindBrokerLeaderResponse\"\x00\x12y\n" + "\x16PublisherToPubBalancer\x12+.messaging_pb.PublisherToPubBalancerRequest\x1a,.messaging_pb.PublisherToPubBalancerResponse\"\x00(\x010\x01\x12Z\n" + @@ -4316,7 +4551,8 @@ const file_mq_broker_proto_rawDesc = "" + "\x0ePublishMessage\x12#.messaging_pb.PublishMessageRequest\x1a$.messaging_pb.PublishMessageResponse\"\x00(\x010\x01\x12g\n" + "\x10SubscribeMessage\x12%.messaging_pb.SubscribeMessageRequest\x1a&.messaging_pb.SubscribeMessageResponse\"\x00(\x010\x01\x12d\n" + "\x0fPublishFollowMe\x12$.messaging_pb.PublishFollowMeRequest\x1a%.messaging_pb.PublishFollowMeResponse\"\x00(\x010\x01\x12h\n" + - "\x11SubscribeFollowMe\x12&.messaging_pb.SubscribeFollowMeRequest\x1a'.messaging_pb.SubscribeFollowMeResponse\"\x00(\x01\x12q\n" + + "\x11SubscribeFollowMe\x12&.messaging_pb.SubscribeFollowMeRequest\x1a'.messaging_pb.SubscribeFollowMeResponse\"\x00(\x01\x12W\n" + + "\fFetchMessage\x12!.messaging_pb.FetchMessageRequest\x1a\".messaging_pb.FetchMessageResponse\"\x00\x12q\n" + "\x14GetUnflushedMessages\x12).messaging_pb.GetUnflushedMessagesRequest\x1a*.messaging_pb.GetUnflushedMessagesResponse\"\x000\x01\x12r\n" + "\x15GetPartitionRangeInfo\x12*.messaging_pb.GetPartitionRangeInfoRequest\x1a+.messaging_pb.GetPartitionRangeInfoResponse\"\x00BO\n" + "\fseaweedfs.mqB\x11MessageQueueProtoZ,github.com/seaweedfs/seaweedfs/weed/pb/mq_pbb\x06proto3" @@ -4333,7 +4569,7 @@ func file_mq_broker_proto_rawDescGZIP() []byte { return file_mq_broker_proto_rawDescData } -var file_mq_broker_proto_msgTypes = make([]protoimpl.MessageInfo, 68) +var file_mq_broker_proto_msgTypes = make([]protoimpl.MessageInfo, 70) var file_mq_broker_proto_goTypes = []any{ (*FindBrokerLeaderRequest)(nil), // 0: messaging_pb.FindBrokerLeaderRequest (*FindBrokerLeaderResponse)(nil), // 1: messaging_pb.FindBrokerLeaderResponse @@ -4375,162 +4611,169 @@ var file_mq_broker_proto_goTypes = []any{ (*SubscribeMessageResponse)(nil), // 37: messaging_pb.SubscribeMessageResponse (*SubscribeFollowMeRequest)(nil), // 38: messaging_pb.SubscribeFollowMeRequest (*SubscribeFollowMeResponse)(nil), // 39: messaging_pb.SubscribeFollowMeResponse - (*ClosePublishersRequest)(nil), // 40: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 41: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 42: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 43: messaging_pb.CloseSubscribersResponse - (*GetUnflushedMessagesRequest)(nil), // 44: messaging_pb.GetUnflushedMessagesRequest - (*GetUnflushedMessagesResponse)(nil), // 45: messaging_pb.GetUnflushedMessagesResponse - (*GetPartitionRangeInfoRequest)(nil), // 46: messaging_pb.GetPartitionRangeInfoRequest - (*GetPartitionRangeInfoResponse)(nil), // 47: messaging_pb.GetPartitionRangeInfoResponse - (*OffsetRangeInfo)(nil), // 48: messaging_pb.OffsetRangeInfo - (*TimestampRangeInfo)(nil), // 49: messaging_pb.TimestampRangeInfo - nil, // 50: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 51: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 52: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 53: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage - (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 54: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 55: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 56: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment - (*PublishMessageRequest_InitMessage)(nil), // 57: messaging_pb.PublishMessageRequest.InitMessage - (*PublishFollowMeRequest_InitMessage)(nil), // 58: messaging_pb.PublishFollowMeRequest.InitMessage - (*PublishFollowMeRequest_FlushMessage)(nil), // 59: messaging_pb.PublishFollowMeRequest.FlushMessage - (*PublishFollowMeRequest_CloseMessage)(nil), // 60: messaging_pb.PublishFollowMeRequest.CloseMessage - (*SubscribeMessageRequest_InitMessage)(nil), // 61: messaging_pb.SubscribeMessageRequest.InitMessage - (*SubscribeMessageRequest_AckMessage)(nil), // 62: messaging_pb.SubscribeMessageRequest.AckMessage - (*SubscribeMessageRequest_SeekMessage)(nil), // 63: messaging_pb.SubscribeMessageRequest.SeekMessage - (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 64: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage - (*SubscribeFollowMeRequest_InitMessage)(nil), // 65: messaging_pb.SubscribeFollowMeRequest.InitMessage - (*SubscribeFollowMeRequest_AckMessage)(nil), // 66: messaging_pb.SubscribeFollowMeRequest.AckMessage - (*SubscribeFollowMeRequest_CloseMessage)(nil), // 67: messaging_pb.SubscribeFollowMeRequest.CloseMessage - (*schema_pb.Topic)(nil), // 68: schema_pb.Topic - (*schema_pb.Partition)(nil), // 69: schema_pb.Partition - (*schema_pb.RecordType)(nil), // 70: schema_pb.RecordType - (*filer_pb.LogEntry)(nil), // 71: filer_pb.LogEntry - (*schema_pb.PartitionOffset)(nil), // 72: schema_pb.PartitionOffset - (schema_pb.OffsetType)(0), // 73: schema_pb.OffsetType + (*FetchMessageRequest)(nil), // 40: messaging_pb.FetchMessageRequest + (*FetchMessageResponse)(nil), // 41: messaging_pb.FetchMessageResponse + (*ClosePublishersRequest)(nil), // 42: messaging_pb.ClosePublishersRequest + (*ClosePublishersResponse)(nil), // 43: messaging_pb.ClosePublishersResponse + (*CloseSubscribersRequest)(nil), // 44: messaging_pb.CloseSubscribersRequest + (*CloseSubscribersResponse)(nil), // 45: messaging_pb.CloseSubscribersResponse + (*GetUnflushedMessagesRequest)(nil), // 46: messaging_pb.GetUnflushedMessagesRequest + (*GetUnflushedMessagesResponse)(nil), // 47: messaging_pb.GetUnflushedMessagesResponse + (*GetPartitionRangeInfoRequest)(nil), // 48: messaging_pb.GetPartitionRangeInfoRequest + (*GetPartitionRangeInfoResponse)(nil), // 49: messaging_pb.GetPartitionRangeInfoResponse + (*OffsetRangeInfo)(nil), // 50: messaging_pb.OffsetRangeInfo + (*TimestampRangeInfo)(nil), // 51: messaging_pb.TimestampRangeInfo + nil, // 52: messaging_pb.BrokerStats.StatsEntry + (*PublisherToPubBalancerRequest_InitMessage)(nil), // 53: messaging_pb.PublisherToPubBalancerRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 54: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 55: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage + (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 56: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage + (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 57: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 58: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment + (*PublishMessageRequest_InitMessage)(nil), // 59: messaging_pb.PublishMessageRequest.InitMessage + (*PublishFollowMeRequest_InitMessage)(nil), // 60: messaging_pb.PublishFollowMeRequest.InitMessage + (*PublishFollowMeRequest_FlushMessage)(nil), // 61: messaging_pb.PublishFollowMeRequest.FlushMessage + (*PublishFollowMeRequest_CloseMessage)(nil), // 62: messaging_pb.PublishFollowMeRequest.CloseMessage + (*SubscribeMessageRequest_InitMessage)(nil), // 63: messaging_pb.SubscribeMessageRequest.InitMessage + (*SubscribeMessageRequest_AckMessage)(nil), // 64: messaging_pb.SubscribeMessageRequest.AckMessage + (*SubscribeMessageRequest_SeekMessage)(nil), // 65: messaging_pb.SubscribeMessageRequest.SeekMessage + (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 66: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage + (*SubscribeFollowMeRequest_InitMessage)(nil), // 67: messaging_pb.SubscribeFollowMeRequest.InitMessage + (*SubscribeFollowMeRequest_AckMessage)(nil), // 68: messaging_pb.SubscribeFollowMeRequest.AckMessage + (*SubscribeFollowMeRequest_CloseMessage)(nil), // 69: messaging_pb.SubscribeFollowMeRequest.CloseMessage + (*schema_pb.Topic)(nil), // 70: schema_pb.Topic + (*schema_pb.Partition)(nil), // 71: schema_pb.Partition + (*schema_pb.RecordType)(nil), // 72: schema_pb.RecordType + (*filer_pb.LogEntry)(nil), // 73: filer_pb.LogEntry + (*schema_pb.PartitionOffset)(nil), // 74: schema_pb.PartitionOffset + (schema_pb.OffsetType)(0), // 75: schema_pb.OffsetType } var file_mq_broker_proto_depIdxs = []int32{ - 50, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry - 68, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> schema_pb.Topic - 69, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> schema_pb.Partition - 51, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 52, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 70, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> schema_pb.Topic + 71, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> schema_pb.Partition + 53, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage 2, // 4: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats - 68, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> schema_pb.Topic + 70, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> schema_pb.Topic 8, // 6: messaging_pb.ConfigureTopicRequest.retention:type_name -> messaging_pb.TopicRetention - 70, // 7: messaging_pb.ConfigureTopicRequest.message_record_type:type_name -> schema_pb.RecordType + 72, // 7: messaging_pb.ConfigureTopicRequest.message_record_type:type_name -> schema_pb.RecordType 17, // 8: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment 8, // 9: messaging_pb.ConfigureTopicResponse.retention:type_name -> messaging_pb.TopicRetention - 70, // 10: messaging_pb.ConfigureTopicResponse.message_record_type:type_name -> schema_pb.RecordType - 68, // 11: messaging_pb.ListTopicsResponse.topics:type_name -> schema_pb.Topic - 68, // 12: messaging_pb.TopicExistsRequest.topic:type_name -> schema_pb.Topic - 68, // 13: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> schema_pb.Topic - 68, // 14: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> schema_pb.Topic + 72, // 10: messaging_pb.ConfigureTopicResponse.message_record_type:type_name -> schema_pb.RecordType + 70, // 11: messaging_pb.ListTopicsResponse.topics:type_name -> schema_pb.Topic + 70, // 12: messaging_pb.TopicExistsRequest.topic:type_name -> schema_pb.Topic + 70, // 13: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> schema_pb.Topic + 70, // 14: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> schema_pb.Topic 17, // 15: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 69, // 16: messaging_pb.BrokerPartitionAssignment.partition:type_name -> schema_pb.Partition - 68, // 17: messaging_pb.GetTopicConfigurationRequest.topic:type_name -> schema_pb.Topic - 68, // 18: messaging_pb.GetTopicConfigurationResponse.topic:type_name -> schema_pb.Topic + 71, // 16: messaging_pb.BrokerPartitionAssignment.partition:type_name -> schema_pb.Partition + 70, // 17: messaging_pb.GetTopicConfigurationRequest.topic:type_name -> schema_pb.Topic + 70, // 18: messaging_pb.GetTopicConfigurationResponse.topic:type_name -> schema_pb.Topic 17, // 19: messaging_pb.GetTopicConfigurationResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment 8, // 20: messaging_pb.GetTopicConfigurationResponse.retention:type_name -> messaging_pb.TopicRetention - 70, // 21: messaging_pb.GetTopicConfigurationResponse.message_record_type:type_name -> schema_pb.RecordType - 68, // 22: messaging_pb.GetTopicPublishersRequest.topic:type_name -> schema_pb.Topic + 72, // 21: messaging_pb.GetTopicConfigurationResponse.message_record_type:type_name -> schema_pb.RecordType + 70, // 22: messaging_pb.GetTopicPublishersRequest.topic:type_name -> schema_pb.Topic 24, // 23: messaging_pb.GetTopicPublishersResponse.publishers:type_name -> messaging_pb.TopicPublisher - 68, // 24: messaging_pb.GetTopicSubscribersRequest.topic:type_name -> schema_pb.Topic + 70, // 24: messaging_pb.GetTopicSubscribersRequest.topic:type_name -> schema_pb.Topic 25, // 25: messaging_pb.GetTopicSubscribersResponse.subscribers:type_name -> messaging_pb.TopicSubscriber - 69, // 26: messaging_pb.TopicPublisher.partition:type_name -> schema_pb.Partition - 69, // 27: messaging_pb.TopicSubscriber.partition:type_name -> schema_pb.Partition - 68, // 28: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> schema_pb.Topic + 71, // 26: messaging_pb.TopicPublisher.partition:type_name -> schema_pb.Partition + 71, // 27: messaging_pb.TopicSubscriber.partition:type_name -> schema_pb.Partition + 70, // 28: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> schema_pb.Topic 17, // 29: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 52, // 30: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - 54, // 31: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage - 53, // 32: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage - 55, // 33: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 56, // 34: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment + 54, // 30: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + 56, // 31: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage + 55, // 32: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage + 57, // 33: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + 58, // 34: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment 30, // 35: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage - 57, // 36: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 59, // 36: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage 31, // 37: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage - 58, // 38: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage + 60, // 38: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage 31, // 39: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage - 59, // 40: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage - 60, // 41: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage - 61, // 42: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage - 62, // 43: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage - 63, // 44: messaging_pb.SubscribeMessageRequest.seek:type_name -> messaging_pb.SubscribeMessageRequest.SeekMessage - 64, // 45: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage + 61, // 40: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage + 62, // 41: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage + 63, // 42: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage + 64, // 43: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage + 65, // 44: messaging_pb.SubscribeMessageRequest.seek:type_name -> messaging_pb.SubscribeMessageRequest.SeekMessage + 66, // 45: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage 31, // 46: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage - 65, // 47: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage - 66, // 48: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage - 67, // 49: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage - 68, // 50: messaging_pb.ClosePublishersRequest.topic:type_name -> schema_pb.Topic - 68, // 51: messaging_pb.CloseSubscribersRequest.topic:type_name -> schema_pb.Topic - 68, // 52: messaging_pb.GetUnflushedMessagesRequest.topic:type_name -> schema_pb.Topic - 69, // 53: messaging_pb.GetUnflushedMessagesRequest.partition:type_name -> schema_pb.Partition - 71, // 54: messaging_pb.GetUnflushedMessagesResponse.message:type_name -> filer_pb.LogEntry - 68, // 55: messaging_pb.GetPartitionRangeInfoRequest.topic:type_name -> schema_pb.Topic - 69, // 56: messaging_pb.GetPartitionRangeInfoRequest.partition:type_name -> schema_pb.Partition - 48, // 57: messaging_pb.GetPartitionRangeInfoResponse.offset_range:type_name -> messaging_pb.OffsetRangeInfo - 49, // 58: messaging_pb.GetPartitionRangeInfoResponse.timestamp_range:type_name -> messaging_pb.TimestampRangeInfo - 3, // 59: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 68, // 60: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> schema_pb.Topic - 69, // 61: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> schema_pb.Partition - 69, // 62: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> schema_pb.Partition - 17, // 63: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment - 69, // 64: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> schema_pb.Partition - 68, // 65: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic - 69, // 66: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> schema_pb.Partition - 68, // 67: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic - 69, // 68: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition - 68, // 69: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic - 72, // 70: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> schema_pb.PartitionOffset - 73, // 71: messaging_pb.SubscribeMessageRequest.InitMessage.offset_type:type_name -> schema_pb.OffsetType - 73, // 72: messaging_pb.SubscribeMessageRequest.SeekMessage.offset_type:type_name -> schema_pb.OffsetType - 68, // 73: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic - 69, // 74: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition - 0, // 75: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 4, // 76: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 6, // 77: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 11, // 78: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 13, // 79: messaging_pb.SeaweedMessaging.TopicExists:input_type -> messaging_pb.TopicExistsRequest - 9, // 80: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 15, // 81: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 18, // 82: messaging_pb.SeaweedMessaging.GetTopicConfiguration:input_type -> messaging_pb.GetTopicConfigurationRequest - 20, // 83: messaging_pb.SeaweedMessaging.GetTopicPublishers:input_type -> messaging_pb.GetTopicPublishersRequest - 22, // 84: messaging_pb.SeaweedMessaging.GetTopicSubscribers:input_type -> messaging_pb.GetTopicSubscribersRequest - 26, // 85: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 40, // 86: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 42, // 87: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 28, // 88: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 32, // 89: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest - 36, // 90: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest - 34, // 91: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest - 38, // 92: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest - 44, // 93: messaging_pb.SeaweedMessaging.GetUnflushedMessages:input_type -> messaging_pb.GetUnflushedMessagesRequest - 46, // 94: messaging_pb.SeaweedMessaging.GetPartitionRangeInfo:input_type -> messaging_pb.GetPartitionRangeInfoRequest - 1, // 95: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 5, // 96: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 7, // 97: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 12, // 98: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 14, // 99: messaging_pb.SeaweedMessaging.TopicExists:output_type -> messaging_pb.TopicExistsResponse - 10, // 100: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 16, // 101: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 19, // 102: messaging_pb.SeaweedMessaging.GetTopicConfiguration:output_type -> messaging_pb.GetTopicConfigurationResponse - 21, // 103: messaging_pb.SeaweedMessaging.GetTopicPublishers:output_type -> messaging_pb.GetTopicPublishersResponse - 23, // 104: messaging_pb.SeaweedMessaging.GetTopicSubscribers:output_type -> messaging_pb.GetTopicSubscribersResponse - 27, // 105: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 41, // 106: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 43, // 107: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 29, // 108: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 33, // 109: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse - 37, // 110: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse - 35, // 111: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse - 39, // 112: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse - 45, // 113: messaging_pb.SeaweedMessaging.GetUnflushedMessages:output_type -> messaging_pb.GetUnflushedMessagesResponse - 47, // 114: messaging_pb.SeaweedMessaging.GetPartitionRangeInfo:output_type -> messaging_pb.GetPartitionRangeInfoResponse - 95, // [95:115] is the sub-list for method output_type - 75, // [75:95] is the sub-list for method input_type - 75, // [75:75] is the sub-list for extension type_name - 75, // [75:75] is the sub-list for extension extendee - 0, // [0:75] is the sub-list for field type_name + 67, // 47: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage + 68, // 48: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage + 69, // 49: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage + 70, // 50: messaging_pb.FetchMessageRequest.topic:type_name -> schema_pb.Topic + 71, // 51: messaging_pb.FetchMessageRequest.partition:type_name -> schema_pb.Partition + 31, // 52: messaging_pb.FetchMessageResponse.messages:type_name -> messaging_pb.DataMessage + 70, // 53: messaging_pb.ClosePublishersRequest.topic:type_name -> schema_pb.Topic + 70, // 54: messaging_pb.CloseSubscribersRequest.topic:type_name -> schema_pb.Topic + 70, // 55: messaging_pb.GetUnflushedMessagesRequest.topic:type_name -> schema_pb.Topic + 71, // 56: messaging_pb.GetUnflushedMessagesRequest.partition:type_name -> schema_pb.Partition + 73, // 57: messaging_pb.GetUnflushedMessagesResponse.message:type_name -> filer_pb.LogEntry + 70, // 58: messaging_pb.GetPartitionRangeInfoRequest.topic:type_name -> schema_pb.Topic + 71, // 59: messaging_pb.GetPartitionRangeInfoRequest.partition:type_name -> schema_pb.Partition + 50, // 60: messaging_pb.GetPartitionRangeInfoResponse.offset_range:type_name -> messaging_pb.OffsetRangeInfo + 51, // 61: messaging_pb.GetPartitionRangeInfoResponse.timestamp_range:type_name -> messaging_pb.TimestampRangeInfo + 3, // 62: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats + 70, // 63: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> schema_pb.Topic + 71, // 64: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> schema_pb.Partition + 71, // 65: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> schema_pb.Partition + 17, // 66: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment + 71, // 67: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> schema_pb.Partition + 70, // 68: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic + 71, // 69: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> schema_pb.Partition + 70, // 70: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic + 71, // 71: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition + 70, // 72: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic + 74, // 73: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> schema_pb.PartitionOffset + 75, // 74: messaging_pb.SubscribeMessageRequest.InitMessage.offset_type:type_name -> schema_pb.OffsetType + 75, // 75: messaging_pb.SubscribeMessageRequest.SeekMessage.offset_type:type_name -> schema_pb.OffsetType + 70, // 76: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic + 71, // 77: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition + 0, // 78: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 4, // 79: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 6, // 80: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 11, // 81: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 13, // 82: messaging_pb.SeaweedMessaging.TopicExists:input_type -> messaging_pb.TopicExistsRequest + 9, // 83: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 15, // 84: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 18, // 85: messaging_pb.SeaweedMessaging.GetTopicConfiguration:input_type -> messaging_pb.GetTopicConfigurationRequest + 20, // 86: messaging_pb.SeaweedMessaging.GetTopicPublishers:input_type -> messaging_pb.GetTopicPublishersRequest + 22, // 87: messaging_pb.SeaweedMessaging.GetTopicSubscribers:input_type -> messaging_pb.GetTopicSubscribersRequest + 26, // 88: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 42, // 89: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 44, // 90: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 28, // 91: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 32, // 92: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 36, // 93: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 34, // 94: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 38, // 95: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest + 40, // 96: messaging_pb.SeaweedMessaging.FetchMessage:input_type -> messaging_pb.FetchMessageRequest + 46, // 97: messaging_pb.SeaweedMessaging.GetUnflushedMessages:input_type -> messaging_pb.GetUnflushedMessagesRequest + 48, // 98: messaging_pb.SeaweedMessaging.GetPartitionRangeInfo:input_type -> messaging_pb.GetPartitionRangeInfoRequest + 1, // 99: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 5, // 100: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 7, // 101: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 12, // 102: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 14, // 103: messaging_pb.SeaweedMessaging.TopicExists:output_type -> messaging_pb.TopicExistsResponse + 10, // 104: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 16, // 105: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 19, // 106: messaging_pb.SeaweedMessaging.GetTopicConfiguration:output_type -> messaging_pb.GetTopicConfigurationResponse + 21, // 107: messaging_pb.SeaweedMessaging.GetTopicPublishers:output_type -> messaging_pb.GetTopicPublishersResponse + 23, // 108: messaging_pb.SeaweedMessaging.GetTopicSubscribers:output_type -> messaging_pb.GetTopicSubscribersResponse + 27, // 109: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 43, // 110: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 45, // 111: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 29, // 112: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 33, // 113: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 37, // 114: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 35, // 115: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 39, // 116: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse + 41, // 117: messaging_pb.SeaweedMessaging.FetchMessage:output_type -> messaging_pb.FetchMessageResponse + 47, // 118: messaging_pb.SeaweedMessaging.GetUnflushedMessages:output_type -> messaging_pb.GetUnflushedMessagesResponse + 49, // 119: messaging_pb.SeaweedMessaging.GetPartitionRangeInfo:output_type -> messaging_pb.GetPartitionRangeInfoResponse + 99, // [99:120] is the sub-list for method output_type + 78, // [78:99] is the sub-list for method input_type + 78, // [78:78] is the sub-list for extension type_name + 78, // [78:78] is the sub-list for extension extendee + 0, // [0:78] is the sub-list for field type_name } func init() { file_mq_broker_proto_init() } @@ -4581,7 +4824,7 @@ func file_mq_broker_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_mq_broker_proto_rawDesc), len(file_mq_broker_proto_rawDesc)), NumEnums: 0, - NumMessages: 68, + NumMessages: 70, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/pb/mq_pb/mq_broker_grpc.pb.go b/weed/pb/mq_pb/mq_broker_grpc.pb.go index e8544b57f..77ff7df52 100644 --- a/weed/pb/mq_pb/mq_broker_grpc.pb.go +++ b/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, diff --git a/weed/util/log_buffer/log_read_integration_test.go b/weed/util/log_buffer/log_read_integration_test.go new file mode 100644 index 000000000..8066ba07c --- /dev/null +++ b/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") + } +} diff --git a/weed/util/log_buffer/log_read_stateless.go b/weed/util/log_buffer/log_read_stateless.go new file mode 100644 index 000000000..4339962d4 --- /dev/null +++ b/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 + } +} diff --git a/weed/util/log_buffer/log_read_stateless_test.go b/weed/util/log_buffer/log_read_stateless_test.go new file mode 100644 index 000000000..948a929ba --- /dev/null +++ b/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) + } +}