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.
 
 
 
 
 
 

315 lines
9.6 KiB

package integration
import (
"context"
"fmt"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"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"
)
// CreateTopic creates a new topic in both Kafka registry and SeaweedMQ
func (h *SeaweedMQHandler) CreateTopic(name string, partitions int32) error {
return h.CreateTopicWithSchema(name, partitions, nil)
}
// CreateTopicWithSchema creates a topic with optional value schema
func (h *SeaweedMQHandler) CreateTopicWithSchema(name string, partitions int32, recordType *schema_pb.RecordType) error {
return h.CreateTopicWithSchemas(name, partitions, nil, recordType)
}
// CreateTopicWithSchemas creates a topic with optional key and value schemas
func (h *SeaweedMQHandler) CreateTopicWithSchemas(name string, partitions int32, keyRecordType *schema_pb.RecordType, valueRecordType *schema_pb.RecordType) error {
// Check if topic already exists in filer
if h.checkTopicInFiler(name) {
return fmt.Errorf("topic %s already exists", name)
}
// Create SeaweedMQ topic reference
seaweedTopic := &schema_pb.Topic{
Namespace: "kafka",
Name: name,
}
// Configure topic with SeaweedMQ broker via gRPC
if len(h.brokerAddresses) > 0 {
brokerAddress := h.brokerAddresses[0] // Use first available broker
glog.V(1).Infof("Configuring topic %s with broker %s", name, brokerAddress)
// Load security configuration for broker connection
util.LoadSecurityConfiguration()
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.mq")
err := pb.WithBrokerGrpcClient(false, brokerAddress, grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
// Convert dual schemas to flat schema format
var flatSchema *schema_pb.RecordType
var keyColumns []string
if keyRecordType != nil || valueRecordType != nil {
flatSchema, keyColumns = schema.CombineFlatSchemaFromKeyValue(keyRecordType, valueRecordType)
}
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: seaweedTopic,
PartitionCount: partitions,
MessageRecordType: flatSchema,
KeyColumns: keyColumns,
})
if err != nil {
return fmt.Errorf("configure topic with broker: %w", err)
}
glog.V(1).Infof("successfully configured topic %s with broker", name)
return nil
})
if err != nil {
return fmt.Errorf("failed to configure topic %s with broker %s: %w", name, brokerAddress, err)
}
} else {
glog.Warningf("No brokers available - creating topic %s in gateway memory only (testing mode)", name)
}
// Topic is now stored in filer only via SeaweedMQ broker
// No need to create in-memory topic info structure
// Offset management now handled directly by SMQ broker - no initialization needed
// Invalidate cache after successful topic creation
h.InvalidateTopicExistsCache(name)
glog.V(1).Infof("Topic %s created successfully with %d partitions", name, partitions)
return nil
}
// CreateTopicWithRecordType creates a topic with flat schema and key columns
func (h *SeaweedMQHandler) CreateTopicWithRecordType(name string, partitions int32, flatSchema *schema_pb.RecordType, keyColumns []string) error {
// Check if topic already exists in filer
if h.checkTopicInFiler(name) {
return fmt.Errorf("topic %s already exists", name)
}
// Create SeaweedMQ topic reference
seaweedTopic := &schema_pb.Topic{
Namespace: "kafka",
Name: name,
}
// Configure topic with SeaweedMQ broker via gRPC
if len(h.brokerAddresses) > 0 {
brokerAddress := h.brokerAddresses[0] // Use first available broker
glog.V(1).Infof("Configuring topic %s with broker %s", name, brokerAddress)
// Load security configuration for broker connection
util.LoadSecurityConfiguration()
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.mq")
err := pb.WithBrokerGrpcClient(false, brokerAddress, grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: seaweedTopic,
PartitionCount: partitions,
MessageRecordType: flatSchema,
KeyColumns: keyColumns,
})
if err != nil {
return fmt.Errorf("failed to configure topic: %w", err)
}
glog.V(1).Infof("successfully configured topic %s with broker", name)
return nil
})
if err != nil {
return err
}
} else {
glog.Warningf("No broker addresses configured, topic %s not created in SeaweedMQ", name)
}
// Topic is now stored in filer only via SeaweedMQ broker
// No need to create in-memory topic info structure
glog.V(1).Infof("Topic %s created successfully with %d partitions using flat schema", name, partitions)
return nil
}
// DeleteTopic removes a topic from both Kafka registry and SeaweedMQ
func (h *SeaweedMQHandler) DeleteTopic(name string) error {
// Check if topic exists in filer
if !h.checkTopicInFiler(name) {
return fmt.Errorf("topic %s does not exist", name)
}
// Get topic info to determine partition count for cleanup
topicInfo, exists := h.GetTopicInfo(name)
if !exists {
return fmt.Errorf("topic %s info not found", name)
}
// Close all publisher sessions for this topic
for partitionID := int32(0); partitionID < topicInfo.Partitions; partitionID++ {
if h.brokerClient != nil {
h.brokerClient.ClosePublisher(name, partitionID)
}
}
// Topic removal from filer would be handled by SeaweedMQ broker
// No in-memory cache to clean up
// Offset management handled by SMQ broker - no cleanup needed
return nil
}
// TopicExists checks if a topic exists in SeaweedMQ broker (includes in-memory topics)
// Uses a 5-second cache to reduce broker queries
func (h *SeaweedMQHandler) TopicExists(name string) bool {
// Check cache first
h.topicExistsCacheMu.RLock()
if entry, found := h.topicExistsCache[name]; found {
if time.Now().Before(entry.expiresAt) {
h.topicExistsCacheMu.RUnlock()
return entry.exists
}
}
h.topicExistsCacheMu.RUnlock()
// Cache miss or expired - query broker
var exists bool
// Check via SeaweedMQ broker (includes in-memory topics)
if h.brokerClient != nil {
var err error
exists, err = h.brokerClient.TopicExists(name)
if err != nil {
// Don't cache errors
return false
}
} else {
// Return false if broker is unavailable
return false
}
// Update cache
h.topicExistsCacheMu.Lock()
h.topicExistsCache[name] = &topicExistsCacheEntry{
exists: exists,
expiresAt: time.Now().Add(h.topicExistsCacheTTL),
}
h.topicExistsCacheMu.Unlock()
return exists
}
// InvalidateTopicExistsCache removes a topic from the existence cache
// Should be called after creating or deleting a topic
func (h *SeaweedMQHandler) InvalidateTopicExistsCache(name string) {
h.topicExistsCacheMu.Lock()
delete(h.topicExistsCache, name)
h.topicExistsCacheMu.Unlock()
}
// GetTopicInfo returns information about a topic from broker
func (h *SeaweedMQHandler) GetTopicInfo(name string) (*KafkaTopicInfo, bool) {
// Get topic configuration from broker
if h.brokerClient != nil {
config, err := h.brokerClient.GetTopicConfiguration(name)
if err == nil && config != nil {
topicInfo := &KafkaTopicInfo{
Name: name,
Partitions: config.PartitionCount,
CreatedAt: config.CreatedAtNs,
}
return topicInfo, true
}
glog.V(2).Infof("Failed to get topic configuration for %s from broker: %v", name, err)
}
// Fallback: check if topic exists in filer (for backward compatibility)
if !h.checkTopicInFiler(name) {
return nil, false
}
// Return default info if broker query failed but topic exists in filer
topicInfo := &KafkaTopicInfo{
Name: name,
Partitions: 1, // Default to 1 partition if broker query failed
CreatedAt: 0,
}
return topicInfo, true
}
// ListTopics returns all topic names from SeaweedMQ broker (includes in-memory topics)
func (h *SeaweedMQHandler) ListTopics() []string {
// Get topics from SeaweedMQ broker (includes in-memory topics)
if h.brokerClient != nil {
topics, err := h.brokerClient.ListTopics()
if err == nil {
return topics
}
}
// Return empty list if broker is unavailable
return []string{}
}
// checkTopicInFiler checks if a topic exists in the filer
func (h *SeaweedMQHandler) checkTopicInFiler(topicName string) bool {
if h.filerClientAccessor == nil {
return false
}
var exists bool
h.filerClientAccessor.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Directory: "/topics/kafka",
Name: topicName,
}
_, err := client.LookupDirectoryEntry(context.Background(), request)
exists = (err == nil)
return nil // Don't propagate error, just check existence
})
return exists
}
// listTopicsFromFiler lists all topics from the filer
func (h *SeaweedMQHandler) listTopicsFromFiler() []string {
if h.filerClientAccessor == nil {
return []string{}
}
var topics []string
h.filerClientAccessor.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.ListEntriesRequest{
Directory: "/topics/kafka",
}
stream, err := client.ListEntries(context.Background(), request)
if err != nil {
return nil // Don't propagate error, just return empty list
}
for {
resp, err := stream.Recv()
if err != nil {
break // End of stream or error
}
if resp.Entry != nil && resp.Entry.IsDirectory {
topics = append(topics, resp.Entry.Name)
} else if resp.Entry != nil {
}
}
return nil
})
return topics
}