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.
1301 lines
44 KiB
1301 lines
44 KiB
package protocol
|
|
|
|
import (
|
|
"context"
|
|
"encoding/binary"
|
|
"fmt"
|
|
"hash/crc32"
|
|
"strings"
|
|
"time"
|
|
"unicode/utf8"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/compression"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/integration"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/schema"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
|
"google.golang.org/protobuf/proto"
|
|
)
|
|
|
|
// partitionFetchResult holds the result of fetching from a single partition
|
|
type partitionFetchResult struct {
|
|
topicIndex int
|
|
partitionIndex int
|
|
recordBatch []byte
|
|
highWaterMark int64
|
|
errorCode int16
|
|
fetchDuration time.Duration
|
|
}
|
|
|
|
func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
// Parse the Fetch request to get the requested topics and partitions
|
|
fetchRequest, err := h.parseFetchRequest(apiVersion, requestBody)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("parse fetch request: %w", err)
|
|
}
|
|
|
|
// Basic long-polling to avoid client busy-looping when there's no data.
|
|
var throttleTimeMs int32 = 0
|
|
// Only long-poll when all referenced topics exist; unknown topics should not block
|
|
allTopicsExist := func() bool {
|
|
for _, topic := range fetchRequest.Topics {
|
|
if !h.seaweedMQHandler.TopicExists(topic.Name) {
|
|
return false
|
|
}
|
|
}
|
|
return true
|
|
}
|
|
hasDataAvailable := func() bool {
|
|
// Check if any requested partition has data available
|
|
// Compare fetch offset with high water mark
|
|
for _, topic := range fetchRequest.Topics {
|
|
if !h.seaweedMQHandler.TopicExists(topic.Name) {
|
|
continue
|
|
}
|
|
for _, partition := range topic.Partitions {
|
|
hwm, err := h.seaweedMQHandler.GetLatestOffset(topic.Name, partition.PartitionID)
|
|
if err != nil {
|
|
continue
|
|
}
|
|
// Normalize fetch offset
|
|
effectiveOffset := partition.FetchOffset
|
|
if effectiveOffset == -2 { // earliest
|
|
effectiveOffset = 0
|
|
} else if effectiveOffset == -1 { // latest
|
|
effectiveOffset = hwm
|
|
}
|
|
// If fetch offset < hwm, data is available
|
|
if effectiveOffset < hwm {
|
|
return true
|
|
}
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
// Long-poll when client requests it via MaxWaitTime and there's no data
|
|
// Even if MinBytes=0, we should honor MaxWaitTime to reduce polling overhead
|
|
maxWaitMs := fetchRequest.MaxWaitTime
|
|
|
|
// Long-poll if: (1) client wants to wait (maxWaitMs > 0), (2) no data available, (3) topics exist
|
|
// NOTE: We long-poll even if MinBytes=0, since the client specified a wait time
|
|
hasData := hasDataAvailable()
|
|
topicsExist := allTopicsExist()
|
|
shouldLongPoll := maxWaitMs > 0 && !hasData && topicsExist
|
|
|
|
if shouldLongPoll {
|
|
start := time.Now()
|
|
// Use the client's requested wait time (already capped at 1s)
|
|
maxPollTime := time.Duration(maxWaitMs) * time.Millisecond
|
|
deadline := start.Add(maxPollTime)
|
|
pollLoop:
|
|
for time.Now().Before(deadline) {
|
|
// Use context-aware sleep instead of blocking time.Sleep
|
|
select {
|
|
case <-ctx.Done():
|
|
throttleTimeMs = int32(time.Since(start) / time.Millisecond)
|
|
break pollLoop
|
|
case <-time.After(10 * time.Millisecond):
|
|
// Continue with polling
|
|
}
|
|
if hasDataAvailable() {
|
|
// Data became available during polling - return immediately with NO throttle
|
|
// Throttle time should only be used for quota enforcement, not for long-poll timing
|
|
throttleTimeMs = 0
|
|
break pollLoop
|
|
}
|
|
}
|
|
// If we got here without breaking early, we hit the timeout
|
|
// Long-poll timeout is NOT throttling - throttle time should only be used for quota/rate limiting
|
|
// Do NOT set throttle time based on long-poll duration
|
|
throttleTimeMs = 0
|
|
}
|
|
|
|
// Build the response
|
|
response := make([]byte, 0, 1024)
|
|
totalAppendedRecordBytes := 0
|
|
|
|
// NOTE: Correlation ID is NOT included in the response body
|
|
// The wire protocol layer (writeResponseWithTimeout) writes: [Size][CorrelationID][Body]
|
|
// Kafka clients read the correlation ID separately from the 8-byte header, then read Size-4 bytes of body
|
|
// If we include correlation ID here, clients will see it twice and fail with "4 extra bytes" errors
|
|
|
|
// Fetch v1+ has throttle_time_ms at the beginning
|
|
if apiVersion >= 1 {
|
|
throttleBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(throttleBytes, uint32(throttleTimeMs))
|
|
response = append(response, throttleBytes...)
|
|
}
|
|
|
|
// Fetch v7+ has error_code and session_id
|
|
if apiVersion >= 7 {
|
|
response = append(response, 0, 0) // error_code (2 bytes, 0 = no error)
|
|
response = append(response, 0, 0, 0, 0) // session_id (4 bytes, 0 = no session)
|
|
}
|
|
|
|
// Check if this version uses flexible format (v12+)
|
|
isFlexible := IsFlexibleVersion(1, apiVersion) // API key 1 = Fetch
|
|
|
|
// Topics count - write the actual number of topics in the request
|
|
// Kafka protocol: we MUST return all requested topics in the response (even with empty data)
|
|
topicsCount := len(fetchRequest.Topics)
|
|
if isFlexible {
|
|
// Flexible versions use compact array format (count + 1)
|
|
response = append(response, EncodeUvarint(uint32(topicsCount+1))...)
|
|
} else {
|
|
topicsCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(topicsCountBytes, uint32(topicsCount))
|
|
response = append(response, topicsCountBytes...)
|
|
}
|
|
|
|
// ====================================================================
|
|
// PERSISTENT PARTITION READERS
|
|
// Use per-connection persistent goroutines that maintain offset position
|
|
// and stream forward, eliminating repeated lookups and reducing broker CPU
|
|
// ====================================================================
|
|
|
|
// Get connection context to access persistent partition readers
|
|
connContext := h.getConnectionContextFromRequest(ctx)
|
|
if connContext == nil {
|
|
glog.Errorf("FETCH CORR=%d: Connection context not available - cannot use persistent readers",
|
|
correlationID)
|
|
return nil, fmt.Errorf("connection context not available")
|
|
}
|
|
|
|
glog.V(4).Infof("[%s] FETCH CORR=%d: Processing %d topics with %d total partitions",
|
|
connContext.ConnectionID, correlationID, len(fetchRequest.Topics),
|
|
func() int {
|
|
count := 0
|
|
for _, t := range fetchRequest.Topics {
|
|
count += len(t.Partitions)
|
|
}
|
|
return count
|
|
}())
|
|
|
|
// Collect results from persistent readers
|
|
// Dispatch all requests concurrently, then wait for all results in parallel
|
|
// to avoid sequential timeout accumulation
|
|
type pendingFetch struct {
|
|
topicName string
|
|
partitionID int32
|
|
resultChan chan *partitionFetchResult
|
|
}
|
|
|
|
pending := make([]pendingFetch, 0)
|
|
|
|
// Phase 1: Dispatch all fetch requests to partition readers (non-blocking)
|
|
for _, topic := range fetchRequest.Topics {
|
|
isSchematizedTopic := false
|
|
if h.IsSchemaEnabled() {
|
|
isSchematizedTopic = h.isSchematizedTopic(topic.Name)
|
|
}
|
|
|
|
for _, partition := range topic.Partitions {
|
|
key := TopicPartitionKey{Topic: topic.Name, Partition: partition.PartitionID}
|
|
|
|
// All topics (including system topics) use persistent readers for in-memory access
|
|
// This enables instant notification and avoids ForceFlush dependencies
|
|
|
|
// Get or create persistent reader for this partition
|
|
reader := h.getOrCreatePartitionReader(ctx, connContext, key, partition.FetchOffset)
|
|
if reader == nil {
|
|
// Failed to create reader - add empty pending
|
|
glog.Errorf("[%s] Failed to get/create partition reader for %s[%d]",
|
|
connContext.ConnectionID, topic.Name, partition.PartitionID)
|
|
nilChan := make(chan *partitionFetchResult, 1)
|
|
nilChan <- &partitionFetchResult{errorCode: 3} // UNKNOWN_TOPIC_OR_PARTITION
|
|
pending = append(pending, pendingFetch{
|
|
topicName: topic.Name,
|
|
partitionID: partition.PartitionID,
|
|
resultChan: nilChan,
|
|
})
|
|
continue
|
|
}
|
|
|
|
// Signal reader to fetch (don't wait for result yet)
|
|
resultChan := make(chan *partitionFetchResult, 1)
|
|
fetchReq := &partitionFetchRequest{
|
|
requestedOffset: partition.FetchOffset,
|
|
maxBytes: partition.MaxBytes,
|
|
maxWaitMs: maxWaitMs, // Pass MaxWaitTime from Kafka fetch request
|
|
resultChan: resultChan,
|
|
isSchematized: isSchematizedTopic,
|
|
apiVersion: apiVersion,
|
|
}
|
|
|
|
// Try to send request (increased timeout for CI environments with slow disk I/O)
|
|
select {
|
|
case reader.fetchChan <- fetchReq:
|
|
// Request sent successfully, add to pending
|
|
pending = append(pending, pendingFetch{
|
|
topicName: topic.Name,
|
|
partitionID: partition.PartitionID,
|
|
resultChan: resultChan,
|
|
})
|
|
case <-time.After(200 * time.Millisecond):
|
|
// Channel full, return empty result
|
|
glog.Warningf("[%s] Reader channel full for %s[%d], returning empty",
|
|
connContext.ConnectionID, topic.Name, partition.PartitionID)
|
|
emptyChan := make(chan *partitionFetchResult, 1)
|
|
emptyChan <- &partitionFetchResult{}
|
|
pending = append(pending, pendingFetch{
|
|
topicName: topic.Name,
|
|
partitionID: partition.PartitionID,
|
|
resultChan: emptyChan,
|
|
})
|
|
}
|
|
}
|
|
}
|
|
|
|
// Phase 2: Wait for all results with adequate timeout for CI environments
|
|
// We MUST return a result for every requested partition or Sarama will error
|
|
results := make([]*partitionFetchResult, len(pending))
|
|
// Use 95% of client's MaxWaitTime to ensure we return BEFORE client timeout
|
|
// This maximizes data collection time while leaving a safety buffer for:
|
|
// - Response serialization, network transmission, client processing
|
|
// For 500ms client timeout: 475ms internal fetch, 25ms buffer
|
|
// For 100ms client timeout: 95ms internal fetch, 5ms buffer
|
|
effectiveDeadlineMs := time.Duration(maxWaitMs) * 95 / 100
|
|
deadline := time.After(effectiveDeadlineMs * time.Millisecond)
|
|
if maxWaitMs < 20 {
|
|
// For very short timeouts (< 20ms), use full timeout to maximize data collection
|
|
deadline = time.After(time.Duration(maxWaitMs) * time.Millisecond)
|
|
}
|
|
|
|
// Collect results one by one with shared deadline
|
|
for i, pf := range pending {
|
|
select {
|
|
case result := <-pf.resultChan:
|
|
results[i] = result
|
|
case <-deadline:
|
|
// Deadline expired, return empty for this and all remaining partitions
|
|
for j := i; j < len(pending); j++ {
|
|
results[j] = &partitionFetchResult{}
|
|
}
|
|
glog.V(3).Infof("[%s] Fetch deadline expired, returning empty for %d remaining partitions",
|
|
connContext.ConnectionID, len(pending)-i)
|
|
goto done
|
|
case <-ctx.Done():
|
|
// Context cancelled, return empty for remaining
|
|
for j := i; j < len(pending); j++ {
|
|
results[j] = &partitionFetchResult{}
|
|
}
|
|
goto done
|
|
}
|
|
}
|
|
done:
|
|
|
|
// ====================================================================
|
|
// BUILD RESPONSE FROM FETCHED DATA
|
|
// Now assemble the response in the correct order using fetched results
|
|
// ====================================================================
|
|
|
|
// Verify we have results for all requested partitions
|
|
// Sarama requires a response block for EVERY requested partition to avoid ErrIncompleteResponse
|
|
expectedResultCount := 0
|
|
for _, topic := range fetchRequest.Topics {
|
|
expectedResultCount += len(topic.Partitions)
|
|
}
|
|
if len(results) != expectedResultCount {
|
|
glog.Errorf("[%s] Result count mismatch: expected %d, got %d - this will cause ErrIncompleteResponse",
|
|
connContext.ConnectionID, expectedResultCount, len(results))
|
|
// Pad with empty results if needed (safety net - shouldn't happen with fixed code)
|
|
for len(results) < expectedResultCount {
|
|
results = append(results, &partitionFetchResult{})
|
|
}
|
|
}
|
|
|
|
// Process each requested topic
|
|
resultIdx := 0
|
|
for _, topic := range fetchRequest.Topics {
|
|
topicNameBytes := []byte(topic.Name)
|
|
|
|
// Topic name length and name
|
|
if isFlexible {
|
|
// Flexible versions use compact string format (length + 1)
|
|
response = append(response, EncodeUvarint(uint32(len(topicNameBytes)+1))...)
|
|
} else {
|
|
response = append(response, byte(len(topicNameBytes)>>8), byte(len(topicNameBytes)))
|
|
}
|
|
response = append(response, topicNameBytes...)
|
|
|
|
// Partitions count for this topic
|
|
partitionsCount := len(topic.Partitions)
|
|
if isFlexible {
|
|
// Flexible versions use compact array format (count + 1)
|
|
response = append(response, EncodeUvarint(uint32(partitionsCount+1))...)
|
|
} else {
|
|
partitionsCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionsCountBytes, uint32(partitionsCount))
|
|
response = append(response, partitionsCountBytes...)
|
|
}
|
|
|
|
// Process each requested partition (using pre-fetched results)
|
|
for _, partition := range topic.Partitions {
|
|
// Get the pre-fetched result for this partition
|
|
result := results[resultIdx]
|
|
resultIdx++
|
|
|
|
// Partition ID
|
|
partitionIDBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionIDBytes, uint32(partition.PartitionID))
|
|
response = append(response, partitionIDBytes...)
|
|
|
|
// Error code (2 bytes) - use the result's error code
|
|
response = append(response, byte(result.errorCode>>8), byte(result.errorCode))
|
|
|
|
// Use the pre-fetched high water mark from concurrent fetch
|
|
highWaterMark := result.highWaterMark
|
|
|
|
// High water mark (8 bytes)
|
|
highWaterMarkBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(highWaterMarkBytes, uint64(highWaterMark))
|
|
response = append(response, highWaterMarkBytes...)
|
|
|
|
// Fetch v4+ has last_stable_offset and log_start_offset
|
|
if apiVersion >= 4 {
|
|
// Last stable offset (8 bytes) - same as high water mark for non-transactional
|
|
response = append(response, highWaterMarkBytes...)
|
|
// Log start offset (8 bytes) - 0 for simplicity
|
|
response = append(response, 0, 0, 0, 0, 0, 0, 0, 0)
|
|
|
|
// Aborted transactions count (4 bytes) = 0
|
|
response = append(response, 0, 0, 0, 0)
|
|
}
|
|
|
|
// Use the pre-fetched record batch
|
|
recordBatch := result.recordBatch
|
|
|
|
// Records size - flexible versions (v12+) use compact format: varint(size+1)
|
|
if isFlexible {
|
|
if len(recordBatch) == 0 {
|
|
response = append(response, 0) // null records = 0 in compact format
|
|
} else {
|
|
response = append(response, EncodeUvarint(uint32(len(recordBatch)+1))...)
|
|
}
|
|
} else {
|
|
// Non-flexible versions use int32(size)
|
|
recordsSizeBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(recordsSizeBytes, uint32(len(recordBatch)))
|
|
response = append(response, recordsSizeBytes...)
|
|
}
|
|
|
|
// Records data
|
|
response = append(response, recordBatch...)
|
|
totalAppendedRecordBytes += len(recordBatch)
|
|
|
|
// Tagged fields for flexible versions (v12+) after each partition
|
|
if isFlexible {
|
|
response = append(response, 0) // Empty tagged fields
|
|
}
|
|
}
|
|
|
|
// Tagged fields for flexible versions (v12+) after each topic
|
|
if isFlexible {
|
|
response = append(response, 0) // Empty tagged fields
|
|
}
|
|
}
|
|
|
|
// Tagged fields for flexible versions (v12+) at the end of response
|
|
if isFlexible {
|
|
response = append(response, 0) // Empty tagged fields
|
|
}
|
|
|
|
// Verify topics count hasn't been corrupted
|
|
if !isFlexible {
|
|
// Topics count position depends on API version:
|
|
// v0: byte 0 (no throttle_time_ms, no error_code, no session_id)
|
|
// v1-v6: byte 4 (after throttle_time_ms)
|
|
// v7+: byte 10 (after throttle_time_ms, error_code, session_id)
|
|
var topicsCountPos int
|
|
if apiVersion == 0 {
|
|
topicsCountPos = 0
|
|
} else if apiVersion < 7 {
|
|
topicsCountPos = 4
|
|
} else {
|
|
topicsCountPos = 10
|
|
}
|
|
|
|
if len(response) >= topicsCountPos+4 {
|
|
actualTopicsCount := binary.BigEndian.Uint32(response[topicsCountPos : topicsCountPos+4])
|
|
if actualTopicsCount != uint32(topicsCount) {
|
|
glog.Errorf("FETCH CORR=%d v%d: Topics count CORRUPTED! Expected %d, found %d at response[%d:%d]=%02x %02x %02x %02x",
|
|
correlationID, apiVersion, topicsCount, actualTopicsCount, topicsCountPos, topicsCountPos+4,
|
|
response[topicsCountPos], response[topicsCountPos+1], response[topicsCountPos+2], response[topicsCountPos+3])
|
|
}
|
|
}
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// FetchRequest represents a parsed Kafka Fetch request
|
|
type FetchRequest struct {
|
|
ReplicaID int32
|
|
MaxWaitTime int32
|
|
MinBytes int32
|
|
MaxBytes int32
|
|
IsolationLevel int8
|
|
Topics []FetchTopic
|
|
}
|
|
|
|
type FetchTopic struct {
|
|
Name string
|
|
Partitions []FetchPartition
|
|
}
|
|
|
|
type FetchPartition struct {
|
|
PartitionID int32
|
|
FetchOffset int64
|
|
LogStartOffset int64
|
|
MaxBytes int32
|
|
}
|
|
|
|
// parseFetchRequest parses a Kafka Fetch request
|
|
func (h *Handler) parseFetchRequest(apiVersion uint16, requestBody []byte) (*FetchRequest, error) {
|
|
if len(requestBody) < 12 {
|
|
return nil, fmt.Errorf("fetch request too short: %d bytes", len(requestBody))
|
|
}
|
|
|
|
offset := 0
|
|
request := &FetchRequest{}
|
|
|
|
// Check if this version uses flexible format (v12+)
|
|
isFlexible := IsFlexibleVersion(1, apiVersion) // API key 1 = Fetch
|
|
|
|
// NOTE: client_id is already handled by HandleConn and stripped from requestBody
|
|
// Request body starts directly with fetch-specific fields
|
|
|
|
// Replica ID (4 bytes) - always fixed
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for replica_id")
|
|
}
|
|
request.ReplicaID = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Max wait time (4 bytes) - always fixed
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for max_wait_time")
|
|
}
|
|
request.MaxWaitTime = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Min bytes (4 bytes) - always fixed
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for min_bytes")
|
|
}
|
|
request.MinBytes = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Max bytes (4 bytes) - only in v3+, always fixed
|
|
if apiVersion >= 3 {
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for max_bytes")
|
|
}
|
|
request.MaxBytes = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
}
|
|
|
|
// Isolation level (1 byte) - only in v4+, always fixed
|
|
if apiVersion >= 4 {
|
|
if offset+1 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for isolation_level")
|
|
}
|
|
request.IsolationLevel = int8(requestBody[offset])
|
|
offset += 1
|
|
}
|
|
|
|
// Session ID (4 bytes) and Session Epoch (4 bytes) - only in v7+, always fixed
|
|
if apiVersion >= 7 {
|
|
if offset+8 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for session_id and epoch")
|
|
}
|
|
offset += 8 // Skip session_id and session_epoch
|
|
}
|
|
|
|
// Topics count - flexible uses compact array, non-flexible uses INT32
|
|
var topicsCount int
|
|
if isFlexible {
|
|
// Compact array: length+1 encoded as varint
|
|
length, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode topics compact array: %w", err)
|
|
}
|
|
topicsCount = int(length)
|
|
offset += consumed
|
|
} else {
|
|
// Regular array: INT32 length
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for topics count")
|
|
}
|
|
topicsCount = int(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
}
|
|
|
|
// Parse topics
|
|
request.Topics = make([]FetchTopic, topicsCount)
|
|
for i := 0; i < topicsCount; i++ {
|
|
// Topic name - flexible uses compact string, non-flexible uses STRING (INT16 length)
|
|
var topicName string
|
|
if isFlexible {
|
|
// Compact string: length+1 encoded as varint
|
|
name, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode topic name compact string: %w", err)
|
|
}
|
|
topicName = name
|
|
offset += consumed
|
|
} else {
|
|
// Regular string: INT16 length + bytes
|
|
if offset+2 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for topic name length")
|
|
}
|
|
topicNameLength := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
|
|
if offset+topicNameLength > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for topic name")
|
|
}
|
|
topicName = string(requestBody[offset : offset+topicNameLength])
|
|
offset += topicNameLength
|
|
}
|
|
request.Topics[i].Name = topicName
|
|
|
|
// Partitions count - flexible uses compact array, non-flexible uses INT32
|
|
var partitionsCount int
|
|
if isFlexible {
|
|
// Compact array: length+1 encoded as varint
|
|
length, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode partitions compact array: %w", err)
|
|
}
|
|
partitionsCount = int(length)
|
|
offset += consumed
|
|
} else {
|
|
// Regular array: INT32 length
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for partitions count")
|
|
}
|
|
partitionsCount = int(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
}
|
|
|
|
// Parse partitions
|
|
request.Topics[i].Partitions = make([]FetchPartition, partitionsCount)
|
|
for j := 0; j < partitionsCount; j++ {
|
|
// Partition ID (4 bytes) - always fixed
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for partition ID")
|
|
}
|
|
request.Topics[i].Partitions[j].PartitionID = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Current leader epoch (4 bytes) - only in v9+, always fixed
|
|
if apiVersion >= 9 {
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for current leader epoch")
|
|
}
|
|
offset += 4 // Skip current leader epoch
|
|
}
|
|
|
|
// Fetch offset (8 bytes) - always fixed
|
|
if offset+8 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for fetch offset")
|
|
}
|
|
request.Topics[i].Partitions[j].FetchOffset = int64(binary.BigEndian.Uint64(requestBody[offset : offset+8]))
|
|
offset += 8
|
|
|
|
// Log start offset (8 bytes) - only in v5+, always fixed
|
|
if apiVersion >= 5 {
|
|
if offset+8 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for log start offset")
|
|
}
|
|
request.Topics[i].Partitions[j].LogStartOffset = int64(binary.BigEndian.Uint64(requestBody[offset : offset+8]))
|
|
offset += 8
|
|
}
|
|
|
|
// Partition max bytes (4 bytes) - always fixed
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for partition max bytes")
|
|
}
|
|
request.Topics[i].Partitions[j].MaxBytes = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Tagged fields for partition (only in flexible versions v12+)
|
|
if isFlexible {
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode partition tagged fields: %w", err)
|
|
}
|
|
offset += consumed
|
|
}
|
|
}
|
|
|
|
// Tagged fields for topic (only in flexible versions v12+)
|
|
if isFlexible {
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode topic tagged fields: %w", err)
|
|
}
|
|
offset += consumed
|
|
}
|
|
}
|
|
|
|
// Forgotten topics data (only in v7+)
|
|
if apiVersion >= 7 {
|
|
// Skip forgotten topics array - we don't use incremental fetch yet
|
|
var forgottenTopicsCount int
|
|
if isFlexible {
|
|
length, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode forgotten topics compact array: %w", err)
|
|
}
|
|
forgottenTopicsCount = int(length)
|
|
offset += consumed
|
|
} else {
|
|
if offset+4 > len(requestBody) {
|
|
// End of request, no forgotten topics
|
|
return request, nil
|
|
}
|
|
forgottenTopicsCount = int(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
}
|
|
|
|
// Skip forgotten topics if present
|
|
for i := 0; i < forgottenTopicsCount && offset < len(requestBody); i++ {
|
|
// Skip topic name
|
|
if isFlexible {
|
|
_, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
break
|
|
}
|
|
offset += consumed
|
|
} else {
|
|
if offset+2 > len(requestBody) {
|
|
break
|
|
}
|
|
nameLen := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2 + nameLen
|
|
}
|
|
|
|
// Skip partitions array
|
|
if isFlexible {
|
|
length, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
break
|
|
}
|
|
offset += consumed
|
|
// Skip partition IDs (4 bytes each)
|
|
offset += int(length) * 4
|
|
} else {
|
|
if offset+4 > len(requestBody) {
|
|
break
|
|
}
|
|
partCount := int(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4 + partCount*4
|
|
}
|
|
|
|
// Skip tagged fields if flexible
|
|
if isFlexible {
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
break
|
|
}
|
|
offset += consumed
|
|
}
|
|
}
|
|
}
|
|
|
|
// Rack ID (only in v11+) - optional string
|
|
if apiVersion >= 11 && offset < len(requestBody) {
|
|
if isFlexible {
|
|
_, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err == nil {
|
|
offset += consumed
|
|
}
|
|
} else {
|
|
if offset+2 <= len(requestBody) {
|
|
rackIDLen := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
if rackIDLen >= 0 && offset+2+rackIDLen <= len(requestBody) {
|
|
offset += 2 + rackIDLen
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// Top-level tagged fields (only in flexible versions v12+)
|
|
if isFlexible && offset < len(requestBody) {
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
// Don't fail on trailing tagged fields parsing
|
|
} else {
|
|
offset += consumed
|
|
}
|
|
}
|
|
|
|
return request, nil
|
|
}
|
|
|
|
// constructRecordBatchFromSMQ creates a Kafka record batch from SeaweedMQ records
|
|
func (h *Handler) constructRecordBatchFromSMQ(topicName string, fetchOffset int64, smqRecords []integration.SMQRecord) []byte {
|
|
if len(smqRecords) == 0 {
|
|
return []byte{}
|
|
}
|
|
|
|
// Create record batch using the SMQ records
|
|
batch := make([]byte, 0, 512)
|
|
|
|
// Record batch header
|
|
baseOffsetBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(baseOffsetBytes, uint64(fetchOffset))
|
|
batch = append(batch, baseOffsetBytes...) // base offset (8 bytes)
|
|
|
|
// Calculate batch length (will be filled after we know the size)
|
|
batchLengthPos := len(batch)
|
|
batch = append(batch, 0, 0, 0, 0) // batch length placeholder (4 bytes)
|
|
|
|
// Partition leader epoch (4 bytes) - use 0 (real Kafka uses 0, not -1)
|
|
batch = append(batch, 0x00, 0x00, 0x00, 0x00)
|
|
|
|
// Magic byte (1 byte) - v2 format
|
|
batch = append(batch, 2)
|
|
|
|
// CRC placeholder (4 bytes) - will be calculated later
|
|
crcPos := len(batch)
|
|
batch = append(batch, 0, 0, 0, 0)
|
|
|
|
// Attributes (2 bytes) - no compression, etc.
|
|
batch = append(batch, 0, 0)
|
|
|
|
// Last offset delta (4 bytes)
|
|
lastOffsetDelta := int32(len(smqRecords) - 1)
|
|
lastOffsetDeltaBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(lastOffsetDeltaBytes, uint32(lastOffsetDelta))
|
|
batch = append(batch, lastOffsetDeltaBytes...)
|
|
|
|
// Base timestamp (8 bytes) - convert from nanoseconds to milliseconds for Kafka compatibility
|
|
baseTimestamp := smqRecords[0].GetTimestamp() / 1000000 // Convert nanoseconds to milliseconds
|
|
baseTimestampBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(baseTimestampBytes, uint64(baseTimestamp))
|
|
batch = append(batch, baseTimestampBytes...)
|
|
|
|
// Max timestamp (8 bytes) - convert from nanoseconds to milliseconds for Kafka compatibility
|
|
maxTimestamp := baseTimestamp
|
|
if len(smqRecords) > 1 {
|
|
maxTimestamp = smqRecords[len(smqRecords)-1].GetTimestamp() / 1000000 // Convert nanoseconds to milliseconds
|
|
}
|
|
maxTimestampBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(maxTimestampBytes, uint64(maxTimestamp))
|
|
batch = append(batch, maxTimestampBytes...)
|
|
|
|
// Producer ID (8 bytes) - use -1 for no producer ID
|
|
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)
|
|
|
|
// Producer epoch (2 bytes) - use -1 for no producer epoch
|
|
batch = append(batch, 0xFF, 0xFF)
|
|
|
|
// Base sequence (4 bytes) - use -1 for no base sequence
|
|
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF)
|
|
|
|
// Records count (4 bytes)
|
|
recordCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(recordCountBytes, uint32(len(smqRecords)))
|
|
batch = append(batch, recordCountBytes...)
|
|
|
|
// Add individual records from SMQ records
|
|
for i, smqRecord := range smqRecords {
|
|
// Build individual record
|
|
recordBytes := make([]byte, 0, 128)
|
|
|
|
// Record attributes (1 byte)
|
|
recordBytes = append(recordBytes, 0)
|
|
|
|
// Timestamp delta (varint) - calculate from base timestamp (both in milliseconds)
|
|
recordTimestampMs := smqRecord.GetTimestamp() / 1000000 // Convert nanoseconds to milliseconds
|
|
timestampDelta := recordTimestampMs - baseTimestamp // Both in milliseconds now
|
|
recordBytes = append(recordBytes, encodeVarint(timestampDelta)...)
|
|
|
|
// Offset delta (varint)
|
|
offsetDelta := int64(i)
|
|
recordBytes = append(recordBytes, encodeVarint(offsetDelta)...)
|
|
|
|
// Key length and key (varint + data) - decode RecordValue to get original Kafka message
|
|
key := h.decodeRecordValueToKafkaMessage(topicName, smqRecord.GetKey())
|
|
if key == nil {
|
|
recordBytes = append(recordBytes, encodeVarint(-1)...) // null key
|
|
} else {
|
|
recordBytes = append(recordBytes, encodeVarint(int64(len(key)))...)
|
|
recordBytes = append(recordBytes, key...)
|
|
}
|
|
|
|
// Value length and value (varint + data) - decode RecordValue to get original Kafka message
|
|
value := h.decodeRecordValueToKafkaMessage(topicName, smqRecord.GetValue())
|
|
|
|
if value == nil {
|
|
recordBytes = append(recordBytes, encodeVarint(-1)...) // null value
|
|
} else {
|
|
recordBytes = append(recordBytes, encodeVarint(int64(len(value)))...)
|
|
recordBytes = append(recordBytes, value...)
|
|
}
|
|
|
|
// Headers count (varint) - 0 headers
|
|
recordBytes = append(recordBytes, encodeVarint(0)...)
|
|
|
|
// Prepend record length (varint)
|
|
recordLength := int64(len(recordBytes))
|
|
batch = append(batch, encodeVarint(recordLength)...)
|
|
batch = append(batch, recordBytes...)
|
|
}
|
|
|
|
// Fill in the batch length
|
|
batchLength := uint32(len(batch) - batchLengthPos - 4)
|
|
binary.BigEndian.PutUint32(batch[batchLengthPos:batchLengthPos+4], batchLength)
|
|
|
|
// Calculate CRC32 for the batch
|
|
// Kafka CRC calculation covers: partition leader epoch + magic + attributes + ... (everything after batch length)
|
|
// Skip: BaseOffset(8) + BatchLength(4) = 12 bytes
|
|
crcData := batch[crcPos+4:] // CRC covers ONLY from attributes (byte 21) onwards // Skip CRC field itself, include rest
|
|
crc := crc32.Checksum(crcData, crc32.MakeTable(crc32.Castagnoli))
|
|
binary.BigEndian.PutUint32(batch[crcPos:crcPos+4], crc)
|
|
|
|
return batch
|
|
}
|
|
|
|
// encodeVarint encodes a signed integer using Kafka's varint encoding
|
|
func encodeVarint(value int64) []byte {
|
|
// Kafka uses zigzag encoding for signed integers
|
|
zigzag := uint64((value << 1) ^ (value >> 63))
|
|
|
|
var buf []byte
|
|
for zigzag >= 0x80 {
|
|
buf = append(buf, byte(zigzag)|0x80)
|
|
zigzag >>= 7
|
|
}
|
|
buf = append(buf, byte(zigzag))
|
|
return buf
|
|
}
|
|
|
|
// SchematizedRecord holds both key and value for schematized messages
|
|
type SchematizedRecord struct {
|
|
Key []byte
|
|
Value []byte
|
|
}
|
|
|
|
// createEmptyRecordBatch creates an empty Kafka record batch using the new parser
|
|
func (h *Handler) createEmptyRecordBatch(baseOffset int64) []byte {
|
|
// Use the new record batch creation function with no compression
|
|
emptyRecords := []byte{}
|
|
batch, err := CreateRecordBatch(baseOffset, emptyRecords, compression.None)
|
|
if err != nil {
|
|
// Fallback to manual creation if there's an error
|
|
return h.createEmptyRecordBatchManual(baseOffset)
|
|
}
|
|
return batch
|
|
}
|
|
|
|
// createEmptyRecordBatchManual creates an empty Kafka record batch manually (fallback)
|
|
func (h *Handler) createEmptyRecordBatchManual(baseOffset int64) []byte {
|
|
// Create a minimal empty record batch
|
|
batch := make([]byte, 0, 61) // Standard record batch header size
|
|
|
|
// Base offset (8 bytes)
|
|
baseOffsetBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(baseOffsetBytes, uint64(baseOffset))
|
|
batch = append(batch, baseOffsetBytes...)
|
|
|
|
// Batch length (4 bytes) - will be filled at the end
|
|
lengthPlaceholder := len(batch)
|
|
batch = append(batch, 0, 0, 0, 0)
|
|
|
|
// Partition leader epoch (4 bytes) - 0 for simplicity
|
|
batch = append(batch, 0, 0, 0, 0)
|
|
|
|
// Magic byte (1 byte) - version 2
|
|
batch = append(batch, 2)
|
|
|
|
// CRC32 (4 bytes) - placeholder, should be calculated
|
|
batch = append(batch, 0, 0, 0, 0)
|
|
|
|
// Attributes (2 bytes) - no compression, no transactional
|
|
batch = append(batch, 0, 0)
|
|
|
|
// Last offset delta (4 bytes) - 0 for empty batch
|
|
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF)
|
|
|
|
// First timestamp (8 bytes) - current time
|
|
timestamp := time.Now().UnixMilli()
|
|
timestampBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(timestampBytes, uint64(timestamp))
|
|
batch = append(batch, timestampBytes...)
|
|
|
|
// Max timestamp (8 bytes) - same as first for empty batch
|
|
batch = append(batch, timestampBytes...)
|
|
|
|
// Producer ID (8 bytes) - -1 for non-transactional
|
|
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)
|
|
|
|
// Producer Epoch (2 bytes) - -1 for non-transactional
|
|
batch = append(batch, 0xFF, 0xFF)
|
|
|
|
// Base Sequence (4 bytes) - -1 for non-transactional
|
|
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF)
|
|
|
|
// Record count (4 bytes) - 0 for empty batch
|
|
batch = append(batch, 0, 0, 0, 0)
|
|
|
|
// Fill in the batch length
|
|
batchLength := len(batch) - 12 // Exclude base offset and length field itself
|
|
binary.BigEndian.PutUint32(batch[lengthPlaceholder:lengthPlaceholder+4], uint32(batchLength))
|
|
|
|
return batch
|
|
}
|
|
|
|
// isSchematizedTopic checks if a topic uses schema management
|
|
func (h *Handler) isSchematizedTopic(topicName string) bool {
|
|
// System topics (_schemas, __consumer_offsets, etc.) should NEVER use schema encoding
|
|
// They have their own internal formats and should be passed through as-is
|
|
if h.isSystemTopic(topicName) {
|
|
return false
|
|
}
|
|
|
|
if !h.IsSchemaEnabled() {
|
|
return false
|
|
}
|
|
|
|
// Check multiple indicators for schematized topics:
|
|
|
|
// Check Confluent Schema Registry naming conventions
|
|
return h.matchesSchemaRegistryConvention(topicName)
|
|
}
|
|
|
|
// matchesSchemaRegistryConvention checks Confluent Schema Registry naming patterns
|
|
func (h *Handler) matchesSchemaRegistryConvention(topicName string) bool {
|
|
// Common Schema Registry subject patterns:
|
|
// - topicName-value (for message values)
|
|
// - topicName-key (for message keys)
|
|
// - topicName (direct topic name as subject)
|
|
|
|
if len(topicName) > 6 && topicName[len(topicName)-6:] == "-value" {
|
|
return true
|
|
}
|
|
if len(topicName) > 4 && topicName[len(topicName)-4:] == "-key" {
|
|
return true
|
|
}
|
|
|
|
// Check if the topic has registered schema subjects in Schema Registry
|
|
// Use standard Kafka naming convention: <topic>-value and <topic>-key
|
|
if h.schemaManager != nil {
|
|
// Check with -value suffix (standard pattern for value schemas)
|
|
latestSchemaValue, err := h.schemaManager.GetLatestSchema(topicName + "-value")
|
|
if err == nil {
|
|
// Since we retrieved schema from registry, ensure topic config is updated
|
|
h.ensureTopicSchemaFromLatestSchema(topicName, latestSchemaValue)
|
|
return true
|
|
}
|
|
|
|
// Check with -key suffix (for key schemas)
|
|
latestSchemaKey, err := h.schemaManager.GetLatestSchema(topicName + "-key")
|
|
if err == nil {
|
|
// Since we retrieved key schema from registry, ensure topic config is updated
|
|
h.ensureTopicKeySchemaFromLatestSchema(topicName, latestSchemaKey)
|
|
return true
|
|
}
|
|
}
|
|
|
|
return false
|
|
}
|
|
|
|
// getSchemaMetadataForTopic retrieves schema metadata for a topic
|
|
func (h *Handler) getSchemaMetadataForTopic(topicName string) (map[string]string, error) {
|
|
if !h.IsSchemaEnabled() {
|
|
return nil, fmt.Errorf("schema management not enabled")
|
|
}
|
|
|
|
// Try multiple approaches to get schema metadata from Schema Registry
|
|
|
|
// 1. Try to get schema from registry using topic name as subject
|
|
metadata, err := h.getSchemaMetadataFromRegistry(topicName)
|
|
if err == nil {
|
|
return metadata, nil
|
|
}
|
|
|
|
// 2. Try with -value suffix (common pattern)
|
|
metadata, err = h.getSchemaMetadataFromRegistry(topicName + "-value")
|
|
if err == nil {
|
|
return metadata, nil
|
|
}
|
|
|
|
// 3. Try with -key suffix
|
|
metadata, err = h.getSchemaMetadataFromRegistry(topicName + "-key")
|
|
if err == nil {
|
|
return metadata, nil
|
|
}
|
|
|
|
return nil, fmt.Errorf("no schema found in registry for topic %s (tried %s, %s-value, %s-key)", topicName, topicName, topicName, topicName)
|
|
}
|
|
|
|
// getSchemaMetadataFromRegistry retrieves schema metadata from Schema Registry
|
|
func (h *Handler) getSchemaMetadataFromRegistry(subject string) (map[string]string, error) {
|
|
if h.schemaManager == nil {
|
|
return nil, fmt.Errorf("schema manager not available")
|
|
}
|
|
|
|
// Get latest schema for the subject
|
|
cachedSchema, err := h.schemaManager.GetLatestSchema(subject)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get schema for subject %s: %w", subject, err)
|
|
}
|
|
|
|
// Since we retrieved schema from registry, ensure topic config is updated
|
|
// Extract topic name from subject (remove -key or -value suffix if present)
|
|
topicName := h.extractTopicFromSubject(subject)
|
|
if topicName != "" {
|
|
h.ensureTopicSchemaFromLatestSchema(topicName, cachedSchema)
|
|
}
|
|
|
|
// Build metadata map
|
|
// Detect format from schema content
|
|
// Simple format detection - assume Avro for now
|
|
format := schema.FormatAvro
|
|
|
|
metadata := map[string]string{
|
|
"schema_id": fmt.Sprintf("%d", cachedSchema.LatestID),
|
|
"schema_format": format.String(),
|
|
"schema_subject": subject,
|
|
"schema_version": fmt.Sprintf("%d", cachedSchema.Version),
|
|
"schema_content": cachedSchema.Schema,
|
|
}
|
|
|
|
return metadata, nil
|
|
}
|
|
|
|
// ensureTopicSchemaFromLatestSchema ensures topic configuration is updated when latest schema is retrieved
|
|
func (h *Handler) ensureTopicSchemaFromLatestSchema(topicName string, latestSchema *schema.CachedSubject) {
|
|
if latestSchema == nil {
|
|
return
|
|
}
|
|
|
|
// Convert CachedSubject to CachedSchema format for reuse
|
|
// Note: CachedSubject has different field structure than expected
|
|
cachedSchema := &schema.CachedSchema{
|
|
ID: latestSchema.LatestID,
|
|
Schema: latestSchema.Schema,
|
|
Subject: latestSchema.Subject,
|
|
Version: latestSchema.Version,
|
|
Format: schema.FormatAvro, // Default to Avro, could be improved with format detection
|
|
CachedAt: latestSchema.CachedAt,
|
|
}
|
|
|
|
// Use existing function to handle the schema update
|
|
h.ensureTopicSchemaFromRegistryCache(topicName, cachedSchema)
|
|
}
|
|
|
|
// extractTopicFromSubject extracts the topic name from a schema registry subject
|
|
func (h *Handler) extractTopicFromSubject(subject string) string {
|
|
// Remove common suffixes used in schema registry
|
|
if strings.HasSuffix(subject, "-value") {
|
|
return strings.TrimSuffix(subject, "-value")
|
|
}
|
|
if strings.HasSuffix(subject, "-key") {
|
|
return strings.TrimSuffix(subject, "-key")
|
|
}
|
|
// If no suffix, assume subject name is the topic name
|
|
return subject
|
|
}
|
|
|
|
// ensureTopicKeySchemaFromLatestSchema ensures topic configuration is updated when key schema is retrieved
|
|
func (h *Handler) ensureTopicKeySchemaFromLatestSchema(topicName string, latestSchema *schema.CachedSubject) {
|
|
if latestSchema == nil {
|
|
return
|
|
}
|
|
|
|
// Convert CachedSubject to CachedSchema format for reuse
|
|
// Note: CachedSubject has different field structure than expected
|
|
cachedSchema := &schema.CachedSchema{
|
|
ID: latestSchema.LatestID,
|
|
Schema: latestSchema.Schema,
|
|
Subject: latestSchema.Subject,
|
|
Version: latestSchema.Version,
|
|
Format: schema.FormatAvro, // Default to Avro, could be improved with format detection
|
|
CachedAt: latestSchema.CachedAt,
|
|
}
|
|
|
|
// Use existing function to handle the key schema update
|
|
h.ensureTopicKeySchemaFromRegistryCache(topicName, cachedSchema)
|
|
}
|
|
|
|
// decodeRecordValueToKafkaMessage decodes a RecordValue back to the original Kafka message bytes
|
|
func (h *Handler) decodeRecordValueToKafkaMessage(topicName string, recordValueBytes []byte) []byte {
|
|
if recordValueBytes == nil {
|
|
return nil
|
|
}
|
|
|
|
// For system topics like _schemas, _consumer_offsets, etc.,
|
|
// return the raw bytes as-is. These topics store Kafka's internal format (Avro, etc.)
|
|
// and should NOT be processed as RecordValue protobuf messages.
|
|
if strings.HasPrefix(topicName, "_") {
|
|
return recordValueBytes
|
|
}
|
|
|
|
// CRITICAL: If schema management is not enabled, we should NEVER try to parse as RecordValue
|
|
// All messages are stored as raw bytes when schema management is disabled
|
|
// Attempting to parse them as RecordValue will cause corruption due to protobuf's lenient parsing
|
|
if !h.IsSchemaEnabled() {
|
|
return recordValueBytes
|
|
}
|
|
|
|
// Try to unmarshal as RecordValue
|
|
recordValue := &schema_pb.RecordValue{}
|
|
if err := proto.Unmarshal(recordValueBytes, recordValue); err != nil {
|
|
// Not a RecordValue format - this is normal for Avro/JSON/raw Kafka messages
|
|
// Return raw bytes as-is (Kafka consumers expect this)
|
|
return recordValueBytes
|
|
}
|
|
|
|
// Validate that the unmarshaled RecordValue is actually a valid RecordValue
|
|
// Protobuf unmarshal is lenient and can succeed with garbage data for random bytes
|
|
// We need to check if this looks like a real RecordValue or just random bytes
|
|
if !h.isValidRecordValue(recordValue, recordValueBytes) {
|
|
// Not a valid RecordValue - return raw bytes as-is
|
|
return recordValueBytes
|
|
}
|
|
|
|
// If schema management is enabled, re-encode the RecordValue to Confluent format
|
|
if h.IsSchemaEnabled() {
|
|
if encodedMsg, err := h.encodeRecordValueToConfluentFormat(topicName, recordValue); err == nil {
|
|
return encodedMsg
|
|
} else {
|
|
}
|
|
}
|
|
|
|
// Fallback: convert RecordValue to JSON
|
|
return h.recordValueToJSON(recordValue)
|
|
}
|
|
|
|
// isValidRecordValue checks if a RecordValue looks like a real RecordValue or garbage from random bytes
|
|
// This performs a roundtrip test: marshal the RecordValue and check if it produces similar output
|
|
func (h *Handler) isValidRecordValue(recordValue *schema_pb.RecordValue, originalBytes []byte) bool {
|
|
// Empty or nil Fields means not a valid RecordValue
|
|
if recordValue == nil || recordValue.Fields == nil || len(recordValue.Fields) == 0 {
|
|
return false
|
|
}
|
|
|
|
// Check if field names are valid UTF-8 strings (not binary garbage)
|
|
// Real RecordValue messages have proper field names like "name", "age", etc.
|
|
// Random bytes parsed as protobuf often create non-UTF8 or very short field names
|
|
for fieldName, fieldValue := range recordValue.Fields {
|
|
// Field name should be valid UTF-8
|
|
if !utf8.ValidString(fieldName) {
|
|
return false
|
|
}
|
|
|
|
// Field name should have reasonable length (at least 1 char, at most 1000)
|
|
if len(fieldName) == 0 || len(fieldName) > 1000 {
|
|
return false
|
|
}
|
|
|
|
// Field value should not be nil
|
|
if fieldValue == nil || fieldValue.Kind == nil {
|
|
return false
|
|
}
|
|
}
|
|
|
|
// Roundtrip check: If this is a real RecordValue, marshaling it back should produce
|
|
// similar-sized output. Random bytes that accidentally parse as protobuf will typically
|
|
// produce very different output when marshaled back.
|
|
remarshaled, err := proto.Marshal(recordValue)
|
|
if err != nil {
|
|
return false
|
|
}
|
|
|
|
// Check if the sizes are reasonably similar (within 50% tolerance)
|
|
// Real RecordValue will have similar size, random bytes will be very different
|
|
originalSize := len(originalBytes)
|
|
remarshaledSize := len(remarshaled)
|
|
if originalSize == 0 {
|
|
return false
|
|
}
|
|
|
|
// Calculate size ratio - should be close to 1.0 for real RecordValue
|
|
ratio := float64(remarshaledSize) / float64(originalSize)
|
|
if ratio < 0.5 || ratio > 2.0 {
|
|
// Size differs too much - this is likely random bytes parsed as protobuf
|
|
return false
|
|
}
|
|
|
|
return true
|
|
}
|
|
|
|
// encodeRecordValueToConfluentFormat re-encodes a RecordValue back to Confluent format
|
|
func (h *Handler) encodeRecordValueToConfluentFormat(topicName string, recordValue *schema_pb.RecordValue) ([]byte, error) {
|
|
if recordValue == nil {
|
|
return nil, fmt.Errorf("RecordValue is nil")
|
|
}
|
|
|
|
// Get schema configuration from topic config
|
|
schemaConfig, err := h.getTopicSchemaConfig(topicName)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get topic schema config: %w", err)
|
|
}
|
|
|
|
// Use schema manager to encode RecordValue back to original format
|
|
encodedBytes, err := h.schemaManager.EncodeMessage(recordValue, schemaConfig.ValueSchemaID, schemaConfig.ValueSchemaFormat)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to encode RecordValue: %w", err)
|
|
}
|
|
|
|
return encodedBytes, nil
|
|
}
|
|
|
|
// getTopicSchemaConfig retrieves schema configuration for a topic
|
|
func (h *Handler) getTopicSchemaConfig(topicName string) (*TopicSchemaConfig, error) {
|
|
h.topicSchemaConfigMu.RLock()
|
|
defer h.topicSchemaConfigMu.RUnlock()
|
|
|
|
if h.topicSchemaConfigs == nil {
|
|
return nil, fmt.Errorf("no schema configuration available for topic: %s", topicName)
|
|
}
|
|
|
|
config, exists := h.topicSchemaConfigs[topicName]
|
|
if !exists {
|
|
return nil, fmt.Errorf("no schema configuration found for topic: %s", topicName)
|
|
}
|
|
|
|
return config, nil
|
|
}
|
|
|
|
// recordValueToJSON converts a RecordValue to JSON bytes (fallback)
|
|
func (h *Handler) recordValueToJSON(recordValue *schema_pb.RecordValue) []byte {
|
|
if recordValue == nil || recordValue.Fields == nil {
|
|
return []byte("{}")
|
|
}
|
|
|
|
// Simple JSON conversion - in a real implementation, this would be more sophisticated
|
|
jsonStr := "{"
|
|
first := true
|
|
for fieldName, fieldValue := range recordValue.Fields {
|
|
if !first {
|
|
jsonStr += ","
|
|
}
|
|
first = false
|
|
|
|
jsonStr += fmt.Sprintf(`"%s":`, fieldName)
|
|
|
|
switch v := fieldValue.Kind.(type) {
|
|
case *schema_pb.Value_StringValue:
|
|
jsonStr += fmt.Sprintf(`"%s"`, v.StringValue)
|
|
case *schema_pb.Value_BytesValue:
|
|
jsonStr += fmt.Sprintf(`"%s"`, string(v.BytesValue))
|
|
case *schema_pb.Value_Int32Value:
|
|
jsonStr += fmt.Sprintf(`%d`, v.Int32Value)
|
|
case *schema_pb.Value_Int64Value:
|
|
jsonStr += fmt.Sprintf(`%d`, v.Int64Value)
|
|
case *schema_pb.Value_BoolValue:
|
|
jsonStr += fmt.Sprintf(`%t`, v.BoolValue)
|
|
default:
|
|
jsonStr += `null`
|
|
}
|
|
}
|
|
jsonStr += "}"
|
|
|
|
return []byte(jsonStr)
|
|
}
|