You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
4195 lines
145 KiB
4195 lines
145 KiB
package protocol
|
|
|
|
import (
|
|
"bufio"
|
|
"bytes"
|
|
"context"
|
|
"encoding/binary"
|
|
"fmt"
|
|
"io"
|
|
"net"
|
|
"os"
|
|
"strconv"
|
|
"strings"
|
|
"sync"
|
|
"time"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/consumer"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/consumer_offset"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/integration"
|
|
"github.com/seaweedfs/seaweedfs/weed/mq/kafka/schema"
|
|
mqschema "github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/security"
|
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
|
)
|
|
|
|
// GetAdvertisedAddress returns the host:port that should be advertised to clients
|
|
// This handles the Docker networking issue where internal IPs aren't reachable by external clients
|
|
func (h *Handler) GetAdvertisedAddress(gatewayAddr string) (string, int) {
|
|
host, port := "localhost", 9093
|
|
|
|
// Try to parse the gateway address if provided to get the port
|
|
if gatewayAddr != "" {
|
|
if _, gatewayPort, err := net.SplitHostPort(gatewayAddr); err == nil {
|
|
if gatewayPortInt, err := strconv.Atoi(gatewayPort); err == nil {
|
|
port = gatewayPortInt // Only use the port, not the host
|
|
}
|
|
}
|
|
}
|
|
|
|
// Override with environment variable if set, otherwise always use localhost for external clients
|
|
if advertisedHost := os.Getenv("KAFKA_ADVERTISED_HOST"); advertisedHost != "" {
|
|
host = advertisedHost
|
|
} else {
|
|
host = "localhost"
|
|
}
|
|
|
|
return host, port
|
|
}
|
|
|
|
// TopicInfo holds basic information about a topic
|
|
type TopicInfo struct {
|
|
Name string
|
|
Partitions int32
|
|
CreatedAt int64
|
|
}
|
|
|
|
// TopicPartitionKey uniquely identifies a topic partition
|
|
type TopicPartitionKey struct {
|
|
Topic string
|
|
Partition int32
|
|
}
|
|
|
|
// contextKey is a type for context keys to avoid collisions
|
|
type contextKey string
|
|
|
|
const (
|
|
// connContextKey is the context key for storing ConnectionContext
|
|
connContextKey contextKey = "connectionContext"
|
|
)
|
|
|
|
// kafkaRequest represents a Kafka API request to be processed
|
|
type kafkaRequest struct {
|
|
correlationID uint32
|
|
apiKey uint16
|
|
apiVersion uint16
|
|
requestBody []byte
|
|
ctx context.Context
|
|
connContext *ConnectionContext // Per-connection context to avoid race conditions
|
|
}
|
|
|
|
// kafkaResponse represents a Kafka API response
|
|
type kafkaResponse struct {
|
|
correlationID uint32
|
|
apiKey uint16
|
|
apiVersion uint16
|
|
response []byte
|
|
err error
|
|
}
|
|
|
|
const (
|
|
// DefaultKafkaNamespace is the default namespace for Kafka topics in SeaweedMQ
|
|
DefaultKafkaNamespace = "kafka"
|
|
)
|
|
|
|
// APIKey represents a Kafka API key type for better type safety
|
|
type APIKey uint16
|
|
|
|
// Kafka API Keys
|
|
const (
|
|
APIKeyProduce APIKey = 0
|
|
APIKeyFetch APIKey = 1
|
|
APIKeyListOffsets APIKey = 2
|
|
APIKeyMetadata APIKey = 3
|
|
APIKeyOffsetCommit APIKey = 8
|
|
APIKeyOffsetFetch APIKey = 9
|
|
APIKeyFindCoordinator APIKey = 10
|
|
APIKeyJoinGroup APIKey = 11
|
|
APIKeyHeartbeat APIKey = 12
|
|
APIKeyLeaveGroup APIKey = 13
|
|
APIKeySyncGroup APIKey = 14
|
|
APIKeyDescribeGroups APIKey = 15
|
|
APIKeyListGroups APIKey = 16
|
|
APIKeyApiVersions APIKey = 18
|
|
APIKeyCreateTopics APIKey = 19
|
|
APIKeyDeleteTopics APIKey = 20
|
|
APIKeyInitProducerId APIKey = 22
|
|
APIKeyDescribeConfigs APIKey = 32
|
|
APIKeyDescribeCluster APIKey = 60
|
|
)
|
|
|
|
// SeaweedMQHandlerInterface defines the interface for SeaweedMQ integration
|
|
type SeaweedMQHandlerInterface interface {
|
|
TopicExists(topic string) bool
|
|
ListTopics() []string
|
|
CreateTopic(topic string, partitions int32) error
|
|
CreateTopicWithSchemas(name string, partitions int32, keyRecordType *schema_pb.RecordType, valueRecordType *schema_pb.RecordType) error
|
|
DeleteTopic(topic string) error
|
|
GetTopicInfo(topic string) (*integration.KafkaTopicInfo, bool)
|
|
// Ledger methods REMOVED - SMQ handles Kafka offsets natively
|
|
ProduceRecord(topicName string, partitionID int32, key, value []byte) (int64, error)
|
|
ProduceRecordValue(topicName string, partitionID int32, key []byte, recordValueBytes []byte) (int64, error)
|
|
// GetStoredRecords retrieves records from SMQ storage (optional - for advanced implementations)
|
|
// ctx is used to control the fetch timeout (should match Kafka fetch request's MaxWaitTime)
|
|
GetStoredRecords(ctx context.Context, topic string, partition int32, fromOffset int64, maxRecords int) ([]integration.SMQRecord, error)
|
|
// GetEarliestOffset returns the earliest available offset for a topic partition
|
|
GetEarliestOffset(topic string, partition int32) (int64, error)
|
|
// GetLatestOffset returns the latest available offset for a topic partition
|
|
GetLatestOffset(topic string, partition int32) (int64, error)
|
|
// WithFilerClient executes a function with a filer client for accessing SeaweedMQ metadata
|
|
WithFilerClient(streamingMode bool, fn func(client filer_pb.SeaweedFilerClient) error) error
|
|
// GetBrokerAddresses returns the discovered SMQ broker addresses for Metadata responses
|
|
GetBrokerAddresses() []string
|
|
// CreatePerConnectionBrokerClient creates an isolated BrokerClient for each TCP connection
|
|
CreatePerConnectionBrokerClient() (*integration.BrokerClient, error)
|
|
// SetProtocolHandler sets the protocol handler reference for connection context access
|
|
SetProtocolHandler(handler integration.ProtocolHandler)
|
|
Close() error
|
|
}
|
|
|
|
// ConsumerOffsetStorage defines the interface for storing consumer offsets
|
|
// This is used by OffsetCommit and OffsetFetch protocol handlers
|
|
type ConsumerOffsetStorage interface {
|
|
CommitOffset(group, topic string, partition int32, offset int64, metadata string) error
|
|
FetchOffset(group, topic string, partition int32) (int64, string, error)
|
|
FetchAllOffsets(group string) (map[TopicPartition]OffsetMetadata, error)
|
|
DeleteGroup(group string) error
|
|
Close() error
|
|
}
|
|
|
|
// TopicPartition uniquely identifies a topic partition for offset storage
|
|
type TopicPartition struct {
|
|
Topic string
|
|
Partition int32
|
|
}
|
|
|
|
// OffsetMetadata contains offset and associated metadata
|
|
type OffsetMetadata struct {
|
|
Offset int64
|
|
Metadata string
|
|
}
|
|
|
|
// TopicSchemaConfig holds schema configuration for a topic
|
|
type TopicSchemaConfig struct {
|
|
// Value schema configuration
|
|
ValueSchemaID uint32
|
|
ValueSchemaFormat schema.Format
|
|
|
|
// Key schema configuration (optional)
|
|
KeySchemaID uint32
|
|
KeySchemaFormat schema.Format
|
|
HasKeySchema bool // indicates if key schema is configured
|
|
}
|
|
|
|
// Legacy accessors for backward compatibility
|
|
func (c *TopicSchemaConfig) SchemaID() uint32 {
|
|
return c.ValueSchemaID
|
|
}
|
|
|
|
func (c *TopicSchemaConfig) SchemaFormat() schema.Format {
|
|
return c.ValueSchemaFormat
|
|
}
|
|
|
|
// getTopicSchemaFormat returns the schema format string for a topic
|
|
func (h *Handler) getTopicSchemaFormat(topic string) string {
|
|
h.topicSchemaConfigMu.RLock()
|
|
defer h.topicSchemaConfigMu.RUnlock()
|
|
|
|
if config, exists := h.topicSchemaConfigs[topic]; exists {
|
|
return config.ValueSchemaFormat.String()
|
|
}
|
|
return "" // Empty string means schemaless or format unknown
|
|
}
|
|
|
|
// stringPtr returns a pointer to the given string
|
|
func stringPtr(s string) *string {
|
|
return &s
|
|
}
|
|
|
|
// Handler processes Kafka protocol requests from clients using SeaweedMQ
|
|
type Handler struct {
|
|
// SeaweedMQ integration
|
|
seaweedMQHandler SeaweedMQHandlerInterface
|
|
|
|
// SMQ offset storage removed - using ConsumerOffsetStorage instead
|
|
|
|
// Consumer offset storage for Kafka protocol OffsetCommit/OffsetFetch
|
|
consumerOffsetStorage ConsumerOffsetStorage
|
|
|
|
// Consumer group coordination
|
|
groupCoordinator *consumer.GroupCoordinator
|
|
|
|
// Response caching to reduce CPU usage for repeated requests
|
|
metadataCache *ResponseCache
|
|
coordinatorCache *ResponseCache
|
|
|
|
// Coordinator registry for distributed coordinator assignment
|
|
coordinatorRegistry CoordinatorRegistryInterface
|
|
|
|
// Schema management (optional, for schematized topics)
|
|
schemaManager *schema.Manager
|
|
useSchema bool
|
|
brokerClient *schema.BrokerClient
|
|
|
|
// Topic schema configuration cache
|
|
topicSchemaConfigs map[string]*TopicSchemaConfig
|
|
topicSchemaConfigMu sync.RWMutex
|
|
|
|
// Track registered schemas to prevent duplicate registrations
|
|
registeredSchemas map[string]bool // key: "topic:schemaID" or "topic-key:schemaID"
|
|
registeredSchemasMu sync.RWMutex
|
|
|
|
filerClient filer_pb.SeaweedFilerClient
|
|
|
|
// SMQ broker addresses discovered from masters for Metadata responses
|
|
smqBrokerAddresses []string
|
|
|
|
// Gateway address for coordinator registry
|
|
gatewayAddress string
|
|
|
|
// Connection contexts stored per connection ID (thread-safe)
|
|
// Replaces the race-prone shared connContext field
|
|
connContexts sync.Map // map[string]*ConnectionContext
|
|
|
|
// Schema Registry URL for delayed initialization
|
|
schemaRegistryURL string
|
|
|
|
// Default partition count for auto-created topics
|
|
defaultPartitions int32
|
|
}
|
|
|
|
// NewHandler creates a basic Kafka handler with in-memory storage
|
|
// WARNING: This is for testing ONLY - never use in production!
|
|
// For production use with persistent storage, use NewSeaweedMQBrokerHandler instead
|
|
func NewHandler() *Handler {
|
|
// Production safety check - prevent accidental production use
|
|
// Comment out for testing: os.Getenv can be used for runtime checks
|
|
panic("NewHandler() with in-memory storage should NEVER be used in production! Use NewSeaweedMQBrokerHandler() with SeaweedMQ masters for production, or NewTestHandler() for tests.")
|
|
}
|
|
|
|
// NewTestHandler and NewSimpleTestHandler moved to handler_test.go (test-only file)
|
|
|
|
// All test-related types and implementations moved to handler_test.go (test-only file)
|
|
|
|
// NewTestHandlerWithMock creates a test handler with a custom SeaweedMQHandlerInterface
|
|
// This is useful for unit tests that need a handler but don't want to connect to real SeaweedMQ
|
|
func NewTestHandlerWithMock(mockHandler SeaweedMQHandlerInterface) *Handler {
|
|
return &Handler{
|
|
seaweedMQHandler: mockHandler,
|
|
consumerOffsetStorage: nil, // Unit tests don't need offset storage
|
|
groupCoordinator: consumer.NewGroupCoordinator(),
|
|
registeredSchemas: make(map[string]bool),
|
|
topicSchemaConfigs: make(map[string]*TopicSchemaConfig),
|
|
defaultPartitions: 1,
|
|
}
|
|
}
|
|
|
|
// NewSeaweedMQBrokerHandler creates a new handler with SeaweedMQ broker integration
|
|
func NewSeaweedMQBrokerHandler(masters string, filerGroup string, clientHost string) (*Handler, error) {
|
|
return NewSeaweedMQBrokerHandlerWithDefaults(masters, filerGroup, clientHost, 4) // Default to 4 partitions
|
|
}
|
|
|
|
// NewSeaweedMQBrokerHandlerWithDefaults creates a new handler with SeaweedMQ broker integration and custom defaults
|
|
func NewSeaweedMQBrokerHandlerWithDefaults(masters string, filerGroup string, clientHost string, defaultPartitions int32) (*Handler, error) {
|
|
// Set up SeaweedMQ integration
|
|
smqHandler, err := integration.NewSeaweedMQBrokerHandler(masters, filerGroup, clientHost)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Use the shared filer client accessor from SeaweedMQHandler
|
|
sharedFilerAccessor := smqHandler.GetFilerClientAccessor()
|
|
if sharedFilerAccessor == nil {
|
|
return nil, fmt.Errorf("no shared filer client accessor available from SMQ handler")
|
|
}
|
|
|
|
// Create consumer offset storage (for OffsetCommit/OffsetFetch protocol)
|
|
// Use filer-based storage for persistence across restarts
|
|
consumerOffsetStorage := newOffsetStorageAdapter(
|
|
consumer_offset.NewFilerStorage(sharedFilerAccessor),
|
|
)
|
|
|
|
// Create response caches to reduce CPU usage
|
|
// Metadata cache: 5 second TTL (Schema Registry polls frequently)
|
|
// Coordinator cache: 10 second TTL (less frequent, more stable)
|
|
metadataCache := NewResponseCache(5 * time.Second)
|
|
coordinatorCache := NewResponseCache(10 * time.Second)
|
|
|
|
// Start cleanup loops
|
|
metadataCache.StartCleanupLoop(30 * time.Second)
|
|
coordinatorCache.StartCleanupLoop(60 * time.Second)
|
|
|
|
handler := &Handler{
|
|
seaweedMQHandler: smqHandler,
|
|
consumerOffsetStorage: consumerOffsetStorage,
|
|
groupCoordinator: consumer.NewGroupCoordinator(),
|
|
smqBrokerAddresses: nil, // Will be set by SetSMQBrokerAddresses() when server starts
|
|
registeredSchemas: make(map[string]bool),
|
|
defaultPartitions: defaultPartitions,
|
|
metadataCache: metadataCache,
|
|
coordinatorCache: coordinatorCache,
|
|
}
|
|
|
|
// Set protocol handler reference in SMQ handler for connection context access
|
|
smqHandler.SetProtocolHandler(handler)
|
|
|
|
return handler, nil
|
|
}
|
|
|
|
// AddTopicForTesting creates a topic for testing purposes
|
|
// This delegates to the underlying SeaweedMQ handler
|
|
func (h *Handler) AddTopicForTesting(topicName string, partitions int32) {
|
|
if h.seaweedMQHandler != nil {
|
|
h.seaweedMQHandler.CreateTopic(topicName, partitions)
|
|
}
|
|
}
|
|
|
|
// Delegate methods to SeaweedMQ handler
|
|
|
|
// GetOrCreateLedger method REMOVED - SMQ handles Kafka offsets natively
|
|
|
|
// GetLedger method REMOVED - SMQ handles Kafka offsets natively
|
|
|
|
// Close shuts down the handler and all connections
|
|
func (h *Handler) Close() error {
|
|
// Close group coordinator
|
|
if h.groupCoordinator != nil {
|
|
h.groupCoordinator.Close()
|
|
}
|
|
|
|
// Close broker client if present
|
|
if h.brokerClient != nil {
|
|
if err := h.brokerClient.Close(); err != nil {
|
|
Warning("Failed to close broker client: %v", err)
|
|
}
|
|
}
|
|
|
|
// Close SeaweedMQ handler if present
|
|
if h.seaweedMQHandler != nil {
|
|
return h.seaweedMQHandler.Close()
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// StoreRecordBatch stores a record batch for later retrieval during Fetch operations
|
|
func (h *Handler) StoreRecordBatch(topicName string, partition int32, baseOffset int64, recordBatch []byte) {
|
|
// Record batch storage is now handled by the SeaweedMQ handler
|
|
}
|
|
|
|
// GetRecordBatch retrieves a stored record batch that contains the requested offset
|
|
func (h *Handler) GetRecordBatch(topicName string, partition int32, offset int64) ([]byte, bool) {
|
|
// Record batch retrieval is now handled by the SeaweedMQ handler
|
|
return nil, false
|
|
}
|
|
|
|
// SetSMQBrokerAddresses updates the SMQ broker addresses used in Metadata responses
|
|
func (h *Handler) SetSMQBrokerAddresses(brokerAddresses []string) {
|
|
h.smqBrokerAddresses = brokerAddresses
|
|
}
|
|
|
|
// GetSMQBrokerAddresses returns the SMQ broker addresses
|
|
func (h *Handler) GetSMQBrokerAddresses() []string {
|
|
// First try to get from the SeaweedMQ handler (preferred)
|
|
if h.seaweedMQHandler != nil {
|
|
if brokerAddresses := h.seaweedMQHandler.GetBrokerAddresses(); len(brokerAddresses) > 0 {
|
|
return brokerAddresses
|
|
}
|
|
}
|
|
|
|
// Fallback to manually set addresses
|
|
if len(h.smqBrokerAddresses) > 0 {
|
|
return h.smqBrokerAddresses
|
|
}
|
|
|
|
// Final fallback for testing
|
|
return []string{"localhost:17777"}
|
|
}
|
|
|
|
// GetGatewayAddress returns the current gateway address as a string (for coordinator registry)
|
|
func (h *Handler) GetGatewayAddress() string {
|
|
if h.gatewayAddress != "" {
|
|
return h.gatewayAddress
|
|
}
|
|
// Fallback for testing
|
|
return "localhost:9092"
|
|
}
|
|
|
|
// SetGatewayAddress sets the gateway address for coordinator registry
|
|
func (h *Handler) SetGatewayAddress(address string) {
|
|
h.gatewayAddress = address
|
|
}
|
|
|
|
// SetCoordinatorRegistry sets the coordinator registry for this handler
|
|
func (h *Handler) SetCoordinatorRegistry(registry CoordinatorRegistryInterface) {
|
|
h.coordinatorRegistry = registry
|
|
}
|
|
|
|
// GetCoordinatorRegistry returns the coordinator registry
|
|
func (h *Handler) GetCoordinatorRegistry() CoordinatorRegistryInterface {
|
|
return h.coordinatorRegistry
|
|
}
|
|
|
|
// isDataPlaneAPI returns true if the API key is a data plane operation (Fetch, Produce)
|
|
// Data plane operations can be slow and may block on I/O
|
|
func isDataPlaneAPI(apiKey uint16) bool {
|
|
switch APIKey(apiKey) {
|
|
case APIKeyProduce:
|
|
return true
|
|
case APIKeyFetch:
|
|
return true
|
|
default:
|
|
return false
|
|
}
|
|
}
|
|
|
|
// GetConnectionContext returns the current connection context converted to integration.ConnectionContext
|
|
// This implements the integration.ProtocolHandler interface
|
|
//
|
|
// NOTE: Since this method doesn't receive a context parameter, it returns a "best guess" connection context.
|
|
// In single-connection scenarios (like tests), this works correctly. In high-concurrency scenarios with many
|
|
// simultaneous connections, this may return a connection context from a different connection.
|
|
// For a proper fix, the integration.ProtocolHandler interface would need to be updated to pass context.Context.
|
|
func (h *Handler) GetConnectionContext() *integration.ConnectionContext {
|
|
// Try to find any active connection context
|
|
// In most cases (single connection, or low concurrency), this will return the correct context
|
|
var connCtx *ConnectionContext
|
|
h.connContexts.Range(func(key, value interface{}) bool {
|
|
if ctx, ok := value.(*ConnectionContext); ok {
|
|
connCtx = ctx
|
|
return false // Stop iteration after finding first context
|
|
}
|
|
return true
|
|
})
|
|
|
|
if connCtx == nil {
|
|
return nil
|
|
}
|
|
|
|
// Convert protocol.ConnectionContext to integration.ConnectionContext
|
|
return &integration.ConnectionContext{
|
|
ClientID: connCtx.ClientID,
|
|
ConsumerGroup: connCtx.ConsumerGroup,
|
|
MemberID: connCtx.MemberID,
|
|
BrokerClient: connCtx.BrokerClient,
|
|
}
|
|
}
|
|
|
|
// HandleConn processes a single client connection
|
|
func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
|
connectionID := fmt.Sprintf("%s->%s", conn.RemoteAddr(), conn.LocalAddr())
|
|
|
|
// Record connection metrics
|
|
RecordConnectionMetrics()
|
|
|
|
// Create cancellable context for this connection
|
|
// This ensures all requests are cancelled when the connection closes
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
defer cancel()
|
|
|
|
// CRITICAL: Create per-connection BrokerClient for isolated gRPC streams
|
|
// This prevents different connections from interfering with each other's Fetch requests
|
|
// In mock/unit test mode, this may not be available, so we continue without it
|
|
var connBrokerClient *integration.BrokerClient
|
|
connBrokerClient, err := h.seaweedMQHandler.CreatePerConnectionBrokerClient()
|
|
if err != nil {
|
|
// Continue without broker client for unit test/mock mode
|
|
connBrokerClient = nil
|
|
}
|
|
|
|
// RACE CONDITION FIX: Create connection-local context and pass through request pipeline
|
|
// Store in thread-safe map to enable lookup from methods that don't have direct access
|
|
connContext := &ConnectionContext{
|
|
RemoteAddr: conn.RemoteAddr(),
|
|
LocalAddr: conn.LocalAddr(),
|
|
ConnectionID: connectionID,
|
|
BrokerClient: connBrokerClient,
|
|
}
|
|
|
|
// Store in thread-safe map for later retrieval
|
|
h.connContexts.Store(connectionID, connContext)
|
|
|
|
defer func() {
|
|
// Close all partition readers first
|
|
cleanupPartitionReaders(connContext)
|
|
// Close the per-connection broker client
|
|
if connBrokerClient != nil {
|
|
if closeErr := connBrokerClient.Close(); closeErr != nil {
|
|
Error("[%s] Error closing BrokerClient: %v", connectionID, closeErr)
|
|
}
|
|
}
|
|
// Remove connection context from map
|
|
h.connContexts.Delete(connectionID)
|
|
RecordDisconnectionMetrics()
|
|
conn.Close()
|
|
}()
|
|
|
|
r := bufio.NewReader(conn)
|
|
w := bufio.NewWriter(conn)
|
|
defer w.Flush()
|
|
|
|
// Use default timeout config
|
|
timeoutConfig := DefaultTimeoutConfig()
|
|
|
|
// Track consecutive read timeouts to detect stale/CLOSE_WAIT connections
|
|
consecutiveTimeouts := 0
|
|
const maxConsecutiveTimeouts = 3 // Give up after 3 timeouts in a row
|
|
|
|
// CRITICAL: Separate control plane from data plane
|
|
// Control plane: Metadata, Heartbeat, JoinGroup, etc. (must be fast, never block)
|
|
// Data plane: Fetch, Produce (can be slow, may block on I/O)
|
|
//
|
|
// Architecture:
|
|
// - Main loop routes requests to appropriate channel based on API key
|
|
// - Control goroutine processes control messages (fast, sequential)
|
|
// - Data goroutine processes data messages (can be slow)
|
|
// - Response writer handles responses in order using correlation IDs
|
|
controlChan := make(chan *kafkaRequest, 10)
|
|
dataChan := make(chan *kafkaRequest, 10)
|
|
responseChan := make(chan *kafkaResponse, 100)
|
|
var wg sync.WaitGroup
|
|
|
|
// Response writer - maintains request/response order per connection
|
|
// CRITICAL: While we process requests concurrently (control/data plane),
|
|
// we MUST track the order requests arrive and send responses in that same order.
|
|
// Solution: Track received correlation IDs in a queue, send responses in that queue order.
|
|
correlationQueue := make([]uint32, 0, 100)
|
|
correlationQueueMu := &sync.Mutex{}
|
|
|
|
wg.Add(1)
|
|
go func() {
|
|
defer wg.Done()
|
|
glog.V(2).Infof("[%s] Response writer started", connectionID)
|
|
defer glog.V(2).Infof("[%s] Response writer exiting", connectionID)
|
|
pendingResponses := make(map[uint32]*kafkaResponse)
|
|
nextToSend := 0 // Index in correlationQueue
|
|
|
|
for {
|
|
select {
|
|
case resp, ok := <-responseChan:
|
|
if !ok {
|
|
// responseChan closed, exit
|
|
return
|
|
}
|
|
glog.V(2).Infof("[%s] Response writer received correlation=%d from responseChan", connectionID, resp.correlationID)
|
|
correlationQueueMu.Lock()
|
|
pendingResponses[resp.correlationID] = resp
|
|
|
|
// Send all responses we can in queue order
|
|
for nextToSend < len(correlationQueue) {
|
|
expectedID := correlationQueue[nextToSend]
|
|
readyResp, exists := pendingResponses[expectedID]
|
|
if !exists {
|
|
// Response not ready yet, stop sending
|
|
glog.V(3).Infof("[%s] Response writer: waiting for correlation=%d (nextToSend=%d, queueLen=%d)", connectionID, expectedID, nextToSend, len(correlationQueue))
|
|
break
|
|
}
|
|
|
|
// Send this response
|
|
if readyResp.err != nil {
|
|
Error("[%s] Error processing correlation=%d: %v", connectionID, readyResp.correlationID, readyResp.err)
|
|
} else {
|
|
glog.V(2).Infof("[%s] Response writer: about to write correlation=%d (%d bytes)", connectionID, readyResp.correlationID, len(readyResp.response))
|
|
if writeErr := h.writeResponseWithHeader(w, readyResp.correlationID, readyResp.apiKey, readyResp.apiVersion, readyResp.response, timeoutConfig.WriteTimeout); writeErr != nil {
|
|
glog.Errorf("[%s] Response writer: WRITE ERROR correlation=%d: %v - EXITING", connectionID, readyResp.correlationID, writeErr)
|
|
Error("[%s] Write error correlation=%d: %v", connectionID, readyResp.correlationID, writeErr)
|
|
correlationQueueMu.Unlock()
|
|
return
|
|
}
|
|
glog.V(2).Infof("[%s] Response writer: successfully wrote correlation=%d", connectionID, readyResp.correlationID)
|
|
}
|
|
|
|
// Remove from pending and advance
|
|
delete(pendingResponses, expectedID)
|
|
nextToSend++
|
|
}
|
|
correlationQueueMu.Unlock()
|
|
case <-ctx.Done():
|
|
// Context cancelled, exit immediately to prevent deadlock
|
|
glog.V(2).Infof("[%s] Response writer: context cancelled, exiting", connectionID)
|
|
return
|
|
}
|
|
}
|
|
}()
|
|
|
|
// Control plane processor - fast operations, never blocks
|
|
wg.Add(1)
|
|
go func() {
|
|
defer wg.Done()
|
|
for {
|
|
select {
|
|
case req, ok := <-controlChan:
|
|
if !ok {
|
|
// Channel closed, exit
|
|
return
|
|
}
|
|
glog.V(2).Infof("[%s] Control plane processing correlation=%d, apiKey=%d", connectionID, req.correlationID, req.apiKey)
|
|
|
|
// CRITICAL: Wrap request processing with panic recovery to prevent deadlocks
|
|
// If processRequestSync panics, we MUST still send a response to avoid blocking the response writer
|
|
var response []byte
|
|
var err error
|
|
func() {
|
|
defer func() {
|
|
if r := recover(); r != nil {
|
|
glog.Errorf("[%s] PANIC in control plane correlation=%d: %v", connectionID, req.correlationID, r)
|
|
err = fmt.Errorf("internal server error: panic in request handler: %v", r)
|
|
}
|
|
}()
|
|
response, err = h.processRequestSync(req)
|
|
}()
|
|
|
|
glog.V(2).Infof("[%s] Control plane completed correlation=%d, sending to responseChan", connectionID, req.correlationID)
|
|
select {
|
|
case responseChan <- &kafkaResponse{
|
|
correlationID: req.correlationID,
|
|
apiKey: req.apiKey,
|
|
apiVersion: req.apiVersion,
|
|
response: response,
|
|
err: err,
|
|
}:
|
|
glog.V(2).Infof("[%s] Control plane sent correlation=%d to responseChan", connectionID, req.correlationID)
|
|
case <-ctx.Done():
|
|
// Connection closed, stop processing
|
|
return
|
|
case <-time.After(5 * time.Second):
|
|
glog.Errorf("[%s] DEADLOCK: Control plane timeout sending correlation=%d to responseChan (buffer full?)", connectionID, req.correlationID)
|
|
}
|
|
case <-ctx.Done():
|
|
// Context cancelled, drain remaining requests before exiting
|
|
glog.V(2).Infof("[%s] Control plane: context cancelled, draining remaining requests", connectionID)
|
|
for {
|
|
select {
|
|
case req, ok := <-controlChan:
|
|
if !ok {
|
|
return
|
|
}
|
|
// Process remaining requests with a short timeout
|
|
glog.V(3).Infof("[%s] Control plane: processing drained request correlation=%d", connectionID, req.correlationID)
|
|
response, err := h.processRequestSync(req)
|
|
select {
|
|
case responseChan <- &kafkaResponse{
|
|
correlationID: req.correlationID,
|
|
apiKey: req.apiKey,
|
|
apiVersion: req.apiVersion,
|
|
response: response,
|
|
err: err,
|
|
}:
|
|
glog.V(3).Infof("[%s] Control plane: sent drained response correlation=%d", connectionID, req.correlationID)
|
|
case <-time.After(1 * time.Second):
|
|
glog.Warningf("[%s] Control plane: timeout sending drained response correlation=%d, discarding", connectionID, req.correlationID)
|
|
return
|
|
}
|
|
default:
|
|
// Channel empty, safe to exit
|
|
glog.V(2).Infof("[%s] Control plane: drain complete, exiting", connectionID)
|
|
return
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}()
|
|
|
|
// Data plane processor - can block on I/O
|
|
wg.Add(1)
|
|
go func() {
|
|
defer wg.Done()
|
|
for {
|
|
select {
|
|
case req, ok := <-dataChan:
|
|
if !ok {
|
|
// Channel closed, exit
|
|
return
|
|
}
|
|
glog.V(2).Infof("[%s] Data plane processing correlation=%d, apiKey=%d", connectionID, req.correlationID, req.apiKey)
|
|
|
|
// CRITICAL: Wrap request processing with panic recovery to prevent deadlocks
|
|
// If processRequestSync panics, we MUST still send a response to avoid blocking the response writer
|
|
var response []byte
|
|
var err error
|
|
func() {
|
|
defer func() {
|
|
if r := recover(); r != nil {
|
|
glog.Errorf("[%s] PANIC in data plane correlation=%d: %v", connectionID, req.correlationID, r)
|
|
err = fmt.Errorf("internal server error: panic in request handler: %v", r)
|
|
}
|
|
}()
|
|
response, err = h.processRequestSync(req)
|
|
}()
|
|
|
|
glog.V(2).Infof("[%s] Data plane completed correlation=%d, sending to responseChan", connectionID, req.correlationID)
|
|
// Use select with context to avoid sending on closed channel
|
|
select {
|
|
case responseChan <- &kafkaResponse{
|
|
correlationID: req.correlationID,
|
|
apiKey: req.apiKey,
|
|
apiVersion: req.apiVersion,
|
|
response: response,
|
|
err: err,
|
|
}:
|
|
glog.V(2).Infof("[%s] Data plane sent correlation=%d to responseChan", connectionID, req.correlationID)
|
|
case <-ctx.Done():
|
|
// Connection closed, stop processing
|
|
return
|
|
case <-time.After(5 * time.Second):
|
|
glog.Errorf("[%s] DEADLOCK: Data plane timeout sending correlation=%d to responseChan (buffer full?)", connectionID, req.correlationID)
|
|
}
|
|
case <-ctx.Done():
|
|
// Context cancelled, drain remaining requests before exiting
|
|
glog.V(2).Infof("[%s] Data plane: context cancelled, draining remaining requests", connectionID)
|
|
for {
|
|
select {
|
|
case req, ok := <-dataChan:
|
|
if !ok {
|
|
return
|
|
}
|
|
// Process remaining requests with a short timeout
|
|
glog.V(3).Infof("[%s] Data plane: processing drained request correlation=%d", connectionID, req.correlationID)
|
|
response, err := h.processRequestSync(req)
|
|
select {
|
|
case responseChan <- &kafkaResponse{
|
|
correlationID: req.correlationID,
|
|
apiKey: req.apiKey,
|
|
apiVersion: req.apiVersion,
|
|
response: response,
|
|
err: err,
|
|
}:
|
|
glog.V(3).Infof("[%s] Data plane: sent drained response correlation=%d", connectionID, req.correlationID)
|
|
case <-time.After(1 * time.Second):
|
|
glog.Warningf("[%s] Data plane: timeout sending drained response correlation=%d, discarding", connectionID, req.correlationID)
|
|
return
|
|
}
|
|
default:
|
|
// Channel empty, safe to exit
|
|
glog.V(2).Infof("[%s] Data plane: drain complete, exiting", connectionID)
|
|
return
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}()
|
|
|
|
defer func() {
|
|
// CRITICAL: Close channels in correct order to avoid panics
|
|
// 1. Close input channels to stop accepting new requests
|
|
close(controlChan)
|
|
close(dataChan)
|
|
// 2. Wait for worker goroutines to finish processing and sending responses
|
|
wg.Wait()
|
|
// 3. NOW close responseChan to signal response writer to exit
|
|
close(responseChan)
|
|
}()
|
|
|
|
for {
|
|
// Check if context is cancelled
|
|
select {
|
|
case <-ctx.Done():
|
|
return ctx.Err()
|
|
default:
|
|
}
|
|
|
|
// Set a read deadline for the connection based on context or default timeout
|
|
var readDeadline time.Time
|
|
var timeoutDuration time.Duration
|
|
|
|
if deadline, ok := ctx.Deadline(); ok {
|
|
readDeadline = deadline
|
|
timeoutDuration = time.Until(deadline)
|
|
} else {
|
|
// Use configurable read timeout instead of hardcoded 5 seconds
|
|
timeoutDuration = timeoutConfig.ReadTimeout
|
|
readDeadline = time.Now().Add(timeoutDuration)
|
|
}
|
|
|
|
if err := conn.SetReadDeadline(readDeadline); err != nil {
|
|
return fmt.Errorf("set read deadline: %w", err)
|
|
}
|
|
|
|
// Check context before reading
|
|
select {
|
|
case <-ctx.Done():
|
|
// Give a small delay to ensure proper cleanup
|
|
time.Sleep(100 * time.Millisecond)
|
|
return ctx.Err()
|
|
default:
|
|
// If context is close to being cancelled, set a very short timeout
|
|
if deadline, ok := ctx.Deadline(); ok {
|
|
timeUntilDeadline := time.Until(deadline)
|
|
if timeUntilDeadline < 2*time.Second && timeUntilDeadline > 0 {
|
|
shortDeadline := time.Now().Add(500 * time.Millisecond)
|
|
if err := conn.SetReadDeadline(shortDeadline); err == nil {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// Read message size (4 bytes)
|
|
var sizeBytes [4]byte
|
|
if _, err := io.ReadFull(r, sizeBytes[:]); err != nil {
|
|
if err == io.EOF {
|
|
return nil
|
|
}
|
|
if netErr, ok := err.(net.Error); ok && netErr.Timeout() {
|
|
// CRITICAL FIX: Track consecutive timeouts to detect CLOSE_WAIT connections
|
|
// When remote peer closes, connection enters CLOSE_WAIT and reads keep timing out
|
|
// After several consecutive timeouts with no data, assume connection is dead
|
|
consecutiveTimeouts++
|
|
if consecutiveTimeouts >= maxConsecutiveTimeouts {
|
|
return nil
|
|
}
|
|
// Idle timeout while waiting for next request; keep connection open
|
|
continue
|
|
}
|
|
return fmt.Errorf("read message size: %w", err)
|
|
}
|
|
|
|
// Successfully read data, reset timeout counter
|
|
consecutiveTimeouts = 0
|
|
|
|
// Successfully read the message size
|
|
size := binary.BigEndian.Uint32(sizeBytes[:])
|
|
// Debug("Read message size: %d bytes", size)
|
|
if size == 0 || size > 1024*1024 { // 1MB limit
|
|
// Use standardized error for message size limit
|
|
// Send error response for message too large
|
|
errorResponse := BuildErrorResponse(0, ErrorCodeMessageTooLarge) // correlation ID 0 since we can't parse it yet
|
|
if writeErr := h.writeResponseWithCorrelationID(w, 0, errorResponse, timeoutConfig.WriteTimeout); writeErr != nil {
|
|
}
|
|
return fmt.Errorf("message size %d exceeds limit", size)
|
|
}
|
|
|
|
// Set read deadline for message body
|
|
if err := conn.SetReadDeadline(time.Now().Add(timeoutConfig.ReadTimeout)); err != nil {
|
|
}
|
|
|
|
// Read the message
|
|
messageBuf := make([]byte, size)
|
|
if _, err := io.ReadFull(r, messageBuf); err != nil {
|
|
_ = HandleTimeoutError(err, "read") // errorCode
|
|
return fmt.Errorf("read message: %w", err)
|
|
}
|
|
|
|
// Parse at least the basic header to get API key and correlation ID
|
|
if len(messageBuf) < 8 {
|
|
return fmt.Errorf("message too short")
|
|
}
|
|
|
|
apiKey := binary.BigEndian.Uint16(messageBuf[0:2])
|
|
apiVersion := binary.BigEndian.Uint16(messageBuf[2:4])
|
|
correlationID := binary.BigEndian.Uint32(messageBuf[4:8])
|
|
|
|
// Debug("Parsed header - API Key: %d (%s), Version: %d, Correlation: %d", apiKey, getAPIName(APIKey(apiKey)), apiVersion, correlationID)
|
|
|
|
// Validate API version against what we support
|
|
if err := h.validateAPIVersion(apiKey, apiVersion); err != nil {
|
|
glog.Errorf("API VERSION VALIDATION FAILED: Key=%d (%s), Version=%d, error=%v", apiKey, getAPIName(APIKey(apiKey)), apiVersion, err)
|
|
// Return proper Kafka error response for unsupported version
|
|
response, writeErr := h.buildUnsupportedVersionResponse(correlationID, apiKey, apiVersion)
|
|
if writeErr != nil {
|
|
return fmt.Errorf("build error response: %w", writeErr)
|
|
}
|
|
// CRITICAL: Send error response through response queue to maintain sequential ordering
|
|
// This prevents deadlocks in the response writer which expects all correlation IDs in sequence
|
|
select {
|
|
case responseChan <- &kafkaResponse{
|
|
correlationID: correlationID,
|
|
apiKey: apiKey,
|
|
apiVersion: apiVersion,
|
|
response: response,
|
|
err: nil,
|
|
}:
|
|
// Error response queued successfully, continue reading next request
|
|
continue
|
|
case <-ctx.Done():
|
|
return ctx.Err()
|
|
}
|
|
}
|
|
|
|
// CRITICAL DEBUG: Log that validation passed
|
|
glog.V(4).Infof("API VERSION VALIDATION PASSED: Key=%d (%s), Version=%d, Correlation=%d - proceeding to header parsing",
|
|
apiKey, getAPIName(APIKey(apiKey)), apiVersion, correlationID)
|
|
|
|
// Extract request body - special handling for ApiVersions requests
|
|
var requestBody []byte
|
|
if apiKey == uint16(APIKeyApiVersions) && apiVersion >= 3 {
|
|
// ApiVersions v3+ uses client_software_name + client_software_version, not client_id
|
|
bodyOffset := 8 // Skip api_key(2) + api_version(2) + correlation_id(4)
|
|
|
|
// Skip client_software_name (compact string)
|
|
if len(messageBuf) > bodyOffset {
|
|
clientNameLen := int(messageBuf[bodyOffset]) // compact string length
|
|
if clientNameLen > 0 {
|
|
clientNameLen-- // compact strings encode length+1
|
|
bodyOffset += 1 + clientNameLen
|
|
} else {
|
|
bodyOffset += 1 // just the length byte for null/empty
|
|
}
|
|
}
|
|
|
|
// Skip client_software_version (compact string)
|
|
if len(messageBuf) > bodyOffset {
|
|
clientVersionLen := int(messageBuf[bodyOffset]) // compact string length
|
|
if clientVersionLen > 0 {
|
|
clientVersionLen-- // compact strings encode length+1
|
|
bodyOffset += 1 + clientVersionLen
|
|
} else {
|
|
bodyOffset += 1 // just the length byte for null/empty
|
|
}
|
|
}
|
|
|
|
// Skip tagged fields (should be 0x00 for ApiVersions)
|
|
if len(messageBuf) > bodyOffset {
|
|
bodyOffset += 1 // tagged fields byte
|
|
}
|
|
|
|
requestBody = messageBuf[bodyOffset:]
|
|
} else {
|
|
// Parse header using flexible version utilities for other APIs
|
|
header, parsedRequestBody, parseErr := ParseRequestHeader(messageBuf)
|
|
if parseErr != nil {
|
|
// CRITICAL: Log the parsing error for debugging
|
|
glog.Errorf("REQUEST HEADER PARSING FAILED: API=%d (%s) v%d, correlation=%d, error=%v, msgLen=%d",
|
|
apiKey, getAPIName(APIKey(apiKey)), apiVersion, correlationID, parseErr, len(messageBuf))
|
|
|
|
// Fall back to basic header parsing if flexible version parsing fails
|
|
|
|
// Basic header parsing fallback (original logic)
|
|
bodyOffset := 8
|
|
if len(messageBuf) < bodyOffset+2 {
|
|
glog.Errorf("FALLBACK PARSING FAILED: missing client_id length, msgLen=%d", len(messageBuf))
|
|
return fmt.Errorf("invalid header: missing client_id length")
|
|
}
|
|
clientIDLen := int16(binary.BigEndian.Uint16(messageBuf[bodyOffset : bodyOffset+2]))
|
|
bodyOffset += 2
|
|
if clientIDLen >= 0 {
|
|
if len(messageBuf) < bodyOffset+int(clientIDLen) {
|
|
glog.Errorf("FALLBACK PARSING FAILED: client_id truncated, clientIDLen=%d, msgLen=%d", clientIDLen, len(messageBuf))
|
|
return fmt.Errorf("invalid header: client_id truncated")
|
|
}
|
|
bodyOffset += int(clientIDLen)
|
|
}
|
|
requestBody = messageBuf[bodyOffset:]
|
|
glog.V(2).Infof("FALLBACK PARSING SUCCESS: API=%d (%s) v%d, bodyLen=%d", apiKey, getAPIName(APIKey(apiKey)), apiVersion, len(requestBody))
|
|
} else {
|
|
// Use the successfully parsed request body
|
|
requestBody = parsedRequestBody
|
|
|
|
// Validate parsed header matches what we already extracted
|
|
if header.APIKey != apiKey || header.APIVersion != apiVersion || header.CorrelationID != correlationID {
|
|
// Fall back to basic parsing rather than failing
|
|
bodyOffset := 8
|
|
if len(messageBuf) < bodyOffset+2 {
|
|
return fmt.Errorf("invalid header: missing client_id length")
|
|
}
|
|
clientIDLen := int16(binary.BigEndian.Uint16(messageBuf[bodyOffset : bodyOffset+2]))
|
|
bodyOffset += 2
|
|
if clientIDLen >= 0 {
|
|
if len(messageBuf) < bodyOffset+int(clientIDLen) {
|
|
return fmt.Errorf("invalid header: client_id truncated")
|
|
}
|
|
bodyOffset += int(clientIDLen)
|
|
}
|
|
requestBody = messageBuf[bodyOffset:]
|
|
} else if header.ClientID != nil {
|
|
// Store client ID in connection context for use in fetch requests
|
|
connContext.ClientID = *header.ClientID
|
|
}
|
|
}
|
|
}
|
|
|
|
// CRITICAL: Route request to appropriate processor
|
|
// Control plane: Fast, never blocks (Metadata, Heartbeat, etc.)
|
|
// Data plane: Can be slow (Fetch, Produce)
|
|
|
|
// Attach connection context to the Go context for retrieval in nested calls
|
|
ctxWithConn := context.WithValue(ctx, connContextKey, connContext)
|
|
|
|
req := &kafkaRequest{
|
|
correlationID: correlationID,
|
|
apiKey: apiKey,
|
|
apiVersion: apiVersion,
|
|
requestBody: requestBody,
|
|
ctx: ctxWithConn,
|
|
connContext: connContext, // Pass per-connection context to avoid race conditions
|
|
}
|
|
|
|
// Route to appropriate channel based on API key
|
|
var targetChan chan *kafkaRequest
|
|
if isDataPlaneAPI(apiKey) {
|
|
targetChan = dataChan
|
|
} else {
|
|
targetChan = controlChan
|
|
}
|
|
|
|
// CRITICAL: Only add to correlation queue AFTER successful channel send
|
|
// If we add before and the channel blocks, the correlation ID is in the queue
|
|
// but the request never gets processed, causing response writer deadlock
|
|
select {
|
|
case targetChan <- req:
|
|
// Request queued successfully - NOW add to correlation tracking
|
|
correlationQueueMu.Lock()
|
|
correlationQueue = append(correlationQueue, correlationID)
|
|
correlationQueueMu.Unlock()
|
|
case <-ctx.Done():
|
|
return ctx.Err()
|
|
case <-time.After(10 * time.Second):
|
|
// Channel full for too long - this shouldn't happen with proper backpressure
|
|
glog.Errorf("[%s] CRITICAL: Failed to queue correlation=%d after 10s timeout - channel full!", connectionID, correlationID)
|
|
return fmt.Errorf("request queue full: correlation=%d", correlationID)
|
|
}
|
|
}
|
|
}
|
|
|
|
// processRequestSync processes a single Kafka API request synchronously and returns the response
|
|
func (h *Handler) processRequestSync(req *kafkaRequest) ([]byte, error) {
|
|
// Record request start time for latency tracking
|
|
requestStart := time.Now()
|
|
apiName := getAPIName(APIKey(req.apiKey))
|
|
|
|
var response []byte
|
|
var err error
|
|
|
|
switch APIKey(req.apiKey) {
|
|
case APIKeyApiVersions:
|
|
response, err = h.handleApiVersions(req.correlationID, req.apiVersion)
|
|
|
|
case APIKeyMetadata:
|
|
response, err = h.handleMetadata(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyListOffsets:
|
|
response, err = h.handleListOffsets(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyCreateTopics:
|
|
response, err = h.handleCreateTopics(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyDeleteTopics:
|
|
response, err = h.handleDeleteTopics(req.correlationID, req.requestBody)
|
|
|
|
case APIKeyProduce:
|
|
response, err = h.handleProduce(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyFetch:
|
|
response, err = h.handleFetch(req.ctx, req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyJoinGroup:
|
|
response, err = h.handleJoinGroup(req.connContext, req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeySyncGroup:
|
|
response, err = h.handleSyncGroup(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyOffsetCommit:
|
|
response, err = h.handleOffsetCommit(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyOffsetFetch:
|
|
response, err = h.handleOffsetFetch(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyFindCoordinator:
|
|
response, err = h.handleFindCoordinator(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyHeartbeat:
|
|
response, err = h.handleHeartbeat(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyLeaveGroup:
|
|
response, err = h.handleLeaveGroup(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyDescribeGroups:
|
|
response, err = h.handleDescribeGroups(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyListGroups:
|
|
response, err = h.handleListGroups(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyDescribeConfigs:
|
|
response, err = h.handleDescribeConfigs(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyDescribeCluster:
|
|
response, err = h.handleDescribeCluster(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
case APIKeyInitProducerId:
|
|
response, err = h.handleInitProducerId(req.correlationID, req.apiVersion, req.requestBody)
|
|
|
|
default:
|
|
Warning("Unsupported API key: %d (%s) v%d - Correlation: %d", req.apiKey, apiName, req.apiVersion, req.correlationID)
|
|
err = fmt.Errorf("unsupported API key: %d (version %d)", req.apiKey, req.apiVersion)
|
|
}
|
|
|
|
glog.V(2).Infof("processRequestSync: Switch completed for correlation=%d, about to record metrics", req.correlationID)
|
|
// Record metrics
|
|
requestLatency := time.Since(requestStart)
|
|
if err != nil {
|
|
RecordErrorMetrics(req.apiKey, requestLatency)
|
|
} else {
|
|
RecordRequestMetrics(req.apiKey, requestLatency)
|
|
}
|
|
glog.V(2).Infof("processRequestSync: Metrics recorded for correlation=%d, about to return", req.correlationID)
|
|
|
|
return response, err
|
|
}
|
|
|
|
// ApiKeyInfo represents supported API key information
|
|
type ApiKeyInfo struct {
|
|
ApiKey APIKey
|
|
MinVersion uint16
|
|
MaxVersion uint16
|
|
}
|
|
|
|
// SupportedApiKeys defines all supported API keys and their version ranges
|
|
var SupportedApiKeys = []ApiKeyInfo{
|
|
{APIKeyApiVersions, 0, 4}, // ApiVersions - support up to v4 for Kafka 8.0.0 compatibility
|
|
{APIKeyMetadata, 0, 7}, // Metadata - support up to v7
|
|
{APIKeyProduce, 0, 7}, // Produce
|
|
{APIKeyFetch, 0, 7}, // Fetch
|
|
{APIKeyListOffsets, 0, 2}, // ListOffsets
|
|
{APIKeyCreateTopics, 0, 5}, // CreateTopics
|
|
{APIKeyDeleteTopics, 0, 4}, // DeleteTopics
|
|
{APIKeyFindCoordinator, 0, 3}, // FindCoordinator - v3+ supports flexible responses
|
|
{APIKeyJoinGroup, 0, 6}, // JoinGroup
|
|
{APIKeySyncGroup, 0, 5}, // SyncGroup
|
|
{APIKeyOffsetCommit, 0, 2}, // OffsetCommit
|
|
{APIKeyOffsetFetch, 0, 5}, // OffsetFetch
|
|
{APIKeyHeartbeat, 0, 4}, // Heartbeat
|
|
{APIKeyLeaveGroup, 0, 4}, // LeaveGroup
|
|
{APIKeyDescribeGroups, 0, 5}, // DescribeGroups
|
|
{APIKeyListGroups, 0, 4}, // ListGroups
|
|
{APIKeyDescribeConfigs, 0, 4}, // DescribeConfigs
|
|
{APIKeyInitProducerId, 0, 4}, // InitProducerId - support up to v4 for transactional producers
|
|
{APIKeyDescribeCluster, 0, 1}, // DescribeCluster - for AdminClient compatibility (KIP-919)
|
|
}
|
|
|
|
func (h *Handler) handleApiVersions(correlationID uint32, apiVersion uint16) ([]byte, error) {
|
|
// Send correct flexible or non-flexible response based on API version
|
|
// This fixes the AdminClient "collection size 2184558" error by using proper varint encoding
|
|
response := make([]byte, 0, 512)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID
|
|
// Do NOT include it in the response body
|
|
|
|
// === RESPONSE BODY ===
|
|
// Error code (2 bytes) - always fixed-length
|
|
response = append(response, 0, 0) // No error
|
|
|
|
// API Keys Array - CRITICAL FIX: Use correct encoding based on version
|
|
if apiVersion >= 3 {
|
|
// FLEXIBLE FORMAT: Compact array with varint length - THIS FIXES THE ADMINCLIENT BUG!
|
|
response = append(response, CompactArrayLength(uint32(len(SupportedApiKeys)))...)
|
|
|
|
// Add API key entries with per-element tagged fields
|
|
for _, api := range SupportedApiKeys {
|
|
response = append(response, byte(api.ApiKey>>8), byte(api.ApiKey)) // api_key (2 bytes)
|
|
response = append(response, byte(api.MinVersion>>8), byte(api.MinVersion)) // min_version (2 bytes)
|
|
response = append(response, byte(api.MaxVersion>>8), byte(api.MaxVersion)) // max_version (2 bytes)
|
|
response = append(response, 0x00) // Per-element tagged fields (varint: empty)
|
|
}
|
|
|
|
} else {
|
|
// NON-FLEXIBLE FORMAT: Regular array with fixed 4-byte length
|
|
response = append(response, 0, 0, 0, byte(len(SupportedApiKeys))) // Array length (4 bytes)
|
|
|
|
// Add API key entries without tagged fields
|
|
for _, api := range SupportedApiKeys {
|
|
response = append(response, byte(api.ApiKey>>8), byte(api.ApiKey)) // api_key (2 bytes)
|
|
response = append(response, byte(api.MinVersion>>8), byte(api.MinVersion)) // min_version (2 bytes)
|
|
response = append(response, byte(api.MaxVersion>>8), byte(api.MaxVersion)) // max_version (2 bytes)
|
|
}
|
|
}
|
|
|
|
// Throttle time (for v1+) - always fixed-length
|
|
if apiVersion >= 1 {
|
|
response = append(response, 0, 0, 0, 0) // throttle_time_ms = 0 (4 bytes)
|
|
}
|
|
|
|
// Response-level tagged fields (for v3+ flexible versions)
|
|
if apiVersion >= 3 {
|
|
response = append(response, 0x00) // Empty response-level tagged fields (varint: single byte 0)
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// handleMetadataV0 implements the Metadata API response in version 0 format.
|
|
// v0 response layout:
|
|
// correlation_id(4) + brokers(ARRAY) + topics(ARRAY)
|
|
// broker: node_id(4) + host(STRING) + port(4)
|
|
// topic: error_code(2) + name(STRING) + partitions(ARRAY)
|
|
// partition: error_code(2) + partition_id(4) + leader(4) + replicas(ARRAY<int32>) + isr(ARRAY<int32>)
|
|
func (h *Handler) HandleMetadataV0(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
response := make([]byte, 0, 256)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID
|
|
// Do NOT include it in the response body
|
|
|
|
// Brokers array length (4 bytes) - 1 broker (this gateway)
|
|
response = append(response, 0, 0, 0, 1)
|
|
|
|
// Broker 0: node_id(4) + host(STRING) + port(4)
|
|
response = append(response, 0, 0, 0, 1) // node_id = 1 (consistent with partitions)
|
|
|
|
// Get advertised address for client connections
|
|
host, port := h.GetAdvertisedAddress(h.GetGatewayAddress())
|
|
|
|
// Host (STRING: 2 bytes length + bytes) - validate length fits in uint16
|
|
if len(host) > 65535 {
|
|
return nil, fmt.Errorf("host name too long: %d bytes", len(host))
|
|
}
|
|
hostLen := uint16(len(host))
|
|
response = append(response, byte(hostLen>>8), byte(hostLen))
|
|
response = append(response, []byte(host)...)
|
|
|
|
// Port (4 bytes) - validate port range
|
|
if port < 0 || port > 65535 {
|
|
return nil, fmt.Errorf("invalid port number: %d", port)
|
|
}
|
|
portBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(portBytes, uint32(port))
|
|
response = append(response, portBytes...)
|
|
|
|
// Parse requested topics (empty means all)
|
|
requestedTopics := h.parseMetadataTopics(requestBody)
|
|
glog.V(0).Infof("[METADATA v0] Requested topics: %v (empty=all)", requestedTopics)
|
|
|
|
// Determine topics to return using SeaweedMQ handler
|
|
var topicsToReturn []string
|
|
if len(requestedTopics) == 0 {
|
|
topicsToReturn = h.seaweedMQHandler.ListTopics()
|
|
} else {
|
|
for _, name := range requestedTopics {
|
|
if h.seaweedMQHandler.TopicExists(name) {
|
|
topicsToReturn = append(topicsToReturn, name)
|
|
}
|
|
}
|
|
}
|
|
|
|
// Topics array length (4 bytes)
|
|
topicsCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(topicsCountBytes, uint32(len(topicsToReturn)))
|
|
response = append(response, topicsCountBytes...)
|
|
|
|
// Topic entries
|
|
for _, topicName := range topicsToReturn {
|
|
// error_code(2) = 0
|
|
response = append(response, 0, 0)
|
|
|
|
// name (STRING)
|
|
nameBytes := []byte(topicName)
|
|
nameLen := uint16(len(nameBytes))
|
|
response = append(response, byte(nameLen>>8), byte(nameLen))
|
|
response = append(response, nameBytes...)
|
|
|
|
// Get actual partition count from topic info
|
|
topicInfo, exists := h.seaweedMQHandler.GetTopicInfo(topicName)
|
|
partitionCount := h.GetDefaultPartitions() // Use configurable default
|
|
if exists && topicInfo != nil {
|
|
partitionCount = topicInfo.Partitions
|
|
}
|
|
|
|
// partitions array length (4 bytes)
|
|
partitionsBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionsBytes, uint32(partitionCount))
|
|
response = append(response, partitionsBytes...)
|
|
|
|
// Create partition entries for each partition
|
|
for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
|
|
// partition: error_code(2) + partition_id(4) + leader(4)
|
|
response = append(response, 0, 0) // error_code
|
|
|
|
// partition_id (4 bytes)
|
|
partitionIDBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionIDBytes, uint32(partitionID))
|
|
response = append(response, partitionIDBytes...)
|
|
|
|
response = append(response, 0, 0, 0, 1) // leader = 1 (this broker)
|
|
|
|
// replicas: array length(4) + one broker id (1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
|
|
// isr: array length(4) + one broker id (1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
}
|
|
}
|
|
|
|
for range topicsToReturn {
|
|
}
|
|
return response, nil
|
|
}
|
|
|
|
func (h *Handler) HandleMetadataV1(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// Simplified Metadata v1 implementation - based on working v0 + v1 additions
|
|
// v1 adds: ControllerID (after brokers), Rack (for brokers), IsInternal (for topics)
|
|
|
|
// Parse requested topics (empty means all)
|
|
requestedTopics := h.parseMetadataTopics(requestBody)
|
|
glog.V(0).Infof("[METADATA v1] Requested topics: %v (empty=all)", requestedTopics)
|
|
|
|
// Determine topics to return using SeaweedMQ handler
|
|
var topicsToReturn []string
|
|
if len(requestedTopics) == 0 {
|
|
topicsToReturn = h.seaweedMQHandler.ListTopics()
|
|
} else {
|
|
for _, name := range requestedTopics {
|
|
if h.seaweedMQHandler.TopicExists(name) {
|
|
topicsToReturn = append(topicsToReturn, name)
|
|
}
|
|
}
|
|
}
|
|
|
|
// Build response using same approach as v0 but with v1 additions
|
|
response := make([]byte, 0, 256)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// Brokers array length (4 bytes) - 1 broker (this gateway)
|
|
response = append(response, 0, 0, 0, 1)
|
|
|
|
// Broker 0: node_id(4) + host(STRING) + port(4) + rack(STRING)
|
|
response = append(response, 0, 0, 0, 1) // node_id = 1
|
|
|
|
// Get advertised address for client connections
|
|
host, port := h.GetAdvertisedAddress(h.GetGatewayAddress())
|
|
|
|
// Host (STRING: 2 bytes length + bytes) - validate length fits in uint16
|
|
if len(host) > 65535 {
|
|
return nil, fmt.Errorf("host name too long: %d bytes", len(host))
|
|
}
|
|
hostLen := uint16(len(host))
|
|
response = append(response, byte(hostLen>>8), byte(hostLen))
|
|
response = append(response, []byte(host)...)
|
|
|
|
// Port (4 bytes) - validate port range
|
|
if port < 0 || port > 65535 {
|
|
return nil, fmt.Errorf("invalid port number: %d", port)
|
|
}
|
|
portBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(portBytes, uint32(port))
|
|
response = append(response, portBytes...)
|
|
|
|
// Rack (STRING: 2 bytes length + bytes) - v1 addition, non-nullable empty string
|
|
response = append(response, 0, 0) // empty string
|
|
|
|
// ControllerID (4 bytes) - v1 addition
|
|
response = append(response, 0, 0, 0, 1) // controller_id = 1
|
|
|
|
// Topics array length (4 bytes)
|
|
topicsCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(topicsCountBytes, uint32(len(topicsToReturn)))
|
|
response = append(response, topicsCountBytes...)
|
|
|
|
// Topics
|
|
for _, topicName := range topicsToReturn {
|
|
// error_code (2 bytes)
|
|
response = append(response, 0, 0)
|
|
|
|
// topic name (STRING: 2 bytes length + bytes)
|
|
topicLen := uint16(len(topicName))
|
|
response = append(response, byte(topicLen>>8), byte(topicLen))
|
|
response = append(response, []byte(topicName)...)
|
|
|
|
// is_internal (1 byte) - v1 addition
|
|
response = append(response, 0) // false
|
|
|
|
// Get actual partition count from topic info
|
|
topicInfo, exists := h.seaweedMQHandler.GetTopicInfo(topicName)
|
|
partitionCount := h.GetDefaultPartitions() // Use configurable default
|
|
if exists && topicInfo != nil {
|
|
partitionCount = topicInfo.Partitions
|
|
}
|
|
|
|
// partitions array length (4 bytes)
|
|
partitionsBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionsBytes, uint32(partitionCount))
|
|
response = append(response, partitionsBytes...)
|
|
|
|
// Create partition entries for each partition
|
|
for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
|
|
// partition: error_code(2) + partition_id(4) + leader_id(4) + replicas(ARRAY) + isr(ARRAY)
|
|
response = append(response, 0, 0) // error_code
|
|
|
|
// partition_id (4 bytes)
|
|
partitionIDBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partitionIDBytes, uint32(partitionID))
|
|
response = append(response, partitionIDBytes...)
|
|
|
|
response = append(response, 0, 0, 0, 1) // leader_id = 1
|
|
|
|
// replicas: array length(4) + one broker id (1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
|
|
// isr: array length(4) + one broker id (1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
response = append(response, 0, 0, 0, 1)
|
|
}
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// HandleMetadataV2 implements Metadata API v2 with ClusterID field
|
|
func (h *Handler) HandleMetadataV2(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// Metadata v2 adds ClusterID field (nullable string)
|
|
// v2 response layout: correlation_id(4) + brokers(ARRAY) + cluster_id(NULLABLE_STRING) + controller_id(4) + topics(ARRAY)
|
|
|
|
// Parse requested topics (empty means all)
|
|
requestedTopics := h.parseMetadataTopics(requestBody)
|
|
glog.V(0).Infof("[METADATA v2] Requested topics: %v (empty=all)", requestedTopics)
|
|
|
|
// Determine topics to return using SeaweedMQ handler
|
|
var topicsToReturn []string
|
|
if len(requestedTopics) == 0 {
|
|
topicsToReturn = h.seaweedMQHandler.ListTopics()
|
|
} else {
|
|
for _, name := range requestedTopics {
|
|
if h.seaweedMQHandler.TopicExists(name) {
|
|
topicsToReturn = append(topicsToReturn, name)
|
|
}
|
|
}
|
|
}
|
|
|
|
var buf bytes.Buffer
|
|
|
|
// Correlation ID (4 bytes)
|
|
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID
|
|
// Do NOT include it in the response body
|
|
|
|
// Brokers array (4 bytes length + brokers) - 1 broker (this gateway)
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Get advertised address for client connections
|
|
host, port := h.GetAdvertisedAddress(h.GetGatewayAddress())
|
|
|
|
nodeID := int32(1) // Single gateway node
|
|
|
|
// Broker: node_id(4) + host(STRING) + port(4) + rack(STRING) + cluster_id(NULLABLE_STRING)
|
|
binary.Write(&buf, binary.BigEndian, nodeID)
|
|
|
|
// Host (STRING: 2 bytes length + data) - validate length fits in int16
|
|
if len(host) > 32767 {
|
|
return nil, fmt.Errorf("host name too long: %d bytes", len(host))
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int16(len(host)))
|
|
buf.WriteString(host)
|
|
|
|
// Port (4 bytes) - validate port range
|
|
if port < 0 || port > 65535 {
|
|
return nil, fmt.Errorf("invalid port number: %d", port)
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int32(port))
|
|
|
|
// Rack (STRING: 2 bytes length + data) - v1+ addition, non-nullable
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // Empty string
|
|
|
|
// ClusterID (NULLABLE_STRING: 2 bytes length + data) - v2 addition
|
|
// Schema Registry requires a non-null cluster ID
|
|
clusterID := "seaweedfs-kafka-gateway"
|
|
binary.Write(&buf, binary.BigEndian, int16(len(clusterID)))
|
|
buf.WriteString(clusterID)
|
|
|
|
// ControllerID (4 bytes) - v1+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Topics array (4 bytes length + topics)
|
|
binary.Write(&buf, binary.BigEndian, int32(len(topicsToReturn)))
|
|
|
|
for _, topicName := range topicsToReturn {
|
|
// ErrorCode (2 bytes)
|
|
binary.Write(&buf, binary.BigEndian, int16(0))
|
|
|
|
// Name (STRING: 2 bytes length + data)
|
|
binary.Write(&buf, binary.BigEndian, int16(len(topicName)))
|
|
buf.WriteString(topicName)
|
|
|
|
// IsInternal (1 byte) - v1+ addition
|
|
buf.WriteByte(0) // false
|
|
|
|
// Get actual partition count from topic info
|
|
topicInfo, exists := h.seaweedMQHandler.GetTopicInfo(topicName)
|
|
partitionCount := h.GetDefaultPartitions() // Use configurable default
|
|
if exists && topicInfo != nil {
|
|
partitionCount = topicInfo.Partitions
|
|
}
|
|
|
|
// Partitions array (4 bytes length + partitions)
|
|
binary.Write(&buf, binary.BigEndian, partitionCount)
|
|
|
|
// Create partition entries for each partition
|
|
for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
|
|
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // LeaderID
|
|
|
|
// ReplicaNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
|
|
// IsrNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
}
|
|
}
|
|
|
|
response := buf.Bytes()
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// HandleMetadataV3V4 implements Metadata API v3/v4 with ThrottleTimeMs field
|
|
func (h *Handler) HandleMetadataV3V4(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// Metadata v3/v4 adds ThrottleTimeMs field at the beginning
|
|
// v3/v4 response layout: correlation_id(4) + throttle_time_ms(4) + brokers(ARRAY) + cluster_id(NULLABLE_STRING) + controller_id(4) + topics(ARRAY)
|
|
|
|
// Parse requested topics (empty means all)
|
|
requestedTopics := h.parseMetadataTopics(requestBody)
|
|
glog.V(0).Infof("[METADATA v3/v4] Requested topics: %v (empty=all)", requestedTopics)
|
|
|
|
// Determine topics to return using SeaweedMQ handler
|
|
var topicsToReturn []string
|
|
if len(requestedTopics) == 0 {
|
|
topicsToReturn = h.seaweedMQHandler.ListTopics()
|
|
} else {
|
|
for _, name := range requestedTopics {
|
|
if h.seaweedMQHandler.TopicExists(name) {
|
|
topicsToReturn = append(topicsToReturn, name)
|
|
}
|
|
}
|
|
}
|
|
|
|
var buf bytes.Buffer
|
|
|
|
// Correlation ID (4 bytes)
|
|
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID
|
|
// Do NOT include it in the response body
|
|
|
|
// ThrottleTimeMs (4 bytes) - v3+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(0)) // No throttling
|
|
|
|
// Brokers array (4 bytes length + brokers) - 1 broker (this gateway)
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Get advertised address for client connections
|
|
host, port := h.GetAdvertisedAddress(h.GetGatewayAddress())
|
|
|
|
nodeID := int32(1) // Single gateway node
|
|
|
|
// Broker: node_id(4) + host(STRING) + port(4) + rack(STRING) + cluster_id(NULLABLE_STRING)
|
|
binary.Write(&buf, binary.BigEndian, nodeID)
|
|
|
|
// Host (STRING: 2 bytes length + data) - validate length fits in int16
|
|
if len(host) > 32767 {
|
|
return nil, fmt.Errorf("host name too long: %d bytes", len(host))
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int16(len(host)))
|
|
buf.WriteString(host)
|
|
|
|
// Port (4 bytes) - validate port range
|
|
if port < 0 || port > 65535 {
|
|
return nil, fmt.Errorf("invalid port number: %d", port)
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int32(port))
|
|
|
|
// Rack (STRING: 2 bytes length + data) - v1+ addition, non-nullable
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // Empty string
|
|
|
|
// ClusterID (NULLABLE_STRING: 2 bytes length + data) - v2+ addition
|
|
// Schema Registry requires a non-null cluster ID
|
|
clusterID := "seaweedfs-kafka-gateway"
|
|
binary.Write(&buf, binary.BigEndian, int16(len(clusterID)))
|
|
buf.WriteString(clusterID)
|
|
|
|
// ControllerID (4 bytes) - v1+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Topics array (4 bytes length + topics)
|
|
binary.Write(&buf, binary.BigEndian, int32(len(topicsToReturn)))
|
|
|
|
for _, topicName := range topicsToReturn {
|
|
// ErrorCode (2 bytes)
|
|
binary.Write(&buf, binary.BigEndian, int16(0))
|
|
|
|
// Name (STRING: 2 bytes length + data)
|
|
binary.Write(&buf, binary.BigEndian, int16(len(topicName)))
|
|
buf.WriteString(topicName)
|
|
|
|
// IsInternal (1 byte) - v1+ addition
|
|
buf.WriteByte(0) // false
|
|
|
|
// Get actual partition count from topic info
|
|
topicInfo, exists := h.seaweedMQHandler.GetTopicInfo(topicName)
|
|
partitionCount := h.GetDefaultPartitions() // Use configurable default
|
|
if exists && topicInfo != nil {
|
|
partitionCount = topicInfo.Partitions
|
|
}
|
|
|
|
// Partitions array (4 bytes length + partitions)
|
|
binary.Write(&buf, binary.BigEndian, partitionCount)
|
|
|
|
// Create partition entries for each partition
|
|
for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
|
|
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // LeaderID
|
|
|
|
// ReplicaNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
|
|
// IsrNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
}
|
|
}
|
|
|
|
response := buf.Bytes()
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// HandleMetadataV5V6 implements Metadata API v5/v6 with OfflineReplicas field
|
|
func (h *Handler) HandleMetadataV5V6(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
return h.handleMetadataV5ToV8(correlationID, requestBody, 5)
|
|
}
|
|
|
|
// HandleMetadataV7 implements Metadata API v7 with LeaderEpoch field (REGULAR FORMAT, NOT FLEXIBLE)
|
|
func (h *Handler) HandleMetadataV7(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// CRITICAL: Metadata v7 uses REGULAR arrays/strings (like v5/v6), NOT compact format
|
|
// Only v9+ uses compact format (flexible responses)
|
|
return h.handleMetadataV5ToV8(correlationID, requestBody, 7)
|
|
}
|
|
|
|
// handleMetadataV5ToV8 handles Metadata v5-v8 with regular (non-compact) encoding
|
|
// v5/v6: adds OfflineReplicas field to partitions
|
|
// v7: adds LeaderEpoch field to partitions
|
|
// v8: adds ClusterAuthorizedOperations field
|
|
// All use REGULAR arrays/strings (NOT compact) - only v9+ uses compact format
|
|
func (h *Handler) handleMetadataV5ToV8(correlationID uint32, requestBody []byte, apiVersion int) ([]byte, error) {
|
|
// v5-v8 response layout: throttle_time_ms(4) + brokers(ARRAY) + cluster_id(NULLABLE_STRING) + controller_id(4) + topics(ARRAY) [+ cluster_authorized_operations(4) for v8]
|
|
// Each partition includes: error_code(2) + partition_index(4) + leader_id(4) [+ leader_epoch(4) for v7+] + replica_nodes(ARRAY) + isr_nodes(ARRAY) + offline_replicas(ARRAY)
|
|
|
|
// Parse requested topics (empty means all)
|
|
requestedTopics := h.parseMetadataTopics(requestBody)
|
|
glog.V(0).Infof("[METADATA v%d] Requested topics: %v (empty=all)", apiVersion, requestedTopics)
|
|
|
|
// Determine topics to return using SeaweedMQ handler
|
|
var topicsToReturn []string
|
|
if len(requestedTopics) == 0 {
|
|
topicsToReturn = h.seaweedMQHandler.ListTopics()
|
|
} else {
|
|
// FIXED: Proper topic existence checking (removed the hack)
|
|
// Now that CreateTopics v5 works, we use proper Kafka workflow:
|
|
// 1. Check which requested topics actually exist
|
|
// 2. Auto-create system topics if they don't exist
|
|
// 3. Only return existing topics in metadata
|
|
// 4. Client will call CreateTopics for non-existent topics
|
|
// 5. Then request metadata again to see the created topics
|
|
for _, topic := range requestedTopics {
|
|
if isSystemTopic(topic) {
|
|
// Always try to auto-create system topics during metadata requests
|
|
glog.V(0).Infof("[METADATA v%d] Ensuring system topic %s exists during metadata request", apiVersion, topic)
|
|
if !h.seaweedMQHandler.TopicExists(topic) {
|
|
glog.V(0).Infof("[METADATA v%d] Auto-creating system topic %s during metadata request", apiVersion, topic)
|
|
if err := h.createTopicWithSchemaSupport(topic, 1); err != nil {
|
|
glog.V(0).Infof("[METADATA v%d] Failed to auto-create system topic %s: %v", apiVersion, topic, err)
|
|
// Continue without adding to topicsToReturn - client will get UNKNOWN_TOPIC_OR_PARTITION
|
|
} else {
|
|
glog.V(0).Infof("[METADATA v%d] Successfully auto-created system topic %s", apiVersion, topic)
|
|
}
|
|
} else {
|
|
glog.V(0).Infof("[METADATA v%d] System topic %s already exists", apiVersion, topic)
|
|
}
|
|
topicsToReturn = append(topicsToReturn, topic)
|
|
} else if h.seaweedMQHandler.TopicExists(topic) {
|
|
topicsToReturn = append(topicsToReturn, topic)
|
|
}
|
|
}
|
|
glog.V(0).Infof("[METADATA v%d] Returning topics: %v (requested: %v)", apiVersion, topicsToReturn, requestedTopics)
|
|
}
|
|
|
|
var buf bytes.Buffer
|
|
|
|
// Correlation ID (4 bytes)
|
|
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID
|
|
// Do NOT include it in the response body
|
|
|
|
// ThrottleTimeMs (4 bytes) - v3+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(0)) // No throttling
|
|
|
|
// Brokers array (4 bytes length + brokers) - 1 broker (this gateway)
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Get advertised address for client connections
|
|
host, port := h.GetAdvertisedAddress(h.GetGatewayAddress())
|
|
|
|
nodeID := int32(1) // Single gateway node
|
|
|
|
// Broker: node_id(4) + host(STRING) + port(4) + rack(STRING) + cluster_id(NULLABLE_STRING)
|
|
binary.Write(&buf, binary.BigEndian, nodeID)
|
|
|
|
// Host (STRING: 2 bytes length + data) - validate length fits in int16
|
|
if len(host) > 32767 {
|
|
return nil, fmt.Errorf("host name too long: %d bytes", len(host))
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int16(len(host)))
|
|
buf.WriteString(host)
|
|
|
|
// Port (4 bytes) - validate port range
|
|
if port < 0 || port > 65535 {
|
|
return nil, fmt.Errorf("invalid port number: %d", port)
|
|
}
|
|
binary.Write(&buf, binary.BigEndian, int32(port))
|
|
|
|
// Rack (STRING: 2 bytes length + data) - v1+ addition, non-nullable
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // Empty string
|
|
|
|
// ClusterID (NULLABLE_STRING: 2 bytes length + data) - v2+ addition
|
|
// Schema Registry requires a non-null cluster ID
|
|
clusterID := "seaweedfs-kafka-gateway"
|
|
binary.Write(&buf, binary.BigEndian, int16(len(clusterID)))
|
|
buf.WriteString(clusterID)
|
|
|
|
// ControllerID (4 bytes) - v1+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(1))
|
|
|
|
// Topics array (4 bytes length + topics)
|
|
binary.Write(&buf, binary.BigEndian, int32(len(topicsToReturn)))
|
|
|
|
for _, topicName := range topicsToReturn {
|
|
// ErrorCode (2 bytes)
|
|
binary.Write(&buf, binary.BigEndian, int16(0))
|
|
|
|
// Name (STRING: 2 bytes length + data)
|
|
binary.Write(&buf, binary.BigEndian, int16(len(topicName)))
|
|
buf.WriteString(topicName)
|
|
|
|
// IsInternal (1 byte) - v1+ addition
|
|
buf.WriteByte(0) // false
|
|
|
|
// Get actual partition count from topic info
|
|
topicInfo, exists := h.seaweedMQHandler.GetTopicInfo(topicName)
|
|
partitionCount := h.GetDefaultPartitions() // Use configurable default
|
|
if exists && topicInfo != nil {
|
|
partitionCount = topicInfo.Partitions
|
|
}
|
|
|
|
// Partitions array (4 bytes length + partitions)
|
|
binary.Write(&buf, binary.BigEndian, partitionCount)
|
|
|
|
// Create partition entries for each partition
|
|
for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
|
|
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
|
|
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // LeaderID
|
|
|
|
// LeaderEpoch (4 bytes) - v7+ addition
|
|
if apiVersion >= 7 {
|
|
binary.Write(&buf, binary.BigEndian, int32(0)) // Leader epoch 0
|
|
}
|
|
|
|
// ReplicaNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
|
|
// IsrNodes array (4 bytes length + nodes)
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
|
|
binary.Write(&buf, binary.BigEndian, int32(1)) // NodeID 1
|
|
|
|
// OfflineReplicas array (4 bytes length + nodes) - v5+ addition
|
|
binary.Write(&buf, binary.BigEndian, int32(0)) // No offline replicas
|
|
}
|
|
}
|
|
|
|
// ClusterAuthorizedOperations (4 bytes) - v8+ addition
|
|
if apiVersion >= 8 {
|
|
binary.Write(&buf, binary.BigEndian, int32(-2147483648)) // All operations allowed (bit mask)
|
|
}
|
|
|
|
response := buf.Bytes()
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (h *Handler) parseMetadataTopics(requestBody []byte) []string {
|
|
// Support both v0/v1 parsing: v1 payload starts directly with topics array length (int32),
|
|
// while older assumptions may have included a client_id string first.
|
|
if len(requestBody) < 4 {
|
|
return []string{}
|
|
}
|
|
|
|
// Try path A: interpret first 4 bytes as topics_count
|
|
offset := 0
|
|
topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
if topicsCount == 0xFFFFFFFF { // -1 means all topics
|
|
return []string{}
|
|
}
|
|
if topicsCount <= 1000000 { // sane bound
|
|
offset += 4
|
|
topics := make([]string, 0, topicsCount)
|
|
for i := uint32(0); i < topicsCount && offset+2 <= len(requestBody); i++ {
|
|
nameLen := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
if offset+nameLen > len(requestBody) {
|
|
break
|
|
}
|
|
topics = append(topics, string(requestBody[offset:offset+nameLen]))
|
|
offset += nameLen
|
|
}
|
|
return topics
|
|
}
|
|
|
|
// Path B: assume leading client_id string then topics_count
|
|
if len(requestBody) < 6 {
|
|
return []string{}
|
|
}
|
|
clientIDLen := int(binary.BigEndian.Uint16(requestBody[0:2]))
|
|
offset = 2 + clientIDLen
|
|
if len(requestBody) < offset+4 {
|
|
return []string{}
|
|
}
|
|
topicsCount = binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
if topicsCount == 0xFFFFFFFF {
|
|
return []string{}
|
|
}
|
|
topics := make([]string, 0, topicsCount)
|
|
for i := uint32(0); i < topicsCount && offset+2 <= len(requestBody); i++ {
|
|
nameLen := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
if offset+nameLen > len(requestBody) {
|
|
break
|
|
}
|
|
topics = append(topics, string(requestBody[offset:offset+nameLen]))
|
|
offset += nameLen
|
|
}
|
|
return topics
|
|
}
|
|
|
|
func (h *Handler) handleListOffsets(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
|
|
// Parse minimal request to understand what's being asked (header already stripped)
|
|
offset := 0
|
|
|
|
// v1+ has replica_id(4)
|
|
if apiVersion >= 1 {
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("ListOffsets v%d request missing replica_id", apiVersion)
|
|
}
|
|
_ = int32(binary.BigEndian.Uint32(requestBody[offset : offset+4])) // replicaID
|
|
offset += 4
|
|
}
|
|
|
|
// v2+ adds isolation_level(1)
|
|
if apiVersion >= 2 {
|
|
if len(requestBody) < offset+1 {
|
|
return nil, fmt.Errorf("ListOffsets v%d request missing isolation_level", apiVersion)
|
|
}
|
|
_ = requestBody[offset] // isolationLevel
|
|
offset += 1
|
|
}
|
|
|
|
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)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// Throttle time (4 bytes, 0 = no throttling) - v2+ only
|
|
if apiVersion >= 2 {
|
|
response = append(response, 0, 0, 0, 0)
|
|
}
|
|
|
|
// Topics count (will be updated later with actual count)
|
|
topicsCountBytes := make([]byte, 4)
|
|
topicsCountOffset := len(response) // Remember where to update the count
|
|
binary.BigEndian.PutUint32(topicsCountBytes, topicsCount)
|
|
response = append(response, topicsCountBytes...)
|
|
|
|
// Track how many topics we actually process
|
|
actualTopicsCount := uint32(0)
|
|
|
|
// 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)
|
|
|
|
// Use direct SMQ reading - no ledgers needed
|
|
// SMQ handles offset management internally
|
|
var responseTimestamp int64
|
|
var responseOffset int64
|
|
|
|
switch timestamp {
|
|
case -2: // earliest offset
|
|
// Get the actual earliest offset from SMQ
|
|
earliestOffset, err := h.seaweedMQHandler.GetEarliestOffset(string(topicName), int32(partitionID))
|
|
if err != nil {
|
|
responseOffset = 0 // fallback to 0
|
|
} else {
|
|
responseOffset = earliestOffset
|
|
}
|
|
responseTimestamp = 0 // No specific timestamp for earliest
|
|
if strings.HasPrefix(string(topicName), "_schemas") {
|
|
glog.Infof("SCHEMA REGISTRY LISTOFFSETS EARLIEST: topic=%s partition=%d returning offset=%d", string(topicName), partitionID, responseOffset)
|
|
}
|
|
case -1: // latest offset
|
|
// Get the actual latest offset from SMQ
|
|
if h.seaweedMQHandler == nil {
|
|
responseOffset = 0
|
|
} else {
|
|
latestOffset, err := h.seaweedMQHandler.GetLatestOffset(string(topicName), int32(partitionID))
|
|
if err != nil {
|
|
responseOffset = 0 // fallback to 0
|
|
} else {
|
|
responseOffset = latestOffset
|
|
}
|
|
}
|
|
responseTimestamp = 0 // No specific timestamp for latest
|
|
default: // specific timestamp - find offset by timestamp
|
|
// For timestamp-based lookup, we need to implement this properly
|
|
// For now, return 0 as fallback
|
|
responseOffset = 0
|
|
responseTimestamp = timestamp
|
|
}
|
|
|
|
// Ensure we never return a timestamp as offset - this was the bug!
|
|
if responseOffset > 1000000000 { // If offset looks like a timestamp
|
|
responseOffset = 0
|
|
}
|
|
|
|
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...)
|
|
}
|
|
|
|
// Successfully processed this topic
|
|
actualTopicsCount++
|
|
}
|
|
|
|
// CRITICAL FIX: Update the topics count in the response header with the actual count
|
|
// This prevents ErrIncompleteResponse when request parsing fails mid-way
|
|
if actualTopicsCount != topicsCount {
|
|
binary.BigEndian.PutUint32(response[topicsCountOffset:topicsCountOffset+4], actualTopicsCount)
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (h *Handler) handleCreateTopics(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
|
|
if len(requestBody) < 2 {
|
|
return nil, fmt.Errorf("CreateTopics request too short")
|
|
}
|
|
|
|
// Parse based on API version
|
|
switch apiVersion {
|
|
case 0, 1:
|
|
response, err := h.handleCreateTopicsV0V1(correlationID, requestBody)
|
|
return response, err
|
|
case 2, 3, 4:
|
|
// kafka-go sends v2-4 in regular format, not compact
|
|
response, err := h.handleCreateTopicsV2To4(correlationID, requestBody)
|
|
return response, err
|
|
case 5:
|
|
// v5+ uses flexible format with compact arrays
|
|
response, err := h.handleCreateTopicsV2Plus(correlationID, apiVersion, requestBody)
|
|
return response, err
|
|
default:
|
|
return nil, fmt.Errorf("unsupported CreateTopics API version: %d", apiVersion)
|
|
}
|
|
}
|
|
|
|
// handleCreateTopicsV2To4 handles CreateTopics API versions 2-4 (auto-detect regular vs compact format)
|
|
func (h *Handler) handleCreateTopicsV2To4(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// Auto-detect format: kafka-go sends regular format, tests send compact format
|
|
if len(requestBody) < 1 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4 request too short")
|
|
}
|
|
|
|
// Detect format by checking first byte
|
|
// Compact format: first byte is compact array length (usually 0x02 for 1 topic)
|
|
// Regular format: first 4 bytes are regular array count (usually 0x00000001 for 1 topic)
|
|
isCompactFormat := false
|
|
if len(requestBody) >= 4 {
|
|
// Check if this looks like a regular 4-byte array count
|
|
regularCount := binary.BigEndian.Uint32(requestBody[0:4])
|
|
// If the "regular count" is very large (> 1000), it's probably compact format
|
|
// Also check if first byte is small (typical compact array length)
|
|
if regularCount > 1000 || (requestBody[0] <= 10 && requestBody[0] > 0) {
|
|
isCompactFormat = true
|
|
}
|
|
} else if requestBody[0] <= 10 && requestBody[0] > 0 {
|
|
isCompactFormat = true
|
|
}
|
|
|
|
if isCompactFormat {
|
|
// Delegate to the compact format handler
|
|
response, err := h.handleCreateTopicsV2Plus(correlationID, 2, requestBody)
|
|
return response, err
|
|
}
|
|
|
|
// Handle regular format
|
|
offset := 0
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4 request too short for topics array")
|
|
}
|
|
|
|
topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// Parse topics
|
|
topics := make([]struct {
|
|
name string
|
|
partitions uint32
|
|
replication uint16
|
|
}, 0, topicsCount)
|
|
for i := uint32(0); i < topicsCount; i++ {
|
|
if len(requestBody) < offset+2 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated topic name length")
|
|
}
|
|
nameLen := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2
|
|
if len(requestBody) < offset+int(nameLen) {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated topic name")
|
|
}
|
|
topicName := string(requestBody[offset : offset+int(nameLen)])
|
|
offset += int(nameLen)
|
|
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated num_partitions")
|
|
}
|
|
numPartitions := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
if len(requestBody) < offset+2 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated replication_factor")
|
|
}
|
|
replication := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2
|
|
|
|
// Assignments array (array of partition assignments) - skip contents
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated assignments count")
|
|
}
|
|
assignments := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
for j := uint32(0); j < assignments; j++ {
|
|
// partition_id (int32) + replicas (array int32)
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated assignment partition id")
|
|
}
|
|
offset += 4
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated replicas count")
|
|
}
|
|
replicasCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
// skip replica ids
|
|
offset += int(replicasCount) * 4
|
|
}
|
|
|
|
// Configs array (array of (name,value) strings) - skip contents
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated configs count")
|
|
}
|
|
configs := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
for j := uint32(0); j < configs; j++ {
|
|
// name (string)
|
|
if len(requestBody) < offset+2 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated config name length")
|
|
}
|
|
nameLen := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2 + int(nameLen)
|
|
// value (nullable string)
|
|
if len(requestBody) < offset+2 {
|
|
return nil, fmt.Errorf("CreateTopics v2-4: truncated config value length")
|
|
}
|
|
valueLen := int16(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
if valueLen >= 0 {
|
|
offset += int(valueLen)
|
|
}
|
|
}
|
|
|
|
topics = append(topics, struct {
|
|
name string
|
|
partitions uint32
|
|
replication uint16
|
|
}{topicName, numPartitions, replication})
|
|
}
|
|
|
|
// timeout_ms
|
|
if len(requestBody) >= offset+4 {
|
|
_ = binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
}
|
|
// validate_only (boolean)
|
|
if len(requestBody) >= offset+1 {
|
|
_ = requestBody[offset]
|
|
offset += 1
|
|
}
|
|
|
|
// Build response
|
|
response := make([]byte, 0, 128)
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
// throttle_time_ms (4 bytes)
|
|
response = append(response, 0, 0, 0, 0)
|
|
// topics array count (int32)
|
|
countBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(countBytes, uint32(len(topics)))
|
|
response = append(response, countBytes...)
|
|
// per-topic responses
|
|
for _, t := range topics {
|
|
// topic name (string)
|
|
nameLen := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(nameLen, uint16(len(t.name)))
|
|
response = append(response, nameLen...)
|
|
response = append(response, []byte(t.name)...)
|
|
// error_code (int16)
|
|
var errCode uint16 = 0
|
|
if h.seaweedMQHandler.TopicExists(t.name) {
|
|
errCode = 36 // TOPIC_ALREADY_EXISTS
|
|
} else if t.partitions == 0 {
|
|
errCode = 37 // INVALID_PARTITIONS
|
|
} else if t.replication == 0 {
|
|
errCode = 38 // INVALID_REPLICATION_FACTOR
|
|
} else {
|
|
// Use schema-aware topic creation
|
|
if err := h.createTopicWithSchemaSupport(t.name, int32(t.partitions)); err != nil {
|
|
errCode = 1 // UNKNOWN_SERVER_ERROR
|
|
}
|
|
}
|
|
eb := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(eb, errCode)
|
|
response = append(response, eb...)
|
|
// error_message (nullable string) -> null
|
|
response = append(response, 0xFF, 0xFF)
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (h *Handler) handleCreateTopicsV0V1(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
|
|
if len(requestBody) < 4 {
|
|
return nil, fmt.Errorf("CreateTopics v0/v1 request too short")
|
|
}
|
|
|
|
offset := 0
|
|
|
|
// Parse topics array (regular array format: count + topics)
|
|
topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// Build response
|
|
response := make([]byte, 0, 256)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// Topics array count (4 bytes in v0/v1)
|
|
topicsCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(topicsCountBytes, topicsCount)
|
|
response = append(response, topicsCountBytes...)
|
|
|
|
// Process each topic
|
|
for i := uint32(0); i < topicsCount && offset < len(requestBody); i++ {
|
|
// Parse topic name (regular string: length + bytes)
|
|
if len(requestBody) < offset+2 {
|
|
break
|
|
}
|
|
topicNameLength := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2
|
|
|
|
if len(requestBody) < offset+int(topicNameLength) {
|
|
break
|
|
}
|
|
topicName := string(requestBody[offset : offset+int(topicNameLength)])
|
|
offset += int(topicNameLength)
|
|
|
|
// Parse num_partitions (4 bytes)
|
|
if len(requestBody) < offset+4 {
|
|
break
|
|
}
|
|
numPartitions := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// Parse replication_factor (2 bytes)
|
|
if len(requestBody) < offset+2 {
|
|
break
|
|
}
|
|
replicationFactor := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2
|
|
|
|
// Parse assignments array (4 bytes count, then assignments)
|
|
if len(requestBody) < offset+4 {
|
|
break
|
|
}
|
|
assignmentsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// Skip assignments for now (simplified)
|
|
for j := uint32(0); j < assignmentsCount && offset < len(requestBody); j++ {
|
|
// Skip partition_id (4 bytes)
|
|
if len(requestBody) >= offset+4 {
|
|
offset += 4
|
|
}
|
|
// Skip replicas array (4 bytes count + replica_ids)
|
|
if len(requestBody) >= offset+4 {
|
|
replicasCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
offset += int(replicasCount) * 4 // Skip replica IDs
|
|
}
|
|
}
|
|
|
|
// Parse configs array (4 bytes count, then configs)
|
|
if len(requestBody) >= offset+4 {
|
|
configsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// Skip configs (simplified)
|
|
for j := uint32(0); j < configsCount && offset < len(requestBody); j++ {
|
|
// Skip config name (string: 2 bytes length + bytes)
|
|
if len(requestBody) >= offset+2 {
|
|
configNameLength := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2 + int(configNameLength)
|
|
}
|
|
// Skip config value (string: 2 bytes length + bytes)
|
|
if len(requestBody) >= offset+2 {
|
|
configValueLength := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2 + int(configValueLength)
|
|
}
|
|
}
|
|
}
|
|
|
|
// Build response for this topic
|
|
// Topic name (string: length + bytes)
|
|
topicNameLengthBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(topicNameLengthBytes, uint16(len(topicName)))
|
|
response = append(response, topicNameLengthBytes...)
|
|
response = append(response, []byte(topicName)...)
|
|
|
|
// Determine error code and message
|
|
var errorCode uint16 = 0
|
|
|
|
// Apply defaults for invalid values
|
|
if numPartitions <= 0 {
|
|
numPartitions = uint32(h.GetDefaultPartitions()) // Use configurable default
|
|
}
|
|
if replicationFactor <= 0 {
|
|
replicationFactor = 1 // Default to 1 replica
|
|
}
|
|
|
|
// Use SeaweedMQ integration
|
|
if h.seaweedMQHandler.TopicExists(topicName) {
|
|
errorCode = 36 // TOPIC_ALREADY_EXISTS
|
|
} else {
|
|
// Create the topic in SeaweedMQ with schema support
|
|
if err := h.createTopicWithSchemaSupport(topicName, int32(numPartitions)); err != nil {
|
|
errorCode = 1 // UNKNOWN_SERVER_ERROR
|
|
}
|
|
}
|
|
|
|
// Error code (2 bytes)
|
|
errorCodeBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(errorCodeBytes, errorCode)
|
|
response = append(response, errorCodeBytes...)
|
|
}
|
|
|
|
// Parse timeout_ms (4 bytes) - at the end of request
|
|
if len(requestBody) >= offset+4 {
|
|
_ = binary.BigEndian.Uint32(requestBody[offset : offset+4]) // timeoutMs
|
|
offset += 4
|
|
}
|
|
|
|
// Parse validate_only (1 byte) - only in v1
|
|
if len(requestBody) >= offset+1 {
|
|
_ = requestBody[offset] != 0 // validateOnly
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// handleCreateTopicsV2Plus handles CreateTopics API versions 2+ (flexible versions with compact arrays/strings)
|
|
// For simplicity and consistency with existing response builder, this parses the flexible request,
|
|
// converts it into the non-flexible v2-v4 body format, and reuses handleCreateTopicsV2To4 to build the response.
|
|
func (h *Handler) handleCreateTopicsV2Plus(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
offset := 0
|
|
|
|
// ADMIN CLIENT COMPATIBILITY FIX:
|
|
// AdminClient's CreateTopics v5 request DOES start with top-level tagged fields (usually empty)
|
|
// Parse them first, then the topics compact array
|
|
|
|
// Parse top-level tagged fields first (usually 0x00 for empty)
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
// Don't fail - AdminClient might not always include tagged fields properly
|
|
// Just log and continue with topics parsing
|
|
} else {
|
|
offset += consumed
|
|
}
|
|
|
|
// Topics (compact array) - Now correctly positioned after tagged fields
|
|
topicsCount, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topics compact array: %w", apiVersion, err)
|
|
}
|
|
offset += consumed
|
|
|
|
type topicSpec struct {
|
|
name string
|
|
partitions uint32
|
|
replication uint16
|
|
}
|
|
topics := make([]topicSpec, 0, topicsCount)
|
|
|
|
for i := uint32(0); i < topicsCount; i++ {
|
|
// Topic name (compact string)
|
|
name, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] name: %w", apiVersion, i, err)
|
|
}
|
|
offset += consumed
|
|
|
|
if len(requestBody) < offset+6 {
|
|
return nil, fmt.Errorf("CreateTopics v%d: truncated partitions/replication for topic[%d]", apiVersion, i)
|
|
}
|
|
|
|
partitions := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
replication := binary.BigEndian.Uint16(requestBody[offset : offset+2])
|
|
offset += 2
|
|
|
|
// ADMIN CLIENT COMPATIBILITY: AdminClient uses little-endian for replication factor
|
|
// This violates Kafka protocol spec but we need to handle it for compatibility
|
|
if replication == 256 {
|
|
replication = 1 // AdminClient sent 0x01 0x00, intended as little-endian 1
|
|
}
|
|
|
|
// Apply defaults for invalid values
|
|
if partitions <= 0 {
|
|
partitions = uint32(h.GetDefaultPartitions()) // Use configurable default
|
|
}
|
|
if replication <= 0 {
|
|
replication = 1 // Default to 1 replica
|
|
}
|
|
|
|
// FIX 2: Assignments (compact array) - this was missing!
|
|
assignCount, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] assignments array: %w", apiVersion, i, err)
|
|
}
|
|
offset += consumed
|
|
|
|
// Skip assignment entries (partition_id + replicas array)
|
|
for j := uint32(0); j < assignCount; j++ {
|
|
// partition_id (int32)
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v%d: truncated assignment[%d] partition_id", apiVersion, j)
|
|
}
|
|
offset += 4
|
|
|
|
// replicas (compact array of int32)
|
|
replicasCount, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode assignment[%d] replicas: %w", apiVersion, j, err)
|
|
}
|
|
offset += consumed
|
|
|
|
// Skip replica broker IDs (int32 each)
|
|
if len(requestBody) < offset+int(replicasCount)*4 {
|
|
return nil, fmt.Errorf("CreateTopics v%d: truncated assignment[%d] replicas", apiVersion, j)
|
|
}
|
|
offset += int(replicasCount) * 4
|
|
|
|
// Assignment tagged fields
|
|
_, consumed, err = DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode assignment[%d] tagged fields: %w", apiVersion, j, err)
|
|
}
|
|
offset += consumed
|
|
}
|
|
|
|
// Configs (compact array) - skip entries
|
|
cfgCount, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] configs array: %w", apiVersion, i, err)
|
|
}
|
|
offset += consumed
|
|
|
|
for j := uint32(0); j < cfgCount; j++ {
|
|
// name (compact string)
|
|
_, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] config[%d] name: %w", apiVersion, i, j, err)
|
|
}
|
|
offset += consumed
|
|
|
|
// value (nullable compact string)
|
|
_, consumed, err = DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] config[%d] value: %w", apiVersion, i, j, err)
|
|
}
|
|
offset += consumed
|
|
|
|
// tagged fields for each config
|
|
_, consumed, err = DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] config[%d] tagged fields: %w", apiVersion, i, j, err)
|
|
}
|
|
offset += consumed
|
|
}
|
|
|
|
// Tagged fields for topic
|
|
_, consumed, err = DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("CreateTopics v%d: decode topic[%d] tagged fields: %w", apiVersion, i, err)
|
|
}
|
|
offset += consumed
|
|
|
|
topics = append(topics, topicSpec{name: name, partitions: partitions, replication: replication})
|
|
}
|
|
|
|
for range topics {
|
|
}
|
|
|
|
// timeout_ms (int32)
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("CreateTopics v%d: missing timeout_ms", apiVersion)
|
|
}
|
|
timeoutMs := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
// validate_only (boolean)
|
|
if len(requestBody) < offset+1 {
|
|
return nil, fmt.Errorf("CreateTopics v%d: missing validate_only flag", apiVersion)
|
|
}
|
|
validateOnly := requestBody[offset] != 0
|
|
offset += 1
|
|
|
|
// Remaining bytes after parsing - could be additional fields
|
|
if offset < len(requestBody) {
|
|
}
|
|
|
|
// Reconstruct a non-flexible v2-like request body and reuse existing handler
|
|
// Format: topics(ARRAY) + timeout_ms(INT32) + validate_only(BOOLEAN)
|
|
var legacyBody []byte
|
|
|
|
// topics count (int32)
|
|
legacyBody = append(legacyBody, 0, 0, 0, byte(len(topics)))
|
|
if len(topics) > 0 {
|
|
legacyBody[len(legacyBody)-1] = byte(len(topics))
|
|
}
|
|
|
|
for _, t := range topics {
|
|
// topic name (STRING)
|
|
nameLen := uint16(len(t.name))
|
|
legacyBody = append(legacyBody, byte(nameLen>>8), byte(nameLen))
|
|
legacyBody = append(legacyBody, []byte(t.name)...)
|
|
|
|
// num_partitions (INT32)
|
|
legacyBody = append(legacyBody, byte(t.partitions>>24), byte(t.partitions>>16), byte(t.partitions>>8), byte(t.partitions))
|
|
|
|
// replication_factor (INT16)
|
|
legacyBody = append(legacyBody, byte(t.replication>>8), byte(t.replication))
|
|
|
|
// assignments array (INT32 count = 0)
|
|
legacyBody = append(legacyBody, 0, 0, 0, 0)
|
|
|
|
// configs array (INT32 count = 0)
|
|
legacyBody = append(legacyBody, 0, 0, 0, 0)
|
|
}
|
|
|
|
// timeout_ms
|
|
legacyBody = append(legacyBody, byte(timeoutMs>>24), byte(timeoutMs>>16), byte(timeoutMs>>8), byte(timeoutMs))
|
|
|
|
// validate_only
|
|
if validateOnly {
|
|
legacyBody = append(legacyBody, 1)
|
|
} else {
|
|
legacyBody = append(legacyBody, 0)
|
|
}
|
|
|
|
// Build response directly instead of delegating to avoid circular dependency
|
|
response := make([]byte, 0, 128)
|
|
|
|
// NOTE: Correlation ID and header tagged fields are handled by writeResponseWithHeader
|
|
// Do NOT include them in the response body
|
|
|
|
// throttle_time_ms (4 bytes) - first field in CreateTopics response body
|
|
response = append(response, 0, 0, 0, 0)
|
|
|
|
// topics (compact array) - V5 FLEXIBLE FORMAT
|
|
topicCount := len(topics)
|
|
|
|
// Debug: log response size at each step
|
|
debugResponseSize := func(step string) {
|
|
}
|
|
debugResponseSize("After correlation ID and throttle_time_ms")
|
|
|
|
// Compact array: length is encoded as UNSIGNED_VARINT(actualLength + 1)
|
|
response = append(response, EncodeUvarint(uint32(topicCount+1))...)
|
|
debugResponseSize("After topics array length")
|
|
|
|
// For each topic
|
|
for _, t := range topics {
|
|
// name (compact string): length is encoded as UNSIGNED_VARINT(actualLength + 1)
|
|
nameBytes := []byte(t.name)
|
|
response = append(response, EncodeUvarint(uint32(len(nameBytes)+1))...)
|
|
response = append(response, nameBytes...)
|
|
|
|
// TopicId - Not present in v5, only added in v7+
|
|
// v5 CreateTopics response does not include TopicId field
|
|
|
|
// error_code (int16)
|
|
var errCode uint16 = 0
|
|
|
|
// ADMIN CLIENT COMPATIBILITY: Apply defaults before error checking
|
|
actualPartitions := t.partitions
|
|
if actualPartitions == 0 {
|
|
actualPartitions = 1 // Default to 1 partition if 0 requested
|
|
}
|
|
actualReplication := t.replication
|
|
if actualReplication == 0 {
|
|
actualReplication = 1 // Default to 1 replication if 0 requested
|
|
}
|
|
|
|
// ADMIN CLIENT COMPATIBILITY: Always return success for existing topics
|
|
// AdminClient expects topic creation to succeed, even if topic already exists
|
|
if h.seaweedMQHandler.TopicExists(t.name) {
|
|
errCode = 0 // SUCCESS - AdminClient can handle this gracefully
|
|
} else {
|
|
// Use corrected values for error checking and topic creation with schema support
|
|
if err := h.createTopicWithSchemaSupport(t.name, int32(actualPartitions)); err != nil {
|
|
errCode = 1 // UNKNOWN_SERVER_ERROR
|
|
}
|
|
}
|
|
eb := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(eb, errCode)
|
|
response = append(response, eb...)
|
|
|
|
// error_message (compact nullable string) - ADMINCLIENT 7.4.0-CE COMPATIBILITY FIX
|
|
// For "_schemas" topic, send null for byte-level compatibility with Java reference
|
|
// For other topics, send empty string to avoid NPE in AdminClient response handling
|
|
if t.name == "_schemas" {
|
|
response = append(response, 0) // Null = 0
|
|
} else {
|
|
response = append(response, 1) // Empty string = 1 (0 chars + 1)
|
|
}
|
|
|
|
// ADDED FOR V5: num_partitions (int32)
|
|
// ADMIN CLIENT COMPATIBILITY: Use corrected values from error checking logic
|
|
partBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(partBytes, actualPartitions)
|
|
response = append(response, partBytes...)
|
|
|
|
// ADDED FOR V5: replication_factor (int16)
|
|
replBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(replBytes, actualReplication)
|
|
response = append(response, replBytes...)
|
|
|
|
// configs (compact nullable array) - ADDED FOR V5
|
|
// ADMINCLIENT 7.4.0-CE NPE FIX: Send empty configs array instead of null
|
|
// AdminClient 7.4.0-ce has NPE when configs=null but were requested
|
|
// Empty array = 1 (0 configs + 1), still achieves ~30-byte response
|
|
response = append(response, 1) // Empty configs array = 1 (0 configs + 1)
|
|
|
|
// Tagged fields for each topic - V5 format per Kafka source
|
|
// Count tagged fields (topicConfigErrorCode only if != 0)
|
|
topicConfigErrorCode := uint16(0) // No error
|
|
numTaggedFields := 0
|
|
if topicConfigErrorCode != 0 {
|
|
numTaggedFields = 1
|
|
}
|
|
|
|
// Write tagged fields count
|
|
response = append(response, EncodeUvarint(uint32(numTaggedFields))...)
|
|
|
|
// Write tagged fields (only if topicConfigErrorCode != 0)
|
|
if topicConfigErrorCode != 0 {
|
|
// Tag 0: TopicConfigErrorCode
|
|
response = append(response, EncodeUvarint(0)...) // Tag number 0
|
|
response = append(response, EncodeUvarint(2)...) // Length (int16 = 2 bytes)
|
|
topicConfigErrBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(topicConfigErrBytes, topicConfigErrorCode)
|
|
response = append(response, topicConfigErrBytes...)
|
|
}
|
|
|
|
debugResponseSize(fmt.Sprintf("After topic '%s'", t.name))
|
|
}
|
|
|
|
// Top-level tagged fields for v5 flexible response (empty)
|
|
response = append(response, 0) // Empty tagged fields = 0
|
|
debugResponseSize("Final response")
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (h *Handler) handleDeleteTopics(correlationID uint32, requestBody []byte) ([]byte, error) {
|
|
// Parse minimal DeleteTopics request
|
|
// Request format: client_id + timeout(4) + topics_array
|
|
|
|
if len(requestBody) < 6 { // client_id_size(2) + timeout(4)
|
|
return nil, fmt.Errorf("DeleteTopics request too short")
|
|
}
|
|
|
|
// Skip client_id
|
|
clientIDSize := binary.BigEndian.Uint16(requestBody[0:2])
|
|
offset := 2 + int(clientIDSize)
|
|
|
|
if len(requestBody) < offset+8 { // timeout(4) + topics_count(4)
|
|
return nil, fmt.Errorf("DeleteTopics request missing data")
|
|
}
|
|
|
|
// Skip timeout
|
|
offset += 4
|
|
|
|
topicsCount := binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
|
|
response := make([]byte, 0, 256)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// 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 topic (using SeaweedMQ handler)
|
|
|
|
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) {
|
|
break
|
|
}
|
|
|
|
topicName := string(requestBody[offset : offset+int(topicNameSize)])
|
|
offset += int(topicNameSize)
|
|
|
|
// Response: topic_name + error_code(2) + error_message
|
|
response = append(response, byte(topicNameSize>>8), byte(topicNameSize))
|
|
response = append(response, []byte(topicName)...)
|
|
|
|
// Check if topic exists and delete it
|
|
var errorCode uint16 = 0
|
|
var errorMessage string = ""
|
|
|
|
// Use SeaweedMQ integration
|
|
if !h.seaweedMQHandler.TopicExists(topicName) {
|
|
errorCode = 3 // UNKNOWN_TOPIC_OR_PARTITION
|
|
errorMessage = "Unknown topic"
|
|
} else {
|
|
// Delete the topic from SeaweedMQ
|
|
if err := h.seaweedMQHandler.DeleteTopic(topicName); err != nil {
|
|
errorCode = 1 // UNKNOWN_SERVER_ERROR
|
|
errorMessage = err.Error()
|
|
}
|
|
}
|
|
|
|
// Error code
|
|
response = append(response, byte(errorCode>>8), byte(errorCode))
|
|
|
|
// Error message (nullable string)
|
|
if errorMessage == "" {
|
|
response = append(response, 0xFF, 0xFF) // null string
|
|
} else {
|
|
errorMsgLen := uint16(len(errorMessage))
|
|
response = append(response, byte(errorMsgLen>>8), byte(errorMsgLen))
|
|
response = append(response, []byte(errorMessage)...)
|
|
}
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// validateAPIVersion checks if we support the requested API version
|
|
func (h *Handler) validateAPIVersion(apiKey, apiVersion uint16) error {
|
|
supportedVersions := map[APIKey][2]uint16{
|
|
APIKeyApiVersions: {0, 4}, // ApiVersions: v0-v4 (Kafka 8.0.0 compatibility)
|
|
APIKeyMetadata: {0, 7}, // Metadata: v0-v7
|
|
APIKeyProduce: {0, 7}, // Produce: v0-v7
|
|
APIKeyFetch: {0, 7}, // Fetch: v0-v7
|
|
APIKeyListOffsets: {0, 2}, // ListOffsets: v0-v2
|
|
APIKeyCreateTopics: {0, 5}, // CreateTopics: v0-v5 (updated to match implementation)
|
|
APIKeyDeleteTopics: {0, 4}, // DeleteTopics: v0-v4
|
|
APIKeyFindCoordinator: {0, 3}, // FindCoordinator: v0-v3 (v3+ uses flexible format)
|
|
APIKeyJoinGroup: {0, 6}, // JoinGroup: cap to v6 (first flexible version)
|
|
APIKeySyncGroup: {0, 5}, // SyncGroup: v0-v5
|
|
APIKeyOffsetCommit: {0, 2}, // OffsetCommit: v0-v2
|
|
APIKeyOffsetFetch: {0, 5}, // OffsetFetch: v0-v5 (updated to match implementation)
|
|
APIKeyHeartbeat: {0, 4}, // Heartbeat: v0-v4
|
|
APIKeyLeaveGroup: {0, 4}, // LeaveGroup: v0-v4
|
|
APIKeyDescribeGroups: {0, 5}, // DescribeGroups: v0-v5
|
|
APIKeyListGroups: {0, 4}, // ListGroups: v0-v4
|
|
APIKeyDescribeConfigs: {0, 4}, // DescribeConfigs: v0-v4
|
|
APIKeyInitProducerId: {0, 4}, // InitProducerId: v0-v4
|
|
APIKeyDescribeCluster: {0, 1}, // DescribeCluster: v0-v1 (KIP-919, AdminClient compatibility)
|
|
}
|
|
|
|
if versionRange, exists := supportedVersions[APIKey(apiKey)]; exists {
|
|
minVer, maxVer := versionRange[0], versionRange[1]
|
|
if apiVersion < minVer || apiVersion > maxVer {
|
|
return fmt.Errorf("unsupported API version %d for API key %d (supported: %d-%d)",
|
|
apiVersion, apiKey, minVer, maxVer)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
return fmt.Errorf("unsupported API key: %d", apiKey)
|
|
}
|
|
|
|
// buildUnsupportedVersionResponse creates a proper Kafka error response
|
|
func (h *Handler) buildUnsupportedVersionResponse(correlationID uint32, apiKey, apiVersion uint16) ([]byte, error) {
|
|
errorMsg := fmt.Sprintf("Unsupported version %d for API key", apiVersion)
|
|
return BuildErrorResponseWithMessage(correlationID, ErrorCodeUnsupportedVersion, errorMsg), nil
|
|
}
|
|
|
|
// handleMetadata routes to the appropriate version-specific handler
|
|
func (h *Handler) handleMetadata(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
switch apiVersion {
|
|
case 0:
|
|
return h.HandleMetadataV0(correlationID, requestBody)
|
|
case 1:
|
|
return h.HandleMetadataV1(correlationID, requestBody)
|
|
case 2:
|
|
return h.HandleMetadataV2(correlationID, requestBody)
|
|
case 3, 4:
|
|
return h.HandleMetadataV3V4(correlationID, requestBody)
|
|
case 5, 6:
|
|
return h.HandleMetadataV5V6(correlationID, requestBody)
|
|
case 7:
|
|
return h.HandleMetadataV7(correlationID, requestBody)
|
|
default:
|
|
// For versions > 7, use the V7 handler (flexible format)
|
|
if apiVersion > 7 {
|
|
return h.HandleMetadataV7(correlationID, requestBody)
|
|
}
|
|
return nil, fmt.Errorf("metadata version %d not implemented yet", apiVersion)
|
|
}
|
|
}
|
|
|
|
// getAPIName returns a human-readable name for Kafka API keys (for debugging)
|
|
func getAPIName(apiKey APIKey) string {
|
|
switch apiKey {
|
|
case APIKeyProduce:
|
|
return "Produce"
|
|
case APIKeyFetch:
|
|
return "Fetch"
|
|
case APIKeyListOffsets:
|
|
return "ListOffsets"
|
|
case APIKeyMetadata:
|
|
return "Metadata"
|
|
case APIKeyOffsetCommit:
|
|
return "OffsetCommit"
|
|
case APIKeyOffsetFetch:
|
|
return "OffsetFetch"
|
|
case APIKeyFindCoordinator:
|
|
return "FindCoordinator"
|
|
case APIKeyJoinGroup:
|
|
return "JoinGroup"
|
|
case APIKeyHeartbeat:
|
|
return "Heartbeat"
|
|
case APIKeyLeaveGroup:
|
|
return "LeaveGroup"
|
|
case APIKeySyncGroup:
|
|
return "SyncGroup"
|
|
case APIKeyDescribeGroups:
|
|
return "DescribeGroups"
|
|
case APIKeyListGroups:
|
|
return "ListGroups"
|
|
case APIKeyApiVersions:
|
|
return "ApiVersions"
|
|
case APIKeyCreateTopics:
|
|
return "CreateTopics"
|
|
case APIKeyDeleteTopics:
|
|
return "DeleteTopics"
|
|
case APIKeyDescribeConfigs:
|
|
return "DescribeConfigs"
|
|
case APIKeyInitProducerId:
|
|
return "InitProducerId"
|
|
case APIKeyDescribeCluster:
|
|
return "DescribeCluster"
|
|
default:
|
|
return "Unknown"
|
|
}
|
|
}
|
|
|
|
// handleDescribeConfigs handles DescribeConfigs API requests (API key 32)
|
|
func (h *Handler) handleDescribeConfigs(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
|
|
// Parse request to extract resources
|
|
resources, err := h.parseDescribeConfigsRequest(requestBody, apiVersion)
|
|
if err != nil {
|
|
Error("DescribeConfigs parsing error: %v", err)
|
|
return nil, fmt.Errorf("failed to parse DescribeConfigs request: %w", err)
|
|
}
|
|
|
|
isFlexible := apiVersion >= 4
|
|
if !isFlexible {
|
|
// Legacy (non-flexible) response for v0-3
|
|
response := make([]byte, 0, 2048)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// Throttle time (0ms)
|
|
throttleBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(throttleBytes, 0)
|
|
response = append(response, throttleBytes...)
|
|
|
|
// Resources array length
|
|
resourcesBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(resourcesBytes, uint32(len(resources)))
|
|
response = append(response, resourcesBytes...)
|
|
|
|
// For each resource, return appropriate configs
|
|
for _, resource := range resources {
|
|
resourceResponse := h.buildDescribeConfigsResourceResponse(resource, apiVersion)
|
|
response = append(response, resourceResponse...)
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// Flexible response for v4+
|
|
response := make([]byte, 0, 2048)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
|
|
// throttle_time_ms (4 bytes)
|
|
response = append(response, 0, 0, 0, 0)
|
|
|
|
// Results (compact array)
|
|
response = append(response, EncodeUvarint(uint32(len(resources)+1))...)
|
|
|
|
for _, res := range resources {
|
|
// ErrorCode (int16) = 0
|
|
response = append(response, 0, 0)
|
|
// ErrorMessage (compact nullable string) = null (0)
|
|
response = append(response, 0)
|
|
// ResourceType (int8)
|
|
response = append(response, byte(res.ResourceType))
|
|
// ResourceName (compact string)
|
|
nameBytes := []byte(res.ResourceName)
|
|
response = append(response, EncodeUvarint(uint32(len(nameBytes)+1))...)
|
|
response = append(response, nameBytes...)
|
|
|
|
// Build configs for this resource
|
|
var cfgs []ConfigEntry
|
|
if res.ResourceType == 2 { // Topic
|
|
cfgs = h.getTopicConfigs(res.ResourceName, res.ConfigNames)
|
|
// Ensure cleanup.policy is compact for _schemas
|
|
if res.ResourceName == "_schemas" {
|
|
replaced := false
|
|
for i := range cfgs {
|
|
if cfgs[i].Name == "cleanup.policy" {
|
|
cfgs[i].Value = "compact"
|
|
replaced = true
|
|
break
|
|
}
|
|
}
|
|
if !replaced {
|
|
cfgs = append(cfgs, ConfigEntry{Name: "cleanup.policy", Value: "compact"})
|
|
}
|
|
}
|
|
} else if res.ResourceType == 4 { // Broker
|
|
cfgs = h.getBrokerConfigs(res.ConfigNames)
|
|
} else {
|
|
cfgs = []ConfigEntry{}
|
|
}
|
|
|
|
// Configs (compact array)
|
|
response = append(response, EncodeUvarint(uint32(len(cfgs)+1))...)
|
|
|
|
for _, cfg := range cfgs {
|
|
// name (compact string)
|
|
cb := []byte(cfg.Name)
|
|
response = append(response, EncodeUvarint(uint32(len(cb)+1))...)
|
|
response = append(response, cb...)
|
|
|
|
// value (compact nullable string)
|
|
vb := []byte(cfg.Value)
|
|
if len(vb) == 0 {
|
|
response = append(response, 0) // null
|
|
} else {
|
|
response = append(response, EncodeUvarint(uint32(len(vb)+1))...)
|
|
response = append(response, vb...)
|
|
}
|
|
|
|
// readOnly (bool)
|
|
if cfg.ReadOnly {
|
|
response = append(response, 1)
|
|
} else {
|
|
response = append(response, 0)
|
|
}
|
|
|
|
// configSource (int8): DEFAULT_CONFIG = 5
|
|
response = append(response, byte(5))
|
|
|
|
// isSensitive (bool)
|
|
if cfg.Sensitive {
|
|
response = append(response, 1)
|
|
} else {
|
|
response = append(response, 0)
|
|
}
|
|
|
|
// synonyms (compact array) - empty
|
|
response = append(response, 1)
|
|
|
|
// config_type (int8) - STRING = 1
|
|
response = append(response, byte(1))
|
|
|
|
// documentation (compact nullable string) - null
|
|
response = append(response, 0)
|
|
|
|
// per-config tagged fields (empty)
|
|
response = append(response, 0)
|
|
}
|
|
|
|
// Per-result tagged fields (empty)
|
|
response = append(response, 0)
|
|
}
|
|
|
|
// Top-level tagged fields (empty)
|
|
response = append(response, 0)
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// isFlexibleResponse determines if an API response should use flexible format (with header tagged fields)
|
|
// Based on Kafka protocol specifications: most APIs become flexible at v3+, but some differ
|
|
func isFlexibleResponse(apiKey uint16, apiVersion uint16) bool {
|
|
// Reference: kafka-go/protocol/response.go:119 and sarama/response_header.go:21
|
|
// Flexible responses have headerVersion >= 1, which adds tagged fields after correlation ID
|
|
|
|
switch APIKey(apiKey) {
|
|
case APIKeyProduce:
|
|
return apiVersion >= 9
|
|
case APIKeyFetch:
|
|
return apiVersion >= 12
|
|
case APIKeyMetadata:
|
|
// Metadata v9+ uses flexible responses (v7-8 use compact arrays/strings but NOT flexible headers)
|
|
return apiVersion >= 9
|
|
case APIKeyOffsetCommit:
|
|
return apiVersion >= 8
|
|
case APIKeyOffsetFetch:
|
|
return apiVersion >= 6
|
|
case APIKeyFindCoordinator:
|
|
return apiVersion >= 3
|
|
case APIKeyJoinGroup:
|
|
return apiVersion >= 6
|
|
case APIKeyHeartbeat:
|
|
return apiVersion >= 4
|
|
case APIKeyLeaveGroup:
|
|
return apiVersion >= 4
|
|
case APIKeySyncGroup:
|
|
return apiVersion >= 4
|
|
case APIKeyApiVersions:
|
|
// CRITICAL: AdminClient compatibility requires header version 0 (no tagged fields)
|
|
// Even though ApiVersions v3+ technically supports flexible responses, AdminClient
|
|
// expects the header to NOT include tagged fields. This is a known quirk.
|
|
return false // Always use non-flexible header for ApiVersions
|
|
case APIKeyCreateTopics:
|
|
return apiVersion >= 5
|
|
case APIKeyDeleteTopics:
|
|
return apiVersion >= 4
|
|
case APIKeyInitProducerId:
|
|
return apiVersion >= 2 // Flexible from v2+ (KIP-360)
|
|
case APIKeyDescribeConfigs:
|
|
return apiVersion >= 4
|
|
case APIKeyDescribeCluster:
|
|
return true // All versions (0+) are flexible
|
|
default:
|
|
// For unknown APIs, assume non-flexible (safer default)
|
|
return false
|
|
}
|
|
}
|
|
|
|
// writeResponseWithHeader writes a Kafka response following the wire protocol:
|
|
// [Size: 4 bytes][Correlation ID: 4 bytes][Tagged Fields (if flexible)][Body]
|
|
func (h *Handler) writeResponseWithHeader(w *bufio.Writer, correlationID uint32, apiKey uint16, apiVersion uint16, responseBody []byte, timeout time.Duration) error {
|
|
// Kafka wire protocol format (from kafka-go/protocol/response.go:116-138 and sarama/response_header.go:10-27):
|
|
// [4 bytes: size = len(everything after this)]
|
|
// [4 bytes: correlation ID]
|
|
// [varint: header tagged fields (0x00 for empty) - ONLY for flexible responses with headerVersion >= 1]
|
|
// [N bytes: response body]
|
|
|
|
// Determine if this response should be flexible
|
|
isFlexible := isFlexibleResponse(apiKey, apiVersion)
|
|
|
|
// Calculate total size: correlation ID (4) + tagged fields (1 if flexible) + body
|
|
totalSize := 4 + len(responseBody)
|
|
if isFlexible {
|
|
totalSize += 1 // Add 1 byte for empty tagged fields (0x00)
|
|
}
|
|
|
|
// Build complete response in memory for hex dump logging
|
|
fullResponse := make([]byte, 0, 4+totalSize)
|
|
|
|
// Write size
|
|
sizeBuf := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(sizeBuf, uint32(totalSize))
|
|
fullResponse = append(fullResponse, sizeBuf...)
|
|
|
|
// Write correlation ID
|
|
correlationBuf := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(correlationBuf, correlationID)
|
|
fullResponse = append(fullResponse, correlationBuf...)
|
|
|
|
// Write header-level tagged fields for flexible responses
|
|
if isFlexible {
|
|
// Empty tagged fields = 0x00 (varint 0)
|
|
fullResponse = append(fullResponse, 0x00)
|
|
}
|
|
|
|
// Write response body
|
|
fullResponse = append(fullResponse, responseBody...)
|
|
|
|
// Write to connection
|
|
if _, err := w.Write(fullResponse); err != nil {
|
|
return fmt.Errorf("write response: %w", err)
|
|
}
|
|
|
|
// Flush
|
|
if err := w.Flush(); err != nil {
|
|
return fmt.Errorf("flush response: %w", err)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// hexDump formats bytes as a hex dump with ASCII representation
|
|
func hexDump(data []byte) string {
|
|
var result strings.Builder
|
|
for i := 0; i < len(data); i += 16 {
|
|
// Offset
|
|
result.WriteString(fmt.Sprintf("%04x ", i))
|
|
|
|
// Hex bytes
|
|
for j := 0; j < 16; j++ {
|
|
if i+j < len(data) {
|
|
result.WriteString(fmt.Sprintf("%02x ", data[i+j]))
|
|
} else {
|
|
result.WriteString(" ")
|
|
}
|
|
if j == 7 {
|
|
result.WriteString(" ")
|
|
}
|
|
}
|
|
|
|
// ASCII representation
|
|
result.WriteString(" |")
|
|
for j := 0; j < 16 && i+j < len(data); j++ {
|
|
b := data[i+j]
|
|
if b >= 32 && b < 127 {
|
|
result.WriteByte(b)
|
|
} else {
|
|
result.WriteByte('.')
|
|
}
|
|
}
|
|
result.WriteString("|\n")
|
|
}
|
|
return result.String()
|
|
}
|
|
|
|
// writeResponseWithCorrelationID is deprecated - use writeResponseWithHeader instead
|
|
// Kept for compatibility with direct callers that don't have API info
|
|
func (h *Handler) writeResponseWithCorrelationID(w *bufio.Writer, correlationID uint32, responseBody []byte, timeout time.Duration) error {
|
|
// Assume non-flexible for backward compatibility
|
|
return h.writeResponseWithHeader(w, correlationID, 0, 0, responseBody, timeout)
|
|
}
|
|
|
|
// writeResponseWithTimeout writes a Kafka response with timeout handling
|
|
// DEPRECATED: Use writeResponseWithCorrelationID instead
|
|
func (h *Handler) writeResponseWithTimeout(w *bufio.Writer, response []byte, timeout time.Duration) error {
|
|
// This old function expects response to include correlation ID at the start
|
|
// For backward compatibility with any remaining callers
|
|
|
|
// Write response size (4 bytes)
|
|
responseSizeBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(responseSizeBytes, uint32(len(response)))
|
|
|
|
if _, err := w.Write(responseSizeBytes); err != nil {
|
|
return fmt.Errorf("write response size: %w", err)
|
|
}
|
|
|
|
// Write response data
|
|
if _, err := w.Write(response); err != nil {
|
|
return fmt.Errorf("write response data: %w", err)
|
|
}
|
|
|
|
// Flush the buffer
|
|
if err := w.Flush(); err != nil {
|
|
return fmt.Errorf("flush response: %w", err)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// EnableSchemaManagement enables schema management with the given configuration
|
|
func (h *Handler) EnableSchemaManagement(config schema.ManagerConfig) error {
|
|
manager, err := schema.NewManagerWithHealthCheck(config)
|
|
if err != nil {
|
|
return fmt.Errorf("failed to create schema manager: %w", err)
|
|
}
|
|
|
|
h.schemaManager = manager
|
|
h.useSchema = true
|
|
|
|
return nil
|
|
}
|
|
|
|
// EnableBrokerIntegration enables mq.broker integration for schematized messages
|
|
func (h *Handler) EnableBrokerIntegration(brokers []string) error {
|
|
if !h.IsSchemaEnabled() {
|
|
return fmt.Errorf("schema management must be enabled before broker integration")
|
|
}
|
|
|
|
brokerClient := schema.NewBrokerClient(schema.BrokerClientConfig{
|
|
Brokers: brokers,
|
|
SchemaManager: h.schemaManager,
|
|
})
|
|
|
|
h.brokerClient = brokerClient
|
|
return nil
|
|
}
|
|
|
|
// DisableSchemaManagement disables schema management and broker integration
|
|
func (h *Handler) DisableSchemaManagement() {
|
|
if h.brokerClient != nil {
|
|
h.brokerClient.Close()
|
|
h.brokerClient = nil
|
|
}
|
|
h.schemaManager = nil
|
|
h.useSchema = false
|
|
}
|
|
|
|
// SetSchemaRegistryURL sets the Schema Registry URL for delayed initialization
|
|
func (h *Handler) SetSchemaRegistryURL(url string) {
|
|
h.schemaRegistryURL = url
|
|
}
|
|
|
|
// SetDefaultPartitions sets the default partition count for auto-created topics
|
|
func (h *Handler) SetDefaultPartitions(partitions int32) {
|
|
h.defaultPartitions = partitions
|
|
}
|
|
|
|
// GetDefaultPartitions returns the default partition count for auto-created topics
|
|
func (h *Handler) GetDefaultPartitions() int32 {
|
|
if h.defaultPartitions <= 0 {
|
|
return 4 // Fallback default
|
|
}
|
|
return h.defaultPartitions
|
|
}
|
|
|
|
// IsSchemaEnabled returns whether schema management is enabled
|
|
func (h *Handler) IsSchemaEnabled() bool {
|
|
// Try to initialize schema management if not already done
|
|
if !h.useSchema && h.schemaRegistryURL != "" {
|
|
h.tryInitializeSchemaManagement()
|
|
}
|
|
return h.useSchema && h.schemaManager != nil
|
|
}
|
|
|
|
// tryInitializeSchemaManagement attempts to initialize schema management
|
|
// This is called lazily when schema functionality is first needed
|
|
func (h *Handler) tryInitializeSchemaManagement() {
|
|
if h.useSchema || h.schemaRegistryURL == "" {
|
|
return // Already initialized or no URL provided
|
|
}
|
|
|
|
schemaConfig := schema.ManagerConfig{
|
|
RegistryURL: h.schemaRegistryURL,
|
|
}
|
|
|
|
if err := h.EnableSchemaManagement(schemaConfig); err != nil {
|
|
return
|
|
}
|
|
|
|
}
|
|
|
|
// IsBrokerIntegrationEnabled returns true if broker integration is enabled
|
|
func (h *Handler) IsBrokerIntegrationEnabled() bool {
|
|
return h.IsSchemaEnabled() && h.brokerClient != nil
|
|
}
|
|
|
|
// commitOffsetToSMQ commits offset using SMQ storage
|
|
func (h *Handler) commitOffsetToSMQ(key ConsumerOffsetKey, offsetValue int64, metadata string) error {
|
|
// Use new consumer offset storage if available, fall back to SMQ storage
|
|
if h.consumerOffsetStorage != nil {
|
|
return h.consumerOffsetStorage.CommitOffset(key.ConsumerGroup, key.Topic, key.Partition, offsetValue, metadata)
|
|
}
|
|
|
|
// No SMQ offset storage - only use consumer offset storage
|
|
return fmt.Errorf("offset storage not initialized")
|
|
}
|
|
|
|
// fetchOffsetFromSMQ fetches offset using SMQ storage
|
|
func (h *Handler) fetchOffsetFromSMQ(key ConsumerOffsetKey) (int64, string, error) {
|
|
// Use new consumer offset storage if available, fall back to SMQ storage
|
|
if h.consumerOffsetStorage != nil {
|
|
return h.consumerOffsetStorage.FetchOffset(key.ConsumerGroup, key.Topic, key.Partition)
|
|
}
|
|
|
|
// SMQ offset storage removed - no fallback
|
|
return -1, "", fmt.Errorf("offset storage not initialized")
|
|
}
|
|
|
|
// DescribeConfigsResource represents a resource in a DescribeConfigs request
|
|
type DescribeConfigsResource struct {
|
|
ResourceType int8 // 2 = Topic, 4 = Broker
|
|
ResourceName string
|
|
ConfigNames []string // Empty means return all configs
|
|
}
|
|
|
|
// parseDescribeConfigsRequest parses a DescribeConfigs request body
|
|
func (h *Handler) parseDescribeConfigsRequest(requestBody []byte, apiVersion uint16) ([]DescribeConfigsResource, error) {
|
|
if len(requestBody) < 1 {
|
|
return nil, fmt.Errorf("request too short")
|
|
}
|
|
|
|
offset := 0
|
|
|
|
// DescribeConfigs v4+ uses flexible protocol (compact arrays with varint)
|
|
isFlexible := apiVersion >= 4
|
|
|
|
var resourcesLength uint32
|
|
if isFlexible {
|
|
// Debug: log the first 8 bytes of the request body
|
|
debugBytes := requestBody[offset:]
|
|
if len(debugBytes) > 8 {
|
|
debugBytes = debugBytes[:8]
|
|
}
|
|
|
|
// FIX: Skip top-level tagged fields for DescribeConfigs v4+ flexible protocol
|
|
// The request body starts with tagged fields count (usually 0x00 = empty)
|
|
_, consumed, err := DecodeTaggedFields(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("DescribeConfigs v%d: decode top-level tagged fields: %w", apiVersion, err)
|
|
}
|
|
offset += consumed
|
|
|
|
// Resources (compact array) - Now correctly positioned after tagged fields
|
|
resourcesLength, consumed, err = DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode resources compact array: %w", err)
|
|
}
|
|
offset += consumed
|
|
} else {
|
|
// Regular array: length is int32
|
|
if len(requestBody) < 4 {
|
|
return nil, fmt.Errorf("request too short for regular array")
|
|
}
|
|
resourcesLength = binary.BigEndian.Uint32(requestBody[offset : offset+4])
|
|
offset += 4
|
|
}
|
|
|
|
// Validate resources length to prevent panic
|
|
if resourcesLength > 100 { // Reasonable limit
|
|
return nil, fmt.Errorf("invalid resources length: %d", resourcesLength)
|
|
}
|
|
|
|
resources := make([]DescribeConfigsResource, 0, resourcesLength)
|
|
|
|
for i := uint32(0); i < resourcesLength; i++ {
|
|
if offset+1 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for resource type")
|
|
}
|
|
|
|
// Resource type (1 byte)
|
|
resourceType := int8(requestBody[offset])
|
|
offset++
|
|
|
|
// Resource name (string - compact for v4+, regular for v0-3)
|
|
var resourceName string
|
|
if isFlexible {
|
|
// Compact string: length is encoded as UNSIGNED_VARINT(actualLength + 1)
|
|
name, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode resource name compact string: %w", err)
|
|
}
|
|
resourceName = name
|
|
offset += consumed
|
|
} else {
|
|
// Regular string: length is int16
|
|
if offset+2 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for resource name length")
|
|
}
|
|
nameLength := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
|
|
// Validate name length to prevent panic
|
|
if nameLength < 0 || nameLength > 1000 { // Reasonable limit
|
|
return nil, fmt.Errorf("invalid resource name length: %d", nameLength)
|
|
}
|
|
|
|
if offset+nameLength > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for resource name")
|
|
}
|
|
resourceName = string(requestBody[offset : offset+nameLength])
|
|
offset += nameLength
|
|
}
|
|
|
|
// Config names array (compact for v4+, regular for v0-3)
|
|
var configNames []string
|
|
if isFlexible {
|
|
// Compact array: length is encoded as UNSIGNED_VARINT(actualLength + 1)
|
|
// For nullable arrays, 0 means null, 1 means empty
|
|
configNamesCount, consumed, err := DecodeCompactArrayLength(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode config names compact array: %w", err)
|
|
}
|
|
offset += consumed
|
|
|
|
// Parse each config name as compact string (if not null)
|
|
if configNamesCount > 0 {
|
|
for j := uint32(0); j < configNamesCount; j++ {
|
|
configName, consumed, err := DecodeFlexibleString(requestBody[offset:])
|
|
if err != nil {
|
|
return nil, fmt.Errorf("decode config name[%d] compact string: %w", j, err)
|
|
}
|
|
offset += consumed
|
|
configNames = append(configNames, configName)
|
|
}
|
|
}
|
|
} else {
|
|
// Regular array: length is int32
|
|
if offset+4 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for config names length")
|
|
}
|
|
configNamesLength := int32(binary.BigEndian.Uint32(requestBody[offset : offset+4]))
|
|
offset += 4
|
|
|
|
// Validate config names length to prevent panic
|
|
// Note: -1 means null/empty array in Kafka protocol
|
|
if configNamesLength < -1 || configNamesLength > 1000 { // Reasonable limit
|
|
return nil, fmt.Errorf("invalid config names length: %d", configNamesLength)
|
|
}
|
|
|
|
// Handle null array case
|
|
if configNamesLength == -1 {
|
|
configNamesLength = 0
|
|
}
|
|
|
|
configNames = make([]string, 0, configNamesLength)
|
|
for j := int32(0); j < configNamesLength; j++ {
|
|
if offset+2 > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for config name length")
|
|
}
|
|
configNameLength := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
|
|
// Validate config name length to prevent panic
|
|
if configNameLength < 0 || configNameLength > 500 { // Reasonable limit
|
|
return nil, fmt.Errorf("invalid config name length: %d", configNameLength)
|
|
}
|
|
|
|
if offset+configNameLength > len(requestBody) {
|
|
return nil, fmt.Errorf("insufficient data for config name")
|
|
}
|
|
configName := string(requestBody[offset : offset+configNameLength])
|
|
offset += configNameLength
|
|
|
|
configNames = append(configNames, configName)
|
|
}
|
|
}
|
|
|
|
resources = append(resources, DescribeConfigsResource{
|
|
ResourceType: resourceType,
|
|
ResourceName: resourceName,
|
|
ConfigNames: configNames,
|
|
})
|
|
}
|
|
|
|
return resources, nil
|
|
}
|
|
|
|
// buildDescribeConfigsResourceResponse builds the response for a single resource
|
|
func (h *Handler) buildDescribeConfigsResourceResponse(resource DescribeConfigsResource, apiVersion uint16) []byte {
|
|
response := make([]byte, 0, 512)
|
|
|
|
// Error code (0 = no error)
|
|
errorCodeBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(errorCodeBytes, 0)
|
|
response = append(response, errorCodeBytes...)
|
|
|
|
// Error message (null string = -1 length)
|
|
errorMsgBytes := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(errorMsgBytes, 0xFFFF) // -1 as uint16
|
|
response = append(response, errorMsgBytes...)
|
|
|
|
// Resource type
|
|
response = append(response, byte(resource.ResourceType))
|
|
|
|
// Resource name
|
|
nameBytes := make([]byte, 2+len(resource.ResourceName))
|
|
binary.BigEndian.PutUint16(nameBytes[0:2], uint16(len(resource.ResourceName)))
|
|
copy(nameBytes[2:], []byte(resource.ResourceName))
|
|
response = append(response, nameBytes...)
|
|
|
|
// Get configs for this resource
|
|
configs := h.getConfigsForResource(resource)
|
|
|
|
// Config entries array length
|
|
configCountBytes := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(configCountBytes, uint32(len(configs)))
|
|
response = append(response, configCountBytes...)
|
|
|
|
// Add each config entry
|
|
for _, config := range configs {
|
|
configBytes := h.buildConfigEntry(config, apiVersion)
|
|
response = append(response, configBytes...)
|
|
}
|
|
|
|
return response
|
|
}
|
|
|
|
// ConfigEntry represents a single configuration entry
|
|
type ConfigEntry struct {
|
|
Name string
|
|
Value string
|
|
ReadOnly bool
|
|
IsDefault bool
|
|
Sensitive bool
|
|
}
|
|
|
|
// getConfigsForResource returns appropriate configs for a resource
|
|
func (h *Handler) getConfigsForResource(resource DescribeConfigsResource) []ConfigEntry {
|
|
switch resource.ResourceType {
|
|
case 2: // Topic
|
|
return h.getTopicConfigs(resource.ResourceName, resource.ConfigNames)
|
|
case 4: // Broker
|
|
return h.getBrokerConfigs(resource.ConfigNames)
|
|
default:
|
|
return []ConfigEntry{}
|
|
}
|
|
}
|
|
|
|
// getTopicConfigs returns topic-level configurations
|
|
func (h *Handler) getTopicConfigs(topicName string, requestedConfigs []string) []ConfigEntry {
|
|
// Default topic configs that admin clients commonly request
|
|
allConfigs := map[string]ConfigEntry{
|
|
"cleanup.policy": {
|
|
Name: "cleanup.policy",
|
|
Value: "delete",
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"retention.ms": {
|
|
Name: "retention.ms",
|
|
Value: "604800000", // 7 days in milliseconds
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"retention.bytes": {
|
|
Name: "retention.bytes",
|
|
Value: "-1", // Unlimited
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"segment.ms": {
|
|
Name: "segment.ms",
|
|
Value: "86400000", // 1 day in milliseconds
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"max.message.bytes": {
|
|
Name: "max.message.bytes",
|
|
Value: "1048588", // ~1MB
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"min.insync.replicas": {
|
|
Name: "min.insync.replicas",
|
|
Value: "1",
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
}
|
|
|
|
// If specific configs requested, filter to those
|
|
if len(requestedConfigs) > 0 {
|
|
filteredConfigs := make([]ConfigEntry, 0, len(requestedConfigs))
|
|
for _, configName := range requestedConfigs {
|
|
if config, exists := allConfigs[configName]; exists {
|
|
filteredConfigs = append(filteredConfigs, config)
|
|
}
|
|
}
|
|
return filteredConfigs
|
|
}
|
|
|
|
// Return all configs
|
|
configs := make([]ConfigEntry, 0, len(allConfigs))
|
|
for _, config := range allConfigs {
|
|
configs = append(configs, config)
|
|
}
|
|
return configs
|
|
}
|
|
|
|
// getBrokerConfigs returns broker-level configurations
|
|
func (h *Handler) getBrokerConfigs(requestedConfigs []string) []ConfigEntry {
|
|
// Default broker configs that admin clients commonly request
|
|
allConfigs := map[string]ConfigEntry{
|
|
"log.retention.hours": {
|
|
Name: "log.retention.hours",
|
|
Value: "168", // 7 days
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"log.segment.bytes": {
|
|
Name: "log.segment.bytes",
|
|
Value: "1073741824", // 1GB
|
|
ReadOnly: false,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"num.network.threads": {
|
|
Name: "num.network.threads",
|
|
Value: "3",
|
|
ReadOnly: true,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
"num.io.threads": {
|
|
Name: "num.io.threads",
|
|
Value: "8",
|
|
ReadOnly: true,
|
|
IsDefault: true,
|
|
Sensitive: false,
|
|
},
|
|
}
|
|
|
|
// If specific configs requested, filter to those
|
|
if len(requestedConfigs) > 0 {
|
|
filteredConfigs := make([]ConfigEntry, 0, len(requestedConfigs))
|
|
for _, configName := range requestedConfigs {
|
|
if config, exists := allConfigs[configName]; exists {
|
|
filteredConfigs = append(filteredConfigs, config)
|
|
}
|
|
}
|
|
return filteredConfigs
|
|
}
|
|
|
|
// Return all configs
|
|
configs := make([]ConfigEntry, 0, len(allConfigs))
|
|
for _, config := range allConfigs {
|
|
configs = append(configs, config)
|
|
}
|
|
return configs
|
|
}
|
|
|
|
// buildConfigEntry builds the wire format for a single config entry
|
|
func (h *Handler) buildConfigEntry(config ConfigEntry, apiVersion uint16) []byte {
|
|
entry := make([]byte, 0, 256)
|
|
|
|
// Config name
|
|
nameBytes := make([]byte, 2+len(config.Name))
|
|
binary.BigEndian.PutUint16(nameBytes[0:2], uint16(len(config.Name)))
|
|
copy(nameBytes[2:], []byte(config.Name))
|
|
entry = append(entry, nameBytes...)
|
|
|
|
// Config value
|
|
valueBytes := make([]byte, 2+len(config.Value))
|
|
binary.BigEndian.PutUint16(valueBytes[0:2], uint16(len(config.Value)))
|
|
copy(valueBytes[2:], []byte(config.Value))
|
|
entry = append(entry, valueBytes...)
|
|
|
|
// Read only flag
|
|
if config.ReadOnly {
|
|
entry = append(entry, 1)
|
|
} else {
|
|
entry = append(entry, 0)
|
|
}
|
|
|
|
// Is default flag (only for version 0)
|
|
if apiVersion == 0 {
|
|
if config.IsDefault {
|
|
entry = append(entry, 1)
|
|
} else {
|
|
entry = append(entry, 0)
|
|
}
|
|
}
|
|
|
|
// Config source (for versions 1-3)
|
|
if apiVersion >= 1 && apiVersion <= 3 {
|
|
// ConfigSource: 1 = DYNAMIC_TOPIC_CONFIG, 2 = DYNAMIC_BROKER_CONFIG, 4 = STATIC_BROKER_CONFIG, 5 = DEFAULT_CONFIG
|
|
configSource := int8(5) // DEFAULT_CONFIG for all our configs since they're defaults
|
|
entry = append(entry, byte(configSource))
|
|
}
|
|
|
|
// Sensitive flag
|
|
if config.Sensitive {
|
|
entry = append(entry, 1)
|
|
} else {
|
|
entry = append(entry, 0)
|
|
}
|
|
|
|
// Config synonyms (for versions 1-3)
|
|
if apiVersion >= 1 && apiVersion <= 3 {
|
|
// Empty synonyms array (4 bytes for array length = 0)
|
|
synonymsLength := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(synonymsLength, 0)
|
|
entry = append(entry, synonymsLength...)
|
|
}
|
|
|
|
// Config type (for version 3 only)
|
|
if apiVersion == 3 {
|
|
configType := int8(1) // STRING type for all our configs
|
|
entry = append(entry, byte(configType))
|
|
}
|
|
|
|
// Config documentation (for version 3 only)
|
|
if apiVersion == 3 {
|
|
// Null documentation (length = -1)
|
|
docLength := make([]byte, 2)
|
|
binary.BigEndian.PutUint16(docLength, 0xFFFF) // -1 as uint16
|
|
entry = append(entry, docLength...)
|
|
}
|
|
|
|
return entry
|
|
}
|
|
|
|
// registerSchemasViaBrokerAPI registers both key and value schemas via the broker's ConfigureTopic API
|
|
// Only the gateway leader performs the registration to avoid concurrent updates.
|
|
func (h *Handler) registerSchemasViaBrokerAPI(topicName string, valueRecordType *schema_pb.RecordType, keyRecordType *schema_pb.RecordType) error {
|
|
if valueRecordType == nil && keyRecordType == nil {
|
|
return nil
|
|
}
|
|
|
|
// Check coordinator registry for multi-gateway deployments
|
|
// In single-gateway mode, coordinator registry may not be initialized - that's OK
|
|
if reg := h.GetCoordinatorRegistry(); reg != nil {
|
|
// Multi-gateway mode - check if we're the leader
|
|
isLeader := reg.IsLeader()
|
|
|
|
if !isLeader {
|
|
// Not leader - in production multi-gateway setups, skip to avoid conflicts
|
|
// In single-gateway setups where leader election fails, log warning but proceed
|
|
// This ensures schema registration works even if distributed locking has issues
|
|
// Note: Schema registration is idempotent, so duplicate registrations are safe
|
|
} else {
|
|
}
|
|
} else {
|
|
// No coordinator registry - definitely single-gateway mode
|
|
}
|
|
|
|
// Require SeaweedMQ integration to access broker
|
|
if h.seaweedMQHandler == nil {
|
|
return fmt.Errorf("no SeaweedMQ handler available for broker access")
|
|
}
|
|
|
|
// Get broker addresses
|
|
brokerAddresses := h.seaweedMQHandler.GetBrokerAddresses()
|
|
if len(brokerAddresses) == 0 {
|
|
return fmt.Errorf("no broker addresses available")
|
|
}
|
|
|
|
// Use the first available broker
|
|
brokerAddress := brokerAddresses[0]
|
|
|
|
// Load security configuration
|
|
util.LoadSecurityConfiguration()
|
|
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.mq")
|
|
|
|
// Get current topic configuration to preserve partition count
|
|
seaweedTopic := &schema_pb.Topic{
|
|
Namespace: DefaultKafkaNamespace,
|
|
Name: topicName,
|
|
}
|
|
|
|
return pb.WithBrokerGrpcClient(false, brokerAddress, grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
|
// First get current configuration
|
|
getResp, err := client.GetTopicConfiguration(context.Background(), &mq_pb.GetTopicConfigurationRequest{
|
|
Topic: seaweedTopic,
|
|
})
|
|
if err != nil {
|
|
// Convert dual schemas to flat schema format
|
|
var flatSchema *schema_pb.RecordType
|
|
var keyColumns []string
|
|
if keyRecordType != nil || valueRecordType != nil {
|
|
flatSchema, keyColumns = mqschema.CombineFlatSchemaFromKeyValue(keyRecordType, valueRecordType)
|
|
}
|
|
|
|
// If topic doesn't exist, create it with configurable default partition count
|
|
// Get schema format from topic config if available
|
|
schemaFormat := h.getTopicSchemaFormat(topicName)
|
|
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
|
|
Topic: seaweedTopic,
|
|
PartitionCount: h.GetDefaultPartitions(), // Use configurable default
|
|
MessageRecordType: flatSchema,
|
|
KeyColumns: keyColumns,
|
|
SchemaFormat: schemaFormat,
|
|
})
|
|
return err
|
|
}
|
|
|
|
// Convert dual schemas to flat schema format for update
|
|
var flatSchema *schema_pb.RecordType
|
|
var keyColumns []string
|
|
if keyRecordType != nil || valueRecordType != nil {
|
|
flatSchema, keyColumns = mqschema.CombineFlatSchemaFromKeyValue(keyRecordType, valueRecordType)
|
|
}
|
|
|
|
// Update existing topic with new schema
|
|
// Get schema format from topic config if available
|
|
schemaFormat := h.getTopicSchemaFormat(topicName)
|
|
_, err = client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
|
|
Topic: seaweedTopic,
|
|
PartitionCount: getResp.PartitionCount,
|
|
MessageRecordType: flatSchema,
|
|
KeyColumns: keyColumns,
|
|
Retention: getResp.Retention,
|
|
SchemaFormat: schemaFormat,
|
|
})
|
|
return err
|
|
})
|
|
}
|
|
|
|
// handleInitProducerId handles InitProducerId API requests (API key 22)
|
|
// This API is used to initialize a producer for transactional or idempotent operations
|
|
func (h *Handler) handleInitProducerId(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {
|
|
|
|
// InitProducerId Request Format (varies by version):
|
|
// v0-v1: transactional_id(NULLABLE_STRING) + transaction_timeout_ms(INT32)
|
|
// v2+: transactional_id(NULLABLE_STRING) + transaction_timeout_ms(INT32) + producer_id(INT64) + producer_epoch(INT16)
|
|
// v4+: Uses flexible format with tagged fields
|
|
|
|
offset := 0
|
|
|
|
// Parse transactional_id (NULLABLE_STRING or COMPACT_NULLABLE_STRING for flexible versions)
|
|
var transactionalId *string
|
|
if apiVersion >= 4 {
|
|
// Flexible version - use compact nullable string
|
|
if len(requestBody) < offset+1 {
|
|
return nil, fmt.Errorf("InitProducerId request too short for transactional_id")
|
|
}
|
|
|
|
length := int(requestBody[offset])
|
|
offset++
|
|
|
|
if length == 0 {
|
|
// Null string
|
|
transactionalId = nil
|
|
} else {
|
|
// Non-null string (length is encoded as length+1 in compact format)
|
|
actualLength := length - 1
|
|
if len(requestBody) < offset+actualLength {
|
|
return nil, fmt.Errorf("InitProducerId request transactional_id too short")
|
|
}
|
|
if actualLength > 0 {
|
|
id := string(requestBody[offset : offset+actualLength])
|
|
transactionalId = &id
|
|
offset += actualLength
|
|
} else {
|
|
// Empty string
|
|
id := ""
|
|
transactionalId = &id
|
|
}
|
|
}
|
|
} else {
|
|
// Non-flexible version - use regular nullable string
|
|
if len(requestBody) < offset+2 {
|
|
return nil, fmt.Errorf("InitProducerId request too short for transactional_id length")
|
|
}
|
|
|
|
length := int(binary.BigEndian.Uint16(requestBody[offset : offset+2]))
|
|
offset += 2
|
|
|
|
if length == 0xFFFF {
|
|
// Null string (-1 as uint16)
|
|
transactionalId = nil
|
|
} else {
|
|
if len(requestBody) < offset+length {
|
|
return nil, fmt.Errorf("InitProducerId request transactional_id too short")
|
|
}
|
|
if length > 0 {
|
|
id := string(requestBody[offset : offset+length])
|
|
transactionalId = &id
|
|
offset += length
|
|
} else {
|
|
// Empty string
|
|
id := ""
|
|
transactionalId = &id
|
|
}
|
|
}
|
|
}
|
|
_ = transactionalId // Used for logging/tracking, but not in core logic yet
|
|
|
|
// Parse transaction_timeout_ms (INT32)
|
|
if len(requestBody) < offset+4 {
|
|
return nil, fmt.Errorf("InitProducerId request too short for transaction_timeout_ms")
|
|
}
|
|
_ = binary.BigEndian.Uint32(requestBody[offset : offset+4]) // transactionTimeoutMs
|
|
offset += 4
|
|
|
|
// For v2+, there might be additional fields, but we'll ignore them for now
|
|
// as we're providing a basic implementation
|
|
|
|
// Build response
|
|
response := make([]byte, 0, 64)
|
|
|
|
// NOTE: Correlation ID is handled by writeResponseWithHeader
|
|
// Do NOT include it in the response body
|
|
// Note: Header tagged fields are also handled by writeResponseWithHeader for flexible versions
|
|
|
|
// InitProducerId Response Format:
|
|
// throttle_time_ms(INT32) + error_code(INT16) + producer_id(INT64) + producer_epoch(INT16)
|
|
// + tagged_fields (for flexible versions)
|
|
|
|
// Throttle time (4 bytes) - v1+
|
|
if apiVersion >= 1 {
|
|
response = append(response, 0, 0, 0, 0) // No throttling
|
|
}
|
|
|
|
// Error code (2 bytes) - SUCCESS
|
|
response = append(response, 0, 0) // No error
|
|
|
|
// Producer ID (8 bytes) - generate a simple producer ID
|
|
// In a real implementation, this would be managed by a transaction coordinator
|
|
producerId := int64(1000) // Simple fixed producer ID for now
|
|
producerIdBytes := make([]byte, 8)
|
|
binary.BigEndian.PutUint64(producerIdBytes, uint64(producerId))
|
|
response = append(response, producerIdBytes...)
|
|
|
|
// Producer epoch (2 bytes) - start with epoch 0
|
|
response = append(response, 0, 0) // Epoch 0
|
|
|
|
// For flexible versions (v4+), add response body tagged fields
|
|
if apiVersion >= 4 {
|
|
response = append(response, 0x00) // Empty response body tagged fields
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
// createTopicWithSchemaSupport creates a topic with optional schema integration
|
|
// This function creates topics with schema support when schema management is enabled
|
|
func (h *Handler) createTopicWithSchemaSupport(topicName string, partitions int32) error {
|
|
|
|
// For system topics like _schemas, __consumer_offsets, etc., use default schema
|
|
if isSystemTopic(topicName) {
|
|
return h.createTopicWithDefaultFlexibleSchema(topicName, partitions)
|
|
}
|
|
|
|
// Check if Schema Registry URL is configured
|
|
if h.schemaRegistryURL != "" {
|
|
|
|
// Try to initialize schema management if not already done
|
|
if h.schemaManager == nil {
|
|
h.tryInitializeSchemaManagement()
|
|
}
|
|
|
|
// If schema manager is still nil after initialization attempt, Schema Registry is unavailable
|
|
if h.schemaManager == nil {
|
|
return fmt.Errorf("Schema Registry is configured at %s but unavailable - cannot create topic %s without schema validation", h.schemaRegistryURL, topicName)
|
|
}
|
|
|
|
// Schema Registry is available - try to fetch existing schema
|
|
keyRecordType, valueRecordType, err := h.fetchSchemaForTopic(topicName)
|
|
if err != nil {
|
|
// Check if this is a connection error vs schema not found
|
|
if h.isSchemaRegistryConnectionError(err) {
|
|
return fmt.Errorf("Schema Registry is unavailable: %w", err)
|
|
}
|
|
// Schema not found - this is an error when schema management is enforced
|
|
return fmt.Errorf("schema is required for topic %s but no schema found in Schema Registry", topicName)
|
|
}
|
|
|
|
if keyRecordType != nil || valueRecordType != nil {
|
|
// Create topic with schema from Schema Registry
|
|
return h.seaweedMQHandler.CreateTopicWithSchemas(topicName, partitions, keyRecordType, valueRecordType)
|
|
}
|
|
|
|
// No schemas found - this is an error when schema management is enforced
|
|
return fmt.Errorf("schema is required for topic %s but no schema found in Schema Registry", topicName)
|
|
}
|
|
|
|
// Schema Registry URL not configured - create topic without schema (backward compatibility)
|
|
return h.seaweedMQHandler.CreateTopic(topicName, partitions)
|
|
}
|
|
|
|
// createTopicWithDefaultFlexibleSchema creates a topic with a flexible default schema
|
|
// that can handle both Avro and JSON messages when schema management is enabled
|
|
func (h *Handler) createTopicWithDefaultFlexibleSchema(topicName string, partitions int32) error {
|
|
// CRITICAL FIX: System topics like _schemas should be PLAIN Kafka topics without schema management
|
|
// Schema Registry uses _schemas to STORE schemas, so it can't have schema management itself
|
|
// This was causing issues with Schema Registry bootstrap
|
|
|
|
glog.V(0).Infof("Creating system topic %s as PLAIN topic (no schema management)", topicName)
|
|
return h.seaweedMQHandler.CreateTopic(topicName, partitions)
|
|
}
|
|
|
|
// fetchSchemaForTopic attempts to fetch schema information for a topic from Schema Registry
|
|
// Returns key and value RecordTypes if schemas are found
|
|
func (h *Handler) fetchSchemaForTopic(topicName string) (*schema_pb.RecordType, *schema_pb.RecordType, error) {
|
|
if h.schemaManager == nil {
|
|
return nil, nil, fmt.Errorf("schema manager not available")
|
|
}
|
|
|
|
var keyRecordType *schema_pb.RecordType
|
|
var valueRecordType *schema_pb.RecordType
|
|
var lastConnectionError error
|
|
|
|
// Try to fetch value schema using standard Kafka naming convention: <topic>-value
|
|
valueSubject := topicName + "-value"
|
|
cachedSchema, err := h.schemaManager.GetLatestSchema(valueSubject)
|
|
if err != nil {
|
|
// Check if this is a connection error (Schema Registry unavailable)
|
|
if h.isSchemaRegistryConnectionError(err) {
|
|
lastConnectionError = err
|
|
}
|
|
// Not found or connection error - continue to check key schema
|
|
} else if cachedSchema != nil {
|
|
|
|
// Convert schema to RecordType
|
|
recordType, err := h.convertSchemaToRecordType(cachedSchema.Schema, cachedSchema.LatestID)
|
|
if err == nil {
|
|
valueRecordType = recordType
|
|
// Store schema configuration for later use
|
|
h.storeTopicSchemaConfig(topicName, cachedSchema.LatestID, schema.FormatAvro)
|
|
} else {
|
|
}
|
|
}
|
|
|
|
// Try to fetch key schema (optional)
|
|
keySubject := topicName + "-key"
|
|
cachedKeySchema, keyErr := h.schemaManager.GetLatestSchema(keySubject)
|
|
if keyErr != nil {
|
|
if h.isSchemaRegistryConnectionError(keyErr) {
|
|
lastConnectionError = keyErr
|
|
}
|
|
// Not found or connection error - key schema is optional
|
|
} else if cachedKeySchema != nil {
|
|
|
|
// Convert schema to RecordType
|
|
recordType, err := h.convertSchemaToRecordType(cachedKeySchema.Schema, cachedKeySchema.LatestID)
|
|
if err == nil {
|
|
keyRecordType = recordType
|
|
// Store key schema configuration for later use
|
|
h.storeTopicKeySchemaConfig(topicName, cachedKeySchema.LatestID, schema.FormatAvro)
|
|
} else {
|
|
}
|
|
}
|
|
|
|
// If we encountered connection errors, fail fast
|
|
if lastConnectionError != nil && keyRecordType == nil && valueRecordType == nil {
|
|
return nil, nil, fmt.Errorf("Schema Registry is unavailable: %w", lastConnectionError)
|
|
}
|
|
|
|
// Return error if no schemas found (but Schema Registry was reachable)
|
|
if keyRecordType == nil && valueRecordType == nil {
|
|
return nil, nil, fmt.Errorf("no schemas found for topic %s", topicName)
|
|
}
|
|
|
|
return keyRecordType, valueRecordType, nil
|
|
}
|
|
|
|
// isSchemaRegistryConnectionError determines if an error is due to Schema Registry being unavailable
|
|
// vs a schema not being found (404)
|
|
func (h *Handler) isSchemaRegistryConnectionError(err error) bool {
|
|
if err == nil {
|
|
return false
|
|
}
|
|
|
|
errStr := err.Error()
|
|
|
|
// Connection errors (network issues, DNS resolution, etc.)
|
|
if strings.Contains(errStr, "failed to fetch") &&
|
|
(strings.Contains(errStr, "connection refused") ||
|
|
strings.Contains(errStr, "no such host") ||
|
|
strings.Contains(errStr, "timeout") ||
|
|
strings.Contains(errStr, "network is unreachable")) {
|
|
return true
|
|
}
|
|
|
|
// HTTP 5xx errors (server errors)
|
|
if strings.Contains(errStr, "schema registry error 5") {
|
|
return true
|
|
}
|
|
|
|
// HTTP 404 errors are "schema not found", not connection errors
|
|
if strings.Contains(errStr, "schema registry error 404") {
|
|
return false
|
|
}
|
|
|
|
// Other HTTP errors (401, 403, etc.) should be treated as connection/config issues
|
|
if strings.Contains(errStr, "schema registry error") {
|
|
return true
|
|
}
|
|
|
|
return false
|
|
}
|
|
|
|
// convertSchemaToRecordType converts a schema string to a RecordType
|
|
func (h *Handler) convertSchemaToRecordType(schemaStr string, schemaID uint32) (*schema_pb.RecordType, error) {
|
|
// Get the cached schema to determine format
|
|
cachedSchema, err := h.schemaManager.GetSchemaByID(schemaID)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get cached schema: %w", err)
|
|
}
|
|
|
|
// Create appropriate decoder and infer RecordType based on format
|
|
switch cachedSchema.Format {
|
|
case schema.FormatAvro:
|
|
// Create Avro decoder and infer RecordType
|
|
decoder, err := schema.NewAvroDecoder(schemaStr)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to create Avro decoder: %w", err)
|
|
}
|
|
return decoder.InferRecordType()
|
|
|
|
case schema.FormatJSONSchema:
|
|
// Create JSON Schema decoder and infer RecordType
|
|
decoder, err := schema.NewJSONSchemaDecoder(schemaStr)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to create JSON Schema decoder: %w", err)
|
|
}
|
|
return decoder.InferRecordType()
|
|
|
|
case schema.FormatProtobuf:
|
|
// For Protobuf, we need the binary descriptor, not string
|
|
// This is a limitation - Protobuf schemas in Schema Registry are typically stored as binary descriptors
|
|
return nil, fmt.Errorf("Protobuf schema conversion from string not supported - requires binary descriptor")
|
|
|
|
default:
|
|
return nil, fmt.Errorf("unsupported schema format: %v", cachedSchema.Format)
|
|
}
|
|
}
|
|
|
|
// isSystemTopic checks if a topic is a Kafka system topic
|
|
func isSystemTopic(topicName string) bool {
|
|
systemTopics := []string{
|
|
"_schemas",
|
|
"__consumer_offsets",
|
|
"__transaction_state",
|
|
"_confluent-ksql-default__command_topic",
|
|
"_confluent-metrics",
|
|
}
|
|
|
|
for _, systemTopic := range systemTopics {
|
|
if topicName == systemTopic {
|
|
return true
|
|
}
|
|
}
|
|
|
|
// Check for topics starting with underscore (common system topic pattern)
|
|
return len(topicName) > 0 && topicName[0] == '_'
|
|
}
|
|
|
|
// getConnectionContextFromRequest extracts the connection context from the request context
|
|
func (h *Handler) getConnectionContextFromRequest(ctx context.Context) *ConnectionContext {
|
|
if connCtx, ok := ctx.Value(connContextKey).(*ConnectionContext); ok {
|
|
return connCtx
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// getOrCreatePartitionReader gets an existing partition reader or creates a new one
|
|
// This maintains persistent readers per connection that stream forward, eliminating
|
|
// repeated offset lookups and reducing broker CPU load
|
|
func (h *Handler) getOrCreatePartitionReader(ctx context.Context, connCtx *ConnectionContext, key TopicPartitionKey, startOffset int64) *partitionReader {
|
|
// Try to get existing reader
|
|
if val, ok := connCtx.partitionReaders.Load(key); ok {
|
|
return val.(*partitionReader)
|
|
}
|
|
|
|
// Create new reader
|
|
reader := newPartitionReader(ctx, h, connCtx, key.Topic, key.Partition, startOffset)
|
|
|
|
// Store it (handle race condition where another goroutine created one)
|
|
if actual, loaded := connCtx.partitionReaders.LoadOrStore(key, reader); loaded {
|
|
// Another goroutine created it first, close ours and use theirs
|
|
reader.close()
|
|
return actual.(*partitionReader)
|
|
}
|
|
|
|
return reader
|
|
}
|
|
|
|
// cleanupPartitionReaders closes all partition readers for a connection
|
|
// Called when connection is closing
|
|
func cleanupPartitionReaders(connCtx *ConnectionContext) {
|
|
if connCtx == nil {
|
|
return
|
|
}
|
|
|
|
connCtx.partitionReaders.Range(func(key, value interface{}) bool {
|
|
if reader, ok := value.(*partitionReader); ok {
|
|
reader.close()
|
|
}
|
|
return true // Continue iteration
|
|
})
|
|
|
|
glog.V(2).Infof("[%s] Cleaned up partition readers", connCtx.ConnectionID)
|
|
}
|