mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
fix listing topics
This commit is contained in:
parent
3cf9b8d621
commit
1492bf7552
|
@ -3,6 +3,7 @@ package balancer
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -30,7 +31,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
|
||||||
currentTopicPartitions := bs.Stats.Items()
|
currentTopicPartitions := bs.Stats.Items()
|
||||||
for _, topicPartitionStats := range stats.Stats {
|
for _, topicPartitionStats := range stats.Stats {
|
||||||
tps := &TopicPartitionStats{
|
tps := &TopicPartitionStats{
|
||||||
TopicPartition: TopicPartition{
|
TopicPartition: topic.TopicPartition{
|
||||||
Namespace: topicPartitionStats.Topic.Namespace,
|
Namespace: topicPartitionStats.Topic.Namespace,
|
||||||
Topic: topicPartitionStats.Topic.Name,
|
Topic: topicPartitionStats.Topic.Name,
|
||||||
RangeStart: topicPartitionStats.Partition.RangeStart,
|
RangeStart: topicPartitionStats.Partition.RangeStart,
|
||||||
|
@ -52,20 +53,28 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition) {
|
||||||
|
tps := &TopicPartitionStats{
|
||||||
|
TopicPartition: topic.TopicPartition{
|
||||||
|
Namespace: t.Namespace,
|
||||||
|
Topic: t.Name,
|
||||||
|
RangeStart: partition.RangeStart,
|
||||||
|
RangeStop: partition.RangeStop,
|
||||||
|
},
|
||||||
|
ConsumerCount: 0,
|
||||||
|
IsLeader: true,
|
||||||
|
}
|
||||||
|
key := tps.TopicPartition.String()
|
||||||
|
bs.Stats.Set(key, tps)
|
||||||
|
}
|
||||||
|
|
||||||
func (bs *BrokerStats) String() string {
|
func (bs *BrokerStats) String() string {
|
||||||
return fmt.Sprintf("BrokerStats{TopicPartitionCount:%d, ConsumerCount:%d, CpuUsagePercent:%d, Stats:%+v}",
|
return fmt.Sprintf("BrokerStats{TopicPartitionCount:%d, ConsumerCount:%d, CpuUsagePercent:%d, Stats:%+v}",
|
||||||
bs.TopicPartitionCount, bs.ConsumerCount, bs.CpuUsagePercent, bs.Stats.Items())
|
bs.TopicPartitionCount, bs.ConsumerCount, bs.CpuUsagePercent, bs.Stats.Items())
|
||||||
}
|
}
|
||||||
|
|
||||||
type TopicPartition struct {
|
|
||||||
Namespace string
|
|
||||||
Topic string
|
|
||||||
RangeStart int32
|
|
||||||
RangeStop int32
|
|
||||||
}
|
|
||||||
|
|
||||||
type TopicPartitionStats struct {
|
type TopicPartitionStats struct {
|
||||||
TopicPartition
|
topic.TopicPartition
|
||||||
ConsumerCount int32
|
ConsumerCount int32
|
||||||
IsLeader bool
|
IsLeader bool
|
||||||
}
|
}
|
||||||
|
@ -81,7 +90,3 @@ func NewBrokerStats() *BrokerStats {
|
||||||
Stats: cmap.New[*TopicPartitionStats](),
|
Stats: cmap.New[*TopicPartitionStats](),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (tp *TopicPartition) String() string {
|
|
||||||
return fmt.Sprintf("%v.%v-%04d-%04d", tp.Namespace, tp.Topic, tp.RangeStart, tp.RangeStop)
|
|
||||||
}
|
|
||||||
|
|
|
@ -24,11 +24,8 @@ func (b *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish b
|
||||||
RangeStop: topicPartitionStat.RangeStop,
|
RangeStop: topicPartitionStat.RangeStop,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
if topicPartitionStat.IsLeader {
|
// TODO fix follower setting
|
||||||
assignment.LeaderBroker = broker
|
assignment.LeaderBroker = broker
|
||||||
} else {
|
|
||||||
assignment.FollowerBrokers = append(assignment.FollowerBrokers, broker)
|
|
||||||
}
|
|
||||||
assignments = append(assignments, assignment)
|
assignments = append(assignments, assignment)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessagin
|
||||||
if initMessage != nil {
|
if initMessage != nil {
|
||||||
broker.Balancer.Brokers.Set(initMessage.Broker, brokerStats)
|
broker.Balancer.Brokers.Set(initMessage.Broker, brokerStats)
|
||||||
} else {
|
} else {
|
||||||
|
// TODO fix this
|
||||||
return status.Errorf(codes.InvalidArgument, "balancer init message is empty")
|
return status.Errorf(codes.InvalidArgument, "balancer init message is empty")
|
||||||
}
|
}
|
||||||
defer func() {
|
defer func() {
|
||||||
|
@ -42,8 +43,8 @@ func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessagin
|
||||||
if receivedStats := req.GetStats(); receivedStats != nil {
|
if receivedStats := req.GetStats(); receivedStats != nil {
|
||||||
brokerStats.UpdateStats(receivedStats)
|
brokerStats.UpdateStats(receivedStats)
|
||||||
|
|
||||||
glog.V(3).Infof("broker %s stats: %+v", initMessage.Broker, brokerStats)
|
glog.V(4).Infof("broker %s stats: %+v", initMessage.Broker, brokerStats)
|
||||||
glog.V(3).Infof("received stats: %+v", receivedStats)
|
glog.V(4).Infof("received stats: %+v", receivedStats)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -2,6 +2,8 @@ package broker
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/balancer"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
@ -29,12 +31,22 @@ func (broker *MessageQueueBroker) CreateTopic(ctx context.Context, request *mq_p
|
||||||
ret.BrokerPartitionAssignments, err = broker.Balancer.LookupOrAllocateTopicPartitions(request.Topic, true, request.PartitionCount)
|
ret.BrokerPartitionAssignments, err = broker.Balancer.LookupOrAllocateTopicPartitions(request.Topic, true, request.PartitionCount)
|
||||||
|
|
||||||
for _, bpa := range ret.BrokerPartitionAssignments {
|
for _, bpa := range ret.BrokerPartitionAssignments {
|
||||||
|
// fmt.Printf("create topic %s on %s\n", request.Topic, bpa.LeaderBroker)
|
||||||
if doCreateErr := broker.withBrokerClient(false, pb.ServerAddress(bpa.LeaderBroker), func(client mq_pb.SeaweedMessagingClient) error {
|
if doCreateErr := broker.withBrokerClient(false, pb.ServerAddress(bpa.LeaderBroker), func(client mq_pb.SeaweedMessagingClient) error {
|
||||||
_, doCreateErr := client.DoCreateTopic(ctx, &mq_pb.DoCreateTopicRequest{
|
_, doCreateErr := client.DoCreateTopic(ctx, &mq_pb.DoCreateTopicRequest{
|
||||||
Topic: request.Topic,
|
Topic: request.Topic,
|
||||||
Partition: bpa.Partition,
|
Partition: bpa.Partition,
|
||||||
})
|
})
|
||||||
return doCreateErr
|
if doCreateErr != nil {
|
||||||
|
return fmt.Errorf("do create topic %s on %s: %v", request.Topic, bpa.LeaderBroker, doCreateErr)
|
||||||
|
}
|
||||||
|
brokerStats, found := broker.Balancer.Brokers.Get(bpa.LeaderBroker)
|
||||||
|
if !found {
|
||||||
|
brokerStats = balancer.NewBrokerStats()
|
||||||
|
broker.Balancer.Brokers.Set(bpa.LeaderBroker, brokerStats)
|
||||||
|
}
|
||||||
|
brokerStats.RegisterAssignment(request.Topic, bpa.Partition)
|
||||||
|
return nil
|
||||||
}); doCreateErr != nil {
|
}); doCreateErr != nil {
|
||||||
return nil, doCreateErr
|
return nil, doCreateErr
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,6 +2,7 @@ package broker
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
"google.golang.org/grpc/status"
|
"google.golang.org/grpc/status"
|
||||||
|
@ -62,7 +63,7 @@ func (broker *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb
|
||||||
}
|
}
|
||||||
|
|
||||||
ret := &mq_pb.ListTopicsResponse{}
|
ret := &mq_pb.ListTopicsResponse{}
|
||||||
knownTopics := make(map[*mq_pb.Topic]struct{})
|
knownTopics := make(map[string]struct{})
|
||||||
for brokerStatsItem := range broker.Balancer.Brokers.IterBuffered() {
|
for brokerStatsItem := range broker.Balancer.Brokers.IterBuffered() {
|
||||||
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
||||||
for topicPartitionStatsItem := range brokerStats.Stats.IterBuffered() {
|
for topicPartitionStatsItem := range brokerStats.Stats.IterBuffered() {
|
||||||
|
@ -71,9 +72,11 @@ func (broker *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb
|
||||||
Namespace: topicPartitionStat.TopicPartition.Namespace,
|
Namespace: topicPartitionStat.TopicPartition.Namespace,
|
||||||
Name: topicPartitionStat.TopicPartition.Topic,
|
Name: topicPartitionStat.TopicPartition.Topic,
|
||||||
}
|
}
|
||||||
if _, found := knownTopics[topic]; found {
|
topicKey := fmt.Sprintf("%s/%s", topic.Namespace, topic.Name)
|
||||||
|
if _, found := knownTopics[topicKey]; found {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
knownTopics[topicKey] = struct{}{}
|
||||||
ret.Topics = append(ret.Topics, topic)
|
ret.Topics = append(ret.Topics, topic)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -61,6 +61,7 @@ func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return fmt.Errorf("send stats message: %v", err)
|
return fmt.Errorf("send stats message: %v", err)
|
||||||
}
|
}
|
||||||
|
glog.V(4).Infof("sent stats: %+v", stats)
|
||||||
|
|
||||||
time.Sleep(time.Millisecond*5000 + time.Duration(rand.Intn(1000))*time.Millisecond)
|
time.Sleep(time.Millisecond*5000 + time.Duration(rand.Intn(1000))*time.Millisecond)
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,7 +63,13 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
|
||||||
}
|
}
|
||||||
manager.topics.IterCb(func(topic string, localTopic *LocalTopic) {
|
manager.topics.IterCb(func(topic string, localTopic *LocalTopic) {
|
||||||
for _, localPartition := range localTopic.Partitions {
|
for _, localPartition := range localTopic.Partitions {
|
||||||
stats.Stats[topic] = &mq_pb.TopicPartitionStats{
|
topicPartition := &TopicPartition{
|
||||||
|
Namespace: string(localTopic.Namespace),
|
||||||
|
Topic: localTopic.Name,
|
||||||
|
RangeStart: localPartition.RangeStart,
|
||||||
|
RangeStop: localPartition.RangeStop,
|
||||||
|
}
|
||||||
|
stats.Stats[topicPartition.String()] = &mq_pb.TopicPartitionStats{
|
||||||
Topic: &mq_pb.Topic{
|
Topic: &mq_pb.Topic{
|
||||||
Namespace: string(localTopic.Namespace),
|
Namespace: string(localTopic.Namespace),
|
||||||
Name: localTopic.Name,
|
Name: localTopic.Name,
|
||||||
|
@ -75,6 +81,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
|
||||||
},
|
},
|
||||||
ConsumerCount: localPartition.ConsumerCount,
|
ConsumerCount: localPartition.ConsumerCount,
|
||||||
}
|
}
|
||||||
|
// fmt.Printf("collect topic %+v partition %+v\n", topicPartition, localPartition.Partition)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
14
weed/mq/topic/topic_partition.go
Normal file
14
weed/mq/topic/topic_partition.go
Normal file
|
@ -0,0 +1,14 @@
|
||||||
|
package topic
|
||||||
|
|
||||||
|
import "fmt"
|
||||||
|
|
||||||
|
type TopicPartition struct {
|
||||||
|
Namespace string
|
||||||
|
Topic string
|
||||||
|
RangeStart int32
|
||||||
|
RangeStop int32
|
||||||
|
}
|
||||||
|
|
||||||
|
func (tp *TopicPartition) String() string {
|
||||||
|
return fmt.Sprintf("%v.%v-%04d-%04d", tp.Namespace, tp.Topic, tp.RangeStart, tp.RangeStop)
|
||||||
|
}
|
Loading…
Reference in a new issue