Browse Source

mq(kafka): implement ListOffsets protocol handler stub for earliest/latest offset queries with comprehensive tests

pull/7231/head
chrislu 2 months ago
parent
commit
9b6faa1910
  1. 117
      weed/mq/kafka/protocol/handler.go
  2. 277
      weed/mq/kafka/protocol/handler_test.go

117
weed/mq/kafka/protocol/handler.go

@ -63,6 +63,8 @@ func (h *Handler) HandleConn(conn net.Conn) error {
response, err = h.handleApiVersions(correlationID)
case 3: // Metadata
response, err = h.handleMetadata(correlationID, messageBuf[8:]) // skip header
case 2: // ListOffsets
response, err = h.handleListOffsets(correlationID, messageBuf[8:]) // skip header
default:
err = fmt.Errorf("unsupported API key: %d (version %d)", apiKey, apiVersion)
}
@ -103,7 +105,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, 2) // 2 API keys
response = append(response, 0, 0, 0, 3) // 3 API keys
// API Key 18 (ApiVersions): api_key(2) + min_version(2) + max_version(2)
response = append(response, 0, 18) // API key 18
@ -115,6 +117,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 2 (ListOffsets): api_key(2) + min_version(2) + max_version(2)
response = append(response, 0, 2) // API key 2
response = append(response, 0, 0) // min version 0
response = append(response, 0, 5) // max version 5
// Throttle time (4 bytes, 0 = no throttling)
response = append(response, 0, 0, 0, 0)
@ -164,3 +171,111 @@ func (h *Handler) handleMetadata(correlationID uint32, requestBody []byte) ([]by
return response, nil
}
func (h *Handler) handleListOffsets(correlationID uint32, requestBody []byte) ([]byte, error) {
// Parse minimal request to understand what's being asked
// For this stub, we'll just return stub responses for any requested topic/partition
// Request format after client_id: topics_array
if len(requestBody) < 6 { // at minimum need client_id_size(2) + topics_count(4)
return nil, fmt.Errorf("ListOffsets request too short")
}
// Skip client_id: client_id_size(2) + client_id_data
clientIDSize := binary.BigEndian.Uint16(requestBody[0:2])
offset := 2 + int(clientIDSize)
if len(requestBody) < offset+4 {
return nil, fmt.Errorf("ListOffsets request missing topics count")
}
topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
offset += 4
response := make([]byte, 0, 256)
// 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)
// Topics count (same as request)
topicsCountBytes := make([]byte, 4)
binary.BigEndian.PutUint32(topicsCountBytes, topicsCount)
response = append(response, topicsCountBytes...)
// Process each requested 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 := requestBody[offset : offset+int(topicNameSize)]
offset += int(topicNameSize)
// Parse partitions count for this topic
partitionsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
offset += 4
// Response: topic_name_size(2) + topic_name + partitions_array
response = append(response, byte(topicNameSize>>8), byte(topicNameSize))
response = append(response, topicName...)
partitionsCountBytes := make([]byte, 4)
binary.BigEndian.PutUint32(partitionsCountBytes, partitionsCount)
response = append(response, partitionsCountBytes...)
// Process each partition
for j := uint32(0); j < partitionsCount && offset+12 <= len(requestBody); j++ {
// Parse partition request: partition_id(4) + timestamp(8)
partitionID := binary.BigEndian.Uint32(requestBody[offset : offset+4])
timestamp := int64(binary.BigEndian.Uint64(requestBody[offset+4 : offset+12]))
offset += 12
// Response: partition_id(4) + error_code(2) + timestamp(8) + offset(8)
partitionIDBytes := make([]byte, 4)
binary.BigEndian.PutUint32(partitionIDBytes, partitionID)
response = append(response, partitionIDBytes...)
// Error code (0 = no error)
response = append(response, 0, 0)
// For stub: return the original timestamp for timestamp queries, or current time for earliest/latest
var responseTimestamp int64
var responseOffset int64
switch timestamp {
case -2: // earliest offset
responseTimestamp = 0
responseOffset = 0
case -1: // latest offset
responseTimestamp = 1000000000 // some timestamp
responseOffset = 0 // stub: no messages yet
default: // specific timestamp
responseTimestamp = timestamp
responseOffset = 0 // stub: no messages at any timestamp
}
timestampBytes := make([]byte, 8)
binary.BigEndian.PutUint64(timestampBytes, uint64(responseTimestamp))
response = append(response, timestampBytes...)
offsetBytes := make([]byte, 8)
binary.BigEndian.PutUint64(offsetBytes, uint64(responseOffset))
response = append(response, offsetBytes...)
}
}
return response, nil
}

277
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 != 2 {
t.Errorf("expected 2 API keys, got: %d", numAPIKeys)
if numAPIKeys != 3 {
t.Errorf("expected 3 API keys, got: %d", numAPIKeys)
}
// Check API key details: api_key(2) + min_version(2) + max_version(2)
if len(respBuf) < 22 { // need space for 2 API keys
if len(respBuf) < 28 { // need space for 3 API keys
t.Fatalf("response too short for API key data")
}
@ -130,6 +130,21 @@ func TestHandler_ApiVersions(t *testing.T) {
if maxVersion2 != 7 {
t.Errorf("expected max version 7, got: %d", maxVersion2)
}
// Third API key (ListOffsets)
apiKey3 := binary.BigEndian.Uint16(respBuf[22:24])
minVersion3 := binary.BigEndian.Uint16(respBuf[24:26])
maxVersion3 := binary.BigEndian.Uint16(respBuf[26:28])
if apiKey3 != 2 {
t.Errorf("expected API key 2, got: %d", apiKey3)
}
if minVersion3 != 0 {
t.Errorf("expected min version 0, got: %d", minVersion3)
}
if maxVersion3 != 5 {
t.Errorf("expected max version 5, got: %d", maxVersion3)
}
// Close client to end handler
client.Close()
@ -154,7 +169,7 @@ func TestHandler_handleApiVersions(t *testing.T) {
t.Fatalf("handleApiVersions: %v", err)
}
if len(response) < 24 { // minimum expected size (now has 2 API keys)
if len(response) < 30 { // minimum expected size (now has 3 API keys)
t.Fatalf("response too short: %d bytes", len(response))
}
@ -172,8 +187,8 @@ func TestHandler_handleApiVersions(t *testing.T) {
// Check number of API keys
numAPIKeys := binary.BigEndian.Uint32(response[6:10])
if numAPIKeys != 2 {
t.Errorf("expected 2 API keys, got: %d", numAPIKeys)
if numAPIKeys != 3 {
t.Errorf("expected 3 API keys, got: %d", numAPIKeys)
}
// Check first API key (ApiVersions)
@ -187,6 +202,12 @@ func TestHandler_handleApiVersions(t *testing.T) {
if apiKey2 != 3 {
t.Errorf("second API key: got %d, want 3", apiKey2)
}
// Check third API key (ListOffsets)
apiKey3 := binary.BigEndian.Uint16(response[22:24])
if apiKey3 != 2 {
t.Errorf("third API key: got %d, want 2", apiKey3)
}
}
func TestHandler_handleMetadata(t *testing.T) {
@ -249,6 +270,250 @@ func TestHandler_handleMetadata(t *testing.T) {
}
}
func TestHandler_handleListOffsets(t *testing.T) {
h := NewHandler()
correlationID := uint32(123)
// Build a simple ListOffsets request: client_id + topics
// client_id_size(2) + client_id + topics_count(4) + topic + partitions
clientID := "test"
topic := "test-topic"
requestBody := make([]byte, 0, 64)
// Client ID
requestBody = append(requestBody, 0, byte(len(clientID)))
requestBody = append(requestBody, []byte(clientID)...)
// Topics count (1)
requestBody = append(requestBody, 0, 0, 0, 1)
// Topic name
requestBody = append(requestBody, 0, byte(len(topic)))
requestBody = append(requestBody, []byte(topic)...)
// Partitions count (2 partitions)
requestBody = append(requestBody, 0, 0, 0, 2)
// Partition 0: partition_id(4) + timestamp(8) - earliest
requestBody = append(requestBody, 0, 0, 0, 0) // partition 0
requestBody = append(requestBody, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFE) // -2 (earliest)
// Partition 1: partition_id(4) + timestamp(8) - latest
requestBody = append(requestBody, 0, 0, 0, 1) // partition 1
requestBody = append(requestBody, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF) // -1 (latest)
response, err := h.handleListOffsets(correlationID, requestBody)
if err != nil {
t.Fatalf("handleListOffsets: %v", err)
}
if len(response) < 50 { // minimum expected size
t.Fatalf("response too short: %d bytes", len(response))
}
// Check correlation ID
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 topics count
topicsCount := binary.BigEndian.Uint32(response[8:12])
if topicsCount != 1 {
t.Errorf("topics count: got %d, want 1", topicsCount)
}
// Check topic name
offset := 12
topicNameSize := binary.BigEndian.Uint16(response[offset : offset+2])
offset += 2
if topicNameSize != uint16(len(topic)) {
t.Errorf("topic name size: got %d, want %d", topicNameSize, len(topic))
}
responseTopic := string(response[offset : offset+int(topicNameSize)])
offset += int(topicNameSize)
if responseTopic != topic {
t.Errorf("topic name: got %s, want %s", responseTopic, topic)
}
// Check partitions count
partitionsCount := binary.BigEndian.Uint32(response[offset : offset+4])
offset += 4
if partitionsCount != 2 {
t.Errorf("partitions count: got %d, want 2", partitionsCount)
}
// Check partition 0 (earliest)
partitionID := binary.BigEndian.Uint32(response[offset : offset+4])
offset += 4
if partitionID != 0 {
t.Errorf("partition 0 ID: got %d, want 0", partitionID)
}
errorCode := binary.BigEndian.Uint16(response[offset : offset+2])
offset += 2
if errorCode != 0 {
t.Errorf("partition 0 error: got %d, want 0", errorCode)
}
timestamp := int64(binary.BigEndian.Uint64(response[offset : offset+8]))
offset += 8
if timestamp != 0 {
t.Errorf("partition 0 timestamp: got %d, want 0", timestamp)
}
offsetValue := int64(binary.BigEndian.Uint64(response[offset : offset+8]))
offset += 8
if offsetValue != 0 {
t.Errorf("partition 0 offset: got %d, want 0", offsetValue)
}
// Check partition 1 (latest)
partitionID = binary.BigEndian.Uint32(response[offset : offset+4])
offset += 4
if partitionID != 1 {
t.Errorf("partition 1 ID: got %d, want 1", partitionID)
}
errorCode = binary.BigEndian.Uint16(response[offset : offset+2])
offset += 2
if errorCode != 0 {
t.Errorf("partition 1 error: got %d, want 0", errorCode)
}
timestamp = int64(binary.BigEndian.Uint64(response[offset : offset+8]))
offset += 8
if timestamp <= 0 {
t.Errorf("partition 1 timestamp: got %d, want > 0", timestamp)
}
offsetValue = int64(binary.BigEndian.Uint64(response[offset : offset+8]))
if offsetValue != 0 {
t.Errorf("partition 1 offset: got %d, want 0", offsetValue)
}
}
func TestHandler_ListOffsets_EndToEnd(t *testing.T) {
// Create handler
h := NewHandler()
// Create in-memory connection
server, client := net.Pipe()
defer server.Close()
defer client.Close()
// Handle connection in background
done := make(chan error, 1)
go func() {
done <- h.HandleConn(server)
}()
// Create ListOffsets request
correlationID := uint32(555)
clientID := "listoffsets-test"
topic := "my-topic"
message := make([]byte, 0, 128)
message = append(message, 0, 2) // API key 2 (ListOffsets)
message = append(message, 0, 0) // API version 0
// Correlation ID
correlationIDBytes := make([]byte, 4)
binary.BigEndian.PutUint32(correlationIDBytes, correlationID)
message = append(message, correlationIDBytes...)
// Client ID length and string
clientIDLen := uint16(len(clientID))
message = append(message, byte(clientIDLen>>8), byte(clientIDLen))
message = append(message, []byte(clientID)...)
// Topics count (1)
message = append(message, 0, 0, 0, 1)
// Topic name
topicLen := uint16(len(topic))
message = append(message, byte(topicLen>>8), byte(topicLen))
message = append(message, []byte(topic)...)
// Partitions count (1)
message = append(message, 0, 0, 0, 1)
// Partition 0 requesting earliest offset
message = append(message, 0, 0, 0, 0) // partition 0
message = append(message, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFE) // -2 (earliest)
// Write message size and data
messageSize := uint32(len(message))
sizeBuf := make([]byte, 4)
binary.BigEndian.PutUint32(sizeBuf, messageSize)
if _, err := client.Write(sizeBuf); err != nil {
t.Fatalf("write size: %v", err)
}
if _, err := client.Write(message); err != nil {
t.Fatalf("write message: %v", err)
}
// Read response size
var respSizeBuf [4]byte
client.SetReadDeadline(time.Now().Add(5 * time.Second))
if _, err := client.Read(respSizeBuf[:]); err != nil {
t.Fatalf("read response size: %v", err)
}
respSize := binary.BigEndian.Uint32(respSizeBuf[:])
if respSize == 0 || respSize > 1024*1024 {
t.Fatalf("invalid response size: %d", respSize)
}
// Read response data
respBuf := make([]byte, respSize)
if _, err := client.Read(respBuf); err != nil {
t.Fatalf("read response: %v", err)
}
// Parse response: correlation_id(4) + throttle_time(4) + topics
if len(respBuf) < 20 { // minimum response size
t.Fatalf("response too short: %d bytes", len(respBuf))
}
// Check correlation ID
respCorrelationID := binary.BigEndian.Uint32(respBuf[0:4])
if respCorrelationID != correlationID {
t.Errorf("correlation ID mismatch: got %d, want %d", respCorrelationID, correlationID)
}
// Check topics count
topicsCount := binary.BigEndian.Uint32(respBuf[8:12])
if topicsCount != 1 {
t.Errorf("expected 1 topic, got: %d", topicsCount)
}
// Check topic name (skip verification of full response for brevity)
// The important thing is we got a structurally valid response
// Close client to end handler
client.Close()
// Wait for handler to complete
select {
case err := <-done:
if err != nil {
t.Errorf("handler error: %v", err)
}
case <-time.After(2 * time.Second):
t.Errorf("handler did not complete in time")
}
}
func TestHandler_Metadata_EndToEnd(t *testing.T) {
// Create handler
h := NewHandler()

Loading…
Cancel
Save