package broker import ( "context" "fmt" "strings" "time" "github.com/seaweedfs/seaweedfs/weed/filer" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "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" ) // LookupTopicBrokers returns the brokers that are serving the topic func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq_pb.LookupTopicBrokersRequest) (resp *mq_pb.LookupTopicBrokersResponse, err error) { if !b.isLockOwner() { proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.LookupTopicBrokers(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } t := topic.FromPbTopic(request.Topic) ret := &mq_pb.LookupTopicBrokersResponse{} conf := &mq_pb.ConfigureTopicResponse{} ret.Topic = request.Topic if conf, err = b.fca.ReadTopicConfFromFiler(t); err != nil { glog.V(0).Infof("lookup topic %s conf: %v", request.Topic, err) } else { err = b.ensureTopicActiveAssignments(t, conf) ret.BrokerPartitionAssignments = conf.BrokerPartitionAssignments } return ret, err } func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.ListTopicsRequest) (resp *mq_pb.ListTopicsResponse, err error) { glog.V(4).Infof("📋 ListTopics called, isLockOwner=%v", b.isLockOwner()) if !b.isLockOwner() { glog.V(4).Infof("📋 ListTopics proxying to lock owner: %s", b.lockAsBalancer.LockOwner()) proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.ListTopics(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } glog.V(4).Infof("📋 ListTopics starting - getting in-memory topics") ret := &mq_pb.ListTopicsResponse{} // First, get topics from in-memory state (includes unflushed topics) inMemoryTopics := b.localTopicManager.ListTopicsInMemory() glog.V(4).Infof("📋 ListTopics found %d in-memory topics", len(inMemoryTopics)) topicMap := make(map[string]*schema_pb.Topic) // Add in-memory topics to the result for _, topic := range inMemoryTopics { topicMap[topic.String()] = &schema_pb.Topic{ Namespace: topic.Namespace, Name: topic.Name, } } // Then, scan the filer directory structure to find persisted topics (fallback for topics not in memory) // Use a shorter timeout for filer scanning to ensure Metadata requests remain fast filerCtx, filerCancel := context.WithTimeout(ctx, 2*time.Second) defer filerCancel() glog.V(4).Infof("📋 ListTopics scanning filer for persisted topics (2s timeout)") err = b.fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { // List all namespaces under /topics glog.V(4).Infof("📋 ListTopics calling ListEntries for %s", filer.TopicsDir) stream, err := client.ListEntries(filerCtx, &filer_pb.ListEntriesRequest{ Directory: filer.TopicsDir, Limit: 1000, }) if err != nil { glog.V(0).Infof("list namespaces in %s: %v", filer.TopicsDir, err) return err } glog.V(4).Infof("📋 ListTopics got ListEntries stream, processing namespaces...") // Process each namespace for { resp, err := stream.Recv() if err != nil { if err.Error() == "EOF" { break } return err } if !resp.Entry.IsDirectory { continue } namespaceName := resp.Entry.Name namespacePath := fmt.Sprintf("%s/%s", filer.TopicsDir, namespaceName) // List all topics in this namespace topicStream, err := client.ListEntries(filerCtx, &filer_pb.ListEntriesRequest{ Directory: namespacePath, Limit: 1000, }) if err != nil { glog.V(0).Infof("list topics in namespace %s: %v", namespacePath, err) continue } // Process each topic in the namespace for { topicResp, err := topicStream.Recv() if err != nil { if err.Error() == "EOF" { break } glog.V(0).Infof("error reading topic stream in namespace %s: %v", namespaceName, err) break } if !topicResp.Entry.IsDirectory { continue } topicName := topicResp.Entry.Name // Check if topic.conf exists topicPath := fmt.Sprintf("%s/%s", namespacePath, topicName) confResp, err := client.LookupDirectoryEntry(filerCtx, &filer_pb.LookupDirectoryEntryRequest{ Directory: topicPath, Name: filer.TopicConfFile, }) if err != nil { glog.V(0).Infof("lookup topic.conf in %s: %v", topicPath, err) continue } if confResp.Entry != nil { // This is a valid persisted topic - add to map if not already present topicKey := fmt.Sprintf("%s.%s", namespaceName, topicName) if _, exists := topicMap[topicKey]; !exists { topicMap[topicKey] = &schema_pb.Topic{ Namespace: namespaceName, Name: topicName, } } } } } return nil }) // Convert map to slice for response (combines in-memory and persisted topics) for _, topic := range topicMap { ret.Topics = append(ret.Topics, topic) } if err != nil { glog.V(0).Infof("📋 ListTopics: filer scan failed: %v (returning %d in-memory topics)", err, len(inMemoryTopics)) // Still return in-memory topics even if filer fails } else { glog.V(4).Infof("📋 ListTopics completed successfully: %d total topics (in-memory + persisted)", len(ret.Topics)) } return ret, nil } // TopicExists checks if a topic exists in memory or filer // Caches both positive and negative results to reduce filer load func (b *MessageQueueBroker) TopicExists(ctx context.Context, request *mq_pb.TopicExistsRequest) (*mq_pb.TopicExistsResponse, error) { if !b.isLockOwner() { var resp *mq_pb.TopicExistsResponse var err error proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.TopicExists(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } if request.Topic == nil { return &mq_pb.TopicExistsResponse{Exists: false}, nil } // Convert schema_pb.Topic to topic.Topic topicObj := topic.Topic{ Namespace: request.Topic.Namespace, Name: request.Topic.Name, } topicKey := topicObj.String() // First check in-memory state (includes unflushed topics) if b.localTopicManager.TopicExistsInMemory(topicObj) { return &mq_pb.TopicExistsResponse{Exists: true}, nil } // Check cache for filer lookup results (both positive and negative) b.topicExistsCacheMu.RLock() if entry, found := b.topicExistsCache[topicKey]; found { if time.Now().Before(entry.expiresAt) { b.topicExistsCacheMu.RUnlock() glog.V(4).Infof("TopicExists cache HIT for %s: %v", topicKey, entry.exists) return &mq_pb.TopicExistsResponse{Exists: entry.exists}, nil } } b.topicExistsCacheMu.RUnlock() // Cache miss or expired - query filer for persisted topics glog.V(4).Infof("TopicExists cache MISS for %s, querying filer", topicKey) exists := false err := b.fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { topicPath := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, request.Topic.Namespace, request.Topic.Name) confResp, err := client.LookupDirectoryEntry(ctx, &filer_pb.LookupDirectoryEntryRequest{ Directory: topicPath, Name: filer.TopicConfFile, }) if err == nil && confResp.Entry != nil { exists = true } return nil // Don't propagate error, just check existence }) if err != nil { glog.V(0).Infof("check topic existence in filer: %v", err) // Don't cache errors - return false and let next check retry return &mq_pb.TopicExistsResponse{Exists: false}, nil } // Update cache with result (both positive and negative) b.topicExistsCacheMu.Lock() b.topicExistsCache[topicKey] = &topicExistsCacheEntry{ exists: exists, expiresAt: time.Now().Add(b.topicExistsCacheTTL), } b.topicExistsCacheMu.Unlock() glog.V(4).Infof("TopicExists cached result for %s: %v", topicKey, exists) return &mq_pb.TopicExistsResponse{Exists: exists}, nil } // invalidateTopicExistsCache removes a topic from the cache // Should be called when a topic is created or deleted func (b *MessageQueueBroker) invalidateTopicExistsCache(t topic.Topic) { topicKey := t.String() b.topicExistsCacheMu.Lock() delete(b.topicExistsCache, topicKey) b.topicExistsCacheMu.Unlock() glog.V(4).Infof("Invalidated TopicExists cache for %s", topicKey) } // GetTopicConfiguration returns the complete configuration of a topic including schema and partition assignments func (b *MessageQueueBroker) GetTopicConfiguration(ctx context.Context, request *mq_pb.GetTopicConfigurationRequest) (resp *mq_pb.GetTopicConfigurationResponse, err error) { if !b.isLockOwner() { proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.GetTopicConfiguration(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } t := topic.FromPbTopic(request.Topic) var conf *mq_pb.ConfigureTopicResponse var createdAtNs, modifiedAtNs int64 if conf, createdAtNs, modifiedAtNs, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil { glog.V(0).Infof("get topic configuration %s: %v", request.Topic, err) return nil, fmt.Errorf("failed to read topic configuration: %w", err) } // Ensure topic assignments are active err = b.ensureTopicActiveAssignments(t, conf) if err != nil { glog.V(0).Infof("ensure topic active assignments %s: %v", request.Topic, err) return nil, fmt.Errorf("failed to ensure topic assignments: %w", err) } // Build the response with complete configuration including metadata ret := &mq_pb.GetTopicConfigurationResponse{ Topic: request.Topic, PartitionCount: int32(len(conf.BrokerPartitionAssignments)), MessageRecordType: conf.MessageRecordType, KeyColumns: conf.KeyColumns, BrokerPartitionAssignments: conf.BrokerPartitionAssignments, CreatedAtNs: createdAtNs, LastUpdatedNs: modifiedAtNs, Retention: conf.Retention, } return ret, nil } // GetTopicPublishers returns the active publishers for a topic func (b *MessageQueueBroker) GetTopicPublishers(ctx context.Context, request *mq_pb.GetTopicPublishersRequest) (resp *mq_pb.GetTopicPublishersResponse, err error) { if !b.isLockOwner() { proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.GetTopicPublishers(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } t := topic.FromPbTopic(request.Topic) var publishers []*mq_pb.TopicPublisher // Get topic configuration to find partition assignments var conf *mq_pb.ConfigureTopicResponse if conf, _, _, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil { glog.V(0).Infof("get topic configuration for publishers %s: %v", request.Topic, err) return nil, fmt.Errorf("failed to read topic configuration: %w", err) } // Collect publishers from each partition that is hosted on this broker for _, assignment := range conf.BrokerPartitionAssignments { // Only collect from partitions where this broker is the leader if assignment.LeaderBroker == b.option.BrokerAddress().String() { partition := topic.FromPbPartition(assignment.Partition) if localPartition := b.localTopicManager.GetLocalPartition(t, partition); localPartition != nil { // Get publisher information from local partition localPartition.Publishers.ForEachPublisher(func(clientName string, publisher *topic.LocalPublisher) { connectTimeNs, lastSeenTimeNs := publisher.GetTimestamps() lastPublishedOffset, lastAckedOffset := publisher.GetOffsets() publishers = append(publishers, &mq_pb.TopicPublisher{ PublisherName: clientName, ClientId: clientName, // For now, client name is used as client ID Partition: assignment.Partition, ConnectTimeNs: connectTimeNs, LastSeenTimeNs: lastSeenTimeNs, Broker: assignment.LeaderBroker, IsActive: true, LastPublishedOffset: lastPublishedOffset, LastAckedOffset: lastAckedOffset, }) }) } } } return &mq_pb.GetTopicPublishersResponse{ Publishers: publishers, }, nil } // GetTopicSubscribers returns the active subscribers for a topic func (b *MessageQueueBroker) GetTopicSubscribers(ctx context.Context, request *mq_pb.GetTopicSubscribersRequest) (resp *mq_pb.GetTopicSubscribersResponse, err error) { if !b.isLockOwner() { proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.GetTopicSubscribers(ctx, request) return nil }) if proxyErr != nil { return nil, proxyErr } return resp, err } t := topic.FromPbTopic(request.Topic) var subscribers []*mq_pb.TopicSubscriber // Get topic configuration to find partition assignments var conf *mq_pb.ConfigureTopicResponse if conf, _, _, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil { glog.V(0).Infof("get topic configuration for subscribers %s: %v", request.Topic, err) return nil, fmt.Errorf("failed to read topic configuration: %w", err) } // Collect subscribers from each partition that is hosted on this broker for _, assignment := range conf.BrokerPartitionAssignments { // Only collect from partitions where this broker is the leader if assignment.LeaderBroker == b.option.BrokerAddress().String() { partition := topic.FromPbPartition(assignment.Partition) if localPartition := b.localTopicManager.GetLocalPartition(t, partition); localPartition != nil { // Get subscriber information from local partition localPartition.Subscribers.ForEachSubscriber(func(clientName string, subscriber *topic.LocalSubscriber) { // Parse client name to extract consumer group and consumer ID // Format is typically: "consumerGroup/consumerID" consumerGroup := "default" consumerID := clientName if idx := strings.Index(clientName, "/"); idx != -1 { consumerGroup = clientName[:idx] consumerID = clientName[idx+1:] } connectTimeNs, lastSeenTimeNs := subscriber.GetTimestamps() lastReceivedOffset, lastAckedOffset := subscriber.GetOffsets() subscribers = append(subscribers, &mq_pb.TopicSubscriber{ ConsumerGroup: consumerGroup, ConsumerId: consumerID, ClientId: clientName, // Full client name as client ID Partition: assignment.Partition, ConnectTimeNs: connectTimeNs, LastSeenTimeNs: lastSeenTimeNs, Broker: assignment.LeaderBroker, IsActive: true, CurrentOffset: lastAckedOffset, // for compatibility LastReceivedOffset: lastReceivedOffset, }) }) } } } return &mq_pb.GetTopicSubscribersResponse{ Subscribers: subscribers, }, nil } func (b *MessageQueueBroker) isLockOwner() bool { return b.lockAsBalancer.LockOwner() == b.option.BrokerAddress().String() }