From 59f1c3dda55e10439c4fac93b1c0553b097164ea Mon Sep 17 00:00:00 2001 From: chrislu Date: Wed, 10 Sep 2025 12:57:19 -0700 Subject: [PATCH] mq(kafka): implement Fetch handler with record batch construction, high watermark tracking, and comprehensive test coverage for consumer functionality --- weed/mq/kafka/protocol/fetch.go | 298 ++++++++++++++++++++++ weed/mq/kafka/protocol/fetch_test.go | 339 +++++++++++++++++++++++++ weed/mq/kafka/protocol/handler.go | 9 +- weed/mq/kafka/protocol/handler_test.go | 33 ++- 4 files changed, 672 insertions(+), 7 deletions(-) create mode 100644 weed/mq/kafka/protocol/fetch.go create mode 100644 weed/mq/kafka/protocol/fetch_test.go diff --git a/weed/mq/kafka/protocol/fetch.go b/weed/mq/kafka/protocol/fetch.go new file mode 100644 index 000000000..bcdd2ce5e --- /dev/null +++ b/weed/mq/kafka/protocol/fetch.go @@ -0,0 +1,298 @@ +package protocol + +import ( + "encoding/binary" + "fmt" + "time" +) + +func (h *Handler) handleFetch(correlationID uint32, requestBody []byte) ([]byte, error) { + // Parse minimal Fetch request + // Request format: client_id + replica_id(4) + max_wait_time(4) + min_bytes(4) + max_bytes(4) + isolation_level(1) + session_id(4) + epoch(4) + topics_array + + if len(requestBody) < 8 { // client_id_size(2) + replica_id(4) + max_wait_time(4) + ... + return nil, fmt.Errorf("Fetch request too short") + } + + // Skip client_id + clientIDSize := binary.BigEndian.Uint16(requestBody[0:2]) + offset := 2 + int(clientIDSize) + + if len(requestBody) < offset+21 { // replica_id(4) + max_wait_time(4) + min_bytes(4) + max_bytes(4) + isolation_level(1) + session_id(4) + epoch(4) + return nil, fmt.Errorf("Fetch request missing data") + } + + // Parse Fetch parameters + replicaID := int32(binary.BigEndian.Uint32(requestBody[offset : offset+4])) + offset += 4 + maxWaitTime := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + minBytes := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + maxBytes := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + isolationLevel := requestBody[offset] + offset += 1 + sessionID := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + epoch := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + + // For Phase 1, ignore most parameters and focus on basic functionality + _ = replicaID + _ = maxWaitTime + _ = minBytes + _ = maxBytes + _ = isolationLevel + _ = sessionID + _ = epoch + + if len(requestBody) < offset+4 { + return nil, fmt.Errorf("Fetch request missing topics count") + } + + topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + + response := make([]byte, 0, 1024) + + // Correlation ID + correlationIDBytes := make([]byte, 4) + binary.BigEndian.PutUint32(correlationIDBytes, correlationID) + response = append(response, correlationIDBytes...) + + // Throttle time (4 bytes, 0 = no throttling) + response = append(response, 0, 0, 0, 0) + + // Error code (2 bytes, 0 = no error) + response = append(response, 0, 0) + + // Session ID (4 bytes) + sessionIDBytes := make([]byte, 4) + binary.BigEndian.PutUint32(sessionIDBytes, sessionID) + response = append(response, sessionIDBytes...) + + // Topics count (same as request) + topicsCountBytes := make([]byte, 4) + binary.BigEndian.PutUint32(topicsCountBytes, topicsCount) + response = append(response, topicsCountBytes...) + + // Process each topic + for i := uint32(0); i < topicsCount && offset < len(requestBody); i++ { + if len(requestBody) < offset+2 { + break + } + + // Parse topic name + topicNameSize := binary.BigEndian.Uint16(requestBody[offset : offset+2]) + offset += 2 + + if len(requestBody) < offset+int(topicNameSize)+4 { + break + } + + topicName := string(requestBody[offset : offset+int(topicNameSize)]) + offset += int(topicNameSize) + + // Parse partitions count + partitionsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + + // Check if topic exists + h.topicsMu.RLock() + _, topicExists := h.topics[topicName] + h.topicsMu.RUnlock() + + // Response: topic_name_size(2) + topic_name + partitions_array + response = append(response, byte(topicNameSize>>8), byte(topicNameSize)) + response = append(response, []byte(topicName)...) + + partitionsCountBytes := make([]byte, 4) + binary.BigEndian.PutUint32(partitionsCountBytes, partitionsCount) + response = append(response, partitionsCountBytes...) + + // Process each partition + for j := uint32(0); j < partitionsCount && offset < len(requestBody); j++ { + if len(requestBody) < offset+16 { + break + } + + // Parse partition: partition_id(4) + current_leader_epoch(4) + fetch_offset(8) + partitionID := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + currentLeaderEpoch := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + fetchOffset := int64(binary.BigEndian.Uint64(requestBody[offset : offset+8])) + offset += 8 + logStartOffset := int64(binary.BigEndian.Uint64(requestBody[offset : offset+8])) + offset += 8 + partitionMaxBytes := binary.BigEndian.Uint32(requestBody[offset : offset+4]) + offset += 4 + + _ = currentLeaderEpoch + _ = logStartOffset + _ = partitionMaxBytes + + // Response: partition_id(4) + error_code(2) + high_water_mark(8) + last_stable_offset(8) + log_start_offset(8) + aborted_transactions + records + partitionIDBytes := make([]byte, 4) + binary.BigEndian.PutUint32(partitionIDBytes, partitionID) + response = append(response, partitionIDBytes...) + + var errorCode uint16 = 0 + var highWaterMark int64 = 0 + var records []byte + + if !topicExists { + errorCode = 3 // UNKNOWN_TOPIC_OR_PARTITION + } else { + // Get ledger and fetch records + ledger := h.GetLedger(topicName, int32(partitionID)) + if ledger == nil { + errorCode = 3 // UNKNOWN_TOPIC_OR_PARTITION + } else { + highWaterMark = ledger.GetHighWaterMark() + + // For Phase 1, construct simple record batches for any messages in range + if fetchOffset < highWaterMark { + records = h.constructRecordBatch(ledger, fetchOffset, highWaterMark) + } + } + } + + // Error code + response = append(response, byte(errorCode>>8), byte(errorCode)) + + // High water mark (8 bytes) + highWaterMarkBytes := make([]byte, 8) + binary.BigEndian.PutUint64(highWaterMarkBytes, uint64(highWaterMark)) + response = append(response, highWaterMarkBytes...) + + // Last stable offset (8 bytes) - same as high water mark for simplicity + lastStableOffsetBytes := make([]byte, 8) + binary.BigEndian.PutUint64(lastStableOffsetBytes, uint64(highWaterMark)) + response = append(response, lastStableOffsetBytes...) + + // Log start offset (8 bytes) + logStartOffsetBytes := make([]byte, 8) + binary.BigEndian.PutUint64(logStartOffsetBytes, 0) // always 0 for Phase 1 + response = append(response, logStartOffsetBytes...) + + // Aborted transactions count (4 bytes, 0 = none) + response = append(response, 0, 0, 0, 0) + + // Records size and data + recordsSize := uint32(len(records)) + recordsSizeBytes := make([]byte, 4) + binary.BigEndian.PutUint32(recordsSizeBytes, recordsSize) + response = append(response, recordsSizeBytes...) + response = append(response, records...) + } + } + + return response, nil +} + +// constructRecordBatch creates a simplified Kafka record batch for testing +// In a real implementation, this would read actual message data from storage +func (h *Handler) constructRecordBatch(ledger interface{}, fetchOffset, highWaterMark int64) []byte { + // For Phase 1, create a simple record batch with dummy messages + // This simulates what would come from real message storage + + recordsToFetch := highWaterMark - fetchOffset + if recordsToFetch <= 0 { + return []byte{} // no records to fetch + } + + // Limit the number of records for Phase 1 + if recordsToFetch > 10 { + recordsToFetch = 10 + } + + // Create a simple record batch + batch := make([]byte, 0, 256) + + // Record batch header + baseOffsetBytes := make([]byte, 8) + binary.BigEndian.PutUint64(baseOffsetBytes, uint64(fetchOffset)) + batch = append(batch, baseOffsetBytes...) // base offset + + // Calculate batch length (will be filled after we know the size) + batchLengthPos := len(batch) + batch = append(batch, 0, 0, 0, 0) // batch length placeholder + + 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) // CRC32 (simplified) + + // Batch attributes + batch = append(batch, 0, 0) // attributes + + // Last offset delta + lastOffsetDelta := uint32(recordsToFetch - 1) + lastOffsetDeltaBytes := make([]byte, 4) + binary.BigEndian.PutUint32(lastOffsetDeltaBytes, lastOffsetDelta) + batch = append(batch, lastOffsetDeltaBytes...) + + // First timestamp + firstTimestamp := time.Now().UnixNano() + firstTimestampBytes := make([]byte, 8) + binary.BigEndian.PutUint64(firstTimestampBytes, uint64(firstTimestamp)) + batch = append(batch, firstTimestampBytes...) + + // Max timestamp + maxTimestamp := firstTimestamp + int64(recordsToFetch)*1000000 // 1ms apart + maxTimestampBytes := make([]byte, 8) + binary.BigEndian.PutUint64(maxTimestampBytes, uint64(maxTimestamp)) + batch = append(batch, maxTimestampBytes...) + + // Producer ID, Producer Epoch, Base Sequence + 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(recordsToFetch)) + batch = append(batch, recordCountBytes...) + + // Add simple records + for i := int64(0); i < recordsToFetch; i++ { + // Each record: length + attributes + timestamp_delta + offset_delta + key_length + key + value_length + value + headers_count + record := make([]byte, 0, 32) + + // Record attributes + record = append(record, 0) + + // Timestamp delta (varint - simplified to 1 byte) + timestampDelta := byte(i) // simple delta + record = append(record, timestampDelta) + + // Offset delta (varint - simplified to 1 byte) + offsetDelta := byte(i) + record = append(record, offsetDelta) + + // Key length (-1 = null key) + record = append(record, 0xFF) + + // Value (simple test message) + value := fmt.Sprintf("message-%d", fetchOffset+i) + record = append(record, byte(len(value))) // value length + record = append(record, []byte(value)...) // value + + // Headers count (0) + record = append(record, 0) + + // Record length (varint - simplified) + recordLength := byte(len(record)) + batch = append(batch, recordLength) + batch = append(batch, record...) + } + + // Fill in the batch length + batchLength := uint32(len(batch) - batchLengthPos - 4) + binary.BigEndian.PutUint32(batch[batchLengthPos:batchLengthPos+4], batchLength) + + return batch +} diff --git a/weed/mq/kafka/protocol/fetch_test.go b/weed/mq/kafka/protocol/fetch_test.go new file mode 100644 index 000000000..f6c936969 --- /dev/null +++ b/weed/mq/kafka/protocol/fetch_test.go @@ -0,0 +1,339 @@ +package protocol + +import ( + "encoding/binary" + "testing" + "time" +) + +func TestHandler_handleFetch(t *testing.T) { + h := NewHandler() + correlationID := uint32(666) + + // Create a topic and add some records + topicName := "test-topic" + h.topics[topicName] = &TopicInfo{ + Name: topicName, + Partitions: 1, + CreatedAt: time.Now().UnixNano(), + } + + // Add some records to the ledger + ledger := h.GetOrCreateLedger(topicName, 0) + baseOffset := ledger.AssignOffsets(3) + currentTime := time.Now().UnixNano() + ledger.AppendRecord(baseOffset+0, currentTime+0, 100) + ledger.AppendRecord(baseOffset+1, currentTime+1000, 200) + ledger.AppendRecord(baseOffset+2, currentTime+2000, 150) + + // Build a Fetch request + clientID := "test-consumer" + + requestBody := make([]byte, 0, 256) + + // Client ID + requestBody = append(requestBody, 0, byte(len(clientID))) + requestBody = append(requestBody, []byte(clientID)...) + + // Replica ID (-1 for consumer) + requestBody = append(requestBody, 0xFF, 0xFF, 0xFF, 0xFF) + + // Max wait time (5000ms) + requestBody = append(requestBody, 0, 0, 0x13, 0x88) + + // Min bytes (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Max bytes (1MB) + requestBody = append(requestBody, 0, 0x10, 0, 0) + + // Isolation level (0 = read uncommitted) + requestBody = append(requestBody, 0) + + // Session ID (0) + requestBody = append(requestBody, 0, 0, 0, 0) + + // Epoch (0) + requestBody = append(requestBody, 0, 0, 0, 0) + + // Topics count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Topic name + requestBody = append(requestBody, 0, byte(len(topicName))) + requestBody = append(requestBody, []byte(topicName)...) + + // Partitions count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Partition 0 + requestBody = append(requestBody, 0, 0, 0, 0) // partition ID + requestBody = append(requestBody, 0, 0, 0, 0) // current leader epoch + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, byte(baseOffset)) // fetch offset + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, 0) // log start offset + requestBody = append(requestBody, 0, 0, 0x10, 0) // partition max bytes (1MB) + + response, err := h.handleFetch(correlationID, requestBody) + if err != nil { + t.Fatalf("handleFetch: %v", err) + } + + if len(response) < 60 { // minimum expected size + t.Fatalf("response too short: %d bytes", len(response)) + } + + // Check response structure + respCorrelationID := binary.BigEndian.Uint32(response[0:4]) + if respCorrelationID != correlationID { + t.Errorf("correlation ID: got %d, want %d", respCorrelationID, correlationID) + } + + // Check throttle time + throttleTime := binary.BigEndian.Uint32(response[4:8]) + if throttleTime != 0 { + t.Errorf("throttle time: got %d, want 0", throttleTime) + } + + // Check error code + errorCode := binary.BigEndian.Uint16(response[8:10]) + if errorCode != 0 { + t.Errorf("error code: got %d, want 0", errorCode) + } + + // Parse response structure (simplified validation) + offset := 14 // skip correlation_id + throttle_time + error_code + session_id + topicsCount := binary.BigEndian.Uint32(response[offset : offset+4]) + if topicsCount != 1 { + t.Errorf("topics count: got %d, want 1", topicsCount) + } + + offset += 4 + respTopicNameSize := binary.BigEndian.Uint16(response[offset : offset+2]) + offset += 2 + if respTopicNameSize != uint16(len(topicName)) { + t.Errorf("response topic name size: got %d, want %d", respTopicNameSize, len(topicName)) + } + + respTopicName := string(response[offset : offset+int(respTopicNameSize)]) + offset += int(respTopicNameSize) + if respTopicName != topicName { + t.Errorf("response topic name: got %s, want %s", respTopicName, topicName) + } + + // Partitions count + respPartitionsCount := binary.BigEndian.Uint32(response[offset : offset+4]) + offset += 4 + if respPartitionsCount != 1 { + t.Errorf("response partitions count: got %d, want 1", respPartitionsCount) + } + + // Partition ID + partitionID := binary.BigEndian.Uint32(response[offset : offset+4]) + offset += 4 + if partitionID != 0 { + t.Errorf("partition ID: got %d, want 0", partitionID) + } + + // Partition error code + partitionErrorCode := binary.BigEndian.Uint16(response[offset : offset+2]) + offset += 2 + if partitionErrorCode != 0 { + t.Errorf("partition error code: got %d, want 0", partitionErrorCode) + } + + // High water mark + highWaterMark := int64(binary.BigEndian.Uint64(response[offset : offset+8])) + offset += 8 + if highWaterMark != 3 { // baseOffset + 3 records + t.Errorf("high water mark: got %d, want %d", highWaterMark, baseOffset+3) + } + + // Skip last_stable_offset, log_start_offset, aborted_transactions_count + offset += 8 + 8 + 4 + + // Records size + recordsSize := binary.BigEndian.Uint32(response[offset : offset+4]) + offset += 4 + if recordsSize == 0 { + t.Errorf("expected some records, got size 0") + } + + // Verify we have records data + if len(response) < offset+int(recordsSize) { + t.Errorf("response shorter than expected records size") + } +} + +func TestHandler_handleFetch_UnknownTopic(t *testing.T) { + h := NewHandler() + correlationID := uint32(777) + + // Build Fetch request for non-existent topic + clientID := "test-consumer" + topicName := "non-existent-topic" + + requestBody := make([]byte, 0, 128) + + // Client ID + requestBody = append(requestBody, 0, byte(len(clientID))) + requestBody = append(requestBody, []byte(clientID)...) + + // Standard Fetch parameters + requestBody = append(requestBody, 0xFF, 0xFF, 0xFF, 0xFF) // replica ID + requestBody = append(requestBody, 0, 0, 0x13, 0x88) // max wait time + requestBody = append(requestBody, 0, 0, 0, 1) // min bytes + requestBody = append(requestBody, 0, 0x10, 0, 0) // max bytes + requestBody = append(requestBody, 0) // isolation level + requestBody = append(requestBody, 0, 0, 0, 0) // session ID + requestBody = append(requestBody, 0, 0, 0, 0) // epoch + + // Topics count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Topic name + requestBody = append(requestBody, 0, byte(len(topicName))) + requestBody = append(requestBody, []byte(topicName)...) + + // Partitions count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Partition 0 + requestBody = append(requestBody, 0, 0, 0, 0) // partition ID + requestBody = append(requestBody, 0, 0, 0, 0) // current leader epoch + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, 0) // fetch offset + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, 0) // log start offset + requestBody = append(requestBody, 0, 0, 0x10, 0) // partition max bytes + + response, err := h.handleFetch(correlationID, requestBody) + if err != nil { + t.Fatalf("handleFetch: %v", err) + } + + // Parse response to check for UNKNOWN_TOPIC_OR_PARTITION error + offset := 14 + 4 + 2 + len(topicName) + 4 + 4 // skip to partition error code + partitionErrorCode := binary.BigEndian.Uint16(response[offset : offset+2]) + if partitionErrorCode != 3 { // UNKNOWN_TOPIC_OR_PARTITION + t.Errorf("expected UNKNOWN_TOPIC_OR_PARTITION error (3), got: %d", partitionErrorCode) + } +} + +func TestHandler_handleFetch_EmptyPartition(t *testing.T) { + h := NewHandler() + correlationID := uint32(888) + + // Create a topic but don't add any records + topicName := "empty-topic" + h.topics[topicName] = &TopicInfo{ + Name: topicName, + Partitions: 1, + CreatedAt: time.Now().UnixNano(), + } + + // Get ledger but don't add records + ledger := h.GetOrCreateLedger(topicName, 0) + _ = ledger // ledger exists but is empty + + // Build Fetch request + clientID := "test-consumer" + + requestBody := make([]byte, 0, 128) + + // Client ID + requestBody = append(requestBody, 0, byte(len(clientID))) + requestBody = append(requestBody, []byte(clientID)...) + + // Standard parameters + requestBody = append(requestBody, 0xFF, 0xFF, 0xFF, 0xFF) // replica ID + requestBody = append(requestBody, 0, 0, 0x13, 0x88) // max wait time + requestBody = append(requestBody, 0, 0, 0, 1) // min bytes + requestBody = append(requestBody, 0, 0x10, 0, 0) // max bytes + requestBody = append(requestBody, 0) // isolation level + requestBody = append(requestBody, 0, 0, 0, 0) // session ID + requestBody = append(requestBody, 0, 0, 0, 0) // epoch + + // Topics count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Topic name + requestBody = append(requestBody, 0, byte(len(topicName))) + requestBody = append(requestBody, []byte(topicName)...) + + // Partitions count (1) + requestBody = append(requestBody, 0, 0, 0, 1) + + // Partition 0 - fetch from offset 0 + requestBody = append(requestBody, 0, 0, 0, 0) // partition ID + requestBody = append(requestBody, 0, 0, 0, 0) // current leader epoch + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, 0) // fetch offset + requestBody = append(requestBody, 0, 0, 0, 0, 0, 0, 0, 0) // log start offset + requestBody = append(requestBody, 0, 0, 0x10, 0) // partition max bytes + + response, err := h.handleFetch(correlationID, requestBody) + if err != nil { + t.Fatalf("handleFetch: %v", err) + } + + // Parse response - should have no error but empty records + offset := 14 + 4 + 2 + len(topicName) + 4 + 4 // skip to partition error code + partitionErrorCode := binary.BigEndian.Uint16(response[offset : offset+2]) + if partitionErrorCode != 0 { + t.Errorf("partition error code: got %d, want 0", partitionErrorCode) + } + + // High water mark should be 0 + offset += 2 + highWaterMark := int64(binary.BigEndian.Uint64(response[offset : offset+8])) + if highWaterMark != 0 { + t.Errorf("high water mark: got %d, want 0", highWaterMark) + } + + // Skip to records size + offset += 8 + 8 + 4 // skip last_stable_offset, log_start_offset, aborted_transactions_count + recordsSize := binary.BigEndian.Uint32(response[offset : offset+4]) + if recordsSize != 0 { + t.Errorf("records size: got %d, want 0", recordsSize) + } +} + +func TestHandler_constructRecordBatch(t *testing.T) { + h := NewHandler() + + // Test with simple parameters + records := h.constructRecordBatch(nil, 0, 3) + if len(records) == 0 { + t.Errorf("expected some records, got empty") + } + + // Should have proper record batch structure + if len(records) < 61 { // minimum record batch header size + t.Errorf("record batch too small: %d bytes", len(records)) + } + + // Check base offset + baseOffset := int64(binary.BigEndian.Uint64(records[0:8])) + if baseOffset != 0 { + t.Errorf("base offset: got %d, want 0", baseOffset) + } + + // Check magic byte + if records[16] != 2 { + t.Errorf("magic byte: got %d, want 2", records[16]) + } + + // Test with no records to fetch + emptyRecords := h.constructRecordBatch(nil, 5, 5) + if len(emptyRecords) != 0 { + t.Errorf("expected empty records, got %d bytes", len(emptyRecords)) + } + + // Test with large range (should be limited) + largeRecords := h.constructRecordBatch(nil, 0, 100) + if len(largeRecords) == 0 { + t.Errorf("expected some records for large range") + } + + // Should be limited to reasonable size + if len(largeRecords) > 10000 { // arbitrary reasonable limit + t.Errorf("record batch too large: %d bytes", len(largeRecords)) + } +} diff --git a/weed/mq/kafka/protocol/handler.go b/weed/mq/kafka/protocol/handler.go index 4e400d914..fac3437dc 100644 --- a/weed/mq/kafka/protocol/handler.go +++ b/weed/mq/kafka/protocol/handler.go @@ -134,6 +134,8 @@ func (h *Handler) HandleConn(conn net.Conn) error { response, err = h.handleDeleteTopics(correlationID, messageBuf[8:]) // skip header case 0: // Produce response, err = h.handleProduce(correlationID, messageBuf[8:]) // skip header + case 1: // Fetch + response, err = h.handleFetch(correlationID, messageBuf[8:]) // skip header default: err = fmt.Errorf("unsupported API key: %d (version %d)", apiKey, apiVersion) } @@ -174,7 +176,7 @@ func (h *Handler) handleApiVersions(correlationID uint32) ([]byte, error) { response = append(response, 0, 0) // Number of API keys (compact array format in newer versions, but using basic format for simplicity) - response = append(response, 0, 0, 0, 6) // 6 API keys + response = append(response, 0, 0, 0, 7) // 7 API keys // API Key 18 (ApiVersions): api_key(2) + min_version(2) + max_version(2) response = append(response, 0, 18) // API key 18 @@ -206,6 +208,11 @@ func (h *Handler) handleApiVersions(correlationID uint32) ([]byte, error) { response = append(response, 0, 0) // min version 0 response = append(response, 0, 7) // max version 7 + // API Key 1 (Fetch): api_key(2) + min_version(2) + max_version(2) + response = append(response, 0, 1) // API key 1 + response = append(response, 0, 0) // min version 0 + response = append(response, 0, 11) // max version 11 + // Throttle time (4 bytes, 0 = no throttling) response = append(response, 0, 0, 0, 0) diff --git a/weed/mq/kafka/protocol/handler_test.go b/weed/mq/kafka/protocol/handler_test.go index a66603062..1043e6090 100644 --- a/weed/mq/kafka/protocol/handler_test.go +++ b/weed/mq/kafka/protocol/handler_test.go @@ -92,12 +92,12 @@ func TestHandler_ApiVersions(t *testing.T) { // Check number of API keys numAPIKeys := binary.BigEndian.Uint32(respBuf[6:10]) - if numAPIKeys != 6 { - t.Errorf("expected 6 API keys, got: %d", numAPIKeys) + if numAPIKeys != 7 { + t.Errorf("expected 7 API keys, got: %d", numAPIKeys) } // Check API key details: api_key(2) + min_version(2) + max_version(2) - if len(respBuf) < 46 { // need space for 6 API keys + if len(respBuf) < 52 { // need space for 7 API keys t.Fatalf("response too short for API key data") } @@ -190,6 +190,21 @@ func TestHandler_ApiVersions(t *testing.T) { if maxVersion6 != 7 { t.Errorf("expected max version 7, got: %d", maxVersion6) } + + // Seventh API key (Fetch) + apiKey7 := binary.BigEndian.Uint16(respBuf[46:48]) + minVersion7 := binary.BigEndian.Uint16(respBuf[48:50]) + maxVersion7 := binary.BigEndian.Uint16(respBuf[50:52]) + + if apiKey7 != 1 { + t.Errorf("expected API key 1, got: %d", apiKey7) + } + if minVersion7 != 0 { + t.Errorf("expected min version 0, got: %d", minVersion7) + } + if maxVersion7 != 11 { + t.Errorf("expected max version 11, got: %d", maxVersion7) + } // Close client to end handler client.Close() @@ -214,7 +229,7 @@ func TestHandler_handleApiVersions(t *testing.T) { t.Fatalf("handleApiVersions: %v", err) } - if len(response) < 48 { // minimum expected size (now has 6 API keys) + if len(response) < 54 { // minimum expected size (now has 7 API keys) t.Fatalf("response too short: %d bytes", len(response)) } @@ -232,8 +247,8 @@ func TestHandler_handleApiVersions(t *testing.T) { // Check number of API keys numAPIKeys := binary.BigEndian.Uint32(response[6:10]) - if numAPIKeys != 6 { - t.Errorf("expected 6 API keys, got: %d", numAPIKeys) + if numAPIKeys != 7 { + t.Errorf("expected 7 API keys, got: %d", numAPIKeys) } // Check first API key (ApiVersions) @@ -271,6 +286,12 @@ func TestHandler_handleApiVersions(t *testing.T) { if apiKey6 != 0 { t.Errorf("sixth API key: got %d, want 0", apiKey6) } + + // Check seventh API key (Fetch) + apiKey7 := binary.BigEndian.Uint16(response[46:48]) + if apiKey7 != 1 { + t.Errorf("seventh API key: got %d, want 1", apiKey7) + } } func TestHandler_handleMetadata(t *testing.T) {