45 changed files with 809 additions and 1054 deletions
-
2test/kafka/api_sequence_test.go
-
6test/kafka/client_integration_test.go
-
4test/kafka/comprehensive_e2e_test.go
-
2test/kafka/connection_close_debug_test.go
-
2test/kafka/connection_debug_test.go
-
2test/kafka/consumer_group_debug_test.go
-
6test/kafka/consumer_group_test.go
-
2test/kafka/consumer_only_test.go
-
2test/kafka/consumer_test.go
-
6test/kafka/debug_connection_test.go
-
2test/kafka/debug_consumer_group_test.go
-
2test/kafka/debug_produce_response_test.go
-
2test/kafka/debug_produce_v7_test.go
-
2test/kafka/debug_readpartitions_test.go
-
6test/kafka/e2e_test.go
-
4test/kafka/gateway_smoke_test.go
-
2test/kafka/joingroup_debug_test.go
-
2test/kafka/kafka_go_debug_test.go
-
2test/kafka/kafka_go_internal_debug_test.go
-
2test/kafka/kafka_go_metadata_test.go
-
22test/kafka/kafka_go_produce_only_test.go
-
2test/kafka/metadata_comparison_test.go
-
2test/kafka/metadata_debug_test.go
-
2test/kafka/metadata_v1_isolation_test.go
-
293test/kafka/mock_smq_handler.go
-
2test/kafka/network_capture_test.go
-
2test/kafka/parsing_debug_test.go
-
2test/kafka/produce_consume_cycle_test.go
-
2test/kafka/produce_consume_test.go
-
2test/kafka/raw_protocol_test.go
-
11test/kafka/sarama_e2e_test.go
-
21test/kafka/sarama_simple_test.go
-
14test/kafka/sarama_test.go
-
8test/kafka/seaweedmq_integration_test.go
-
23weed/mq/kafka/gateway/server.go
-
100weed/mq/kafka/gateway/server_test.go
-
37weed/mq/kafka/gateway/server_testing.go
-
9weed/mq/kafka/integration/seaweedmq_handler.go
-
31weed/mq/kafka/offset/ledger.go
-
4weed/mq/kafka/protocol/consumer_coordination_test.go
-
85weed/mq/kafka/protocol/fetch.go
-
266weed/mq/kafka/protocol/handler.go
-
558weed/mq/kafka/protocol/handler_test.go
-
299weed/mq/kafka/protocol/handler_testing.go
-
6weed/mq/kafka/protocol/offset_management_test.go
@ -0,0 +1,293 @@ |
|||||
|
package kafka |
||||
|
|
||||
|
import ( |
||||
|
"fmt" |
||||
|
"sync" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/offset" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
// MockSMQHandler provides a realistic SeaweedMQ simulation for testing
|
||||
|
// It behaves like the real SeaweedMQ integration but uses in-memory storage
|
||||
|
type MockSMQHandler struct { |
||||
|
// Topic management
|
||||
|
topicsMu sync.RWMutex |
||||
|
topics map[string]*MockTopicInfo |
||||
|
|
||||
|
// Message storage - simulates SeaweedMQ's persistent storage
|
||||
|
messagesMu sync.RWMutex |
||||
|
messages map[string]map[int32][]*MockSMQRecord // topic -> partition -> []records
|
||||
|
|
||||
|
// Offset management - simulates Kafka offset ledgers
|
||||
|
ledgersMu sync.RWMutex |
||||
|
ledgers map[string]*offset.Ledger // topic-partition -> ledger
|
||||
|
|
||||
|
// Simulated SMQ timestamp tracking
|
||||
|
lastTimestamp map[string]map[int32]int64 // topic -> partition -> last_timestamp
|
||||
|
} |
||||
|
|
||||
|
// MockTopicInfo represents a Kafka topic in the mock SMQ environment
|
||||
|
type MockTopicInfo struct { |
||||
|
Name string |
||||
|
Partitions int32 |
||||
|
CreatedAt int64 |
||||
|
Schema *schema_pb.Topic |
||||
|
} |
||||
|
|
||||
|
// MockSMQRecord represents a record in the mock SeaweedMQ storage
|
||||
|
type MockSMQRecord struct { |
||||
|
Key []byte |
||||
|
Value []byte |
||||
|
Timestamp int64 // SeaweedMQ timestamp (nanoseconds)
|
||||
|
Offset int64 // Kafka offset for this partition
|
||||
|
} |
||||
|
|
||||
|
// Implement SMQRecord interface
|
||||
|
func (r *MockSMQRecord) GetKey() []byte { return r.Key } |
||||
|
func (r *MockSMQRecord) GetValue() []byte { return r.Value } |
||||
|
func (r *MockSMQRecord) GetTimestamp() int64 { return r.Timestamp } |
||||
|
func (r *MockSMQRecord) GetOffset() int64 { return r.Offset } |
||||
|
|
||||
|
// NewMockSMQHandler creates a new mock SeaweedMQ handler for testing
|
||||
|
func NewMockSMQHandler() *MockSMQHandler { |
||||
|
return &MockSMQHandler{ |
||||
|
topics: make(map[string]*MockTopicInfo), |
||||
|
messages: make(map[string]map[int32][]*MockSMQRecord), |
||||
|
ledgers: make(map[string]*offset.Ledger), |
||||
|
lastTimestamp: make(map[string]map[int32]int64), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// TopicExists checks if a topic exists in the mock SMQ
|
||||
|
func (m *MockSMQHandler) TopicExists(topic string) bool { |
||||
|
m.topicsMu.RLock() |
||||
|
defer m.topicsMu.RUnlock() |
||||
|
_, exists := m.topics[topic] |
||||
|
return exists |
||||
|
} |
||||
|
|
||||
|
// ListTopics returns all topics in the mock SMQ
|
||||
|
func (m *MockSMQHandler) ListTopics() []string { |
||||
|
m.topicsMu.RLock() |
||||
|
defer m.topicsMu.RUnlock() |
||||
|
|
||||
|
topics := make([]string, 0, len(m.topics)) |
||||
|
for name := range m.topics { |
||||
|
topics = append(topics, name) |
||||
|
} |
||||
|
return topics |
||||
|
} |
||||
|
|
||||
|
// CreateTopic creates a new topic in the mock SMQ
|
||||
|
func (m *MockSMQHandler) CreateTopic(topic string, partitions int32) error { |
||||
|
m.topicsMu.Lock() |
||||
|
defer m.topicsMu.Unlock() |
||||
|
|
||||
|
if _, exists := m.topics[topic]; exists { |
||||
|
return fmt.Errorf("topic %s already exists", topic) |
||||
|
} |
||||
|
|
||||
|
// Create topic info
|
||||
|
m.topics[topic] = &MockTopicInfo{ |
||||
|
Name: topic, |
||||
|
Partitions: partitions, |
||||
|
CreatedAt: time.Now().UnixNano(), |
||||
|
Schema: &schema_pb.Topic{ |
||||
|
Name: topic, |
||||
|
// PartitionCount removed - not part of schema_pb.Topic
|
||||
|
}, |
||||
|
} |
||||
|
|
||||
|
// Initialize message storage for all partitions
|
||||
|
m.messagesMu.Lock() |
||||
|
m.messages[topic] = make(map[int32][]*MockSMQRecord) |
||||
|
for i := int32(0); i < partitions; i++ { |
||||
|
m.messages[topic][i] = make([]*MockSMQRecord, 0, 1000) |
||||
|
} |
||||
|
m.messagesMu.Unlock() |
||||
|
|
||||
|
// Initialize timestamp tracking
|
||||
|
if m.lastTimestamp[topic] == nil { |
||||
|
m.lastTimestamp[topic] = make(map[int32]int64) |
||||
|
} |
||||
|
for i := int32(0); i < partitions; i++ { |
||||
|
m.lastTimestamp[topic][i] = 0 |
||||
|
} |
||||
|
|
||||
|
fmt.Printf("MOCK SMQ: Created topic %s with %d partitions\n", topic, partitions) |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// DeleteTopic removes a topic from the mock SMQ
|
||||
|
func (m *MockSMQHandler) DeleteTopic(topic string) error { |
||||
|
m.topicsMu.Lock() |
||||
|
defer m.topicsMu.Unlock() |
||||
|
|
||||
|
if _, exists := m.topics[topic]; !exists { |
||||
|
return fmt.Errorf("topic %s does not exist", topic) |
||||
|
} |
||||
|
|
||||
|
delete(m.topics, topic) |
||||
|
|
||||
|
// Clean up messages
|
||||
|
m.messagesMu.Lock() |
||||
|
delete(m.messages, topic) |
||||
|
m.messagesMu.Unlock() |
||||
|
|
||||
|
// Clean up ledgers
|
||||
|
m.ledgersMu.Lock() |
||||
|
keysToDelete := make([]string, 0) |
||||
|
for key := range m.ledgers { |
||||
|
if key[:len(topic)+1] == topic+"-" { |
||||
|
keysToDelete = append(keysToDelete, key) |
||||
|
} |
||||
|
} |
||||
|
for _, key := range keysToDelete { |
||||
|
delete(m.ledgers, key) |
||||
|
} |
||||
|
m.ledgersMu.Unlock() |
||||
|
|
||||
|
fmt.Printf("MOCK SMQ: Deleted topic %s\n", topic) |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// GetOrCreateLedger gets or creates a Kafka offset ledger for a topic-partition
|
||||
|
func (m *MockSMQHandler) GetOrCreateLedger(topic string, partition int32) *offset.Ledger { |
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
|
||||
|
m.ledgersMu.Lock() |
||||
|
defer m.ledgersMu.Unlock() |
||||
|
|
||||
|
if ledger, exists := m.ledgers[key]; exists { |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
// Create new ledger
|
||||
|
ledger := offset.NewLedger() |
||||
|
m.ledgers[key] = ledger |
||||
|
|
||||
|
// Ensure topic is created
|
||||
|
if !m.TopicExists(topic) { |
||||
|
m.CreateTopic(topic, partition+1) // Ensure enough partitions
|
||||
|
} |
||||
|
|
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
// GetLedger retrieves an existing ledger or returns nil
|
||||
|
func (m *MockSMQHandler) GetLedger(topic string, partition int32) *offset.Ledger { |
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
|
||||
|
m.ledgersMu.RLock() |
||||
|
defer m.ledgersMu.RUnlock() |
||||
|
|
||||
|
return m.ledgers[key] |
||||
|
} |
||||
|
|
||||
|
// ProduceRecord publishes a record to the mock SeaweedMQ and updates Kafka offset tracking
|
||||
|
func (m *MockSMQHandler) ProduceRecord(topic string, partition int32, key, value []byte) (int64, error) { |
||||
|
// Verify topic exists
|
||||
|
if !m.TopicExists(topic) { |
||||
|
return 0, fmt.Errorf("topic %s does not exist", topic) |
||||
|
} |
||||
|
|
||||
|
// Get current timestamp (simulate SeaweedMQ timestamp)
|
||||
|
timestamp := time.Now().UnixNano() |
||||
|
|
||||
|
// Get or create Kafka offset ledger
|
||||
|
ledger := m.GetOrCreateLedger(topic, partition) |
||||
|
|
||||
|
// Assign Kafka offset
|
||||
|
kafkaOffset := ledger.AssignOffsets(1) |
||||
|
|
||||
|
// Record in ledger (simulates Kafka offset -> SMQ timestamp mapping)
|
||||
|
messageSize := int32(len(value)) |
||||
|
if err := ledger.AppendRecord(kafkaOffset, timestamp, messageSize); err != nil { |
||||
|
return 0, fmt.Errorf("failed to append to ledger: %w", err) |
||||
|
} |
||||
|
|
||||
|
// Store message in mock SMQ storage (simulates persistent storage)
|
||||
|
m.messagesMu.Lock() |
||||
|
defer m.messagesMu.Unlock() |
||||
|
|
||||
|
if m.messages[topic] == nil { |
||||
|
m.messages[topic] = make(map[int32][]*MockSMQRecord) |
||||
|
} |
||||
|
if m.messages[topic][partition] == nil { |
||||
|
m.messages[topic][partition] = make([]*MockSMQRecord, 0, 1000) |
||||
|
} |
||||
|
|
||||
|
// Create message record (simulate SMQ storage)
|
||||
|
record := &MockSMQRecord{ |
||||
|
Key: append([]byte(nil), key...), // Deep copy key
|
||||
|
Value: append([]byte(nil), value...), // Deep copy value
|
||||
|
Timestamp: timestamp, |
||||
|
Offset: kafkaOffset, |
||||
|
} |
||||
|
|
||||
|
// Append to partition (simulate SMQ append-only log)
|
||||
|
m.messages[topic][partition] = append(m.messages[topic][partition], record) |
||||
|
|
||||
|
// Update last timestamp
|
||||
|
m.lastTimestamp[topic][partition] = timestamp |
||||
|
|
||||
|
fmt.Printf("MOCK SMQ: Stored record - topic:%s, partition:%d, kafka_offset:%d, smq_timestamp:%d, key:%s, value:%s\n", |
||||
|
topic, partition, kafkaOffset, timestamp, string(key), string(value)) |
||||
|
|
||||
|
return kafkaOffset, nil |
||||
|
} |
||||
|
|
||||
|
// GetStoredRecords retrieves records from mock SMQ storage starting from a given Kafka offset
|
||||
|
func (m *MockSMQHandler) GetStoredRecords(topic string, partition int32, fromKafkaOffset int64, maxRecords int) ([]offset.SMQRecord, error) { |
||||
|
m.messagesMu.RLock() |
||||
|
defer m.messagesMu.RUnlock() |
||||
|
|
||||
|
if m.messages[topic] == nil || m.messages[topic][partition] == nil { |
||||
|
return nil, nil // No messages
|
||||
|
} |
||||
|
|
||||
|
records := m.messages[topic][partition] |
||||
|
result := make([]offset.SMQRecord, 0, maxRecords) |
||||
|
|
||||
|
// Find records starting from the given Kafka offset
|
||||
|
for _, record := range records { |
||||
|
if record.Offset >= fromKafkaOffset && len(result) < maxRecords { |
||||
|
result = append(result, record) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
fmt.Printf("MOCK SMQ: Retrieved %d records from topic:%s, partition:%d, from_offset:%d\n", |
||||
|
len(result), topic, partition, fromKafkaOffset) |
||||
|
|
||||
|
return result, nil |
||||
|
} |
||||
|
|
||||
|
// Close shuts down the mock SMQ handler
|
||||
|
func (m *MockSMQHandler) Close() error { |
||||
|
fmt.Printf("MOCK SMQ: Handler closed\n") |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// GetPartitionCount returns the number of partitions for a topic
|
||||
|
func (m *MockSMQHandler) GetPartitionCount(topic string) int32 { |
||||
|
m.topicsMu.RLock() |
||||
|
defer m.topicsMu.RUnlock() |
||||
|
|
||||
|
if topicInfo, exists := m.topics[topic]; exists { |
||||
|
return topicInfo.Partitions |
||||
|
} |
||||
|
return 0 |
||||
|
} |
||||
|
|
||||
|
// GetMessageCount returns the total number of messages stored for a topic-partition
|
||||
|
func (m *MockSMQHandler) GetMessageCount(topic string, partition int32) int { |
||||
|
m.messagesMu.RLock() |
||||
|
defer m.messagesMu.RUnlock() |
||||
|
|
||||
|
if m.messages[topic] == nil || m.messages[topic][partition] == nil { |
||||
|
return 0 |
||||
|
} |
||||
|
return len(m.messages[topic][partition]) |
||||
|
} |
||||
@ -1,100 +0,0 @@ |
|||||
package gateway |
|
||||
|
|
||||
import ( |
|
||||
"net" |
|
||||
"strings" |
|
||||
"testing" |
|
||||
"time" |
|
||||
) |
|
||||
|
|
||||
func TestServerStartAndClose(t *testing.T) { |
|
||||
// Skip this test as it requires a real SeaweedMQ Agent
|
|
||||
t.Skip("This test requires SeaweedMQ Agent integration - run manually with agent available") |
|
||||
|
|
||||
srv := NewServer(Options{ |
|
||||
Listen: ":0", |
|
||||
Masters: "localhost:9333", // Use masters instead of AgentAddress
|
|
||||
}) |
|
||||
if err := srv.Start(); err != nil { |
|
||||
t.Fatalf("start: %v", err) |
|
||||
} |
|
||||
// ensure listener is open and accepting
|
|
||||
// try to dial the actual chosen port
|
|
||||
// Find the actual address
|
|
||||
var addr string |
|
||||
if srv.ln == nil { |
|
||||
t.Fatalf("listener not set") |
|
||||
} |
|
||||
addr = srv.ln.Addr().String() |
|
||||
c, err := net.DialTimeout("tcp", addr, time.Second) |
|
||||
if err != nil { |
|
||||
t.Fatalf("dial: %v", err) |
|
||||
} |
|
||||
_ = c.Close() |
|
||||
if err := srv.Close(); err != nil { |
|
||||
t.Fatalf("close: %v", err) |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
func TestGetListenerAddr(t *testing.T) { |
|
||||
// Skip this test as it requires a real SeaweedMQ Agent
|
|
||||
t.Skip("This test requires SeaweedMQ Agent integration - run manually with agent available") |
|
||||
|
|
||||
// Test with localhost binding - should return the actual address
|
|
||||
srv := NewServer(Options{ |
|
||||
Listen: "127.0.0.1:0", |
|
||||
Masters: "localhost:9333", // Would need real agent for this test
|
|
||||
}) |
|
||||
if err := srv.Start(); err != nil { |
|
||||
t.Fatalf("start: %v", err) |
|
||||
} |
|
||||
defer srv.Close() |
|
||||
|
|
||||
host, port := srv.GetListenerAddr() |
|
||||
if host != "127.0.0.1" { |
|
||||
t.Errorf("expected 127.0.0.1, got %s", host) |
|
||||
} |
|
||||
if port <= 0 { |
|
||||
t.Errorf("expected valid port, got %d", port) |
|
||||
} |
|
||||
|
|
||||
// Test IPv6 all interfaces binding - should resolve to non-loopback IP
|
|
||||
srv6 := NewServer(Options{ |
|
||||
Listen: "[::]:0", |
|
||||
Masters: "localhost:9333", // Would need real agent for this test
|
|
||||
}) |
|
||||
if err := srv6.Start(); err != nil { |
|
||||
t.Fatalf("start IPv6: %v", err) |
|
||||
} |
|
||||
defer srv6.Close() |
|
||||
|
|
||||
host6, port6 := srv6.GetListenerAddr() |
|
||||
// Should not be localhost when bound to all interfaces
|
|
||||
if host6 == "localhost" { |
|
||||
t.Errorf("IPv6 all interfaces should not resolve to localhost, got %s", host6) |
|
||||
} |
|
||||
if port6 <= 0 { |
|
||||
t.Errorf("expected valid port, got %d", port6) |
|
||||
} |
|
||||
t.Logf("IPv6 all interfaces resolved to: %s:%d", host6, port6) |
|
||||
} |
|
||||
|
|
||||
func TestResolveAdvertisedAddress(t *testing.T) { |
|
||||
addr := resolveAdvertisedAddress() |
|
||||
if addr == "" { |
|
||||
t.Error("resolveAdvertisedAddress returned empty string") |
|
||||
} |
|
||||
|
|
||||
// Should be a valid IP address
|
|
||||
ip := net.ParseIP(addr) |
|
||||
if ip == nil { |
|
||||
t.Errorf("resolveAdvertisedAddress returned invalid IP: %s", addr) |
|
||||
} |
|
||||
|
|
||||
// Should not be IPv6 (we prefer IPv4 for Kafka compatibility)
|
|
||||
if strings.Contains(addr, ":") { |
|
||||
t.Errorf("Expected IPv4 address, got IPv6: %s", addr) |
|
||||
} |
|
||||
|
|
||||
t.Logf("Resolved advertised address: %s", addr) |
|
||||
} |
|
||||
@ -0,0 +1,37 @@ |
|||||
|
package gateway |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/protocol" |
||||
|
) |
||||
|
|
||||
|
// NewTestServer creates a server for testing with in-memory handlers
|
||||
|
// This should ONLY be used for testing - never in production
|
||||
|
// WARNING: This function includes test-only components in production binary
|
||||
|
func NewTestServer(opts Options) *Server { |
||||
|
ctx, cancel := context.WithCancel(context.Background()) |
||||
|
|
||||
|
// Use test handler with storage capability
|
||||
|
handler := protocol.NewTestHandler() |
||||
|
|
||||
|
return &Server{ |
||||
|
opts: opts, |
||||
|
ctx: ctx, |
||||
|
cancel: cancel, |
||||
|
handler: handler, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// NewTestServerWithHandler creates a test server with a custom handler
|
||||
|
// This allows tests to inject specific handlers for different scenarios
|
||||
|
func NewTestServerWithHandler(opts Options, handler *protocol.Handler) *Server { |
||||
|
ctx, cancel := context.WithCancel(context.Background()) |
||||
|
|
||||
|
return &Server{ |
||||
|
opts: opts, |
||||
|
ctx: ctx, |
||||
|
cancel: cancel, |
||||
|
handler: handler, |
||||
|
} |
||||
|
} |
||||
@ -1,558 +0,0 @@ |
|||||
package protocol |
|
||||
|
|
||||
import ( |
|
||||
"context" |
|
||||
"encoding/binary" |
|
||||
"net" |
|
||||
"testing" |
|
||||
"time" |
|
||||
) |
|
||||
|
|
||||
func TestHandler_ApiVersions(t *testing.T) { |
|
||||
// Create handler
|
|
||||
h := NewTestHandler() |
|
||||
|
|
||||
// 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(context.Background(), server) |
|
||||
}() |
|
||||
|
|
||||
// Create ApiVersions request manually
|
|
||||
// Request format: api_key(2) + api_version(2) + correlation_id(4) + client_id_size(2) + client_id + body
|
|
||||
correlationID := uint32(12345) |
|
||||
clientID := "test-client" |
|
||||
|
|
||||
message := make([]byte, 0, 64) |
|
||||
message = append(message, 0, 18) // API key 18 (ApiVersions)
|
|
||||
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 for ApiVersions
|
|
||||
|
|
||||
// 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) + error_code(2) + num_api_keys(4) + api_keys + throttle_time(4)
|
|
||||
if len(respBuf) < 14 { // 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 error code
|
|
||||
errorCode := binary.BigEndian.Uint16(respBuf[4:6]) |
|
||||
if errorCode != 0 { |
|
||||
t.Errorf("expected no error, got error code: %d", errorCode) |
|
||||
} |
|
||||
|
|
||||
// Check number of API keys
|
|
||||
numAPIKeys := binary.BigEndian.Uint32(respBuf[6:10]) |
|
||||
if numAPIKeys != 14 { |
|
||||
t.Errorf("expected 14 API keys, got: %d", numAPIKeys) |
|
||||
} |
|
||||
|
|
||||
// Check API key details: api_key(2) + min_version(2) + max_version(2)
|
|
||||
if len(respBuf) < 52 { // need space for 7 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) |
|
||||
} |
|
||||
if minVersion != 0 { |
|
||||
t.Errorf("expected min version 0, got: %d", minVersion) |
|
||||
} |
|
||||
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) |
|
||||
} |
|
||||
|
|
||||
// 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 != 2 { |
|
||||
t.Errorf("expected max version 2, got: %d", maxVersion3) |
|
||||
} |
|
||||
|
|
||||
// Fourth API key (CreateTopics)
|
|
||||
apiKey4 := binary.BigEndian.Uint16(respBuf[28:30]) |
|
||||
minVersion4 := binary.BigEndian.Uint16(respBuf[30:32]) |
|
||||
maxVersion4 := binary.BigEndian.Uint16(respBuf[32:34]) |
|
||||
|
|
||||
if apiKey4 != 19 { |
|
||||
t.Errorf("expected API key 19, got: %d", apiKey4) |
|
||||
} |
|
||||
if minVersion4 != 0 { |
|
||||
t.Errorf("expected min version 0, got: %d", minVersion4) |
|
||||
} |
|
||||
if maxVersion4 != 4 { |
|
||||
t.Errorf("expected max version 4, got: %d", maxVersion4) |
|
||||
} |
|
||||
|
|
||||
// Fifth API key (DeleteTopics)
|
|
||||
apiKey5 := binary.BigEndian.Uint16(respBuf[34:36]) |
|
||||
minVersion5 := binary.BigEndian.Uint16(respBuf[36:38]) |
|
||||
maxVersion5 := binary.BigEndian.Uint16(respBuf[38:40]) |
|
||||
|
|
||||
if apiKey5 != 20 { |
|
||||
t.Errorf("expected API key 20, got: %d", apiKey5) |
|
||||
} |
|
||||
if minVersion5 != 0 { |
|
||||
t.Errorf("expected min version 0, got: %d", minVersion5) |
|
||||
} |
|
||||
if maxVersion5 != 4 { |
|
||||
t.Errorf("expected max version 4, got: %d", maxVersion5) |
|
||||
} |
|
||||
|
|
||||
// Sixth API key (Produce)
|
|
||||
apiKey6 := binary.BigEndian.Uint16(respBuf[40:42]) |
|
||||
minVersion6 := binary.BigEndian.Uint16(respBuf[42:44]) |
|
||||
maxVersion6 := binary.BigEndian.Uint16(respBuf[44:46]) |
|
||||
|
|
||||
if apiKey6 != 0 { |
|
||||
t.Errorf("expected API key 0, got: %d", apiKey6) |
|
||||
} |
|
||||
if minVersion6 != 0 { |
|
||||
t.Errorf("expected min version 0, got: %d", minVersion6) |
|
||||
} |
|
||||
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 != 7 { |
|
||||
t.Errorf("expected max version 7, got: %d", maxVersion7) |
|
||||
} |
|
||||
|
|
||||
// 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_handleApiVersions(t *testing.T) { |
|
||||
h := NewTestHandler() |
|
||||
correlationID := uint32(999) |
|
||||
|
|
||||
response, err := h.handleApiVersions(correlationID) |
|
||||
if err != nil { |
|
||||
t.Fatalf("handleApiVersions: %v", err) |
|
||||
} |
|
||||
|
|
||||
if len(response) < 90 { // minimum expected size (now has 13 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 number of API keys
|
|
||||
numAPIKeys := binary.BigEndian.Uint32(response[6:10]) |
|
||||
if numAPIKeys != 14 { |
|
||||
t.Errorf("expected 14 API keys, got: %d", numAPIKeys) |
|
||||
} |
|
||||
|
|
||||
// Check first API key (ApiVersions)
|
|
||||
apiKey := binary.BigEndian.Uint16(response[10:12]) |
|
||||
if apiKey != 18 { |
|
||||
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) |
|
||||
} |
|
||||
|
|
||||
// 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) { |
|
||||
h := NewTestHandler() |
|
||||
correlationID := uint32(456) |
|
||||
|
|
||||
// Empty request body for minimal test
|
|
||||
requestBody := []byte{} |
|
||||
|
|
||||
response, err := h.handleMetadata(correlationID, 0, requestBody) |
|
||||
if err != nil { |
|
||||
t.Fatalf("handleMetadata: %v", err) |
|
||||
} |
|
||||
|
|
||||
if len(response) < 31 { // minimum expected size for v0 (calculated)
|
|
||||
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 brokers count
|
|
||||
brokersCount := binary.BigEndian.Uint32(response[4:8]) |
|
||||
if brokersCount != 1 { |
|
||||
t.Errorf("brokers count: got %d, want 1", brokersCount) |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
func TestHandler_handleListOffsets(t *testing.T) { |
|
||||
h := NewTestHandler() |
|
||||
correlationID := uint32(123) |
|
||||
|
|
||||
// Build a simple ListOffsets v0 request body (header stripped): topics
|
|
||||
// topics_count(4) + topic + partitions
|
|
||||
topic := "test-topic" |
|
||||
|
|
||||
requestBody := make([]byte, 0, 64) |
|
||||
|
|
||||
// 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, 0, requestBody) |
|
||||
if err != nil { |
|
||||
t.Fatalf("handleListOffsets: %v", err) |
|
||||
} |
|
||||
|
|
||||
if len(response) < 20 { // 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) |
|
||||
} |
|
||||
|
|
||||
// For v0, throttle time is not present; topics count is next
|
|
||||
topicsCount := binary.BigEndian.Uint32(response[4:8]) |
|
||||
if topicsCount != 1 { |
|
||||
t.Errorf("topics count: got %d, want 1", topicsCount) |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
func TestHandler_ListOffsets_EndToEnd(t *testing.T) { |
|
||||
// Create handler
|
|
||||
h := NewTestHandler() |
|
||||
|
|
||||
// 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(context.Background(), 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) + 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 for v0 (no throttle time in v0)
|
|
||||
topicsCount := binary.BigEndian.Uint32(respBuf[4:8]) |
|
||||
if topicsCount != 1 { |
|
||||
t.Errorf("expected 1 topic, got: %d", topicsCount) |
|
||||
} |
|
||||
|
|
||||
// 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 := NewTestHandler() |
|
||||
|
|
||||
// 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(context.Background(), 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) + brokers + topics (v0 has no throttle time)
|
|
||||
if len(respBuf) < 31 { // minimum response size for v0
|
|
||||
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 (immediately after correlation ID in v0)
|
|
||||
brokersCount := binary.BigEndian.Uint32(respBuf[4:8]) |
|
||||
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") |
|
||||
} |
|
||||
} |
|
||||
@ -0,0 +1,299 @@ |
|||||
|
package protocol |
||||
|
|
||||
|
import ( |
||||
|
"fmt" |
||||
|
"sync" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/consumer" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/offset" |
||||
|
) |
||||
|
|
||||
|
// MessageRecord represents a stored message (TEST ONLY)
|
||||
|
type MessageRecord struct { |
||||
|
Key []byte |
||||
|
Value []byte |
||||
|
Timestamp int64 |
||||
|
} |
||||
|
|
||||
|
// basicSeaweedMQHandler is a minimal in-memory implementation for testing (TEST ONLY)
|
||||
|
type basicSeaweedMQHandler struct { |
||||
|
topics map[string]bool |
||||
|
ledgers map[string]*offset.Ledger |
||||
|
// messages stores actual message content indexed by topic-partition-offset
|
||||
|
messages map[string]map[int32]map[int64]*MessageRecord // topic -> partition -> offset -> message
|
||||
|
mu sync.RWMutex |
||||
|
} |
||||
|
|
||||
|
// testSeaweedMQHandler is a minimal mock implementation for testing (TEST ONLY)
|
||||
|
type testSeaweedMQHandler struct { |
||||
|
topics map[string]bool |
||||
|
ledgers map[string]*offset.Ledger |
||||
|
mu sync.RWMutex |
||||
|
} |
||||
|
|
||||
|
// NewTestHandler creates a handler for testing purposes without requiring SeaweedMQ masters
|
||||
|
// This should ONLY be used in tests - uses basicSeaweedMQHandler for message storage simulation
|
||||
|
func NewTestHandler() *Handler { |
||||
|
return &Handler{ |
||||
|
groupCoordinator: consumer.NewGroupCoordinator(), |
||||
|
brokerHost: "localhost", |
||||
|
brokerPort: 9092, |
||||
|
seaweedMQHandler: &basicSeaweedMQHandler{ |
||||
|
topics: make(map[string]bool), |
||||
|
ledgers: make(map[string]*offset.Ledger), |
||||
|
messages: make(map[string]map[int32]map[int64]*MessageRecord), |
||||
|
}, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// NewSimpleTestHandler creates a minimal test handler without message storage
|
||||
|
// This should ONLY be used for basic protocol tests that don't need message content
|
||||
|
func NewSimpleTestHandler() *Handler { |
||||
|
return &Handler{ |
||||
|
groupCoordinator: consumer.NewGroupCoordinator(), |
||||
|
brokerHost: "localhost", |
||||
|
brokerPort: 9092, |
||||
|
seaweedMQHandler: &testSeaweedMQHandler{ |
||||
|
topics: make(map[string]bool), |
||||
|
ledgers: make(map[string]*offset.Ledger), |
||||
|
}, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// ===== basicSeaweedMQHandler implementation (TEST ONLY) =====
|
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) TopicExists(topic string) bool { |
||||
|
return b.topics[topic] |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) ListTopics() []string { |
||||
|
topics := make([]string, 0, len(b.topics)) |
||||
|
for topic := range b.topics { |
||||
|
topics = append(topics, topic) |
||||
|
} |
||||
|
return topics |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) CreateTopic(topic string, partitions int32) error { |
||||
|
b.topics[topic] = true |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) DeleteTopic(topic string) error { |
||||
|
delete(b.topics, topic) |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) GetOrCreateLedger(topic string, partition int32) *offset.Ledger { |
||||
|
b.mu.Lock() |
||||
|
defer b.mu.Unlock() |
||||
|
|
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
if ledger, exists := b.ledgers[key]; exists { |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
// Create new ledger
|
||||
|
ledger := offset.NewLedger() |
||||
|
b.ledgers[key] = ledger |
||||
|
|
||||
|
// Also create the topic if it doesn't exist
|
||||
|
b.topics[topic] = true |
||||
|
|
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) GetLedger(topic string, partition int32) *offset.Ledger { |
||||
|
b.mu.RLock() |
||||
|
defer b.mu.RUnlock() |
||||
|
|
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
if ledger, exists := b.ledgers[key]; exists { |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
// Return nil if ledger doesn't exist (topic doesn't exist)
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) ProduceRecord(topicName string, partitionID int32, key, value []byte) (int64, error) { |
||||
|
// Get or create the ledger first (this will acquire and release the lock)
|
||||
|
ledger := b.GetOrCreateLedger(topicName, partitionID) |
||||
|
|
||||
|
// Now acquire the lock for the rest of the operation
|
||||
|
b.mu.Lock() |
||||
|
defer b.mu.Unlock() |
||||
|
|
||||
|
// Assign an offset and append the record
|
||||
|
offset := ledger.AssignOffsets(1) |
||||
|
timestamp := time.Now().UnixNano() |
||||
|
size := int32(len(value)) |
||||
|
|
||||
|
if err := ledger.AppendRecord(offset, timestamp, size); err != nil { |
||||
|
return 0, fmt.Errorf("failed to append record: %w", err) |
||||
|
} |
||||
|
|
||||
|
// Store the actual message content
|
||||
|
if b.messages[topicName] == nil { |
||||
|
b.messages[topicName] = make(map[int32]map[int64]*MessageRecord) |
||||
|
} |
||||
|
if b.messages[topicName][partitionID] == nil { |
||||
|
b.messages[topicName][partitionID] = make(map[int64]*MessageRecord) |
||||
|
} |
||||
|
|
||||
|
// Make copies of key and value to avoid referencing the original slices
|
||||
|
keyCopy := make([]byte, len(key)) |
||||
|
copy(keyCopy, key) |
||||
|
valueCopy := make([]byte, len(value)) |
||||
|
copy(valueCopy, value) |
||||
|
|
||||
|
b.messages[topicName][partitionID][offset] = &MessageRecord{ |
||||
|
Key: keyCopy, |
||||
|
Value: valueCopy, |
||||
|
Timestamp: timestamp, |
||||
|
} |
||||
|
|
||||
|
return offset, nil |
||||
|
} |
||||
|
|
||||
|
// GetStoredMessages retrieves stored messages for a topic-partition from a given offset (TEST ONLY)
|
||||
|
func (b *basicSeaweedMQHandler) GetStoredMessages(topicName string, partitionID int32, fromOffset int64, maxMessages int) []*MessageRecord { |
||||
|
b.mu.RLock() |
||||
|
defer b.mu.RUnlock() |
||||
|
|
||||
|
if b.messages[topicName] == nil || b.messages[topicName][partitionID] == nil { |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
partitionMessages := b.messages[topicName][partitionID] |
||||
|
var result []*MessageRecord |
||||
|
|
||||
|
// Collect messages starting from fromOffset
|
||||
|
for offset := fromOffset; offset < fromOffset+int64(maxMessages); offset++ { |
||||
|
if msg, exists := partitionMessages[offset]; exists { |
||||
|
result = append(result, msg) |
||||
|
} else { |
||||
|
// No more consecutive messages
|
||||
|
break |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return result |
||||
|
} |
||||
|
|
||||
|
// BasicSMQRecord implements SMQRecord interface for basicSeaweedMQHandler (TEST ONLY)
|
||||
|
type BasicSMQRecord struct { |
||||
|
*MessageRecord |
||||
|
offset int64 |
||||
|
} |
||||
|
|
||||
|
func (r *BasicSMQRecord) GetKey() []byte { return r.Key } |
||||
|
func (r *BasicSMQRecord) GetValue() []byte { return r.Value } |
||||
|
func (r *BasicSMQRecord) GetTimestamp() int64 { return r.Timestamp } |
||||
|
func (r *BasicSMQRecord) GetOffset() int64 { return r.offset } |
||||
|
|
||||
|
// GetStoredRecords retrieves stored message records for basicSeaweedMQHandler (TEST ONLY)
|
||||
|
func (b *basicSeaweedMQHandler) GetStoredRecords(topic string, partition int32, fromOffset int64, maxRecords int) ([]offset.SMQRecord, error) { |
||||
|
messages := b.GetStoredMessages(topic, partition, fromOffset, maxRecords) |
||||
|
if len(messages) == 0 { |
||||
|
return nil, nil |
||||
|
} |
||||
|
|
||||
|
records := make([]offset.SMQRecord, len(messages)) |
||||
|
for i, msg := range messages { |
||||
|
records[i] = &BasicSMQRecord{ |
||||
|
MessageRecord: msg, |
||||
|
offset: fromOffset + int64(i), |
||||
|
} |
||||
|
} |
||||
|
return records, nil |
||||
|
} |
||||
|
|
||||
|
func (b *basicSeaweedMQHandler) Close() error { |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// ===== testSeaweedMQHandler implementation (TEST ONLY) =====
|
||||
|
|
||||
|
func (t *testSeaweedMQHandler) TopicExists(topic string) bool { |
||||
|
return t.topics[topic] |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) ListTopics() []string { |
||||
|
var topics []string |
||||
|
for topic := range t.topics { |
||||
|
topics = append(topics, topic) |
||||
|
} |
||||
|
return topics |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) CreateTopic(topic string, partitions int32) error { |
||||
|
t.topics[topic] = true |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) DeleteTopic(topic string) error { |
||||
|
delete(t.topics, topic) |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) GetOrCreateLedger(topic string, partition int32) *offset.Ledger { |
||||
|
t.mu.Lock() |
||||
|
defer t.mu.Unlock() |
||||
|
|
||||
|
// Mark topic as existing when creating ledger
|
||||
|
t.topics[topic] = true |
||||
|
|
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
if ledger, exists := t.ledgers[key]; exists { |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
ledger := offset.NewLedger() |
||||
|
t.ledgers[key] = ledger |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) GetLedger(topic string, partition int32) *offset.Ledger { |
||||
|
t.mu.RLock() |
||||
|
defer t.mu.RUnlock() |
||||
|
|
||||
|
key := fmt.Sprintf("%s-%d", topic, partition) |
||||
|
if ledger, exists := t.ledgers[key]; exists { |
||||
|
return ledger |
||||
|
} |
||||
|
|
||||
|
// Return nil if ledger doesn't exist (topic doesn't exist)
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) ProduceRecord(topicName string, partitionID int32, key, value []byte) (int64, error) { |
||||
|
// For testing, actually store the record in the ledger
|
||||
|
ledger := t.GetOrCreateLedger(topicName, partitionID) |
||||
|
|
||||
|
// Assign an offset and append the record
|
||||
|
offset := ledger.AssignOffsets(1) |
||||
|
timestamp := time.Now().UnixNano() |
||||
|
size := int32(len(value)) |
||||
|
|
||||
|
if err := ledger.AppendRecord(offset, timestamp, size); err != nil { |
||||
|
return 0, fmt.Errorf("failed to append record: %w", err) |
||||
|
} |
||||
|
|
||||
|
return offset, nil |
||||
|
} |
||||
|
|
||||
|
// GetStoredRecords for testSeaweedMQHandler - returns empty (no storage simulation)
|
||||
|
func (t *testSeaweedMQHandler) GetStoredRecords(topic string, partition int32, fromOffset int64, maxRecords int) ([]offset.SMQRecord, error) { |
||||
|
// Test handler doesn't simulate message storage, return empty
|
||||
|
return nil, nil |
||||
|
} |
||||
|
|
||||
|
func (t *testSeaweedMQHandler) Close() error { |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// AddTopicForTesting moved to handler.go (available to production code for testing)
|
||||
|
|
||||
|
// GetStoredMessages is already defined in the basicSeaweedMQHandler implementation above
|
||||
Write
Preview
Loading…
Cancel
Save
Reference in new issue