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 ( |
|||
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