1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
|
package broker
import (
"context"
"fmt"
"strings"
"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) {
if !b.isLockOwner() {
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
}
ret := &mq_pb.ListTopicsResponse{}
// Scan the filer directory structure to find all topics
err = b.fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
// List all namespaces under /topics
stream, err := client.ListEntries(ctx, &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
}
// 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(ctx, &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(ctx, &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 topic
topic := &schema_pb.Topic{
Namespace: namespaceName,
Name: topicName,
}
ret.Topics = append(ret.Topics, topic)
}
}
}
return nil
})
if err != nil {
glog.V(0).Infof("list topics from filer: %v", err)
// Return empty response on error
return &mq_pb.ListTopicsResponse{}, nil
}
return ret, nil
}
// 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)),
RecordType: conf.RecordType,
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()
}
|