From e5920f55f3e12a12ac99189ec4e23afbf39a7808 Mon Sep 17 00:00:00 2001 From: chrislu Date: Wed, 10 Sep 2025 12:32:33 -0700 Subject: [PATCH] mq(kafka): implement Metadata protocol handler stub with broker discovery and comprehensive tests --- weed/mq/kafka/protocol/handler.go | 65 +++++++- weed/mq/kafka/protocol/handler_test.go | 213 +++++++++++++++++++++++-- 2 files changed, 257 insertions(+), 21 deletions(-) diff --git a/weed/mq/kafka/protocol/handler.go b/weed/mq/kafka/protocol/handler.go index 9d510cb8a..d40ca6a5e 100644 --- a/weed/mq/kafka/protocol/handler.go +++ b/weed/mq/kafka/protocol/handler.go @@ -61,6 +61,8 @@ func (h *Handler) HandleConn(conn net.Conn) error { switch apiKey { case 18: // ApiVersions response, err = h.handleApiVersions(correlationID) + case 3: // Metadata + response, err = h.handleMetadata(correlationID, messageBuf[8:]) // skip header default: err = fmt.Errorf("unsupported API key: %d (version %d)", apiKey, apiVersion) } @@ -89,27 +91,76 @@ func (h *Handler) HandleConn(conn net.Conn) error { func (h *Handler) handleApiVersions(correlationID uint32) ([]byte, error) { // Build ApiVersions response manually // Response format: correlation_id(4) + error_code(2) + num_api_keys(4) + api_keys + throttle_time(4) - + response := make([]byte, 0, 64) - + // Correlation ID correlationIDBytes := make([]byte, 4) binary.BigEndian.PutUint32(correlationIDBytes, correlationID) response = append(response, correlationIDBytes...) - + // Error code (0 = no 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, 1) // 1 API key - + response = append(response, 0, 0, 0, 2) // 2 API keys + // API Key 18 (ApiVersions): api_key(2) + min_version(2) + max_version(2) response = append(response, 0, 18) // API key 18 response = append(response, 0, 0) // min version 0 response = append(response, 0, 3) // max version 3 - + + // API Key 3 (Metadata): api_key(2) + min_version(2) + max_version(2) + response = append(response, 0, 3) // API key 3 + response = append(response, 0, 0) // min version 0 + response = append(response, 0, 7) // max version 7 + // Throttle time (4 bytes, 0 = no throttling) response = append(response, 0, 0, 0, 0) + + return response, nil +} +func (h *Handler) handleMetadata(correlationID uint32, requestBody []byte) ([]byte, error) { + // For now, ignore the request body content (topics filter, etc.) + // Build minimal Metadata response + // Response format: correlation_id(4) + throttle_time(4) + brokers + cluster_id + controller_id + topics + + 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) + + // Brokers array length (4 bytes) - 1 broker (this gateway) + response = append(response, 0, 0, 0, 1) + + // Broker 0: node_id(4) + host + port(4) + rack + response = append(response, 0, 0, 0, 0) // node_id = 0 + + // Host string: length(2) + "localhost" + host := "localhost" + response = append(response, 0, byte(len(host))) + response = append(response, []byte(host)...) + + // Port (4 bytes) - 9092 (standard Kafka port) + response = append(response, 0, 0, 0x23, 0x84) // 9092 in big-endian + + // Rack - nullable string, using null (-1 length) + response = append(response, 0xFF, 0xFF) // null rack + + // Cluster ID - nullable string, using null + response = append(response, 0xFF, 0xFF) // null cluster_id + + // Controller ID (4 bytes) - -1 (no controller) + response = append(response, 0xFF, 0xFF, 0xFF, 0xFF) + + // Topics array length (4 bytes) - 0 topics for now + response = append(response, 0, 0, 0, 0) + return response, nil } diff --git a/weed/mq/kafka/protocol/handler_test.go b/weed/mq/kafka/protocol/handler_test.go index 59cd1dcf0..5b74a1be5 100644 --- a/weed/mq/kafka/protocol/handler_test.go +++ b/weed/mq/kafka/protocol/handler_test.go @@ -92,19 +92,20 @@ func TestHandler_ApiVersions(t *testing.T) { // Check number of API keys numAPIKeys := binary.BigEndian.Uint32(respBuf[6:10]) - if numAPIKeys != 1 { - t.Errorf("expected 1 API key, got: %d", numAPIKeys) + if numAPIKeys != 2 { + t.Errorf("expected 2 API keys, got: %d", numAPIKeys) } - + // Check API key details: api_key(2) + min_version(2) + max_version(2) - if len(respBuf) < 16 { + if len(respBuf) < 22 { // need space for 2 API keys t.Fatalf("response too short for API key data") } - + + // First API key (ApiVersions) apiKey := binary.BigEndian.Uint16(respBuf[10:12]) minVersion := binary.BigEndian.Uint16(respBuf[12:14]) maxVersion := binary.BigEndian.Uint16(respBuf[14:16]) - + if apiKey != 18 { t.Errorf("expected API key 18, got: %d", apiKey) } @@ -114,6 +115,21 @@ func TestHandler_ApiVersions(t *testing.T) { if maxVersion != 3 { t.Errorf("expected max version 3, got: %d", maxVersion) } + + // Second API key (Metadata) + apiKey2 := binary.BigEndian.Uint16(respBuf[16:18]) + minVersion2 := binary.BigEndian.Uint16(respBuf[18:20]) + maxVersion2 := binary.BigEndian.Uint16(respBuf[20:22]) + + if apiKey2 != 3 { + t.Errorf("expected API key 3, got: %d", apiKey2) + } + if minVersion2 != 0 { + t.Errorf("expected min version 0, got: %d", minVersion2) + } + if maxVersion2 != 7 { + t.Errorf("expected max version 7, got: %d", maxVersion2) + } // Close client to end handler client.Close() @@ -132,31 +148,200 @@ func TestHandler_ApiVersions(t *testing.T) { func TestHandler_handleApiVersions(t *testing.T) { h := NewHandler() correlationID := uint32(999) - + response, err := h.handleApiVersions(correlationID) if err != nil { t.Fatalf("handleApiVersions: %v", err) } - - if len(response) < 20 { // minimum expected size + + if len(response) < 24 { // minimum expected size (now has 2 API keys) 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 error code errorCode := binary.BigEndian.Uint16(response[4:6]) if errorCode != 0 { t.Errorf("error code: got %d, want 0", errorCode) } - - // Check API key + + // Check number of API keys + numAPIKeys := binary.BigEndian.Uint32(response[6:10]) + if numAPIKeys != 2 { + t.Errorf("expected 2 API keys, got: %d", numAPIKeys) + } + + // Check first API key (ApiVersions) apiKey := binary.BigEndian.Uint16(response[10:12]) if apiKey != 18 { - t.Errorf("API key: got %d, want 18", apiKey) + t.Errorf("first API key: got %d, want 18", apiKey) + } + + // Check second API key (Metadata) + apiKey2 := binary.BigEndian.Uint16(response[16:18]) + if apiKey2 != 3 { + t.Errorf("second API key: got %d, want 3", apiKey2) + } +} + +func TestHandler_handleMetadata(t *testing.T) { + h := NewHandler() + correlationID := uint32(456) + + // Empty request body for minimal test + requestBody := []byte{} + + response, err := h.handleMetadata(correlationID, requestBody) + if err != nil { + t.Fatalf("handleMetadata: %v", err) + } + + if len(response) < 40 { // 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 brokers count + brokersCount := binary.BigEndian.Uint32(response[8:12]) + if brokersCount != 1 { + t.Errorf("brokers count: got %d, want 1", brokersCount) + } + + // Check first broker node ID + nodeID := binary.BigEndian.Uint32(response[12:16]) + if nodeID != 0 { + t.Errorf("broker node ID: got %d, want 0", nodeID) + } + + // Check host string length + hostLen := binary.BigEndian.Uint16(response[16:18]) + expectedHost := "localhost" + if hostLen != uint16(len(expectedHost)) { + t.Errorf("host length: got %d, want %d", hostLen, len(expectedHost)) + } + + // Check host string + if string(response[18:18+hostLen]) != expectedHost { + t.Errorf("host: got %s, want %s", string(response[18:18+hostLen]), expectedHost) + } + + // Check port + portStart := 18 + int(hostLen) + port := binary.BigEndian.Uint32(response[portStart:portStart+4]) + if port != 9092 { + t.Errorf("port: got %d, want 9092", port) + } +} + +func TestHandler_Metadata_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 Metadata request + correlationID := uint32(789) + clientID := "metadata-test" + + message := make([]byte, 0, 64) + message = append(message, 0, 3) // API key 3 (Metadata) + 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)...) + + // Empty request body (all topics) + message = append(message, 0xFF, 0xFF, 0xFF, 0xFF) // -1 = all topics + + // 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) + brokers + cluster_id + controller_id + topics + if len(respBuf) < 40 { // 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 brokers count + brokersCount := binary.BigEndian.Uint32(respBuf[8:12]) + if brokersCount != 1 { + t.Errorf("expected 1 broker, got: %d", brokersCount) + } + + // 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") } }