Browse Source

mq(kafka): debug Metadata v1 format compatibility with kafka-go ReadPartitions

- Added detailed hex dump comparison between v0 and v1 responses
- Identified v1 adds rack field (2 bytes) and is_internal field (1 byte) = 3 bytes total
- kafka-go still fails with 'multiple Read calls return no data or error'
- Our Metadata v1 format appears correct per protocol spec but incompatible with kafka-go
pull/7231/head
chrislu 2 months ago
parent
commit
5cc05d8ba7
  1. 95
      test/kafka/connection_debug_test.go
  2. 69
      test/kafka/kafka_go_metadata_test.go
  3. 243
      test/kafka/metadata_debug_test.go
  4. 49
      test/kafka/real_kafka_comparison_test.go
  5. 23
      weed/mq/kafka/protocol/handler.go

95
test/kafka/connection_debug_test.go

@ -0,0 +1,95 @@
package kafka
import (
"fmt"
"net"
"testing"
"time"
"github.com/segmentio/kafka-go"
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/gateway"
)
// TestConnectionDebug debugs the exact connection behavior between kafka-go and our gateway
func TestConnectionDebug(t *testing.T) {
// Start gateway server
gatewayServer := gateway.NewServer(gateway.Options{
Listen: ":0", // random port
})
go func() {
if err := gatewayServer.Start(); err != nil {
t.Errorf("Gateway server error: %v", err)
}
}()
defer gatewayServer.Close()
// Wait for server to start
time.Sleep(100 * time.Millisecond)
// Get the actual listening address
host, port := gatewayServer.GetListenerAddr()
brokerAddr := fmt.Sprintf("%s:%d", host, port)
t.Logf("Gateway running on %s", brokerAddr)
// Get handler and configure it
handler := gatewayServer.GetHandler()
handler.SetBrokerAddress(host, port)
// Add test topic
topicName := "conn-debug-topic"
handler.AddTopicForTesting(topicName, 1)
// Test 1: Manual connection that works
t.Log("=== Test 1: Manual TCP connection ===")
conn, err := net.Dial("tcp", brokerAddr)
if err != nil {
t.Fatalf("Manual connection failed: %v", err)
}
// Send a simple request and read response
// This should work based on our previous debug test
conn.Close()
t.Log("Manual connection works fine")
// Test 2: kafka-go Dialer connection
t.Log("=== Test 2: kafka-go Dialer connection ===")
dialer := &kafka.Dialer{
Timeout: 5 * time.Second,
}
kafkaConn, err := dialer.Dial("tcp", brokerAddr)
if err != nil {
t.Fatalf("kafka-go connection failed: %v", err)
}
defer kafkaConn.Close()
t.Log("kafka-go connection established")
// Test 3: Try to read partitions with detailed error handling
t.Log("=== Test 3: kafka-go ReadPartitions with error details ===")
// Set a deadline to avoid hanging
kafkaConn.SetDeadline(time.Now().Add(10 * time.Second))
partitions, err := kafkaConn.ReadPartitions(topicName)
if err != nil {
t.Logf("ReadPartitions failed with error: %v", err)
t.Logf("Error type: %T", err)
// Try to get more details about the error
if netErr, ok := err.(net.Error); ok {
t.Logf("Network error - Timeout: %v, Temporary: %v", netErr.Timeout(), netErr.Temporary())
}
// The error might give us clues about what's wrong
return
}
t.Logf("ReadPartitions succeeded! Found %d partitions", len(partitions))
for i, partition := range partitions {
t.Logf("Partition %d: Topic=%s, ID=%d, Leader=%v", i, partition.Topic, partition.ID, partition.Leader)
}
t.Log("✅ Connection debug test completed!")
}

69
test/kafka/kafka_go_metadata_test.go

@ -0,0 +1,69 @@
package kafka
import (
"fmt"
"testing"
"time"
"github.com/segmentio/kafka-go"
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/gateway"
)
// TestKafkaGoMetadataV1Compatibility tests if our Metadata v1 response is compatible with kafka-go
func TestKafkaGoMetadataV1Compatibility(t *testing.T) {
// Start gateway server
gatewayServer := gateway.NewServer(gateway.Options{
Listen: ":0", // random port
})
go func() {
if err := gatewayServer.Start(); err != nil {
t.Errorf("Gateway server error: %v", err)
}
}()
defer gatewayServer.Close()
// Wait for server to start
time.Sleep(100 * time.Millisecond)
// Get the actual listening address
host, port := gatewayServer.GetListenerAddr()
brokerAddr := fmt.Sprintf("%s:%d", host, port)
t.Logf("Gateway running on %s", brokerAddr)
// Get handler and configure it
handler := gatewayServer.GetHandler()
handler.SetBrokerAddress(host, port)
// Add test topic
topicName := "metadata-test-topic"
handler.AddTopicForTesting(topicName, 1)
// Test kafka-go's ability to read partitions (which uses Metadata v1/v6)
// This is what kafka-go does after JoinGroup in consumer groups
t.Log("=== Testing kafka-go ReadPartitions (Metadata v1) ===")
// Create a connection to test ReadPartitions
dialer := &kafka.Dialer{
Timeout: 5 * time.Second,
}
conn, err := dialer.Dial("tcp", brokerAddr)
if err != nil {
t.Fatalf("Failed to connect: %v", err)
}
defer conn.Close()
// ReadPartitions uses Metadata v1 internally
partitions, err := conn.ReadPartitions(topicName)
if err != nil {
t.Fatalf("ReadPartitions failed: %v", err)
}
t.Logf("ReadPartitions succeeded! Found %d partitions", len(partitions))
for i, partition := range partitions {
t.Logf("Partition %d: Topic=%s, ID=%d, Leader=%v", i, partition.Topic, partition.ID, partition.Leader)
}
// If we get here, our Metadata v1 response is compatible with kafka-go!
t.Log("✅ Metadata v1 compatibility test PASSED!")
}

243
test/kafka/metadata_debug_test.go

@ -0,0 +1,243 @@
package kafka
import (
"encoding/binary"
"fmt"
"net"
"testing"
"time"
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/gateway"
)
// TestMetadataV1DebugCapture captures the exact bytes kafka-go sends and expects
func TestMetadataV1DebugCapture(t *testing.T) {
// Start gateway server
gatewayServer := gateway.NewServer(gateway.Options{
Listen: ":0", // random port
})
go func() {
if err := gatewayServer.Start(); err != nil {
t.Errorf("Gateway server error: %v", err)
}
}()
defer gatewayServer.Close()
// Wait for server to start
time.Sleep(100 * time.Millisecond)
// Get the actual listening address
host, port := gatewayServer.GetListenerAddr()
brokerAddr := fmt.Sprintf("%s:%d", host, port)
t.Logf("Gateway running on %s", brokerAddr)
// Get handler and configure it
handler := gatewayServer.GetHandler()
handler.SetBrokerAddress(host, port)
// Add test topic
topicName := "debug-topic"
handler.AddTopicForTesting(topicName, 1)
// Create raw TCP connection to manually send Metadata v1 request
conn, err := net.Dial("tcp", brokerAddr)
if err != nil {
t.Fatalf("Failed to connect: %v", err)
}
defer conn.Close()
// Send ApiVersions request first
t.Log("=== Sending ApiVersions request ===")
apiVersionsRequest := []byte{
// Request header: api_key(2) + api_version(2) + correlation_id(4) + client_id_len(2) + client_id
0x00, 0x12, // api_key = 18 (ApiVersions)
0x00, 0x00, // api_version = 0
0x00, 0x00, 0x00, 0x01, // correlation_id = 1
0x00, 0x09, // client_id length = 9
'd', 'e', 'b', 'u', 'g', '-', 't', 'e', 's', 't', // client_id = "debug-test"
}
// Prepend message length
messageLen := make([]byte, 4)
binary.BigEndian.PutUint32(messageLen, uint32(len(apiVersionsRequest)))
fullRequest := append(messageLen, apiVersionsRequest...)
_, err = conn.Write(fullRequest)
if err != nil {
t.Fatalf("Failed to send ApiVersions: %v", err)
}
// Read ApiVersions response
responseLen := make([]byte, 4)
_, err = conn.Read(responseLen)
if err != nil {
t.Fatalf("Failed to read ApiVersions response length: %v", err)
}
respLen := binary.BigEndian.Uint32(responseLen)
response := make([]byte, respLen)
_, err = conn.Read(response)
if err != nil {
t.Fatalf("Failed to read ApiVersions response: %v", err)
}
t.Logf("ApiVersions response (%d bytes): %x", len(response), response)
// Now send Metadata v1 request
t.Log("=== Sending Metadata v1 request ===")
metadataRequest := []byte{
// Request header: api_key(2) + api_version(2) + correlation_id(4) + client_id_len(2) + client_id
0x00, 0x03, // api_key = 3 (Metadata)
0x00, 0x01, // api_version = 1
0x00, 0x00, 0x00, 0x02, // correlation_id = 2
0x00, 0x09, // client_id length = 9
'd', 'e', 'b', 'u', 'g', '-', 't', 'e', 's', 't', // client_id = "debug-test"
// Metadata request body: topics_count(4) + topic_name_len(2) + topic_name
0x00, 0x00, 0x00, 0x01, // topics_count = 1
0x00, 0x0B, // topic_name length = 11
'd', 'e', 'b', 'u', 'g', '-', 't', 'o', 'p', 'i', 'c', // topic_name = "debug-topic"
}
// Prepend message length
messageLen = make([]byte, 4)
binary.BigEndian.PutUint32(messageLen, uint32(len(metadataRequest)))
fullRequest = append(messageLen, metadataRequest...)
t.Logf("Sending Metadata v1 request (%d bytes): %x", len(fullRequest), fullRequest)
_, err = conn.Write(fullRequest)
if err != nil {
t.Fatalf("Failed to send Metadata: %v", err)
}
// Read Metadata response
responseLen = make([]byte, 4)
_, err = conn.Read(responseLen)
if err != nil {
t.Fatalf("Failed to read Metadata response length: %v", err)
}
respLen = binary.BigEndian.Uint32(responseLen)
response = make([]byte, respLen)
_, err = conn.Read(response)
if err != nil {
t.Fatalf("Failed to read Metadata response: %v", err)
}
t.Logf("Metadata v1 response (%d bytes): %x", len(response), response)
// Parse the response manually to understand the format
t.Log("=== Parsing Metadata v1 response ===")
offset := 0
// Correlation ID (4 bytes)
correlationID := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Correlation ID: %d", correlationID)
// Brokers array length (4 bytes)
brokersCount := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Brokers count: %d", brokersCount)
// Parse each broker
for i := uint32(0); i < brokersCount; i++ {
// node_id (4 bytes)
nodeID := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Broker %d node_id: %d", i, nodeID)
// host (STRING: 2 bytes length + bytes)
hostLen := binary.BigEndian.Uint16(response[offset:offset+2])
offset += 2
hostBytes := response[offset:offset+int(hostLen)]
offset += int(hostLen)
t.Logf("Broker %d host: %s", i, string(hostBytes))
// port (4 bytes)
portNum := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Broker %d port: %d", i, portNum)
// rack (STRING: 2 bytes length + bytes) - v1 addition
rackLen := binary.BigEndian.Uint16(response[offset:offset+2])
offset += 2
if rackLen > 0 {
rackBytes := response[offset:offset+int(rackLen)]
offset += int(rackLen)
t.Logf("Broker %d rack: %s", i, string(rackBytes))
} else {
t.Logf("Broker %d rack: (empty)", i)
}
}
// Topics array length (4 bytes)
if offset < len(response) {
topicsCount := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Topics count: %d", topicsCount)
// Parse each topic
for i := uint32(0); i < topicsCount && offset < len(response); i++ {
// error_code (2 bytes)
errorCode := binary.BigEndian.Uint16(response[offset:offset+2])
offset += 2
t.Logf("Topic %d error_code: %d", i, errorCode)
// name (STRING: 2 bytes length + bytes)
nameLen := binary.BigEndian.Uint16(response[offset:offset+2])
offset += 2
nameBytes := response[offset:offset+int(nameLen)]
offset += int(nameLen)
t.Logf("Topic %d name: %s", i, string(nameBytes))
// is_internal (1 byte) - v1 addition
if offset < len(response) {
isInternal := response[offset]
offset += 1
t.Logf("Topic %d is_internal: %d", i, isInternal)
}
// partitions array length (4 bytes)
if offset+4 <= len(response) {
partitionsCount := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf("Topic %d partitions count: %d", i, partitionsCount)
// Skip partition details for brevity
for j := uint32(0); j < partitionsCount && offset < len(response); j++ {
// error_code (2) + partition_id (4) + leader (4) + replicas (4+n*4) + isr (4+n*4)
if offset+2 <= len(response) {
partErrorCode := binary.BigEndian.Uint16(response[offset:offset+2])
offset += 2
t.Logf(" Partition %d error_code: %d", j, partErrorCode)
}
if offset+4 <= len(response) {
partitionID := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf(" Partition %d id: %d", j, partitionID)
}
if offset+4 <= len(response) {
leader := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4
t.Logf(" Partition %d leader: %d", j, leader)
}
// Skip replicas and isr arrays for brevity - just advance offset
if offset+4 <= len(response) {
replicasCount := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4 + int(replicasCount)*4
t.Logf(" Partition %d replicas count: %d", j, replicasCount)
}
if offset+4 <= len(response) {
isrCount := binary.BigEndian.Uint32(response[offset:offset+4])
offset += 4 + int(isrCount)*4
t.Logf(" Partition %d isr count: %d", j, isrCount)
}
}
}
}
}
t.Logf("Parsed %d bytes, remaining: %d", offset, len(response)-offset)
}

49
test/kafka/real_kafka_comparison_test.go

@ -0,0 +1,49 @@
package kafka
import (
"testing"
"time"
"github.com/segmentio/kafka-go"
)
// TestRealKafkaComparison tests kafka-go against a real Kafka broker (if available)
// This test will be skipped if no real Kafka broker is running on localhost:9092
func TestRealKafkaComparison(t *testing.T) {
// Try to connect to a real Kafka broker
dialer := &kafka.Dialer{
Timeout: 2 * time.Second,
}
conn, err := dialer.Dial("tcp", "localhost:9092")
if err != nil {
t.Skipf("No real Kafka broker available on localhost:9092: %v", err)
return
}
defer conn.Close()
t.Log("=== Testing kafka-go ReadPartitions against real Kafka ===")
// Test ReadPartitions against real Kafka
partitions, err := conn.ReadPartitions("__consumer_offsets") // This topic should exist
if err != nil {
// Try a different approach - create a test topic first
t.Logf("ReadPartitions failed for __consumer_offsets: %v", err)
// Try to read all partitions
partitions, err = conn.ReadPartitions()
if err != nil {
t.Fatalf("ReadPartitions failed: %v", err)
}
}
t.Logf("ReadPartitions succeeded! Found %d partitions", len(partitions))
for i, partition := range partitions {
if i < 5 { // Limit output
t.Logf("Partition %d: Topic=%s, ID=%d, Leader=%v", i, partition.Topic, partition.ID, partition.Leader)
}
}
// If we get here, kafka-go's ReadPartitions works fine with a real Kafka broker
t.Log("✅ kafka-go ReadPartitions works with real Kafka!")
}

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

@ -313,8 +313,6 @@ func (h *Handler) handleApiVersions(correlationID uint32) ([]byte, error) {
response = append(response, 0, 3) // max version 3 response = append(response, 0, 3) // max version 3
// API Key 3 (Metadata): api_key(2) + min_version(2) + max_version(2) // API Key 3 (Metadata): api_key(2) + min_version(2) + max_version(2)
// Keep v0 only until v1 format issue is resolved
// TODO: Fix Metadata v1 format - kafka-go rejects our v1 response with "Unknown Topic Or Partition"
response = append(response, 0, 3) // API key 3 response = append(response, 0, 3) // API key 3
response = append(response, 0, 0) // min version 0 response = append(response, 0, 0) // min version 0
response = append(response, 0, 1) // max version 1 response = append(response, 0, 1) // max version 1
@ -480,6 +478,9 @@ func (h *Handler) HandleMetadataV0(correlationID uint32, requestBody []byte) ([]
} }
func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]byte, error) { func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]byte, error) {
// TEMPORARY: Use v0 format as base and add only the essential v1 differences
// This is to debug the kafka-go parsing issue
response := make([]byte, 0, 256) response := make([]byte, 0, 256)
// Correlation ID // Correlation ID
@ -490,7 +491,7 @@ func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]
// Brokers array length (4 bytes) - 1 broker (this gateway) // Brokers array length (4 bytes) - 1 broker (this gateway)
response = append(response, 0, 0, 0, 1) response = append(response, 0, 0, 0, 1)
// Broker 0: node_id(4) + host(STRING) + port(4) + rack(STRING)
// Broker 0: node_id(4) + host(STRING) + port(4) + rack(STRING) [v1 adds rack]
response = append(response, 0, 0, 0, 0) // node_id = 0 response = append(response, 0, 0, 0, 0) // node_id = 0
// Use dynamic broker address set by the server // Use dynamic broker address set by the server
@ -508,12 +509,9 @@ func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]
binary.BigEndian.PutUint32(portBytes, uint32(port)) binary.BigEndian.PutUint32(portBytes, uint32(port))
response = append(response, portBytes...) response = append(response, portBytes...)
// Rack (STRING, NOT nullable in v1) - use empty string
// Rack (STRING) - v1 addition: empty string (NOT nullable)
response = append(response, 0x00, 0x00) response = append(response, 0x00, 0x00)
// Controller ID (4 bytes) - use broker 0 as controller
response = append(response, 0x00, 0x00, 0x00, 0x00)
// Parse requested topics (empty means all) // Parse requested topics (empty means all)
requestedTopics := h.parseMetadataTopics(requestBody) requestedTopics := h.parseMetadataTopics(requestBody)
fmt.Printf("DEBUG: 🔍 METADATA v1 REQUEST - Requested: %v (empty=all)\n", requestedTopics) fmt.Printf("DEBUG: 🔍 METADATA v1 REQUEST - Requested: %v (empty=all)\n", requestedTopics)
@ -540,7 +538,7 @@ func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]
binary.BigEndian.PutUint32(topicsCountBytes, uint32(len(topicsToReturn))) binary.BigEndian.PutUint32(topicsCountBytes, uint32(len(topicsToReturn)))
response = append(response, topicsCountBytes...) response = append(response, topicsCountBytes...)
// Topic entries (v1)
// Topic entries - using v0 format first, then add v1 differences
for _, topicName := range topicsToReturn { for _, topicName := range topicsToReturn {
// error_code(2) = 0 // error_code(2) = 0
response = append(response, 0, 0) response = append(response, 0, 0)
@ -551,7 +549,7 @@ func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]
response = append(response, byte(nameLen>>8), byte(nameLen)) response = append(response, byte(nameLen>>8), byte(nameLen))
response = append(response, nameBytes...) response = append(response, nameBytes...)
// is_internal(1) = false (v1 addition)
// is_internal(1) = false - v1 addition: this is the key difference!
response = append(response, 0) response = append(response, 0)
// partitions array length (4 bytes) - 1 partition // partitions array length (4 bytes) - 1 partition
@ -572,6 +570,13 @@ func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]
} }
fmt.Printf("DEBUG: Metadata v1 response for %d topics: %v\n", len(topicsToReturn), topicsToReturn) fmt.Printf("DEBUG: Metadata v1 response for %d topics: %v\n", len(topicsToReturn), topicsToReturn)
fmt.Printf("DEBUG: Metadata v1 response hex dump (%d bytes): %x\n", len(response), response)
// CRITICAL DEBUG: Let's also compare with v0 format
v0Response, _ := h.HandleMetadataV0(correlationID, requestBody)
fmt.Printf("DEBUG: Metadata v0 response hex dump (%d bytes): %x\n", len(v0Response), v0Response)
fmt.Printf("DEBUG: v1 vs v0 length difference: %d bytes\n", len(response) - len(v0Response))
return response, nil return response, nil
} }

Loading…
Cancel
Save