Browse Source
Merge accumulated changes related to message queue (#5098)
Merge accumulated changes related to message queue (#5098)
* balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * tracking topic=>broker * merge * commentpull/5100/head
Chris Lu
1 year ago
committed by
GitHub
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
58 changed files with 3747 additions and 2646 deletions
-
1.gitignore
-
27docker/compose/local-mq-test.yml
-
5weed/cluster/lock_client.go
-
225weed/mq/broker/broker_grpc_admin.go
-
31weed/mq/broker/broker_grpc_balance.go
-
107weed/mq/broker/broker_grpc_configure.go
-
72weed/mq/broker/broker_grpc_create.go
-
28weed/mq/broker/broker_grpc_lookup.go
-
132weed/mq/broker/broker_grpc_pub.go
-
26weed/mq/broker/broker_grpc_pub_balancer.go
-
56weed/mq/broker/broker_grpc_sub.go
-
77weed/mq/broker/broker_grpc_sub_coordinator.go
-
28weed/mq/broker/broker_grpc_topic_partition_control.go
-
89weed/mq/broker/broker_segment_serde.go
-
51weed/mq/broker/broker_server.go
-
30weed/mq/broker/broker_stats.go
-
14weed/mq/client/cmd/weed_pub/publisher.go
-
18weed/mq/client/cmd/weed_sub/subscriber.go
-
73weed/mq/client/pub_client/connect.go
-
114weed/mq/client/pub_client/lookup.go
-
4weed/mq/client/pub_client/publish.go
-
24weed/mq/client/pub_client/publisher.go
-
4weed/mq/client/sub_client/process.go
-
17weed/mq/client/sub_client/subscribe.go
-
16weed/mq/client/sub_client/subscriber.go
-
92weed/mq/coordinator/consumer_group.go
-
36weed/mq/coordinator/coordinator.go
-
4weed/mq/pub_balancer/allocate.go
-
2weed/mq/pub_balancer/allocate_test.go
-
73weed/mq/pub_balancer/balance.go
-
58weed/mq/pub_balancer/balance_action.go
-
43weed/mq/pub_balancer/balance_action_split.go
-
52weed/mq/pub_balancer/balance_brokers.go
-
75weed/mq/pub_balancer/balance_brokers_test.go
-
83weed/mq/pub_balancer/balancer.go
-
77weed/mq/pub_balancer/broker_stats.go
-
19weed/mq/pub_balancer/lookup.go
-
50weed/mq/pub_balancer/partition_list_broker.go
-
127weed/mq/pub_balancer/repair.go
-
97weed/mq/pub_balancer/repair_test.go
-
41weed/mq/sub_coordinator/consumer_group.go
-
86weed/mq/sub_coordinator/coordinator.go
-
119weed/mq/sub_coordinator/partition_consumer_mapping.go
-
312weed/mq/sub_coordinator/partition_consumer_mapping_test.go
-
32weed/mq/sub_coordinator/partition_list.go
-
29weed/mq/topic/local_manager.go
-
49weed/mq/topic/local_partition.go
-
52weed/mq/topic/local_partition_publishers.go
-
49weed/mq/topic/local_partition_subscribers.go
-
58weed/mq/topic/local_topic.go
-
17weed/mq/topic/partition.go
-
48weed/mq/topic/topic.go
-
148weed/pb/mq.proto
-
2688weed/pb/mq_pb/mq.pb.go
-
416weed/pb/mq_pb/mq_grpc.pb.go
-
46weed/shell/command_mq_balance.go
-
16weed/shell/command_mq_topic_configure.go
-
4weed/shell/command_mq_topic_list.go
@ -0,0 +1,27 @@ |
|||||
|
version: '3.9' |
||||
|
|
||||
|
services: |
||||
|
server: |
||||
|
image: chrislusf/seaweedfs:local |
||||
|
ports: |
||||
|
- 9333:9333 |
||||
|
- 19333:19333 |
||||
|
- 8888:8888 |
||||
|
- 18888:18888 |
||||
|
command: "server -ip=server -filer -volume.max=0 -master.volumeSizeLimitMB=8 -volume.preStopSeconds=1" |
||||
|
healthcheck: |
||||
|
test: curl -f http://localhost:8888/healthz |
||||
|
mq_broker: |
||||
|
image: chrislusf/seaweedfs:local |
||||
|
ports: |
||||
|
- 17777:17777 |
||||
|
command: "mq.broker -master=server:9333 -ip=mq_broker" |
||||
|
depends_on: |
||||
|
server: |
||||
|
condition: service_healthy |
||||
|
mq_client: |
||||
|
image: chrislusf/seaweedfs:local |
||||
|
# run a custom command instead of entrypoint |
||||
|
command: "ls -al" |
||||
|
depends_on: |
||||
|
- mq_broker |
@ -0,0 +1,31 @@ |
|||||
|
package broker |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"google.golang.org/grpc/codes" |
||||
|
"google.golang.org/grpc/status" |
||||
|
) |
||||
|
|
||||
|
func (b *MessageQueueBroker) BalanceTopics(ctx context.Context, request *mq_pb.BalanceTopicsRequest) (resp *mq_pb.BalanceTopicsResponse, err error) { |
||||
|
if b.currentBalancer == "" { |
||||
|
return nil, status.Errorf(codes.Unavailable, "no balancer") |
||||
|
} |
||||
|
if !b.lockAsBalancer.IsLocked() { |
||||
|
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
resp, err = client.BalanceTopics(ctx, request) |
||||
|
return nil |
||||
|
}) |
||||
|
if proxyErr != nil { |
||||
|
return nil, proxyErr |
||||
|
} |
||||
|
return resp, err |
||||
|
} |
||||
|
|
||||
|
ret := &mq_pb.BalanceTopicsResponse{} |
||||
|
|
||||
|
actions := b.Balancer.BalancePublishers() |
||||
|
err = b.Balancer.ExecuteBalanceAction(actions, b.grpcDialOption) |
||||
|
|
||||
|
return ret, err |
||||
|
} |
@ -0,0 +1,107 @@ |
|||||
|
package broker |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"google.golang.org/grpc/codes" |
||||
|
"google.golang.org/grpc/status" |
||||
|
) |
||||
|
|
||||
|
// ConfigureTopic Runs on any broker, but proxied to the balancer if not the balancer
|
||||
|
// It generates an assignments based on existing allocations,
|
||||
|
// and then assign the partitions to the brokers.
|
||||
|
func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.ConfigureTopicRequest) (resp *mq_pb.ConfigureTopicResponse, err error) { |
||||
|
if b.currentBalancer == "" { |
||||
|
return nil, status.Errorf(codes.Unavailable, "no balancer") |
||||
|
} |
||||
|
if !b.lockAsBalancer.IsLocked() { |
||||
|
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
resp, err = client.ConfigureTopic(ctx, request) |
||||
|
return nil |
||||
|
}) |
||||
|
if proxyErr != nil { |
||||
|
return nil, proxyErr |
||||
|
} |
||||
|
return resp, err |
||||
|
} |
||||
|
|
||||
|
ret := &mq_pb.ConfigureTopicResponse{} |
||||
|
ret.BrokerPartitionAssignments, err = b.Balancer.LookupOrAllocateTopicPartitions(request.Topic, true, request.PartitionCount) |
||||
|
|
||||
|
for _, bpa := range ret.BrokerPartitionAssignments { |
||||
|
// fmt.Printf("create topic %s on %s\n", request.Topic, bpa.LeaderBroker)
|
||||
|
if doCreateErr := b.withBrokerClient(false, pb.ServerAddress(bpa.LeaderBroker), func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
_, doCreateErr := client.AssignTopicPartitions(ctx, &mq_pb.AssignTopicPartitionsRequest{ |
||||
|
Topic: request.Topic, |
||||
|
BrokerPartitionAssignments: []*mq_pb.BrokerPartitionAssignment{ |
||||
|
{ |
||||
|
Partition: bpa.Partition, |
||||
|
}, |
||||
|
}, |
||||
|
IsLeader: true, |
||||
|
IsDraining: false, |
||||
|
}) |
||||
|
if doCreateErr != nil { |
||||
|
return fmt.Errorf("do create topic %s on %s: %v", request.Topic, bpa.LeaderBroker, doCreateErr) |
||||
|
} |
||||
|
brokerStats, found := b.Balancer.Brokers.Get(bpa.LeaderBroker) |
||||
|
if !found { |
||||
|
brokerStats = pub_balancer.NewBrokerStats() |
||||
|
if !b.Balancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) { |
||||
|
brokerStats, _ = b.Balancer.Brokers.Get(bpa.LeaderBroker) |
||||
|
} |
||||
|
} |
||||
|
brokerStats.RegisterAssignment(request.Topic, bpa.Partition) |
||||
|
return nil |
||||
|
}); doCreateErr != nil { |
||||
|
return nil, doCreateErr |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// TODO revert if some error happens in the middle of the assignments
|
||||
|
|
||||
|
return ret, err |
||||
|
} |
||||
|
|
||||
|
// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment
|
||||
|
func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) { |
||||
|
ret := &mq_pb.AssignTopicPartitionsResponse{} |
||||
|
self := pb.ServerAddress(fmt.Sprintf("%s:%d", b.option.Ip, b.option.Port)) |
||||
|
|
||||
|
// drain existing topic partition subscriptions
|
||||
|
for _, brokerPartition := range request.BrokerPartitionAssignments { |
||||
|
localPartition := topic.FromPbBrokerPartitionAssignment(self, brokerPartition) |
||||
|
if request.IsDraining { |
||||
|
// TODO drain existing topic partition subscriptions
|
||||
|
|
||||
|
b.localTopicManager.RemoveTopicPartition( |
||||
|
topic.FromPbTopic(request.Topic), |
||||
|
localPartition.Partition) |
||||
|
} else { |
||||
|
b.localTopicManager.AddTopicPartition( |
||||
|
topic.FromPbTopic(request.Topic), |
||||
|
localPartition) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// if is leader, notify the followers to drain existing topic partition subscriptions
|
||||
|
if request.IsLeader { |
||||
|
for _, brokerPartition := range request.BrokerPartitionAssignments { |
||||
|
for _, follower := range brokerPartition.FollowerBrokers { |
||||
|
err := pb.WithBrokerGrpcClient(false, follower, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
_, err := client.AssignTopicPartitions(context.Background(), request) |
||||
|
return err |
||||
|
}) |
||||
|
if err != nil { |
||||
|
return ret, err |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return ret, nil |
||||
|
} |
@ -1,72 +0,0 @@ |
|||||
package broker |
|
||||
|
|
||||
import ( |
|
||||
"context" |
|
||||
"fmt" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/balancer" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
|
||||
"google.golang.org/grpc/codes" |
|
||||
"google.golang.org/grpc/status" |
|
||||
) |
|
||||
|
|
||||
// ConfigureTopic Runs on any broker, but proxied to the balancer if not the balancer
|
|
||||
func (broker *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.ConfigureTopicRequest) (resp *mq_pb.ConfigureTopicResponse, err error) { |
|
||||
if broker.currentBalancer == "" { |
|
||||
return nil, status.Errorf(codes.Unavailable, "no balancer") |
|
||||
} |
|
||||
if !broker.lockAsBalancer.IsLocked() { |
|
||||
proxyErr := broker.withBrokerClient(false, broker.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error { |
|
||||
resp, err = client.ConfigureTopic(ctx, request) |
|
||||
return nil |
|
||||
}) |
|
||||
if proxyErr != nil { |
|
||||
return nil, proxyErr |
|
||||
} |
|
||||
return resp, err |
|
||||
} |
|
||||
|
|
||||
ret := &mq_pb.ConfigureTopicResponse{} |
|
||||
ret.BrokerPartitionAssignments, err = broker.Balancer.LookupOrAllocateTopicPartitions(request.Topic, true, request.PartitionCount) |
|
||||
|
|
||||
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 { |
|
||||
_, doCreateErr := client.DoConfigureTopic(ctx, &mq_pb.DoConfigureTopicRequest{ |
|
||||
Topic: request.Topic, |
|
||||
Partition: bpa.Partition, |
|
||||
}) |
|
||||
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() |
|
||||
if !broker.Balancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) { |
|
||||
brokerStats, _ = broker.Balancer.Brokers.Get(bpa.LeaderBroker) |
|
||||
} |
|
||||
} |
|
||||
brokerStats.RegisterAssignment(request.Topic, bpa.Partition) |
|
||||
return nil |
|
||||
}); doCreateErr != nil { |
|
||||
return nil, doCreateErr |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
// TODO revert if some error happens in the middle of the assignments
|
|
||||
|
|
||||
return ret, err |
|
||||
} |
|
||||
|
|
||||
func (broker *MessageQueueBroker) DoConfigureTopic(ctx context.Context, req *mq_pb.DoConfigureTopicRequest) (resp *mq_pb.DoConfigureTopicResponse, err error) { |
|
||||
ret := &mq_pb.DoConfigureTopicResponse{} |
|
||||
t, p := topic.FromPbTopic(req.Topic), topic.FromPbPartition(req.Partition) |
|
||||
localTopicPartition := broker.localTopicManager.GetTopicPartition(t, p) |
|
||||
if localTopicPartition == nil { |
|
||||
localTopicPartition = topic.NewLocalPartition(t, p, true, nil) |
|
||||
broker.localTopicManager.AddTopicPartition(t, localTopicPartition) |
|
||||
} |
|
||||
|
|
||||
return ret, err |
|
||||
} |
|
@ -0,0 +1,77 @@ |
|||||
|
package broker |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/glog" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"google.golang.org/grpc/codes" |
||||
|
"google.golang.org/grpc/status" |
||||
|
) |
||||
|
|
||||
|
// SubscriberToSubCoordinator coordinates the subscribers
|
||||
|
func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMessaging_SubscriberToSubCoordinatorServer) error { |
||||
|
if !b.lockAsBalancer.IsLocked() { |
||||
|
return status.Errorf(codes.Unavailable, "not current broker balancer") |
||||
|
} |
||||
|
req, err := stream.Recv() |
||||
|
if err != nil { |
||||
|
return err |
||||
|
} |
||||
|
|
||||
|
var cgi *sub_coordinator.ConsumerGroupInstance |
||||
|
// process init message
|
||||
|
initMessage := req.GetInit() |
||||
|
if initMessage != nil { |
||||
|
cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic) |
||||
|
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic) |
||||
|
} else { |
||||
|
return status.Errorf(codes.InvalidArgument, "subscriber init message is empty") |
||||
|
} |
||||
|
defer func() { |
||||
|
b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic) |
||||
|
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err) |
||||
|
}() |
||||
|
|
||||
|
ctx := stream.Context() |
||||
|
|
||||
|
// process ack messages
|
||||
|
go func() { |
||||
|
for { |
||||
|
_, err := stream.Recv() |
||||
|
if err != nil { |
||||
|
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err) |
||||
|
} |
||||
|
|
||||
|
select { |
||||
|
case <-ctx.Done(): |
||||
|
err := ctx.Err() |
||||
|
if err == context.Canceled { |
||||
|
// Client disconnected
|
||||
|
return |
||||
|
} |
||||
|
return |
||||
|
default: |
||||
|
// Continue processing the request
|
||||
|
} |
||||
|
} |
||||
|
}() |
||||
|
|
||||
|
// send commands to subscriber
|
||||
|
for { |
||||
|
select { |
||||
|
case <-ctx.Done(): |
||||
|
err := ctx.Err() |
||||
|
if err == context.Canceled { |
||||
|
// Client disconnected
|
||||
|
return err |
||||
|
} |
||||
|
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err) |
||||
|
return err |
||||
|
case message := <- cgi.ResponseChan: |
||||
|
if err := stream.Send(message); err != nil { |
||||
|
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
} |
@ -0,0 +1,28 @@ |
|||||
|
package broker |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
) |
||||
|
|
||||
|
func (b *MessageQueueBroker) ClosePublishers(ctx context.Context, request *mq_pb.ClosePublishersRequest) (resp *mq_pb.ClosePublishersResponse, err error) { |
||||
|
resp = &mq_pb.ClosePublishersResponse{} |
||||
|
|
||||
|
t := topic.FromPbTopic(request.Topic) |
||||
|
|
||||
|
b.localTopicManager.ClosePublishers(t, request.UnixTimeNs) |
||||
|
|
||||
|
// wait until all publishers are closed
|
||||
|
b.localTopicManager.WaitUntilNoPublishers(t) |
||||
|
|
||||
|
return |
||||
|
} |
||||
|
|
||||
|
func (b *MessageQueueBroker) CloseSubscribers(ctx context.Context, request *mq_pb.CloseSubscribersRequest) (resp *mq_pb.CloseSubscribersResponse, err error) { |
||||
|
resp = &mq_pb.CloseSubscribersResponse{} |
||||
|
|
||||
|
b.localTopicManager.CloseSubscribers(topic.FromPbTopic(request.Topic), request.UnixTimeNs) |
||||
|
|
||||
|
return |
||||
|
} |
@ -1,89 +0,0 @@ |
|||||
package broker |
|
||||
|
|
||||
import ( |
|
||||
"bytes" |
|
||||
"fmt" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/filer" |
|
||||
"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" |
|
||||
jsonpb "google.golang.org/protobuf/encoding/protojson" |
|
||||
"time" |
|
||||
) |
|
||||
|
|
||||
func (broker *MessageQueueBroker) checkSegmentOnFiler(segment *topic.Segment) (brokers []pb.ServerAddress, err error) { |
|
||||
info, found, err := broker.readSegmentInfoOnFiler(segment) |
|
||||
if err != nil { |
|
||||
return |
|
||||
} |
|
||||
if !found { |
|
||||
return |
|
||||
} |
|
||||
for _, b := range info.Brokers { |
|
||||
brokers = append(brokers, pb.ServerAddress(b)) |
|
||||
} |
|
||||
|
|
||||
return |
|
||||
} |
|
||||
|
|
||||
func (broker *MessageQueueBroker) saveSegmentBrokersOnFiler(segment *topic.Segment, brokers []pb.ServerAddress) (err error) { |
|
||||
var nodes []string |
|
||||
for _, b := range brokers { |
|
||||
nodes = append(nodes, string(b)) |
|
||||
} |
|
||||
broker.saveSegmentInfoToFiler(segment, &mq_pb.SegmentInfo{ |
|
||||
Segment: segment.ToPbSegment(), |
|
||||
StartTsNs: time.Now().UnixNano(), |
|
||||
Brokers: nodes, |
|
||||
StopTsNs: 0, |
|
||||
PreviousSegments: nil, |
|
||||
NextSegments: nil, |
|
||||
}) |
|
||||
return |
|
||||
} |
|
||||
|
|
||||
func (broker *MessageQueueBroker) readSegmentInfoOnFiler(segment *topic.Segment) (info *mq_pb.SegmentInfo, found bool, err error) { |
|
||||
dir, name := segment.DirAndName() |
|
||||
|
|
||||
found, err = filer_pb.Exists(broker, dir, name, false) |
|
||||
if !found || err != nil { |
|
||||
return |
|
||||
} |
|
||||
|
|
||||
err = pb.WithFilerClient(false, 0, broker.GetFiler(), broker.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { |
|
||||
// read filer conf first
|
|
||||
data, err := filer.ReadInsideFiler(client, dir, name) |
|
||||
if err != nil { |
|
||||
return fmt.Errorf("ReadEntry: %v", err) |
|
||||
} |
|
||||
|
|
||||
// parse into filer conf object
|
|
||||
info = &mq_pb.SegmentInfo{} |
|
||||
if err = jsonpb.Unmarshal(data, info); err != nil { |
|
||||
return err |
|
||||
} |
|
||||
found = true |
|
||||
return nil |
|
||||
}) |
|
||||
|
|
||||
return |
|
||||
} |
|
||||
|
|
||||
func (broker *MessageQueueBroker) saveSegmentInfoToFiler(segment *topic.Segment, info *mq_pb.SegmentInfo) (err error) { |
|
||||
dir, name := segment.DirAndName() |
|
||||
|
|
||||
var buf bytes.Buffer |
|
||||
filer.ProtoToText(&buf, info) |
|
||||
|
|
||||
err = pb.WithFilerClient(false, 0, broker.GetFiler(), broker.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { |
|
||||
// read filer conf first
|
|
||||
err := filer.SaveInsideFiler(client, dir, name, buf.Bytes()) |
|
||||
if err != nil { |
|
||||
return fmt.Errorf("save segment info: %v", err) |
|
||||
} |
|
||||
return nil |
|
||||
}) |
|
||||
|
|
||||
return |
|
||||
} |
|
@ -0,0 +1,73 @@ |
|||||
|
package pub_client |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"google.golang.org/grpc/codes" |
||||
|
"google.golang.org/grpc/status" |
||||
|
"log" |
||||
|
) |
||||
|
|
||||
|
// broker => publish client
|
||||
|
// send init message
|
||||
|
// save the publishing client
|
||||
|
func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress string) (publishClient *PublishClient, err error) { |
||||
|
log.Printf("connecting to %v for topic partition %+v", brokerAddress, partition) |
||||
|
|
||||
|
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption) |
||||
|
if err != nil { |
||||
|
return publishClient, fmt.Errorf("dial broker %s: %v", brokerAddress, err) |
||||
|
} |
||||
|
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection) |
||||
|
stream, err := brokerClient.Publish(context.Background()) |
||||
|
if err != nil { |
||||
|
return publishClient, fmt.Errorf("create publish client: %v", err) |
||||
|
} |
||||
|
publishClient = &PublishClient{ |
||||
|
SeaweedMessaging_PublishClient: stream, |
||||
|
Broker: brokerAddress, |
||||
|
} |
||||
|
if err = publishClient.Send(&mq_pb.PublishRequest{ |
||||
|
Message: &mq_pb.PublishRequest_Init{ |
||||
|
Init: &mq_pb.PublishRequest_InitMessage{ |
||||
|
Topic: &mq_pb.Topic{ |
||||
|
Namespace: p.namespace, |
||||
|
Name: p.topic, |
||||
|
}, |
||||
|
Partition: &mq_pb.Partition{ |
||||
|
RingSize: partition.RingSize, |
||||
|
RangeStart: partition.RangeStart, |
||||
|
RangeStop: partition.RangeStop, |
||||
|
}, |
||||
|
AckInterval: 128, |
||||
|
}, |
||||
|
}, |
||||
|
}); err != nil { |
||||
|
return publishClient, fmt.Errorf("send init message: %v", err) |
||||
|
} |
||||
|
resp, err := stream.Recv() |
||||
|
if err != nil { |
||||
|
return publishClient, fmt.Errorf("recv init response: %v", err) |
||||
|
} |
||||
|
if resp.Error != "" { |
||||
|
return publishClient, fmt.Errorf("init response error: %v", resp.Error) |
||||
|
} |
||||
|
|
||||
|
go func() { |
||||
|
for { |
||||
|
_, err := publishClient.Recv() |
||||
|
if err != nil { |
||||
|
e, ok := status.FromError(err) |
||||
|
if ok && e.Code() == codes.Unknown && e.Message() == "EOF" { |
||||
|
return |
||||
|
} |
||||
|
publishClient.Err = err |
||||
|
fmt.Printf("publish to %s error: %v\n", publishClient.Broker, err) |
||||
|
return |
||||
|
} |
||||
|
} |
||||
|
}() |
||||
|
return publishClient, nil |
||||
|
} |
@ -1,92 +0,0 @@ |
|||||
package coordinator |
|
||||
|
|
||||
import ( |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
|
||||
"sync" |
|
||||
) |
|
||||
|
|
||||
func (cg *ConsumerGroup) SetMinMaxActiveInstances(min, max int32) { |
|
||||
cg.MinimumActiveInstances = min |
|
||||
cg.MaximumActiveInstances = max |
|
||||
} |
|
||||
|
|
||||
func (cg *ConsumerGroup) AddConsumerGroupInstance(clientId string) *ConsumerGroupInstance { |
|
||||
cgi := &ConsumerGroupInstance{ |
|
||||
ClientId: clientId, |
|
||||
} |
|
||||
cg.ConsumerGroupInstances.Set(clientId, cgi) |
|
||||
return cgi |
|
||||
} |
|
||||
|
|
||||
func (cg *ConsumerGroup) RemoveConsumerGroupInstance(clientId string) { |
|
||||
cg.ConsumerGroupInstances.Remove(clientId) |
|
||||
} |
|
||||
|
|
||||
func (cg *ConsumerGroup) CoordinateIfNeeded() { |
|
||||
emptyInstanceCount, activeInstanceCount := int32(0), int32(0) |
|
||||
for cgi := range cg.ConsumerGroupInstances.IterBuffered() { |
|
||||
if cgi.Val.Partition == nil { |
|
||||
// this consumer group instance is not assigned a partition
|
|
||||
// need to assign one
|
|
||||
emptyInstanceCount++ |
|
||||
} else { |
|
||||
activeInstanceCount++ |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
var delta int32 |
|
||||
if emptyInstanceCount > 0 { |
|
||||
if cg.MinimumActiveInstances <= 0 { |
|
||||
// need to assign more partitions
|
|
||||
delta = emptyInstanceCount |
|
||||
} else if activeInstanceCount < cg.MinimumActiveInstances && activeInstanceCount+emptyInstanceCount >= cg.MinimumActiveInstances { |
|
||||
// need to assign more partitions
|
|
||||
delta = cg.MinimumActiveInstances - activeInstanceCount |
|
||||
} |
|
||||
} |
|
||||
|
|
||||
if cg.MaximumActiveInstances > 0 { |
|
||||
if activeInstanceCount > cg.MaximumActiveInstances { |
|
||||
// need to remove some partitions
|
|
||||
delta = cg.MaximumActiveInstances - activeInstanceCount |
|
||||
} |
|
||||
} |
|
||||
if delta == 0 { |
|
||||
return |
|
||||
} |
|
||||
cg.doCoordinate(activeInstanceCount + delta) |
|
||||
} |
|
||||
|
|
||||
func (cg *ConsumerGroup) doCoordinate(target int32) { |
|
||||
// stop existing instances from processing
|
|
||||
var wg sync.WaitGroup |
|
||||
for cgi := range cg.ConsumerGroupInstances.IterBuffered() { |
|
||||
if cgi.Val.Partition != nil { |
|
||||
wg.Add(1) |
|
||||
go func(cgi *ConsumerGroupInstance) { |
|
||||
defer wg.Done() |
|
||||
// stop processing
|
|
||||
// flush internal state
|
|
||||
// wait for all messages to be processed
|
|
||||
// close the connection
|
|
||||
}(cgi.Val) |
|
||||
} |
|
||||
} |
|
||||
wg.Wait() |
|
||||
|
|
||||
partitions := topic.SplitPartitions(target) |
|
||||
|
|
||||
// assign partitions to new instances
|
|
||||
i := 0 |
|
||||
for cgi := range cg.ConsumerGroupInstances.IterBuffered() { |
|
||||
cgi.Val.Partition = partitions[i] |
|
||||
i++ |
|
||||
wg.Add(1) |
|
||||
go func(cgi *ConsumerGroupInstance) { |
|
||||
defer wg.Done() |
|
||||
// start processing
|
|
||||
// start consuming from the last offset
|
|
||||
}(cgi.Val) |
|
||||
} |
|
||||
wg.Wait() |
|
||||
} |
|
@ -1,36 +0,0 @@ |
|||||
package coordinator |
|
||||
|
|
||||
import ( |
|
||||
cmap "github.com/orcaman/concurrent-map/v2" |
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
|
||||
) |
|
||||
|
|
||||
type ConsumerGroupInstance struct { |
|
||||
ClientId string |
|
||||
// the consumer group instance may not have an active partition
|
|
||||
Partition *topic.Partition |
|
||||
// processed message count
|
|
||||
ProcessedMessageCount int64 |
|
||||
} |
|
||||
type ConsumerGroup struct { |
|
||||
// map a client id to a consumer group instance
|
|
||||
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance] |
|
||||
MinimumActiveInstances int32 |
|
||||
MaximumActiveInstances int32 |
|
||||
} |
|
||||
type TopicConsumerGroups struct { |
|
||||
// map a consumer group name to a consumer group
|
|
||||
ConsumerGroups cmap.ConcurrentMap[string, *ConsumerGroup] |
|
||||
} |
|
||||
|
|
||||
// Coordinator coordinates the instances in the consumer group for one topic.
|
|
||||
// It is responsible for:
|
|
||||
// 1. Assigning partitions to consumer instances.
|
|
||||
// 2. Reassigning partitions when a consumer instance is down.
|
|
||||
// 3. Reassigning partitions when a consumer instance is up.
|
|
||||
type Coordinator struct { |
|
||||
// map client id to subscriber
|
|
||||
Subscribers cmap.ConcurrentMap[string, *ConsumerGroupInstance] |
|
||||
// map topic name to consumer groups
|
|
||||
TopicSubscribers cmap.ConcurrentMap[string, map[string]TopicConsumerGroups] |
|
||||
} |
|
@ -1,4 +1,4 @@ |
|||||
package balancer |
|
||||
|
package pub_balancer |
||||
|
|
||||
import ( |
import ( |
||||
cmap "github.com/orcaman/concurrent-map/v2" |
cmap "github.com/orcaman/concurrent-map/v2" |
@ -0,0 +1,73 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"google.golang.org/grpc" |
||||
|
) |
||||
|
|
||||
|
/* |
||||
|
* Assuming a topic has [x,y] number of partitions when publishing, and there are b number of brokers. |
||||
|
* and p is the number of partitions per topic. |
||||
|
* if the broker number b <= x, then p = x. |
||||
|
* if the broker number x < b < y, then x <= p <= b. |
||||
|
* if the broker number b >= y, x <= p <= y |
||||
|
|
||||
|
Balance topic partitions to brokers |
||||
|
=================================== |
||||
|
|
||||
|
When the goal is to make sure that low traffic partitions can be merged, (and p >= x, and after last rebalance interval): |
||||
|
1. Calculate the average load(throughput) of partitions per topic. |
||||
|
2. If any two neighboring partitions have a load that is less than the average load, merge them. |
||||
|
3. If min(b, y) < p, then merge two neighboring partitions that have the least combined load. |
||||
|
|
||||
|
When the goal is to make sure that high traffic partitions can be split, (and p < y and p < b, and after last rebalance interval): |
||||
|
1. Calculate the average number of partitions per broker. |
||||
|
2. If any partition has a load that is more than the average load, split it into two partitions. |
||||
|
|
||||
|
When the goal is to make sure that each broker has the same number of partitions: |
||||
|
1. Calculate the average number of partitions per broker. |
||||
|
2. For the brokers that have more than the average number of partitions, move the partitions to the brokers that have less than the average number of partitions. |
||||
|
|
||||
|
*/ |
||||
|
|
||||
|
type BalanceAction interface { |
||||
|
} |
||||
|
type BalanceActionMerge struct { |
||||
|
Before []topic.TopicPartition |
||||
|
After topic.TopicPartition |
||||
|
} |
||||
|
type BalanceActionSplit struct { |
||||
|
Before topic.TopicPartition |
||||
|
After []topic.TopicPartition |
||||
|
} |
||||
|
|
||||
|
type BalanceActionMove struct { |
||||
|
TopicPartition topic.TopicPartition |
||||
|
SourceBroker string |
||||
|
TargetBroker string |
||||
|
} |
||||
|
|
||||
|
type BalanceActionCreate struct { |
||||
|
TopicPartition topic.TopicPartition |
||||
|
TargetBroker string |
||||
|
} |
||||
|
|
||||
|
// BalancePublishers check the stats of all brokers,
|
||||
|
// and balance the publishers to the brokers.
|
||||
|
func (balancer *Balancer) BalancePublishers() []BalanceAction { |
||||
|
action := BalanceTopicPartitionOnBrokers(balancer.Brokers) |
||||
|
return []BalanceAction{action} |
||||
|
} |
||||
|
|
||||
|
func (balancer *Balancer) ExecuteBalanceAction(actions []BalanceAction, grpcDialOption grpc.DialOption) (err error) { |
||||
|
for _, action := range actions { |
||||
|
switch action.(type) { |
||||
|
case *BalanceActionMove: |
||||
|
err = balancer.ExecuteBalanceActionMove(action.(*BalanceActionMove), grpcDialOption) |
||||
|
} |
||||
|
if err != nil { |
||||
|
return err |
||||
|
} |
||||
|
} |
||||
|
return nil |
||||
|
} |
@ -0,0 +1,58 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"google.golang.org/grpc" |
||||
|
) |
||||
|
|
||||
|
// Balancer <= PublisherToPubBalancer() <= Broker <=> Publish()
|
||||
|
// ExecuteBalanceActionMove from Balancer => AssignTopicPartitions() => Broker => Publish()
|
||||
|
|
||||
|
func (balancer *Balancer) ExecuteBalanceActionMove(move *BalanceActionMove, grpcDialOption grpc.DialOption) error { |
||||
|
if _, found := balancer.Brokers.Get(move.SourceBroker); !found { |
||||
|
return fmt.Errorf("source broker %s not found", move.SourceBroker) |
||||
|
} |
||||
|
if _, found := balancer.Brokers.Get(move.TargetBroker); !found { |
||||
|
return fmt.Errorf("target broker %s not found", move.TargetBroker) |
||||
|
} |
||||
|
|
||||
|
err := pb.WithBrokerGrpcClient(false, move.TargetBroker, grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
_, err := client.AssignTopicPartitions(context.Background(), &mq_pb.AssignTopicPartitionsRequest{ |
||||
|
Topic: move.TopicPartition.Topic.ToPbTopic(), |
||||
|
BrokerPartitionAssignments: []*mq_pb.BrokerPartitionAssignment{ |
||||
|
{ |
||||
|
Partition: move.TopicPartition.ToPbPartition(), |
||||
|
}, |
||||
|
}, |
||||
|
IsLeader: true, |
||||
|
IsDraining: false, |
||||
|
}) |
||||
|
return err |
||||
|
}) |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("assign topic partition %v to %s: %v", move.TopicPartition, move.TargetBroker, err) |
||||
|
} |
||||
|
|
||||
|
err = pb.WithBrokerGrpcClient(false, move.SourceBroker, grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
_, err := client.AssignTopicPartitions(context.Background(), &mq_pb.AssignTopicPartitionsRequest{ |
||||
|
Topic: move.TopicPartition.Topic.ToPbTopic(), |
||||
|
BrokerPartitionAssignments: []*mq_pb.BrokerPartitionAssignment{ |
||||
|
{ |
||||
|
Partition: move.TopicPartition.ToPbPartition(), |
||||
|
}, |
||||
|
}, |
||||
|
IsLeader: true, |
||||
|
IsDraining: true, |
||||
|
}) |
||||
|
return err |
||||
|
}) |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("assign topic partition %v to %s: %v", move.TopicPartition, move.SourceBroker, err) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
|
||||
|
} |
@ -0,0 +1,43 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
/* |
||||
|
Sequence of operations to ensure ordering |
||||
|
|
||||
|
Assuming Publisher P10 is publishing to Topic Partition TP10, and Subscriber S10 is subscribing to Topic TP10. |
||||
|
After splitting Topic TP10 into Topic Partition TP11 and Topic Partition TP21, |
||||
|
Publisher P11 is publishing to Topic Partition TP11, and Publisher P21 is publishing to Topic Partition TP21. |
||||
|
Subscriber S12 is subscribing to Topic Partition TP11, and Subscriber S21 is subscribing to Topic Partition TP21. |
||||
|
|
||||
|
(The last digit is ephoch generation number, which is increasing when the topic partitioning is changed.) |
||||
|
|
||||
|
The diagram is as follows: |
||||
|
P10 -> TP10 -> S10 |
||||
|
|| |
||||
|
\/ |
||||
|
P11 -> TP11 -> S11 |
||||
|
P21 -> TP21 -> S21 |
||||
|
|
||||
|
The following is the sequence of events: |
||||
|
1. Create Topic Partition TP11 and TP21 |
||||
|
2. Close Publisher(s) P10 |
||||
|
3. Close Subscriber(s) S10 |
||||
|
4. Close Topic Partition TP10 |
||||
|
5. Start Publisher P11, P21 |
||||
|
6. Start Subscriber S11, S21 |
||||
|
|
||||
|
The dependency is as follows: |
||||
|
2 => 3 => 4 |
||||
|
| | |
||||
|
v v |
||||
|
1 => (5 | 6) |
||||
|
|
||||
|
And also: |
||||
|
2 => 5 |
||||
|
3 => 6 |
||||
|
|
||||
|
For brokers: |
||||
|
1. Close all publishers for a topic partition |
||||
|
2. Close all subscribers for a topic partition |
||||
|
3. Close the topic partition |
||||
|
|
||||
|
*/ |
@ -0,0 +1,52 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"math/rand" |
||||
|
) |
||||
|
|
||||
|
func BalanceTopicPartitionOnBrokers(brokers cmap.ConcurrentMap[string, *BrokerStats]) BalanceAction { |
||||
|
// 1. calculate the average number of partitions per broker
|
||||
|
var totalPartitionCount int32 |
||||
|
var totalBrokerCount int32 |
||||
|
for brokerStats := range brokers.IterBuffered() { |
||||
|
totalBrokerCount++ |
||||
|
totalPartitionCount += brokerStats.Val.TopicPartitionCount |
||||
|
} |
||||
|
averagePartitionCountPerBroker := totalPartitionCount / totalBrokerCount |
||||
|
minPartitionCountPerBroker := averagePartitionCountPerBroker |
||||
|
maxPartitionCountPerBroker := averagePartitionCountPerBroker |
||||
|
var sourceBroker, targetBroker string |
||||
|
var candidatePartition *topic.TopicPartition |
||||
|
for brokerStats := range brokers.IterBuffered() { |
||||
|
if minPartitionCountPerBroker > brokerStats.Val.TopicPartitionCount { |
||||
|
minPartitionCountPerBroker = brokerStats.Val.TopicPartitionCount |
||||
|
targetBroker = brokerStats.Key |
||||
|
} |
||||
|
if maxPartitionCountPerBroker < brokerStats.Val.TopicPartitionCount { |
||||
|
maxPartitionCountPerBroker = brokerStats.Val.TopicPartitionCount |
||||
|
sourceBroker = brokerStats.Key |
||||
|
// select a random partition from the source broker
|
||||
|
randomePartitionIndex := rand.Intn(int(brokerStats.Val.TopicPartitionCount)) |
||||
|
index := 0 |
||||
|
for topicPartitionStats := range brokerStats.Val.TopicPartitionStats.IterBuffered() { |
||||
|
if index == randomePartitionIndex { |
||||
|
candidatePartition = &topicPartitionStats.Val.TopicPartition |
||||
|
break |
||||
|
} else { |
||||
|
index++ |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
if minPartitionCountPerBroker >= maxPartitionCountPerBroker-1 { |
||||
|
return nil |
||||
|
} |
||||
|
// 2. move the partitions from the source broker to the target broker
|
||||
|
return &BalanceActionMove{ |
||||
|
TopicPartition: *candidatePartition, |
||||
|
SourceBroker: sourceBroker, |
||||
|
TargetBroker: targetBroker, |
||||
|
} |
||||
|
} |
@ -0,0 +1,75 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"reflect" |
||||
|
"testing" |
||||
|
) |
||||
|
|
||||
|
func TestBalanceTopicPartitionOnBrokers(t *testing.T) { |
||||
|
|
||||
|
brokers := cmap.New[*BrokerStats]() |
||||
|
broker1Stats := &BrokerStats{ |
||||
|
TopicPartitionCount: 1, |
||||
|
ConsumerCount: 1, |
||||
|
CpuUsagePercent: 1, |
||||
|
TopicPartitionStats: cmap.New[*TopicPartitionStats](), |
||||
|
} |
||||
|
broker1Stats.TopicPartitionStats.Set("topic1:0", &TopicPartitionStats{ |
||||
|
TopicPartition: topic.TopicPartition{ |
||||
|
Topic: topic.Topic{Namespace: "topic1", Name: "topic1"}, |
||||
|
Partition: topic.Partition{RangeStart: 0, RangeStop: 512, RingSize: 1024}, |
||||
|
}, |
||||
|
ConsumerCount: 1, |
||||
|
IsLeader: true, |
||||
|
}) |
||||
|
broker2Stats := &BrokerStats{ |
||||
|
TopicPartitionCount: 2, |
||||
|
ConsumerCount: 1, |
||||
|
CpuUsagePercent: 1, |
||||
|
TopicPartitionStats: cmap.New[*TopicPartitionStats](), |
||||
|
} |
||||
|
broker2Stats.TopicPartitionStats.Set("topic1:1", &TopicPartitionStats{ |
||||
|
TopicPartition: topic.TopicPartition{ |
||||
|
Topic: topic.Topic{Namespace: "topic1", Name: "topic1"}, |
||||
|
Partition: topic.Partition{RangeStart: 512, RangeStop: 1024, RingSize: 1024}, |
||||
|
}, |
||||
|
ConsumerCount: 1, |
||||
|
IsLeader: true, |
||||
|
}) |
||||
|
broker2Stats.TopicPartitionStats.Set("topic2:0", &TopicPartitionStats{ |
||||
|
TopicPartition: topic.TopicPartition{ |
||||
|
Topic: topic.Topic{Namespace: "topic2", Name: "topic2"}, |
||||
|
Partition: topic.Partition{RangeStart: 0, RangeStop: 1024, RingSize: 1024}, |
||||
|
}, |
||||
|
ConsumerCount: 1, |
||||
|
IsLeader: true, |
||||
|
}) |
||||
|
brokers.Set("broker1", broker1Stats) |
||||
|
brokers.Set("broker2", broker2Stats) |
||||
|
|
||||
|
type args struct { |
||||
|
brokers cmap.ConcurrentMap[string, *BrokerStats] |
||||
|
} |
||||
|
tests := []struct { |
||||
|
name string |
||||
|
args args |
||||
|
want BalanceAction |
||||
|
}{ |
||||
|
{ |
||||
|
name: "test", |
||||
|
args: args{ |
||||
|
brokers: brokers, |
||||
|
}, |
||||
|
want: nil, |
||||
|
}, |
||||
|
} |
||||
|
for _, tt := range tests { |
||||
|
t.Run(tt.name, func(t *testing.T) { |
||||
|
if got := BalanceTopicPartitionOnBrokers(tt.args.brokers); !reflect.DeepEqual(got, tt.want) { |
||||
|
t.Errorf("BalanceTopicPartitionOnBrokers() = %v, want %v", got, tt.want) |
||||
|
} |
||||
|
}) |
||||
|
} |
||||
|
} |
@ -0,0 +1,83 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
) |
||||
|
|
||||
|
const ( |
||||
|
MaxPartitionCount = 8 * 9 * 5 * 7 //2520
|
||||
|
LockBrokerBalancer = "broker_balancer" |
||||
|
) |
||||
|
|
||||
|
// Balancer collects stats from all brokers.
|
||||
|
//
|
||||
|
// When publishers wants to create topics, it picks brokers to assign the topic partitions.
|
||||
|
// When consumers wants to subscribe topics, it tells which brokers are serving the topic partitions.
|
||||
|
//
|
||||
|
// When a partition needs to be split or merged, or a partition needs to be moved to another broker,
|
||||
|
// the balancer will let the broker tell the consumer instance to stop processing the partition.
|
||||
|
// The existing consumer instance will flush the internal state, and then stop processing.
|
||||
|
// Then the balancer will tell the brokers to start sending new messages in the new/moved partition to the consumer instances.
|
||||
|
//
|
||||
|
// Failover to standby consumer instances:
|
||||
|
//
|
||||
|
// A consumer group can have min and max number of consumer instances.
|
||||
|
// For consumer instances joined after the max number, they will be in standby mode.
|
||||
|
//
|
||||
|
// When a consumer instance is down, the broker will notice this and inform the balancer.
|
||||
|
// The balancer will then tell the broker to send the partition to another standby consumer instance.
|
||||
|
type Balancer struct { |
||||
|
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
|
||||
|
// Collected from all brokers when they connect to the broker leader
|
||||
|
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
|
||||
|
} |
||||
|
func NewBalancer() *Balancer { |
||||
|
return &Balancer{ |
||||
|
Brokers: cmap.New[*BrokerStats](), |
||||
|
TopicToBrokers: cmap.New[*PartitionSlotToBrokerList](), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (balancer *Balancer) OnBrokerConnected(broker string) (brokerStats *BrokerStats) { |
||||
|
var found bool |
||||
|
brokerStats, found = balancer.Brokers.Get(broker) |
||||
|
if !found { |
||||
|
brokerStats = NewBrokerStats() |
||||
|
if !balancer.Brokers.SetIfAbsent(broker, brokerStats) { |
||||
|
brokerStats, _ = balancer.Brokers.Get(broker) |
||||
|
} |
||||
|
} |
||||
|
return brokerStats |
||||
|
} |
||||
|
|
||||
|
func (balancer *Balancer) OnBrokerDisconnected(broker string, stats *BrokerStats) { |
||||
|
balancer.Brokers.Remove(broker) |
||||
|
|
||||
|
// update TopicToBrokers
|
||||
|
for _, topic := range stats.Topics { |
||||
|
partitionSlotToBrokerList, found := balancer.TopicToBrokers.Get(topic.String()) |
||||
|
if !found { |
||||
|
continue |
||||
|
} |
||||
|
partitionSlotToBrokerList.RemoveBroker(broker) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *BrokerStats, receivedStats *mq_pb.BrokerStats) { |
||||
|
brokerStats.UpdateStats(receivedStats) |
||||
|
|
||||
|
// update TopicToBrokers
|
||||
|
for _, topicPartitionStats := range receivedStats.Stats { |
||||
|
topic := topicPartitionStats.Topic |
||||
|
partition := topicPartitionStats.Partition |
||||
|
partitionSlotToBrokerList, found := balancer.TopicToBrokers.Get(topic.String()) |
||||
|
if !found { |
||||
|
partitionSlotToBrokerList = NewPartitionSlotToBrokerList(MaxPartitionCount) |
||||
|
if !balancer.TopicToBrokers.SetIfAbsent(topic.String(), partitionSlotToBrokerList) { |
||||
|
partitionSlotToBrokerList, _ = balancer.TopicToBrokers.Get(topic.String()) |
||||
|
} |
||||
|
} |
||||
|
partitionSlotToBrokerList.AddBroker(partition, broker) |
||||
|
} |
||||
|
} |
@ -0,0 +1,50 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
"github.com/seaweedfs/seaweedfs/weed/glog" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
) |
||||
|
|
||||
|
type PartitionSlotToBroker struct { |
||||
|
RangeStart int32 |
||||
|
RangeStop int32 |
||||
|
AssignedBroker string |
||||
|
} |
||||
|
|
||||
|
type PartitionSlotToBrokerList struct { |
||||
|
PartitionSlots []*PartitionSlotToBroker |
||||
|
RingSize int32 |
||||
|
} |
||||
|
|
||||
|
func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList { |
||||
|
return &PartitionSlotToBrokerList{ |
||||
|
RingSize: ringSize, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string) { |
||||
|
for _, partitionSlot := range ps.PartitionSlots { |
||||
|
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop { |
||||
|
if partitionSlot.AssignedBroker == broker { |
||||
|
return |
||||
|
} |
||||
|
if partitionSlot.AssignedBroker != "" { |
||||
|
glog.V(0).Infof("partition %s broker change: %s => %s", partition, partitionSlot.AssignedBroker, broker) |
||||
|
} |
||||
|
partitionSlot.AssignedBroker = broker |
||||
|
return |
||||
|
} |
||||
|
} |
||||
|
ps.PartitionSlots = append(ps.PartitionSlots, &PartitionSlotToBroker{ |
||||
|
RangeStart: partition.RangeStart, |
||||
|
RangeStop: partition.RangeStop, |
||||
|
AssignedBroker: broker, |
||||
|
}) |
||||
|
} |
||||
|
func (ps *PartitionSlotToBrokerList) RemoveBroker(broker string) { |
||||
|
for _, partitionSlot := range ps.PartitionSlots { |
||||
|
if partitionSlot.AssignedBroker == broker { |
||||
|
partitionSlot.AssignedBroker = "" |
||||
|
} |
||||
|
} |
||||
|
} |
@ -0,0 +1,127 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"math/rand" |
||||
|
"modernc.org/mathutil" |
||||
|
"sort" |
||||
|
) |
||||
|
|
||||
|
func (balancer *Balancer) RepairTopics() []BalanceAction { |
||||
|
action := BalanceTopicPartitionOnBrokers(balancer.Brokers) |
||||
|
return []BalanceAction{action} |
||||
|
} |
||||
|
|
||||
|
type TopicPartitionInfo struct { |
||||
|
Leader string |
||||
|
Followers []string |
||||
|
} |
||||
|
|
||||
|
// RepairMissingTopicPartitions check the stats of all brokers,
|
||||
|
// and repair the missing topic partitions on the brokers.
|
||||
|
func RepairMissingTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats]) (actions []BalanceAction) { |
||||
|
|
||||
|
// find all topic partitions
|
||||
|
topicToTopicPartitions := make(map[topic.Topic]map[topic.Partition]*TopicPartitionInfo) |
||||
|
for brokerStatsItem := range brokers.IterBuffered() { |
||||
|
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val |
||||
|
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() { |
||||
|
topicPartitionStat := topicPartitionStatsItem.Val |
||||
|
topicPartitionToInfo, found := topicToTopicPartitions[topicPartitionStat.Topic] |
||||
|
if !found { |
||||
|
topicPartitionToInfo = make(map[topic.Partition]*TopicPartitionInfo) |
||||
|
topicToTopicPartitions[topicPartitionStat.Topic] = topicPartitionToInfo |
||||
|
} |
||||
|
tpi, found := topicPartitionToInfo[topicPartitionStat.Partition] |
||||
|
if !found { |
||||
|
tpi = &TopicPartitionInfo{} |
||||
|
topicPartitionToInfo[topicPartitionStat.Partition] = tpi |
||||
|
} |
||||
|
if topicPartitionStat.IsLeader { |
||||
|
tpi.Leader = broker |
||||
|
} else { |
||||
|
tpi.Followers = append(tpi.Followers, broker) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// collect all brokers as candidates
|
||||
|
candidates := make([]string, 0, brokers.Count()) |
||||
|
for brokerStatsItem := range brokers.IterBuffered() { |
||||
|
candidates = append(candidates, brokerStatsItem.Key) |
||||
|
} |
||||
|
|
||||
|
// find the missing topic partitions
|
||||
|
for t, topicPartitionToInfo := range topicToTopicPartitions { |
||||
|
missingPartitions := EachTopicRepairMissingTopicPartitions(t, topicPartitionToInfo) |
||||
|
for _, partition := range missingPartitions { |
||||
|
actions = append(actions, BalanceActionCreate{ |
||||
|
TopicPartition: topic.TopicPartition{ |
||||
|
Topic: t, |
||||
|
Partition: partition, |
||||
|
}, |
||||
|
TargetBroker: candidates[rand.Intn(len(candidates))], |
||||
|
}) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return actions |
||||
|
} |
||||
|
|
||||
|
func EachTopicRepairMissingTopicPartitions(t topic.Topic, info map[topic.Partition]*TopicPartitionInfo) (missingPartitions []topic.Partition) { |
||||
|
|
||||
|
// find the missing topic partitions
|
||||
|
var partitions []topic.Partition |
||||
|
for partition := range info { |
||||
|
partitions = append(partitions, partition) |
||||
|
} |
||||
|
return findMissingPartitions(partitions, MaxPartitionCount) |
||||
|
} |
||||
|
|
||||
|
// findMissingPartitions find the missing partitions
|
||||
|
func findMissingPartitions(partitions []topic.Partition, ringSize int32) (missingPartitions []topic.Partition) { |
||||
|
// sort the partitions by range start
|
||||
|
sort.Slice(partitions, func(i, j int) bool { |
||||
|
return partitions[i].RangeStart < partitions[j].RangeStart |
||||
|
}) |
||||
|
|
||||
|
// calculate the average partition size
|
||||
|
var covered int32 |
||||
|
for _, partition := range partitions { |
||||
|
covered += partition.RangeStop - partition.RangeStart |
||||
|
} |
||||
|
averagePartitionSize := covered / int32(len(partitions)) |
||||
|
|
||||
|
// find the missing partitions
|
||||
|
var coveredWatermark int32 |
||||
|
i := 0 |
||||
|
for i < len(partitions) { |
||||
|
partition := partitions[i] |
||||
|
if partition.RangeStart > coveredWatermark { |
||||
|
upperBound := mathutil.MinInt32(coveredWatermark+averagePartitionSize, partition.RangeStart) |
||||
|
missingPartitions = append(missingPartitions, topic.Partition{ |
||||
|
RangeStart: coveredWatermark, |
||||
|
RangeStop: upperBound, |
||||
|
RingSize: ringSize, |
||||
|
}) |
||||
|
coveredWatermark = upperBound |
||||
|
if coveredWatermark == partition.RangeStop { |
||||
|
i++ |
||||
|
} |
||||
|
} else { |
||||
|
coveredWatermark = partition.RangeStop |
||||
|
i++ |
||||
|
} |
||||
|
} |
||||
|
for coveredWatermark < ringSize { |
||||
|
upperBound := mathutil.MinInt32(coveredWatermark+averagePartitionSize, ringSize) |
||||
|
missingPartitions = append(missingPartitions, topic.Partition{ |
||||
|
RangeStart: coveredWatermark, |
||||
|
RangeStop: upperBound, |
||||
|
RingSize: ringSize, |
||||
|
}) |
||||
|
coveredWatermark = upperBound |
||||
|
} |
||||
|
return missingPartitions |
||||
|
} |
@ -0,0 +1,97 @@ |
|||||
|
package pub_balancer |
||||
|
|
||||
|
import ( |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"reflect" |
||||
|
"testing" |
||||
|
) |
||||
|
|
||||
|
func Test_findMissingPartitions(t *testing.T) { |
||||
|
type args struct { |
||||
|
partitions []topic.Partition |
||||
|
} |
||||
|
tests := []struct { |
||||
|
name string |
||||
|
args args |
||||
|
wantMissingPartitions []topic.Partition |
||||
|
}{ |
||||
|
{ |
||||
|
name: "one partition", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: nil, |
||||
|
}, |
||||
|
{ |
||||
|
name: "two partitions", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 512}, |
||||
|
{RingSize: 1024, RangeStart: 512, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: nil, |
||||
|
}, |
||||
|
{ |
||||
|
name: "four partitions, missing last two", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 256}, |
||||
|
{RingSize: 1024, RangeStart: 256, RangeStop: 512}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 512, RangeStop: 768}, |
||||
|
{RingSize: 1024, RangeStart: 768, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "four partitions, missing first two", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 512, RangeStop: 768}, |
||||
|
{RingSize: 1024, RangeStart: 768, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 256}, |
||||
|
{RingSize: 1024, RangeStart: 256, RangeStop: 512}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "four partitions, missing middle two", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 256}, |
||||
|
{RingSize: 1024, RangeStart: 768, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 256, RangeStop: 512}, |
||||
|
{RingSize: 1024, RangeStart: 512, RangeStop: 768}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "four partitions, missing three", |
||||
|
args: args{ |
||||
|
partitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 512, RangeStop: 768}, |
||||
|
}, |
||||
|
}, |
||||
|
wantMissingPartitions: []topic.Partition{ |
||||
|
{RingSize: 1024, RangeStart: 0, RangeStop: 256}, |
||||
|
{RingSize: 1024, RangeStart: 256, RangeStop: 512}, |
||||
|
{RingSize: 1024, RangeStart: 768, RangeStop: 1024}, |
||||
|
}, |
||||
|
}, |
||||
|
} |
||||
|
for _, tt := range tests { |
||||
|
t.Run(tt.name, func(t *testing.T) { |
||||
|
if gotMissingPartitions := findMissingPartitions(tt.args.partitions, 1024); !reflect.DeepEqual(gotMissingPartitions, tt.wantMissingPartitions) { |
||||
|
t.Errorf("findMissingPartitions() = %v, want %v", gotMissingPartitions, tt.wantMissingPartitions) |
||||
|
} |
||||
|
}) |
||||
|
} |
||||
|
} |
@ -0,0 +1,41 @@ |
|||||
|
package sub_coordinator |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
) |
||||
|
type ConsumerGroupInstance struct { |
||||
|
InstanceId string |
||||
|
// the consumer group instance may not have an active partition
|
||||
|
Partitions []*topic.Partition |
||||
|
ResponseChan chan *mq_pb.SubscriberToSubCoordinatorResponse |
||||
|
} |
||||
|
type ConsumerGroup struct { |
||||
|
// map a consumer group instance id to a consumer group instance
|
||||
|
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance] |
||||
|
mapping *PartitionConsumerMapping |
||||
|
} |
||||
|
|
||||
|
func NewConsumerGroup() *ConsumerGroup { |
||||
|
return &ConsumerGroup{ |
||||
|
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](), |
||||
|
mapping: NewPartitionConsumerMapping(pub_balancer.MaxPartitionCount), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func NewConsumerGroupInstance(instanceId string) *ConsumerGroupInstance { |
||||
|
return &ConsumerGroupInstance{ |
||||
|
InstanceId: instanceId, |
||||
|
ResponseChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1), |
||||
|
} |
||||
|
} |
||||
|
func (cg *ConsumerGroup) OnAddConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) { |
||||
|
} |
||||
|
func (cg *ConsumerGroup) OnRemoveConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) { |
||||
|
|
||||
|
} |
||||
|
func (cg *ConsumerGroup) OnPartitionListChange() { |
||||
|
|
||||
|
} |
@ -0,0 +1,86 @@ |
|||||
|
package sub_coordinator |
||||
|
|
||||
|
import ( |
||||
|
cmap "github.com/orcaman/concurrent-map/v2" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
) |
||||
|
|
||||
|
|
||||
|
type TopicConsumerGroups struct { |
||||
|
// map a consumer group name to a consumer group
|
||||
|
ConsumerGroups cmap.ConcurrentMap[string, *ConsumerGroup] |
||||
|
} |
||||
|
|
||||
|
// Coordinator coordinates the instances in the consumer group for one topic.
|
||||
|
// It is responsible for:
|
||||
|
// 1. (Maybe) assigning partitions when a consumer instance is up/down.
|
||||
|
|
||||
|
type Coordinator struct { |
||||
|
// map topic name to consumer groups
|
||||
|
TopicSubscribers cmap.ConcurrentMap[string, *TopicConsumerGroups] |
||||
|
balancer *pub_balancer.Balancer |
||||
|
} |
||||
|
|
||||
|
func NewCoordinator(balancer *pub_balancer.Balancer) *Coordinator { |
||||
|
return &Coordinator{ |
||||
|
TopicSubscribers: cmap.New[*TopicConsumerGroups](), |
||||
|
balancer: balancer, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (c *Coordinator) GetTopicConsumerGroups(topic *mq_pb.Topic) *TopicConsumerGroups { |
||||
|
topicName := toTopicName(topic) |
||||
|
tcg, _ := c.TopicSubscribers.Get(topicName) |
||||
|
if tcg == nil { |
||||
|
tcg = &TopicConsumerGroups{ |
||||
|
ConsumerGroups: cmap.New[*ConsumerGroup](), |
||||
|
} |
||||
|
c.TopicSubscribers.Set(topicName, tcg) |
||||
|
} |
||||
|
return tcg |
||||
|
} |
||||
|
func (c *Coordinator) RemoveTopic(topic *mq_pb.Topic) { |
||||
|
topicName := toTopicName(topic) |
||||
|
c.TopicSubscribers.Remove(topicName) |
||||
|
} |
||||
|
|
||||
|
func toTopicName(topic *mq_pb.Topic) string { |
||||
|
topicName := topic.Namespace + "." + topic.Name |
||||
|
return topicName |
||||
|
} |
||||
|
|
||||
|
func (c *Coordinator) AddSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) *ConsumerGroupInstance{ |
||||
|
tcg := c.GetTopicConsumerGroups(topic) |
||||
|
cg, _ := tcg.ConsumerGroups.Get(consumerGroup) |
||||
|
if cg == nil { |
||||
|
cg = NewConsumerGroup() |
||||
|
tcg.ConsumerGroups.Set(consumerGroup, cg) |
||||
|
} |
||||
|
cgi, _ := cg.ConsumerGroupInstances.Get(consumerGroupInstance) |
||||
|
if cgi == nil { |
||||
|
cgi = NewConsumerGroupInstance(consumerGroupInstance) |
||||
|
cg.ConsumerGroupInstances.Set(consumerGroupInstance, cgi) |
||||
|
} |
||||
|
cg.OnAddConsumerGroupInstance(consumerGroupInstance, topic) |
||||
|
return cgi |
||||
|
} |
||||
|
|
||||
|
func (c *Coordinator) RemoveSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) { |
||||
|
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic)) |
||||
|
if tcg == nil { |
||||
|
return |
||||
|
} |
||||
|
cg, _ := tcg.ConsumerGroups.Get(consumerGroup) |
||||
|
if cg == nil { |
||||
|
return |
||||
|
} |
||||
|
cg.ConsumerGroupInstances.Remove(consumerGroupInstance) |
||||
|
cg.OnRemoveConsumerGroupInstance(consumerGroupInstance, topic) |
||||
|
if cg.ConsumerGroupInstances.Count() == 0 { |
||||
|
tcg.ConsumerGroups.Remove(consumerGroup) |
||||
|
} |
||||
|
if tcg.ConsumerGroups.Count() == 0 { |
||||
|
c.RemoveTopic(topic) |
||||
|
} |
||||
|
} |
@ -0,0 +1,119 @@ |
|||||
|
package sub_coordinator |
||||
|
|
||||
|
import ( |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"time" |
||||
|
) |
||||
|
|
||||
|
type PartitionConsumerMapping struct { |
||||
|
currentMapping *PartitionSlotToConsumerInstanceList |
||||
|
prevMappings []*PartitionSlotToConsumerInstanceList |
||||
|
} |
||||
|
|
||||
|
func NewPartitionConsumerMapping(ringSize int32) *PartitionConsumerMapping { |
||||
|
newVersion := time.Now().UnixNano() |
||||
|
return &PartitionConsumerMapping{ |
||||
|
currentMapping: NewPartitionSlotToConsumerInstanceList(ringSize, newVersion), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Balance goal:
|
||||
|
// 1. max processing power utilization
|
||||
|
// 2. allow one consumer instance to be down unexpectedly
|
||||
|
// without affecting the processing power utilization
|
||||
|
|
||||
|
func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitions []*topic.Partition, consumerInstanceIds []string) { |
||||
|
if len(partitions) == 0 || len(consumerInstanceIds) == 0 { |
||||
|
return |
||||
|
} |
||||
|
newVersion := time.Now().UnixNano() |
||||
|
newMapping := NewPartitionSlotToConsumerInstanceList(partitions[0].RingSize, newVersion) |
||||
|
newMapping.PartitionSlots = doBalanceSticky(partitions, consumerInstanceIds, pcm.prevMappings[0]) |
||||
|
if pcm.currentMapping != nil { |
||||
|
pcm.prevMappings = append(pcm.prevMappings, pcm.currentMapping) |
||||
|
} |
||||
|
pcm.currentMapping = newMapping |
||||
|
} |
||||
|
|
||||
|
func doBalanceSticky(partitions []*topic.Partition, consumerInstanceIds []string, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) { |
||||
|
// collect previous consumer instance ids
|
||||
|
prevConsumerInstanceIds := make(map[string]struct{}) |
||||
|
if prevMapping != nil { |
||||
|
for _, prevPartitionSlot := range prevMapping.PartitionSlots { |
||||
|
if prevPartitionSlot.AssignedInstanceId != "" { |
||||
|
prevConsumerInstanceIds[prevPartitionSlot.AssignedInstanceId] = struct{}{} |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
// collect current consumer instance ids
|
||||
|
currConsumerInstanceIds := make(map[string]struct{}) |
||||
|
for _, consumerInstanceId := range consumerInstanceIds { |
||||
|
currConsumerInstanceIds[consumerInstanceId] = struct{}{} |
||||
|
} |
||||
|
|
||||
|
// check deleted consumer instances
|
||||
|
deletedConsumerInstanceIds := make(map[string]struct{}) |
||||
|
for consumerInstanceId := range prevConsumerInstanceIds { |
||||
|
if _, ok := currConsumerInstanceIds[consumerInstanceId]; !ok { |
||||
|
deletedConsumerInstanceIds[consumerInstanceId] = struct{}{} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// convert partition slots from list to a map
|
||||
|
prevPartitionSlotMap := make(map[string]*PartitionSlotToConsumerInstance) |
||||
|
if prevMapping != nil { |
||||
|
for _, partitionSlot := range prevMapping.PartitionSlots { |
||||
|
key := fmt.Sprintf("%d-%d", partitionSlot.RangeStart, partitionSlot.RangeStop) |
||||
|
prevPartitionSlotMap[key] = partitionSlot |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// make a copy of old mapping, skipping the deleted consumer instances
|
||||
|
newPartitionSlots := ToPartitionSlots(partitions) |
||||
|
for _, newPartitionSlot := range newPartitionSlots { |
||||
|
key := fmt.Sprintf("%d-%d", newPartitionSlot.RangeStart, newPartitionSlot.RangeStop) |
||||
|
if prevPartitionSlot, ok := prevPartitionSlotMap[key]; ok { |
||||
|
if _, ok := deletedConsumerInstanceIds[prevPartitionSlot.AssignedInstanceId]; !ok { |
||||
|
newPartitionSlot.AssignedInstanceId = prevPartitionSlot.AssignedInstanceId |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// for all consumer instances, count the average number of partitions
|
||||
|
// that are assigned to them
|
||||
|
consumerInstancePartitionCount := make(map[string]int) |
||||
|
for _, newPartitionSlot := range newPartitionSlots { |
||||
|
if newPartitionSlot.AssignedInstanceId != "" { |
||||
|
consumerInstancePartitionCount[newPartitionSlot.AssignedInstanceId]++ |
||||
|
} |
||||
|
} |
||||
|
// average number of partitions that are assigned to each consumer instance
|
||||
|
averageConsumerInstanceLoad := float32(len(partitions)) / float32(len(consumerInstanceIds)) |
||||
|
|
||||
|
// assign unassigned partition slots to consumer instances that is underloaded
|
||||
|
consumerInstanceIdsIndex := 0 |
||||
|
for _, newPartitionSlot := range newPartitionSlots { |
||||
|
if newPartitionSlot.AssignedInstanceId == "" { |
||||
|
for avoidDeadLoop := len(consumerInstanceIds); avoidDeadLoop > 0; avoidDeadLoop-- { |
||||
|
consumerInstanceId := consumerInstanceIds[consumerInstanceIdsIndex] |
||||
|
if float32(consumerInstancePartitionCount[consumerInstanceId]) < averageConsumerInstanceLoad { |
||||
|
newPartitionSlot.AssignedInstanceId = consumerInstanceId |
||||
|
consumerInstancePartitionCount[consumerInstanceId]++ |
||||
|
consumerInstanceIdsIndex++ |
||||
|
if consumerInstanceIdsIndex >= len(consumerInstanceIds) { |
||||
|
consumerInstanceIdsIndex = 0 |
||||
|
} |
||||
|
break |
||||
|
} else { |
||||
|
consumerInstanceIdsIndex++ |
||||
|
if consumerInstanceIdsIndex >= len(consumerInstanceIds) { |
||||
|
consumerInstanceIdsIndex = 0 |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return newPartitionSlots |
||||
|
} |
@ -0,0 +1,312 @@ |
|||||
|
package sub_coordinator |
||||
|
|
||||
|
import ( |
||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
"reflect" |
||||
|
"testing" |
||||
|
) |
||||
|
|
||||
|
func Test_doBalanceSticky(t *testing.T) { |
||||
|
type args struct { |
||||
|
partitions []*topic.Partition |
||||
|
consumerInstanceIds []string |
||||
|
prevMapping *PartitionSlotToConsumerInstanceList |
||||
|
} |
||||
|
tests := []struct { |
||||
|
name string |
||||
|
args args |
||||
|
wantPartitionSlots []*PartitionSlotToConsumerInstance |
||||
|
}{ |
||||
|
{ |
||||
|
name: "1 consumer instance, 1 partition", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1"}, |
||||
|
prevMapping: nil, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 1 partition", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"}, |
||||
|
prevMapping: nil, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "1 consumer instance, 2 partitions", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1"}, |
||||
|
prevMapping: nil, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 2 partitions", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"}, |
||||
|
prevMapping: nil, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 2 partitions, 1 deleted consumer instance", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"}, |
||||
|
prevMapping: &PartitionSlotToConsumerInstanceList{ |
||||
|
PartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-3", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 2 partitions, 1 new consumer instance", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"}, |
||||
|
prevMapping: &PartitionSlotToConsumerInstanceList{ |
||||
|
PartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-3", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-3", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 2 partitions, 1 new partition", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 100, |
||||
|
RangeStop: 150, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"}, |
||||
|
prevMapping: &PartitionSlotToConsumerInstanceList{ |
||||
|
PartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 100, |
||||
|
RangeStop: 150, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
{ |
||||
|
name: "2 consumer instances, 2 partitions, 1 new partition, 1 new consumer instance", |
||||
|
args: args{ |
||||
|
partitions: []*topic.Partition{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 100, |
||||
|
RangeStop: 150, |
||||
|
}, |
||||
|
}, |
||||
|
consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"}, |
||||
|
prevMapping: &PartitionSlotToConsumerInstanceList{ |
||||
|
PartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
wantPartitionSlots: []*PartitionSlotToConsumerInstance{ |
||||
|
{ |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 50, |
||||
|
AssignedInstanceId: "consumer-instance-1", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 50, |
||||
|
RangeStop: 100, |
||||
|
AssignedInstanceId: "consumer-instance-2", |
||||
|
}, |
||||
|
{ |
||||
|
RangeStart: 100, |
||||
|
RangeStop: 150, |
||||
|
AssignedInstanceId: "consumer-instance-3", |
||||
|
}, |
||||
|
}, |
||||
|
}, |
||||
|
} |
||||
|
for _, tt := range tests { |
||||
|
t.Run(tt.name, func(t *testing.T) { |
||||
|
if gotPartitionSlots := doBalanceSticky(tt.args.partitions, tt.args.consumerInstanceIds, tt.args.prevMapping); !reflect.DeepEqual(gotPartitionSlots, tt.wantPartitionSlots) { |
||||
|
t.Errorf("doBalanceSticky() = %v, want %v", gotPartitionSlots, tt.wantPartitionSlots) |
||||
|
} |
||||
|
}) |
||||
|
} |
||||
|
} |
@ -0,0 +1,32 @@ |
|||||
|
package sub_coordinator |
||||
|
|
||||
|
import "github.com/seaweedfs/seaweedfs/weed/mq/topic" |
||||
|
|
||||
|
type PartitionSlotToConsumerInstance struct { |
||||
|
RangeStart int32 |
||||
|
RangeStop int32 |
||||
|
AssignedInstanceId string |
||||
|
} |
||||
|
|
||||
|
type PartitionSlotToConsumerInstanceList struct { |
||||
|
PartitionSlots []*PartitionSlotToConsumerInstance |
||||
|
RingSize int32 |
||||
|
Version int64 |
||||
|
} |
||||
|
|
||||
|
func NewPartitionSlotToConsumerInstanceList(ringSize int32, version int64) *PartitionSlotToConsumerInstanceList { |
||||
|
return &PartitionSlotToConsumerInstanceList{ |
||||
|
RingSize: ringSize, |
||||
|
Version: version, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func ToPartitionSlots(partitions []*topic.Partition) (partitionSlots []*PartitionSlotToConsumerInstance) { |
||||
|
for _, partition := range partitions { |
||||
|
partitionSlots = append(partitionSlots, &PartitionSlotToConsumerInstance{ |
||||
|
RangeStart: partition.RangeStart, |
||||
|
RangeStop: partition.RangeStop, |
||||
|
}) |
||||
|
} |
||||
|
return |
||||
|
} |
@ -0,0 +1,52 @@ |
|||||
|
package topic |
||||
|
|
||||
|
import "sync" |
||||
|
|
||||
|
type LocalPartitionPublishers struct { |
||||
|
publishers map[string]*LocalPublisher |
||||
|
publishersLock sync.RWMutex |
||||
|
} |
||||
|
type LocalPublisher struct { |
||||
|
} |
||||
|
|
||||
|
func NewLocalPublisher() *LocalPublisher { |
||||
|
return &LocalPublisher{} |
||||
|
} |
||||
|
func (p *LocalPublisher) SignalShutdown() { |
||||
|
} |
||||
|
|
||||
|
func NewLocalPartitionPublishers() *LocalPartitionPublishers { |
||||
|
return &LocalPartitionPublishers{ |
||||
|
publishers: make(map[string]*LocalPublisher), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionPublishers) AddPublisher(clientName string, publisher *LocalPublisher) { |
||||
|
p.publishersLock.Lock() |
||||
|
defer p.publishersLock.Unlock() |
||||
|
|
||||
|
p.publishers[clientName] = publisher |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionPublishers) RemovePublisher(clientName string) { |
||||
|
p.publishersLock.Lock() |
||||
|
defer p.publishersLock.Unlock() |
||||
|
|
||||
|
delete(p.publishers, clientName) |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionPublishers) SignalShutdown() { |
||||
|
p.publishersLock.RLock() |
||||
|
defer p.publishersLock.RUnlock() |
||||
|
|
||||
|
for _, publisher := range p.publishers { |
||||
|
publisher.SignalShutdown() |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionPublishers) IsEmpty() bool { |
||||
|
p.publishersLock.RLock() |
||||
|
defer p.publishersLock.RUnlock() |
||||
|
|
||||
|
return len(p.publishers) == 0 |
||||
|
} |
@ -0,0 +1,49 @@ |
|||||
|
package topic |
||||
|
|
||||
|
import "sync" |
||||
|
|
||||
|
type LocalPartitionSubscribers struct { |
||||
|
Subscribers map[string]*LocalSubscriber |
||||
|
SubscribersLock sync.RWMutex |
||||
|
} |
||||
|
type LocalSubscriber struct { |
||||
|
stopCh chan struct{} |
||||
|
} |
||||
|
|
||||
|
func NewLocalSubscriber() *LocalSubscriber { |
||||
|
return &LocalSubscriber{ |
||||
|
stopCh: make(chan struct{}, 1), |
||||
|
} |
||||
|
} |
||||
|
func (p *LocalSubscriber) SignalShutdown() { |
||||
|
close(p.stopCh) |
||||
|
} |
||||
|
|
||||
|
func NewLocalPartitionSubscribers() *LocalPartitionSubscribers { |
||||
|
return &LocalPartitionSubscribers{ |
||||
|
Subscribers: make(map[string]*LocalSubscriber), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionSubscribers) AddSubscriber(clientName string, Subscriber *LocalSubscriber) { |
||||
|
p.SubscribersLock.Lock() |
||||
|
defer p.SubscribersLock.Unlock() |
||||
|
|
||||
|
p.Subscribers[clientName] = Subscriber |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionSubscribers) RemoveSubscriber(clientName string) { |
||||
|
p.SubscribersLock.Lock() |
||||
|
defer p.SubscribersLock.Unlock() |
||||
|
|
||||
|
delete(p.Subscribers, clientName) |
||||
|
} |
||||
|
|
||||
|
func (p *LocalPartitionSubscribers) SignalShutdown() { |
||||
|
p.SubscribersLock.RLock() |
||||
|
defer p.SubscribersLock.RUnlock() |
||||
|
|
||||
|
for _, Subscriber := range p.Subscribers { |
||||
|
Subscriber.SignalShutdown() |
||||
|
} |
||||
|
} |
2688
weed/pb/mq_pb/mq.pb.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -0,0 +1,46 @@ |
|||||
|
package shell |
||||
|
|
||||
|
import ( |
||||
|
"context" |
||||
|
"fmt" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb" |
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" |
||||
|
"io" |
||||
|
) |
||||
|
|
||||
|
func init() { |
||||
|
Commands = append(Commands, &commandMqBalanceTopics{}) |
||||
|
} |
||||
|
|
||||
|
type commandMqBalanceTopics struct { |
||||
|
} |
||||
|
|
||||
|
func (c *commandMqBalanceTopics) Name() string { |
||||
|
return "mq.balance" |
||||
|
} |
||||
|
|
||||
|
func (c *commandMqBalanceTopics) Help() string { |
||||
|
return `balance topic partitions |
||||
|
|
||||
|
` |
||||
|
} |
||||
|
|
||||
|
func (c *commandMqBalanceTopics) Do(args []string, commandEnv *CommandEnv, writer io.Writer) error { |
||||
|
|
||||
|
// find the broker balancer
|
||||
|
brokerBalancer, err := findBrokerBalancer(commandEnv) |
||||
|
if err != nil { |
||||
|
return err |
||||
|
} |
||||
|
fmt.Fprintf(writer, "current balancer: %s\n", brokerBalancer) |
||||
|
|
||||
|
// balance topics
|
||||
|
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { |
||||
|
_, err := client.BalanceTopics(context.Background(), &mq_pb.BalanceTopicsRequest{}) |
||||
|
if err != nil { |
||||
|
return err |
||||
|
} |
||||
|
return nil |
||||
|
}) |
||||
|
|
||||
|
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue