Browse Source

mq(kafka): implement Fetch handler with record batch construction, high watermark tracking, and comprehensive test coverage for consumer functionality

pull/7231/head
chrislu 2 months ago
parent
commit
59f1c3dda5
  1. 298
      weed/mq/kafka/protocol/fetch.go
  2. 339
      weed/mq/kafka/protocol/fetch_test.go
  3. 9
      weed/mq/kafka/protocol/handler.go
  4. 33
      weed/mq/kafka/protocol/handler_test.go

298
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
}

339
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))
}
}

9
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)

33
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) {

Loading…
Cancel
Save