You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 
 
 
 
 

513 lines
18 KiB

package integration
import (
"context"
"encoding/binary"
"fmt"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
)
// GetStoredRecords retrieves records from SeaweedMQ using the proper subscriber API
// ctx controls the fetch timeout (should match Kafka fetch request's MaxWaitTime)
func (h *SeaweedMQHandler) GetStoredRecords(ctx context.Context, topic string, partition int32, fromOffset int64, maxRecords int) ([]SMQRecord, error) {
glog.V(4).Infof("[FETCH] GetStoredRecords: topic=%s partition=%d fromOffset=%d maxRecords=%d", topic, partition, fromOffset, maxRecords)
// Verify topic exists
if !h.TopicExists(topic) {
return nil, fmt.Errorf("topic %s does not exist", topic)
}
// CRITICAL: Use per-connection BrokerClient to prevent gRPC stream interference
// Each Kafka connection has its own isolated BrokerClient instance
var brokerClient *BrokerClient
consumerGroup := "kafka-fetch-consumer" // default
// CRITICAL FIX: Use stable consumer ID per topic-partition, NOT with timestamp
// Including timestamp would create a new session on every fetch, causing subscriber churn
consumerID := fmt.Sprintf("kafka-fetch-%s-%d", topic, partition) // default, stable per topic-partition
// Get the per-connection broker client from connection context
if h.protocolHandler != nil {
connCtx := h.protocolHandler.GetConnectionContext()
if connCtx != nil {
// Extract per-connection broker client
if connCtx.BrokerClient != nil {
if bc, ok := connCtx.BrokerClient.(*BrokerClient); ok {
brokerClient = bc
glog.V(4).Infof("[FETCH] Using per-connection BrokerClient for topic=%s partition=%d", topic, partition)
}
}
// Extract consumer group and client ID
if connCtx.ConsumerGroup != "" {
consumerGroup = connCtx.ConsumerGroup
glog.V(4).Infof("[FETCH] Using actual consumer group from context: %s", consumerGroup)
}
if connCtx.MemberID != "" {
// Use member ID as base, but still include topic-partition for uniqueness
consumerID = fmt.Sprintf("%s-%s-%d", connCtx.MemberID, topic, partition)
glog.V(4).Infof("[FETCH] Using actual member ID from context: %s", consumerID)
} else if connCtx.ClientID != "" {
// Fallback to client ID if member ID not set (for clients not using consumer groups)
// Include topic-partition to ensure each partition consumer is unique
consumerID = fmt.Sprintf("%s-%s-%d", connCtx.ClientID, topic, partition)
glog.V(4).Infof("[FETCH] Using client ID from context: %s", consumerID)
}
}
}
// Fallback to shared broker client if per-connection client not available
if brokerClient == nil {
glog.Warningf("[FETCH] No per-connection BrokerClient, falling back to shared client")
brokerClient = h.brokerClient
if brokerClient == nil {
return nil, fmt.Errorf("no broker client available")
}
}
// 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(4).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-STATELESS] Failed to fetch records: %v", err)
return nil, fmt.Errorf("failed to fetch records: %v", err)
}
glog.V(4).Infof("[FETCH-STATELESS] Fetched %d records", len(seaweedRecords))
//
// 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
//
// 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))
for i, seaweedRecord := range seaweedRecords {
// CRITICAL FIX: Use the actual offset from SeaweedMQ
// The SeaweedRecord.Offset field now contains the correct offset from the subscriber
kafkaOffset := seaweedRecord.Offset
// CRITICAL: Skip records before the requested offset
// This can happen when the subscriber cache returns old data
if kafkaOffset < fromOffset {
glog.V(4).Infof("[FETCH] Skipping record %d with offset %d (requested fromOffset=%d)", i, kafkaOffset, fromOffset)
continue
}
smqRecord := &SeaweedSMQRecord{
key: seaweedRecord.Key,
value: seaweedRecord.Value,
timestamp: seaweedRecord.Timestamp,
offset: kafkaOffset,
}
smqRecords = append(smqRecords, smqRecord)
glog.V(4).Infof("[FETCH] Record %d: offset=%d, keyLen=%d, valueLen=%d", i, kafkaOffset, len(seaweedRecord.Key), len(seaweedRecord.Value))
}
glog.V(4).Infof("[FETCH] Successfully read %d records from SMQ", len(smqRecords))
return smqRecords, nil
}
// GetEarliestOffset returns the earliest available offset for a topic partition
// ALWAYS queries SMQ broker directly - no ledger involved
func (h *SeaweedMQHandler) GetEarliestOffset(topic string, partition int32) (int64, error) {
// Check if topic exists
if !h.TopicExists(topic) {
return 0, nil // Empty topic starts at offset 0
}
// ALWAYS query SMQ broker directly for earliest offset
if h.brokerClient != nil {
earliestOffset, err := h.brokerClient.GetEarliestOffset(topic, partition)
if err != nil {
return 0, err
}
return earliestOffset, nil
}
// No broker client - this shouldn't happen in production
return 0, fmt.Errorf("broker client not available")
}
// GetLatestOffset returns the latest available offset for a topic partition
// ALWAYS queries SMQ broker directly - no ledger involved
func (h *SeaweedMQHandler) GetLatestOffset(topic string, partition int32) (int64, error) {
// Check if topic exists
if !h.TopicExists(topic) {
return 0, nil // Empty topic
}
// Check cache first
cacheKey := fmt.Sprintf("%s:%d", topic, partition)
h.hwmCacheMu.RLock()
if entry, exists := h.hwmCache[cacheKey]; exists {
if time.Now().Before(entry.expiresAt) {
// Cache hit - return cached value
h.hwmCacheMu.RUnlock()
glog.V(2).Infof("[HWM] Cache HIT for %s: hwm=%d", cacheKey, entry.value)
return entry.value, nil
}
}
h.hwmCacheMu.RUnlock()
// Cache miss or expired - query SMQ broker
if h.brokerClient != nil {
glog.V(2).Infof("[HWM] Cache MISS for %s, querying broker...", cacheKey)
latestOffset, err := h.brokerClient.GetHighWaterMark(topic, partition)
if err != nil {
glog.V(1).Infof("[HWM] ERROR querying broker for %s: %v", cacheKey, err)
return 0, err
}
glog.V(2).Infof("[HWM] Broker returned hwm=%d for %s", latestOffset, cacheKey)
// Update cache
h.hwmCacheMu.Lock()
h.hwmCache[cacheKey] = &hwmCacheEntry{
value: latestOffset,
expiresAt: time.Now().Add(h.hwmCacheTTL),
}
h.hwmCacheMu.Unlock()
return latestOffset, nil
}
// No broker client - this shouldn't happen in production
return 0, fmt.Errorf("broker client not available")
}
// WithFilerClient executes a function with a filer client
func (h *SeaweedMQHandler) WithFilerClient(streamingMode bool, fn func(client filer_pb.SeaweedFilerClient) error) error {
if h.brokerClient == nil {
return fmt.Errorf("no broker client available")
}
return h.brokerClient.WithFilerClient(streamingMode, fn)
}
// GetFilerAddress returns the filer address used by this handler
func (h *SeaweedMQHandler) GetFilerAddress() string {
if h.brokerClient != nil {
return h.brokerClient.GetFilerAddress()
}
return ""
}
// ProduceRecord publishes a record to SeaweedMQ and lets SMQ generate the offset
// ctx controls the publish timeout - if client cancels, broker operation is cancelled
func (h *SeaweedMQHandler) ProduceRecord(ctx context.Context, topic string, partition int32, key []byte, value []byte) (int64, error) {
if len(key) > 0 {
}
if len(value) > 0 {
} else {
}
// Verify topic exists
if !h.TopicExists(topic) {
return 0, fmt.Errorf("topic %s does not exist", topic)
}
// Get current timestamp
timestamp := time.Now().UnixNano()
// Publish to SeaweedMQ and let SMQ generate the offset
var smqOffset int64
var publishErr error
if h.brokerClient == nil {
publishErr = fmt.Errorf("no broker client available")
} else {
smqOffset, publishErr = h.brokerClient.PublishRecord(ctx, topic, partition, key, value, timestamp)
}
if publishErr != nil {
return 0, fmt.Errorf("failed to publish to SeaweedMQ: %v", publishErr)
}
// SMQ should have generated and returned the offset - use it directly as the Kafka offset
// Invalidate HWM cache for this partition to ensure fresh reads
// This is critical for read-your-own-write scenarios (e.g., Schema Registry)
cacheKey := fmt.Sprintf("%s:%d", topic, partition)
h.hwmCacheMu.Lock()
delete(h.hwmCache, cacheKey)
h.hwmCacheMu.Unlock()
return smqOffset, nil
}
// ProduceRecordValue produces a record using RecordValue format to SeaweedMQ
// ALWAYS uses broker's assigned offset - no ledger involved
// ctx controls the publish timeout - if client cancels, broker operation is cancelled
func (h *SeaweedMQHandler) ProduceRecordValue(ctx context.Context, topic string, partition int32, key []byte, recordValueBytes []byte) (int64, error) {
// Verify topic exists
if !h.TopicExists(topic) {
return 0, fmt.Errorf("topic %s does not exist", topic)
}
// Get current timestamp
timestamp := time.Now().UnixNano()
// Publish RecordValue to SeaweedMQ and get the broker-assigned offset
var smqOffset int64
var publishErr error
if h.brokerClient == nil {
publishErr = fmt.Errorf("no broker client available")
} else {
smqOffset, publishErr = h.brokerClient.PublishRecordValue(ctx, topic, partition, key, recordValueBytes, timestamp)
}
if publishErr != nil {
return 0, fmt.Errorf("failed to publish RecordValue to SeaweedMQ: %v", publishErr)
}
// SMQ broker has assigned the offset - use it directly as the Kafka offset
// Invalidate HWM cache for this partition to ensure fresh reads
// This is critical for read-your-own-write scenarios (e.g., Schema Registry)
cacheKey := fmt.Sprintf("%s:%d", topic, partition)
h.hwmCacheMu.Lock()
delete(h.hwmCache, cacheKey)
h.hwmCacheMu.Unlock()
return smqOffset, nil
}
// Ledger methods removed - SMQ broker handles all offset management directly
// FetchRecords DEPRECATED - only used in old tests
func (h *SeaweedMQHandler) FetchRecords(topic string, partition int32, fetchOffset int64, maxBytes int32) ([]byte, error) {
// Verify topic exists
if !h.TopicExists(topic) {
return nil, fmt.Errorf("topic %s does not exist", topic)
}
// DEPRECATED: This function only used in old tests
// Get HWM directly from broker
highWaterMark, err := h.GetLatestOffset(topic, partition)
if err != nil {
return nil, err
}
// If fetch offset is at or beyond high water mark, no records to return
if fetchOffset >= highWaterMark {
return []byte{}, nil
}
// Get or create subscriber session for this topic/partition
var seaweedRecords []*SeaweedRecord
// Calculate how many records to fetch
recordsToFetch := int(highWaterMark - fetchOffset)
if recordsToFetch > 100 {
recordsToFetch = 100 // Limit batch size
}
// Read records using broker client
if h.brokerClient == nil {
return nil, fmt.Errorf("no broker client available")
}
// Use default consumer group/ID since this is a deprecated function
brokerSubscriber, subErr := h.brokerClient.GetOrCreateSubscriber(topic, partition, fetchOffset, "deprecated-consumer-group", "deprecated-consumer")
if subErr != nil {
return nil, fmt.Errorf("failed to get broker subscriber: %v", subErr)
}
// 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
return []byte{}, nil
}
// Map SeaweedMQ records to Kafka offsets and update ledger
kafkaRecords, err := h.mapSeaweedToKafkaOffsets(topic, partition, seaweedRecords, fetchOffset)
if err != nil {
return nil, fmt.Errorf("failed to map offsets: %v", err)
}
// Convert mapped records to Kafka record batch format
return h.convertSeaweedToKafkaRecordBatch(kafkaRecords, fetchOffset, maxBytes)
}
// mapSeaweedToKafkaOffsets maps SeaweedMQ records to proper Kafka offsets
func (h *SeaweedMQHandler) mapSeaweedToKafkaOffsets(topic string, partition int32, seaweedRecords []*SeaweedRecord, startOffset int64) ([]*SeaweedRecord, error) {
if len(seaweedRecords) == 0 {
return seaweedRecords, nil
}
// DEPRECATED: This function only used in old tests
// Just map offsets sequentially
mappedRecords := make([]*SeaweedRecord, 0, len(seaweedRecords))
for i, seaweedRecord := range seaweedRecords {
currentKafkaOffset := startOffset + int64(i)
// Create a copy of the record with proper Kafka offset assignment
mappedRecord := &SeaweedRecord{
Key: seaweedRecord.Key,
Value: seaweedRecord.Value,
Timestamp: seaweedRecord.Timestamp,
Offset: currentKafkaOffset,
}
// Just skip any error handling since this is deprecated
{
// Log warning but continue processing
}
mappedRecords = append(mappedRecords, mappedRecord)
}
return mappedRecords, nil
}
// convertSeaweedToKafkaRecordBatch converts SeaweedMQ records to Kafka record batch format
func (h *SeaweedMQHandler) convertSeaweedToKafkaRecordBatch(seaweedRecords []*SeaweedRecord, fetchOffset int64, maxBytes int32) ([]byte, error) {
if len(seaweedRecords) == 0 {
return []byte{}, nil
}
batch := make([]byte, 0, 512)
// Record batch header
baseOffsetBytes := make([]byte, 8)
binary.BigEndian.PutUint64(baseOffsetBytes, uint64(fetchOffset))
batch = append(batch, baseOffsetBytes...) // base offset
// Batch length (placeholder, will be filled at end)
batchLengthPos := len(batch)
batch = append(batch, 0, 0, 0, 0)
batch = append(batch, 0, 0, 0, 0) // partition leader epoch
batch = append(batch, 2) // magic byte (version 2)
// CRC placeholder
batch = append(batch, 0, 0, 0, 0)
// Batch attributes
batch = append(batch, 0, 0)
// Last offset delta
lastOffsetDelta := uint32(len(seaweedRecords) - 1)
lastOffsetDeltaBytes := make([]byte, 4)
binary.BigEndian.PutUint32(lastOffsetDeltaBytes, lastOffsetDelta)
batch = append(batch, lastOffsetDeltaBytes...)
// Timestamps - use actual timestamps from SeaweedMQ records
var firstTimestamp, maxTimestamp int64
if len(seaweedRecords) > 0 {
firstTimestamp = seaweedRecords[0].Timestamp
maxTimestamp = firstTimestamp
for _, record := range seaweedRecords {
if record.Timestamp > maxTimestamp {
maxTimestamp = record.Timestamp
}
}
}
firstTimestampBytes := make([]byte, 8)
binary.BigEndian.PutUint64(firstTimestampBytes, uint64(firstTimestamp))
batch = append(batch, firstTimestampBytes...)
maxTimestampBytes := make([]byte, 8)
binary.BigEndian.PutUint64(maxTimestampBytes, uint64(maxTimestamp))
batch = append(batch, maxTimestampBytes...)
// Producer info (simplified)
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF) // producer ID (-1)
batch = append(batch, 0xFF, 0xFF) // producer epoch (-1)
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF) // base sequence (-1)
// Record count
recordCountBytes := make([]byte, 4)
binary.BigEndian.PutUint32(recordCountBytes, uint32(len(seaweedRecords)))
batch = append(batch, recordCountBytes...)
// Add actual records from SeaweedMQ
for i, seaweedRecord := range seaweedRecords {
record := h.convertSingleSeaweedRecord(seaweedRecord, int64(i), fetchOffset)
recordLength := byte(len(record))
batch = append(batch, recordLength)
batch = append(batch, record...)
// Check if we're approaching maxBytes limit
if int32(len(batch)) > maxBytes*3/4 {
// Leave room for remaining headers and stop adding records
break
}
}
// Fill in the batch length
batchLength := uint32(len(batch) - batchLengthPos - 4)
binary.BigEndian.PutUint32(batch[batchLengthPos:batchLengthPos+4], batchLength)
return batch, nil
}
// convertSingleSeaweedRecord converts a single SeaweedMQ record to Kafka format
func (h *SeaweedMQHandler) convertSingleSeaweedRecord(seaweedRecord *SeaweedRecord, index, baseOffset int64) []byte {
record := make([]byte, 0, 64)
// Record attributes
record = append(record, 0)
// Timestamp delta (varint - simplified)
timestampDelta := seaweedRecord.Timestamp - baseOffset // Simple delta calculation
if timestampDelta < 0 {
timestampDelta = 0
}
record = append(record, byte(timestampDelta&0xFF)) // Simplified varint encoding
// Offset delta (varint - simplified)
record = append(record, byte(index))
// Key length and key
if len(seaweedRecord.Key) > 0 {
record = append(record, byte(len(seaweedRecord.Key)))
record = append(record, seaweedRecord.Key...)
} else {
// Null key
record = append(record, 0xFF)
}
// Value length and value
if len(seaweedRecord.Value) > 0 {
record = append(record, byte(len(seaweedRecord.Value)))
record = append(record, seaweedRecord.Value...)
} else {
// Empty value
record = append(record, 0)
}
// Headers count (0)
record = append(record, 0)
return record
}