From 580940bf8214ac467694a09436ffc6d97066b97c Mon Sep 17 00:00:00 2001 From: Chris Lu Date: Mon, 11 Dec 2023 12:05:54 -0800 Subject: [PATCH] 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 * comment --- .gitignore | 1 + docker/compose/local-mq-test.yml | 27 + weed/cluster/lock_client.go | 5 +- weed/mq/broker/broker_grpc_admin.go | 225 +- weed/mq/broker/broker_grpc_balance.go | 31 + weed/mq/broker/broker_grpc_configure.go | 107 + weed/mq/broker/broker_grpc_create.go | 72 - weed/mq/broker/broker_grpc_lookup.go | 28 +- weed/mq/broker/broker_grpc_pub.go | 134 +- ...alancer.go => broker_grpc_pub_balancer.go} | 26 +- weed/mq/broker/broker_grpc_sub.go | 56 +- weed/mq/broker/broker_grpc_sub_coordinator.go | 77 + .../broker_grpc_topic_partition_control.go | 28 + weed/mq/broker/broker_segment_serde.go | 89 - weed/mq/broker/broker_server.go | 51 +- weed/mq/broker/broker_stats.go | 30 +- weed/mq/client/cmd/weed_pub/publisher.go | 14 +- weed/mq/client/cmd/weed_sub/subscriber.go | 18 +- weed/mq/client/pub_client/connect.go | 73 + weed/mq/client/pub_client/lookup.go | 124 +- weed/mq/client/pub_client/publish.go | 4 +- weed/mq/client/pub_client/publisher.go | 26 +- weed/mq/client/sub_client/process.go | 4 + weed/mq/client/sub_client/subscribe.go | 17 +- weed/mq/client/sub_client/subscriber.go | 20 +- weed/mq/coordinator/consumer_group.go | 92 - weed/mq/coordinator/coordinator.go | 36 - .../mq/{balancer => pub_balancer}/allocate.go | 4 +- .../allocate_test.go | 2 +- weed/mq/pub_balancer/balance.go | 73 + weed/mq/pub_balancer/balance_action.go | 58 + weed/mq/pub_balancer/balance_action_split.go | 43 + weed/mq/pub_balancer/balance_brokers.go | 52 + weed/mq/pub_balancer/balance_brokers_test.go | 75 + weed/mq/pub_balancer/balancer.go | 83 + .../broker_stats.go} | 77 +- weed/mq/{balancer => pub_balancer}/lookup.go | 19 +- weed/mq/pub_balancer/partition_list_broker.go | 50 + weed/mq/pub_balancer/repair.go | 127 + weed/mq/pub_balancer/repair_test.go | 97 + weed/mq/sub_coordinator/consumer_group.go | 41 + weed/mq/sub_coordinator/coordinator.go | 86 + .../partition_consumer_mapping.go | 119 + .../partition_consumer_mapping_test.go | 312 ++ weed/mq/sub_coordinator/partition_list.go | 32 + weed/mq/topic/local_manager.go | 29 +- weed/mq/topic/local_partition.go | 59 +- weed/mq/topic/local_partition_publishers.go | 52 + weed/mq/topic/local_partition_subscribers.go | 49 + weed/mq/topic/local_topic.go | 58 + weed/mq/topic/partition.go | 17 +- weed/mq/topic/topic.go | 50 +- weed/pb/mq.proto | 154 +- weed/pb/mq_pb/mq.pb.go | 2822 ++++++++--------- weed/pb/mq_pb/mq_grpc.pb.go | 484 ++- weed/shell/command_mq_balance.go | 46 + ...reate.go => command_mq_topic_configure.go} | 16 +- weed/shell/command_mq_topic_list.go | 4 +- 58 files changed, 3853 insertions(+), 2752 deletions(-) create mode 100644 docker/compose/local-mq-test.yml create mode 100644 weed/mq/broker/broker_grpc_balance.go create mode 100644 weed/mq/broker/broker_grpc_configure.go delete mode 100644 weed/mq/broker/broker_grpc_create.go rename weed/mq/broker/{broker_grpc_balancer.go => broker_grpc_pub_balancer.go} (54%) create mode 100644 weed/mq/broker/broker_grpc_sub_coordinator.go create mode 100644 weed/mq/broker/broker_grpc_topic_partition_control.go delete mode 100644 weed/mq/broker/broker_segment_serde.go create mode 100644 weed/mq/client/pub_client/connect.go delete mode 100644 weed/mq/coordinator/consumer_group.go delete mode 100644 weed/mq/coordinator/coordinator.go rename weed/mq/{balancer => pub_balancer}/allocate.go (91%) rename weed/mq/{balancer => pub_balancer}/allocate_test.go (98%) create mode 100644 weed/mq/pub_balancer/balance.go create mode 100644 weed/mq/pub_balancer/balance_action.go create mode 100644 weed/mq/pub_balancer/balance_action_split.go create mode 100644 weed/mq/pub_balancer/balance_brokers.go create mode 100644 weed/mq/pub_balancer/balance_brokers_test.go create mode 100644 weed/mq/pub_balancer/balancer.go rename weed/mq/{balancer/balancer.go => pub_balancer/broker_stats.go} (56%) rename weed/mq/{balancer => pub_balancer}/lookup.go (60%) create mode 100644 weed/mq/pub_balancer/partition_list_broker.go create mode 100644 weed/mq/pub_balancer/repair.go create mode 100644 weed/mq/pub_balancer/repair_test.go create mode 100644 weed/mq/sub_coordinator/consumer_group.go create mode 100644 weed/mq/sub_coordinator/coordinator.go create mode 100644 weed/mq/sub_coordinator/partition_consumer_mapping.go create mode 100644 weed/mq/sub_coordinator/partition_consumer_mapping_test.go create mode 100644 weed/mq/sub_coordinator/partition_list.go create mode 100644 weed/mq/topic/local_partition_publishers.go create mode 100644 weed/mq/topic/local_partition_subscribers.go create mode 100644 weed/shell/command_mq_balance.go rename weed/shell/{command_mq_topic_create.go => command_mq_topic_configure.go} (73%) diff --git a/.gitignore b/.gitignore index 3e09f6e38..4870a7848 100644 --- a/.gitignore +++ b/.gitignore @@ -88,3 +88,4 @@ other/java/hdfs/dependency-reduced-pom.xml # binary file weed/weed weed/mq/client/cmd/weed_pub/weed_pub +docker/weed diff --git a/docker/compose/local-mq-test.yml b/docker/compose/local-mq-test.yml new file mode 100644 index 000000000..d4b071039 --- /dev/null +++ b/docker/compose/local-mq-test.yml @@ -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 diff --git a/weed/cluster/lock_client.go b/weed/cluster/lock_client.go index bb1052656..c21f20874 100644 --- a/weed/cluster/lock_client.go +++ b/weed/cluster/lock_client.go @@ -60,12 +60,12 @@ func (lc *LockClient) StartLock(key string, owner string) (lock *LiveLock) { util.RetryUntil("create lock:"+key, func() error { errorMessage, err := lock.doLock(lock_manager.MaxDuration) if err != nil { - glog.Infof("create lock %s: %s", key, err) + glog.V(0).Infof("create lock %s: %s", key, err) time.Sleep(time.Second) return err } if errorMessage != "" { - glog.Infof("create lock %s: %s", key, errorMessage) + glog.V(4).Infof("create lock %s: %s", key, errorMessage) time.Sleep(time.Second) return fmt.Errorf("%v", errorMessage) } @@ -73,7 +73,6 @@ func (lc *LockClient) StartLock(key string, owner string) (lock *LiveLock) { return nil }, func(err error) (shouldContinue bool) { if err != nil { - glog.Warningf("create lock %s: %s", key, err) time.Sleep(time.Second) } return lock.renewToken == "" diff --git a/weed/mq/broker/broker_grpc_admin.go b/weed/mq/broker/broker_grpc_admin.go index b24bf08a4..1313d09ec 100644 --- a/weed/mq/broker/broker_grpc_admin.go +++ b/weed/mq/broker/broker_grpc_admin.go @@ -3,18 +3,13 @@ package broker import ( "context" "github.com/seaweedfs/seaweedfs/weed/cluster" - "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/mq/topic" - "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/master_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" - "sort" - "sync" ) -func (broker *MessageQueueBroker) FindBrokerLeader(c context.Context, request *mq_pb.FindBrokerLeaderRequest) (*mq_pb.FindBrokerLeaderResponse, error) { +func (b *MessageQueueBroker) FindBrokerLeader(c context.Context, request *mq_pb.FindBrokerLeaderRequest) (*mq_pb.FindBrokerLeaderResponse, error) { ret := &mq_pb.FindBrokerLeaderResponse{} - err := broker.withMasterClient(false, broker.MasterClient.GetMaster(), func(client master_pb.SeaweedClient) error { + err := b.withMasterClient(false, b.MasterClient.GetMaster(), func(client master_pb.SeaweedClient) error { resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{ ClientType: cluster.BrokerType, FilerGroup: request.FilerGroup, @@ -30,219 +25,3 @@ func (broker *MessageQueueBroker) FindBrokerLeader(c context.Context, request *m }) return ret, err } - -func (broker *MessageQueueBroker) AssignSegmentBrokers(c context.Context, request *mq_pb.AssignSegmentBrokersRequest) (*mq_pb.AssignSegmentBrokersResponse, error) { - ret := &mq_pb.AssignSegmentBrokersResponse{} - segment := topic.FromPbSegment(request.Segment) - - // check existing segment locations on filer - existingBrokers, err := broker.checkSegmentOnFiler(segment) - if err != nil { - return ret, err - } - - if len(existingBrokers) > 0 { - // good if the segment is still on the brokers - isActive, err := broker.checkSegmentsOnBrokers(segment, existingBrokers) - if err != nil { - return ret, err - } - if isActive { - for _, broker := range existingBrokers { - ret.Brokers = append(ret.Brokers, string(broker)) - } - return ret, nil - } - } - - // randomly pick up to 10 brokers, and find the ones with the lightest load - selectedBrokers, err := broker.selectBrokers() - if err != nil { - return ret, err - } - - // save the allocated brokers info for this segment on the filer - if err := broker.saveSegmentBrokersOnFiler(segment, selectedBrokers); err != nil { - return ret, err - } - - for _, broker := range selectedBrokers { - ret.Brokers = append(ret.Brokers, string(broker)) - } - return ret, nil -} - -func (broker *MessageQueueBroker) CheckSegmentStatus(c context.Context, request *mq_pb.CheckSegmentStatusRequest) (*mq_pb.CheckSegmentStatusResponse, error) { - ret := &mq_pb.CheckSegmentStatusResponse{} - // TODO add in memory active segment - return ret, nil -} - -func (broker *MessageQueueBroker) CheckBrokerLoad(c context.Context, request *mq_pb.CheckBrokerLoadRequest) (*mq_pb.CheckBrokerLoadResponse, error) { - ret := &mq_pb.CheckBrokerLoadResponse{} - // TODO read broker's load - return ret, nil -} - -// createOrUpdateTopicPartitions creates the topic partitions on the broker -// 1. check -func (broker *MessageQueueBroker) createOrUpdateTopicPartitions(topic *topic.Topic, prevAssignments []*mq_pb.BrokerPartitionAssignment) (err error) { - // create or update each partition - if prevAssignments == nil { - broker.createOrUpdateTopicPartition(topic, nil) - } else { - for _, brokerPartitionAssignment := range prevAssignments { - broker.createOrUpdateTopicPartition(topic, brokerPartitionAssignment) - } - } - return nil -} - -func (broker *MessageQueueBroker) createOrUpdateTopicPartition(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionAssignment) (newAssignment *mq_pb.BrokerPartitionAssignment) { - shouldCreate := broker.confirmBrokerPartitionAssignment(topic, oldAssignment) - if !shouldCreate { - - } - return -} -func (broker *MessageQueueBroker) confirmBrokerPartitionAssignment(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionAssignment) (shouldCreate bool) { - if oldAssignment == nil { - return true - } - for _, b := range oldAssignment.FollowerBrokers { - pb.WithBrokerGrpcClient(false, b, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { - _, err := client.CheckTopicPartitionsStatus(context.Background(), &mq_pb.CheckTopicPartitionsStatusRequest{ - Namespace: string(topic.Namespace), - Topic: topic.Name, - BrokerPartitionAssignment: oldAssignment, - ShouldCancelIfNotMatch: true, - }) - if err != nil { - shouldCreate = true - } - return nil - }) - } - return -} - -func (broker *MessageQueueBroker) checkSegmentsOnBrokers(segment *topic.Segment, brokers []pb.ServerAddress) (active bool, err error) { - var wg sync.WaitGroup - - for _, candidate := range brokers { - wg.Add(1) - go func(candidate pb.ServerAddress) { - defer wg.Done() - broker.withBrokerClient(false, candidate, func(client mq_pb.SeaweedMessagingClient) error { - resp, checkErr := client.CheckSegmentStatus(context.Background(), &mq_pb.CheckSegmentStatusRequest{ - Segment: &mq_pb.Segment{ - Namespace: string(segment.Topic.Namespace), - Topic: segment.Topic.Name, - Id: segment.Id, - }, - }) - if checkErr != nil { - err = checkErr - glog.V(0).Infof("check segment status on %s: %v", candidate, checkErr) - return nil - } - if resp.IsActive == false { - active = false - } - return nil - }) - }(candidate) - } - wg.Wait() - return -} - -func (broker *MessageQueueBroker) selectBrokers() (brokers []pb.ServerAddress, err error) { - candidates, err := broker.selectCandidatesFromMaster(10) - if err != nil { - return - } - brokers, err = broker.pickLightestCandidates(candidates, 3) - return -} - -func (broker *MessageQueueBroker) selectCandidatesFromMaster(limit int32) (candidates []pb.ServerAddress, err error) { - err = broker.withMasterClient(false, broker.MasterClient.GetMaster(), func(client master_pb.SeaweedClient) error { - resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{ - ClientType: cluster.BrokerType, - FilerGroup: broker.option.FilerGroup, - Limit: limit, - }) - if err != nil { - return err - } - if len(resp.ClusterNodes) == 0 { - return nil - } - for _, node := range resp.ClusterNodes { - candidates = append(candidates, pb.ServerAddress(node.Address)) - } - return nil - }) - return -} - -type CandidateStatus struct { - address pb.ServerAddress - messageCount int64 - bytesCount int64 - load int64 -} - -func (broker *MessageQueueBroker) pickLightestCandidates(candidates []pb.ServerAddress, limit int) (selected []pb.ServerAddress, err error) { - - if len(candidates) <= limit { - return candidates, nil - } - - candidateStatuses, err := broker.checkBrokerStatus(candidates) - if err != nil { - return nil, err - } - - sort.Slice(candidateStatuses, func(i, j int) bool { - return candidateStatuses[i].load < candidateStatuses[j].load - }) - - for i, candidate := range candidateStatuses { - if i >= limit { - break - } - selected = append(selected, candidate.address) - } - - return -} - -func (broker *MessageQueueBroker) checkBrokerStatus(candidates []pb.ServerAddress) (candidateStatuses []*CandidateStatus, err error) { - - candidateStatuses = make([]*CandidateStatus, len(candidates)) - var wg sync.WaitGroup - for i, candidate := range candidates { - wg.Add(1) - go func(i int, candidate pb.ServerAddress) { - defer wg.Done() - err = broker.withBrokerClient(false, candidate, func(client mq_pb.SeaweedMessagingClient) error { - resp, checkErr := client.CheckBrokerLoad(context.Background(), &mq_pb.CheckBrokerLoadRequest{}) - if checkErr != nil { - err = checkErr - return err - } - candidateStatuses[i] = &CandidateStatus{ - address: candidate, - messageCount: resp.MessageCount, - bytesCount: resp.BytesCount, - load: resp.MessageCount + resp.BytesCount/(64*1024), - } - return nil - }) - }(i, candidate) - } - wg.Wait() - return -} diff --git a/weed/mq/broker/broker_grpc_balance.go b/weed/mq/broker/broker_grpc_balance.go new file mode 100644 index 000000000..c09161ff9 --- /dev/null +++ b/weed/mq/broker/broker_grpc_balance.go @@ -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 +} diff --git a/weed/mq/broker/broker_grpc_configure.go b/weed/mq/broker/broker_grpc_configure.go new file mode 100644 index 000000000..7f7c8f84b --- /dev/null +++ b/weed/mq/broker/broker_grpc_configure.go @@ -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 +} diff --git a/weed/mq/broker/broker_grpc_create.go b/weed/mq/broker/broker_grpc_create.go deleted file mode 100644 index 15b3efd26..000000000 --- a/weed/mq/broker/broker_grpc_create.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/broker/broker_grpc_lookup.go b/weed/mq/broker/broker_grpc_lookup.go index 74a3a9822..4e9c9e441 100644 --- a/weed/mq/broker/broker_grpc_lookup.go +++ b/weed/mq/broker/broker_grpc_lookup.go @@ -20,12 +20,12 @@ import ( // 2.2 if the topic is found, return the brokers // // 3. unlock the topic -func (broker *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq_pb.LookupTopicBrokersRequest) (resp *mq_pb.LookupTopicBrokersResponse, err error) { - if broker.currentBalancer == "" { +func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq_pb.LookupTopicBrokersRequest) (resp *mq_pb.LookupTopicBrokersResponse, err error) { + if b.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 { + if !b.lockAsBalancer.IsLocked() { + proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.LookupTopicBrokers(ctx, request) return nil }) @@ -37,22 +37,16 @@ func (broker *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, reques ret := &mq_pb.LookupTopicBrokersResponse{} ret.Topic = request.Topic - ret.BrokerPartitionAssignments, err = broker.Balancer.LookupOrAllocateTopicPartitions(ret.Topic, request.IsForPublish, 6) + ret.BrokerPartitionAssignments, err = b.Balancer.LookupOrAllocateTopicPartitions(ret.Topic, request.IsForPublish, 6) return ret, err } -// CheckTopicPartitionsStatus check the topic partitions on the broker -func (broker *MessageQueueBroker) CheckTopicPartitionsStatus(c context.Context, request *mq_pb.CheckTopicPartitionsStatusRequest) (*mq_pb.CheckTopicPartitionsStatusResponse, error) { - ret := &mq_pb.CheckTopicPartitionsStatusResponse{} - return ret, nil -} - -func (broker *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.ListTopicsRequest) (resp *mq_pb.ListTopicsResponse, err error) { - if broker.currentBalancer == "" { +func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.ListTopicsRequest) (resp *mq_pb.ListTopicsResponse, err error) { + if b.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 { + if !b.lockAsBalancer.IsLocked() { + proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error { resp, err = client.ListTopics(ctx, request) return nil }) @@ -64,9 +58,9 @@ func (broker *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb ret := &mq_pb.ListTopicsResponse{} knownTopics := make(map[string]struct{}) - for brokerStatsItem := range broker.Balancer.Brokers.IterBuffered() { + for brokerStatsItem := range b.Balancer.Brokers.IterBuffered() { _, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val - for topicPartitionStatsItem := range brokerStats.Stats.IterBuffered() { + for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() { topicPartitionStat := topicPartitionStatsItem.Val topic := &mq_pb.Topic{ Namespace: topicPartitionStat.TopicPartition.Namespace, diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go index acbffefba..43280e9be 100644 --- a/weed/mq/broker/broker_grpc_pub.go +++ b/weed/mq/broker/broker_grpc_pub.go @@ -5,71 +5,36 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" - "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "google.golang.org/grpc/peer" + "math/rand" + "net" "sync/atomic" "time" ) -// For a new or re-configured topic, or one of the broker went offline, -// the pub clients ask one broker what are the brokers for all the topic partitions. -// The broker will lock the topic on write. -// 1. if the topic is not found, create the topic, and allocate the topic partitions to the brokers -// 2. if the topic is found, return the brokers for the topic partitions -// For a topic to read from, the sub clients ask one broker what are the brokers for all the topic partitions. -// The broker will lock the topic on read. -// 1. if the topic is not found, return error -// 2. if the topic is found, return the brokers for the topic partitions -// -// If the topic needs to be re-balanced, the admin client will lock the topic, -// 1. collect throughput information for all the brokers -// 2. adjust the topic partitions to the brokers -// 3. notify the brokers to add/remove partitions to host -// 3.1 When locking the topic, the partitions and brokers should be remembered in the lock. -// 4. the brokers will stop process incoming messages if not the right partition -// 4.1 the pub clients will need to re-partition the messages and publish to the right brokers for the partition3 -// 4.2 the sub clients will need to change the brokers to read from -// -// The following is from each individual component's perspective: -// For a pub client -// For current topic/partition, ask one broker for the brokers for the topic partitions -// 1. connect to the brokers and keep sending, until the broker returns error, or the broker leader is moved. -// For a sub client -// For current topic/partition, ask one broker for the brokers for the topic partitions -// 1. connect to the brokers and keep reading, until the broker returns error, or the broker leader is moved. -// For a broker -// Upon a pub client lookup: -// 1. lock the topic -// 2. if already has topic partition assignment, check all brokers are healthy -// 3. if not, create topic partition assignment -// 2. return the brokers for the topic partitions -// 3. unlock the topic -// Upon a sub client lookup: -// 1. lock the topic -// 2. if already has topic partition assignment, check all brokers are healthy -// 3. if not, return error -// 2. return the brokers for the topic partitions -// 3. unlock the topic -// For an admin tool -// 0. collect stats from all the brokers, and find the topic worth moving -// 1. lock the topic -// 2. collect throughput information for all the brokers -// 3. adjust the topic partitions to the brokers -// 4. notify the brokers to add/remove partitions to host -// 5. the brokers will stop process incoming messages if not the right partition -// 6. unlock the topic +// PUB +// 1. gRPC API to configure a topic +// 1.1 create a topic with existing partition count +// 1.2 assign partitions to brokers +// 2. gRPC API to lookup topic partitions +// 3. gRPC API to publish by topic partitions -/* -The messages are buffered in memory, and saved to filer under - /topics/////*.msg - /topics////segment - /topics//info/segment_.meta +// SUB +// 1. gRPC API to lookup a topic partitions +// Re-balance topic partitions for publishing +// 1. collect stats from all the brokers +// 2. Rebalance and configure new generation of partitions on brokers +// 3. Tell brokers to close current gneration of publishing. +// Publishers needs to lookup again and publish to the new generation of partitions. +// Re-balance topic partitions for subscribing +// 1. collect stats from all the brokers +// Subscribers needs to listen for new partitions and connect to the brokers. +// Each subscription may not get data. It can act as a backup. -*/ - -func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer) error { +func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer) error { // 1. write to the volume server // 2. find the topic metadata owning filer // 3. write to the filer @@ -85,19 +50,23 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS initMessage := req.GetInit() if initMessage != nil { t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition) - localTopicPartition = broker.localTopicManager.GetTopicPartition(t, p) + localTopicPartition = b.localTopicManager.GetTopicPartition(t, p) if localTopicPartition == nil { - localTopicPartition = topic.NewLocalPartition(t, p, true, nil) - broker.localTopicManager.AddTopicPartition(t, localTopicPartition) + response.Error = fmt.Sprintf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition) + glog.Errorf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition) + return stream.Send(response) } ackInterval = int(initMessage.AckInterval) stream.Send(response) } else { - response.Error = fmt.Sprintf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition) - glog.Errorf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition) + response.Error = fmt.Sprintf("missing init message") + glog.Errorf("missing init message") return stream.Send(response) } + clientName := fmt.Sprintf("%v-%4d/%s/%v", findClientAddress(stream.Context()), rand.Intn(10000), initMessage.Topic, initMessage.Partition) + localTopicPartition.Publishers.AddPublisher(clientName, topic.NewLocalPublisher()) + ackCounter := 0 var ackSequence int64 var isStopping int32 @@ -105,6 +74,7 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS defer func() { atomic.StoreInt32(&isStopping, 1) close(respChan) + localTopicPartition.Publishers.RemovePublisher(clientName) }() go func() { ticker := time.NewTicker(1 * time.Second) @@ -127,6 +97,11 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS } else { return } + case <-localTopicPartition.StopPublishersCh: + respChan <- &mq_pb.PublishResponse{ + AckSequence: ackSequence, + ShouldClose: true, + } } } }() @@ -156,33 +131,22 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS } } - glog.Infof("publish stream closed") + glog.V(0).Infof("topic %v partition %v publish stream closed.", initMessage.Topic, initMessage.Partition) return nil } -// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment -func (broker *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) { - ret := &mq_pb.AssignTopicPartitionsResponse{} - self := pb.ServerAddress(fmt.Sprintf("%s:%d", broker.option.Ip, broker.option.Port)) - - for _, brokerPartition := range request.BrokerPartitionAssignments { - localPartiton := topic.FromPbBrokerPartitionAssignment(self, brokerPartition) - broker.localTopicManager.AddTopicPartition( - topic.FromPbTopic(request.Topic), - localPartiton) - if request.IsLeader { - for _, follower := range localPartiton.FollowerBrokers { - err := pb.WithBrokerGrpcClient(false, follower.String(), broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { - _, err := client.AssignTopicPartitions(context.Background(), request) - return err - }) - if err != nil { - return ret, err - } - } - } +// duplicated from master_grpc_server.go +func findClientAddress(ctx context.Context) string { + // fmt.Printf("FromContext %+v\n", ctx) + pr, ok := peer.FromContext(ctx) + if !ok { + glog.Error("failed to get peer from ctx") + return "" } - - return ret, nil + if pr.Addr == net.Addr(nil) { + glog.Error("failed to get peer address") + return "" + } + return pr.Addr.String() } diff --git a/weed/mq/broker/broker_grpc_balancer.go b/weed/mq/broker/broker_grpc_pub_balancer.go similarity index 54% rename from weed/mq/broker/broker_grpc_balancer.go rename to weed/mq/broker/broker_grpc_pub_balancer.go index 5e23f89df..e3d49f816 100644 --- a/weed/mq/broker/broker_grpc_balancer.go +++ b/weed/mq/broker/broker_grpc_pub_balancer.go @@ -2,15 +2,15 @@ package broker import ( "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) -// ConnectToBalancer receives connections from brokers and collects stats -func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessaging_ConnectToBalancerServer) error { - if !broker.lockAsBalancer.IsLocked() { +// PublisherToPubBalancer receives connections from brokers and collects stats +func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessaging_PublisherToPubBalancerServer) error { + if !b.lockAsBalancer.IsLocked() { return status.Errorf(codes.Unavailable, "not current broker balancer") } req, err := stream.Recv() @@ -20,21 +20,14 @@ func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessagin // process init message initMessage := req.GetInit() - var brokerStats *balancer.BrokerStats + var brokerStats *pub_balancer.BrokerStats if initMessage != nil { - var found bool - brokerStats, found = broker.Balancer.Brokers.Get(initMessage.Broker) - if !found { - brokerStats = balancer.NewBrokerStats() - if !broker.Balancer.Brokers.SetIfAbsent(initMessage.Broker, brokerStats) { - brokerStats, _ = broker.Balancer.Brokers.Get(initMessage.Broker) - } - } + brokerStats = b.Balancer.OnBrokerConnected(initMessage.Broker) } else { return status.Errorf(codes.InvalidArgument, "balancer init message is empty") } defer func() { - broker.Balancer.Brokers.Remove(initMessage.Broker) + b.Balancer.OnBrokerDisconnected(initMessage.Broker, brokerStats) }() // process stats message @@ -43,12 +36,11 @@ func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessagin if err != nil { return err } - if !broker.lockAsBalancer.IsLocked() { + if !b.lockAsBalancer.IsLocked() { return status.Errorf(codes.Unavailable, "not current broker balancer") } if receivedStats := req.GetStats(); receivedStats != nil { - brokerStats.UpdateStats(receivedStats) - + b.Balancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats) glog.V(4).Infof("broker %s stats: %+v", initMessage.Broker, brokerStats) glog.V(4).Infof("received stats: %+v", receivedStats) } diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index 376ced03d..c98ce4684 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -1,6 +1,7 @@ package broker import ( + "context" "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" @@ -9,10 +10,11 @@ import ( "time" ) -func (broker *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream mq_pb.SeaweedMessaging_SubscribeServer) error { +func (b *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream mq_pb.SeaweedMessaging_SubscribeServer) error { - localTopicPartition := broker.localTopicManager.GetTopicPartition(topic.FromPbTopic(req.GetInit().Topic), - topic.FromPbPartition(req.GetInit().Partition)) + t := topic.FromPbTopic(req.GetInit().Topic) + partition := topic.FromPbPartition(req.GetInit().Partition) + localTopicPartition := b.localTopicManager.GetTopicPartition(t, partition) if localTopicPartition == nil { stream.Send(&mq_pb.SubscribeResponse{ Message: &mq_pb.SubscribeResponse_Ctrl{ @@ -25,13 +27,59 @@ func (broker *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream } clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId) + localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber()) + glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition) + isConnected := true + sleepIntervalCount := 0 + defer func() { + isConnected = false + localTopicPartition.Subscribers.RemoveSubscriber(clientName) + glog.V(0).Infof("Subscriber %s on %v %v disconnected", clientName, t, partition) + }() + + ctx := stream.Context() + var startTime time.Time + if startTs := req.GetInit().GetStartTimestampNs(); startTs > 0 { + startTime = time.Unix(0, startTs) + } else { + startTime = time.Now() + } + + localTopicPartition.Subscribe(clientName, startTime, func() bool { + if !isConnected { + return false + } + sleepIntervalCount++ + if sleepIntervalCount > 10 { + sleepIntervalCount = 10 + } + time.Sleep(time.Duration(sleepIntervalCount) * 2339 * time.Millisecond) + + // Check if the client has disconnected by monitoring the context + select { + case <-ctx.Done(): + err := ctx.Err() + if err == context.Canceled { + // Client disconnected + return false + } + glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err) + return false + default: + // Continue processing the request + } + + return true + }, func(logEntry *filer_pb.LogEntry) error { + // reset the sleep interval count + sleepIntervalCount = 0 - localTopicPartition.Subscribe(clientName, time.Now(), func(logEntry *filer_pb.LogEntry) error { value := logEntry.GetData() if err := stream.Send(&mq_pb.SubscribeResponse{Message: &mq_pb.SubscribeResponse_Data{ Data: &mq_pb.DataMessage{ Key: []byte(fmt.Sprintf("key-%d", logEntry.PartitionKeyHash)), Value: value, + TsNs: logEntry.TsNs, }, }}); err != nil { glog.Errorf("Error sending setup response: %v", err) diff --git a/weed/mq/broker/broker_grpc_sub_coordinator.go b/weed/mq/broker/broker_grpc_sub_coordinator.go new file mode 100644 index 000000000..349db3178 --- /dev/null +++ b/weed/mq/broker/broker_grpc_sub_coordinator.go @@ -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) + } + } + } +} diff --git a/weed/mq/broker/broker_grpc_topic_partition_control.go b/weed/mq/broker/broker_grpc_topic_partition_control.go new file mode 100644 index 000000000..66547b010 --- /dev/null +++ b/weed/mq/broker/broker_grpc_topic_partition_control.go @@ -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 +} diff --git a/weed/mq/broker/broker_segment_serde.go b/weed/mq/broker/broker_segment_serde.go deleted file mode 100644 index bb9aecc0b..000000000 --- a/weed/mq/broker/broker_segment_serde.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/broker/broker_server.go b/weed/mq/broker/broker_server.go index 94888e14c..52b34ddbc 100644 --- a/weed/mq/broker/broker_server.go +++ b/weed/mq/broker/broker_server.go @@ -3,7 +3,8 @@ package broker import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "time" @@ -37,20 +38,24 @@ type MessageQueueBroker struct { filers map[pb.ServerAddress]struct{} currentFiler pb.ServerAddress localTopicManager *topic.LocalTopicManager - Balancer *balancer.Balancer + Balancer *pub_balancer.Balancer lockAsBalancer *cluster.LiveLock currentBalancer pb.ServerAddress + Coordinator *sub_coordinator.Coordinator } func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.DialOption) (mqBroker *MessageQueueBroker, err error) { + pub_broker_balancer := pub_balancer.NewBalancer() + mqBroker = &MessageQueueBroker{ option: option, grpcDialOption: grpcDialOption, MasterClient: wdclient.NewMasterClient(grpcDialOption, option.FilerGroup, cluster.BrokerType, pb.NewServerAddress(option.Ip, option.Port, 0), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)), filers: make(map[pb.ServerAddress]struct{}), localTopicManager: topic.NewLocalTopicManager(), - Balancer: balancer.NewBalancer(), + Balancer: pub_broker_balancer, + Coordinator: sub_coordinator.NewCoordinator(pub_broker_balancer), } mqBroker.MasterClient.SetOnPeerUpdateFn(mqBroker.OnBrokerUpdate) @@ -67,10 +72,10 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial time.Sleep(time.Millisecond * 237) } self := fmt.Sprintf("%s:%d", option.Ip, option.Port) - glog.V(1).Infof("broker %s found filer %s", self, mqBroker.currentFiler) + glog.V(0).Infof("broker %s found filer %s", self, mqBroker.currentFiler) lockClient := cluster.NewLockClient(grpcDialOption, mqBroker.currentFiler) - mqBroker.lockAsBalancer = lockClient.StartLock(balancer.LockBrokerBalancer, self) + mqBroker.lockAsBalancer = lockClient.StartLock(pub_balancer.LockBrokerBalancer, self) for { err := mqBroker.BrokerConnectToBalancer(self) if err != nil { @@ -83,22 +88,22 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial return mqBroker, nil } -func (broker *MessageQueueBroker) OnBrokerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) { +func (b *MessageQueueBroker) OnBrokerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) { if update.NodeType != cluster.FilerType { return } address := pb.ServerAddress(update.Address) if update.IsAdd { - broker.filers[address] = struct{}{} - if broker.currentFiler == "" { - broker.currentFiler = address + b.filers[address] = struct{}{} + if b.currentFiler == "" { + b.currentFiler = address } } else { - delete(broker.filers, address) - if broker.currentFiler == address { - for filer := range broker.filers { - broker.currentFiler = filer + delete(b.filers, address) + if b.currentFiler == address { + for filer := range b.filers { + b.currentFiler = filer break } } @@ -106,39 +111,39 @@ func (broker *MessageQueueBroker) OnBrokerUpdate(update *master_pb.ClusterNodeUp } -func (broker *MessageQueueBroker) GetFiler() pb.ServerAddress { - return broker.currentFiler +func (b *MessageQueueBroker) GetFiler() pb.ServerAddress { + return b.currentFiler } -func (broker *MessageQueueBroker) WithFilerClient(streamingMode bool, fn func(filer_pb.SeaweedFilerClient) error) error { +func (b *MessageQueueBroker) WithFilerClient(streamingMode bool, fn func(filer_pb.SeaweedFilerClient) error) error { - return pb.WithFilerClient(streamingMode, 0, broker.GetFiler(), broker.grpcDialOption, fn) + return pb.WithFilerClient(streamingMode, 0, b.GetFiler(), b.grpcDialOption, fn) } -func (broker *MessageQueueBroker) AdjustedUrl(location *filer_pb.Location) string { +func (b *MessageQueueBroker) AdjustedUrl(location *filer_pb.Location) string { return location.Url } -func (broker *MessageQueueBroker) GetDataCenter() string { +func (b *MessageQueueBroker) GetDataCenter() string { return "" } -func (broker *MessageQueueBroker) withMasterClient(streamingMode bool, master pb.ServerAddress, fn func(client master_pb.SeaweedClient) error) error { +func (b *MessageQueueBroker) withMasterClient(streamingMode bool, master pb.ServerAddress, fn func(client master_pb.SeaweedClient) error) error { - return pb.WithMasterClient(streamingMode, master, broker.grpcDialOption, false, func(client master_pb.SeaweedClient) error { + return pb.WithMasterClient(streamingMode, master, b.grpcDialOption, false, func(client master_pb.SeaweedClient) error { return fn(client) }) } -func (broker *MessageQueueBroker) withBrokerClient(streamingMode bool, server pb.ServerAddress, fn func(client mq_pb.SeaweedMessagingClient) error) error { +func (b *MessageQueueBroker) withBrokerClient(streamingMode bool, server pb.ServerAddress, fn func(client mq_pb.SeaweedMessagingClient) error) error { - return pb.WithBrokerGrpcClient(streamingMode, server.String(), broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { + return pb.WithBrokerGrpcClient(streamingMode, server.String(), b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { return fn(client) }) diff --git a/weed/mq/broker/broker_stats.go b/weed/mq/broker/broker_stats.go index 0cc23837b..3cd217519 100644 --- a/weed/mq/broker/broker_stats.go +++ b/weed/mq/broker/broker_stats.go @@ -4,21 +4,22 @@ import ( "context" "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "io" "math/rand" "time" ) // BrokerConnectToBalancer connects to the broker balancer and sends stats -func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error { +func (b *MessageQueueBroker) BrokerConnectToBalancer(self string) error { // find the lock owner var brokerBalancer string - err := broker.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { + err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{ - Name: balancer.LockBrokerBalancer, + Name: pub_balancer.LockBrokerBalancer, }) if err != nil { return err @@ -29,7 +30,7 @@ func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error { if err != nil { return err } - broker.currentBalancer = pb.ServerAddress(brokerBalancer) + b.currentBalancer = pb.ServerAddress(brokerBalancer) glog.V(0).Infof("broker %s found balancer %s", self, brokerBalancer) if brokerBalancer == "" { @@ -37,15 +38,15 @@ func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error { } // connect to the lock owner - err = pb.WithBrokerGrpcClient(false, brokerBalancer, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { - stream, err := client.ConnectToBalancer(context.Background()) + err = pb.WithBrokerGrpcClient(false, brokerBalancer, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { + stream, err := client.PublisherToPubBalancer(context.Background()) if err != nil { return fmt.Errorf("connect to balancer %v: %v", brokerBalancer, err) } defer stream.CloseSend() - err = stream.Send(&mq_pb.ConnectToBalancerRequest{ - Message: &mq_pb.ConnectToBalancerRequest_Init{ - Init: &mq_pb.ConnectToBalancerRequest_InitMessage{ + err = stream.Send(&mq_pb.PublisherToPubBalancerRequest{ + Message: &mq_pb.PublisherToPubBalancerRequest_Init{ + Init: &mq_pb.PublisherToPubBalancerRequest_InitMessage{ Broker: self, }, }, @@ -55,13 +56,16 @@ func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error { } for { - stats := broker.localTopicManager.CollectStats(time.Second * 5) - err = stream.Send(&mq_pb.ConnectToBalancerRequest{ - Message: &mq_pb.ConnectToBalancerRequest_Stats{ + stats := b.localTopicManager.CollectStats(time.Second * 5) + err = stream.Send(&mq_pb.PublisherToPubBalancerRequest{ + Message: &mq_pb.PublisherToPubBalancerRequest_Stats{ Stats: stats, }, }) if err != nil { + if err == io.EOF { + return err + } return fmt.Errorf("send stats message: %v", err) } glog.V(3).Infof("sent stats: %+v", stats) diff --git a/weed/mq/client/cmd/weed_pub/publisher.go b/weed/mq/client/cmd/weed_pub/publisher.go index 03674db3f..ee00be9f8 100644 --- a/weed/mq/client/cmd/weed_pub/publisher.go +++ b/weed/mq/client/cmd/weed_pub/publisher.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" "log" + "strings" "sync" "time" ) @@ -12,6 +13,10 @@ import ( var ( messageCount = flag.Int("n", 1000, "message count") concurrency = flag.Int("c", 4, "concurrency count") + + namespace = flag.String("ns", "test", "namespace") + topic = flag.String("topic", "test", "topic") + seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") ) func doPublish(publisher *pub_client.TopicPublisher, id int) { @@ -29,9 +34,12 @@ func doPublish(publisher *pub_client.TopicPublisher, id int) { func main() { flag.Parse() - publisher := pub_client.NewTopicPublisher( - "test", "test") - if err := publisher.Connect("localhost:17777"); err != nil { + config := &pub_client.PublisherConfiguration{ + CreateTopic: true, + } + publisher := pub_client.NewTopicPublisher(*namespace, *topic, config) + brokers := strings.Split(*seedBrokers, ",") + if err := publisher.Connect(brokers); err != nil { fmt.Println(err) return } diff --git a/weed/mq/client/cmd/weed_sub/subscriber.go b/weed/mq/client/cmd/weed_sub/subscriber.go index 6cb18c574..d5bd8f12d 100644 --- a/weed/mq/client/cmd/weed_sub/subscriber.go +++ b/weed/mq/client/cmd/weed_sub/subscriber.go @@ -1,13 +1,23 @@ package main import ( + "flag" "fmt" "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + "strings" + "time" +) + +var ( + namespace = flag.String("ns", "test", "namespace") + topic = flag.String("topic", "test", "topic") + seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") ) func main() { + flag.Parse() subscriberConfig := &sub_client.SubscriberConfiguration{ ClientId: "testSubscriber", @@ -17,12 +27,14 @@ func main() { } contentConfig := &sub_client.ContentConfiguration{ - Namespace: "test", - Topic: "test", + Namespace: *namespace, + Topic: *topic, Filter: "", + StartTime: time.Now(), } - subscriber := sub_client.NewTopicSubscriber("localhost:17777", subscriberConfig, contentConfig) + brokers := strings.Split(*seedBrokers, ",") + subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig) subscriber.SetEachMessageFunc(func(key, value []byte) bool { println(string(key), "=>", string(value)) diff --git a/weed/mq/client/pub_client/connect.go b/weed/mq/client/pub_client/connect.go new file mode 100644 index 000000000..fc7ff4d77 --- /dev/null +++ b/weed/mq/client/pub_client/connect.go @@ -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 +} diff --git a/weed/mq/client/pub_client/lookup.go b/weed/mq/client/pub_client/lookup.go index 28cb29015..e55bfd256 100644 --- a/weed/mq/client/pub_client/lookup.go +++ b/weed/mq/client/pub_client/lookup.go @@ -5,11 +5,28 @@ import ( "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" ) -func (p *TopicPublisher) doLookup(brokerAddress string) error { +func (p *TopicPublisher) doLookupAndConnect(brokerAddress string) error { + if p.config.CreateTopic { + err := pb.WithBrokerGrpcClient(true, + brokerAddress, + p.grpcDialOption, + func(client mq_pb.SeaweedMessagingClient) error { + _, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{ + Topic: &mq_pb.Topic{ + Namespace: p.namespace, + Name: p.topic, + }, + PartitionCount: p.config.CreateTopicPartitionCount, + }) + return err + }) + if err != nil { + return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err) + } + } + err := pb.WithBrokerGrpcClient(true, brokerAddress, p.grpcDialOption, @@ -22,20 +39,35 @@ func (p *TopicPublisher) doLookup(brokerAddress string) error { }, IsForPublish: true, }) - if err != nil { - return err - } - for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments { - // partition => publishClient - publishClient, redirectTo, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker) + if p.config.CreateTopic && err != nil { + _, err = client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{ + Topic: &mq_pb.Topic{ + Namespace: p.namespace, + Name: p.topic, + }, + PartitionCount: p.config.CreateTopicPartitionCount, + }) if err != nil { return err } - for redirectTo != "" { - publishClient, redirectTo, err = p.doConnect(brokerPartitionAssignment.Partition, redirectTo) - if err != nil { - return err - } + lookupResp, err = client.LookupTopicBrokers(context.Background(), + &mq_pb.LookupTopicBrokersRequest{ + Topic: &mq_pb.Topic{ + Namespace: p.namespace, + Name: p.topic, + }, + IsForPublish: true, + }) + } + if err != nil { + return err + } + + for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments { + // partition => publishClient + publishClient, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker) + if err != nil { + return err } p.partition2Broker.Insert( brokerPartitionAssignment.Partition.RangeStart, @@ -50,67 +82,3 @@ func (p *TopicPublisher) doLookup(brokerAddress string) error { } return nil } - -// broker => publish client -// send init message -// save the publishing client -func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress string) (publishClient *PublishClient, redirectTo string, err error) { - grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption) - if err != nil { - return publishClient, redirectTo, fmt.Errorf("dial broker %s: %v", brokerAddress, err) - } - brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection) - stream, err := brokerClient.Publish(context.Background()) - if err != nil { - return publishClient, redirectTo, 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, redirectTo, fmt.Errorf("send init message: %v", err) - } - resp, err := stream.Recv() - if err != nil { - return publishClient, redirectTo, fmt.Errorf("recv init response: %v", err) - } - if resp.Error != "" { - return publishClient, redirectTo, fmt.Errorf("init response error: %v", resp.Error) - } - if resp.RedirectToBroker != "" { - redirectTo = resp.RedirectToBroker - return publishClient, redirectTo, nil - } - - 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, redirectTo, nil -} diff --git a/weed/mq/client/pub_client/publish.go b/weed/mq/client/pub_client/publish.go index 9495e380c..1e250ede3 100644 --- a/weed/mq/client/pub_client/publish.go +++ b/weed/mq/client/pub_client/publish.go @@ -2,13 +2,13 @@ package pub_client import ( "fmt" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/util" ) func (p *TopicPublisher) Publish(key, value []byte) error { - hashKey := util.HashToInt32(key) % balancer.MaxPartitionCount + hashKey := util.HashToInt32(key) % pub_balancer.MaxPartitionCount if hashKey < 0 { hashKey = -hashKey } diff --git a/weed/mq/client/pub_client/publisher.go b/weed/mq/client/pub_client/publisher.go index bf1711e38..a0c26db36 100644 --- a/weed/mq/client/pub_client/publisher.go +++ b/weed/mq/client/pub_client/publisher.go @@ -1,8 +1,9 @@ package pub_client import ( + "fmt" "github.com/rdleal/intervalst/interval" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" @@ -11,6 +12,8 @@ import ( ) type PublisherConfiguration struct { + CreateTopic bool + CreateTopicPartitionCount int32 } type PublishClient struct { @@ -24,9 +27,10 @@ type TopicPublisher struct { partition2Broker *interval.SearchTree[*PublishClient, int32] grpcDialOption grpc.DialOption sync.Mutex // protects grpc + config *PublisherConfiguration } -func NewTopicPublisher(namespace, topic string) *TopicPublisher { +func NewTopicPublisher(namespace, topic string, config *PublisherConfiguration) *TopicPublisher { return &TopicPublisher{ namespace: namespace, topic: topic, @@ -34,19 +38,27 @@ func NewTopicPublisher(namespace, topic string) *TopicPublisher { return int(a - b) }), grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), + config: config, } } -func (p *TopicPublisher) Connect(bootstrapBroker string) error { - if err := p.doLookup(bootstrapBroker); err != nil { - return err +func (p *TopicPublisher) Connect(bootstrapBrokers []string) (err error) { + if len(bootstrapBrokers) == 0 { + return nil } - return nil + for _, b := range bootstrapBrokers { + err = p.doLookupAndConnect(b) + if err == nil { + return nil + } + fmt.Printf("failed to connect to %s: %v\n\n", b, err) + } + return err } func (p *TopicPublisher) Shutdown() error { - if clients, found := p.partition2Broker.AllIntersections(0, balancer.MaxPartitionCount); found { + if clients, found := p.partition2Broker.AllIntersections(0, pub_balancer.MaxPartitionCount); found { for _, client := range clients { client.CloseSend() } diff --git a/weed/mq/client/sub_client/process.go b/weed/mq/client/sub_client/process.go index 7717a101f..b6bdb14ee 100644 --- a/weed/mq/client/sub_client/process.go +++ b/weed/mq/client/sub_client/process.go @@ -32,6 +32,9 @@ func (sub *TopicSubscriber) doProcess() error { RangeStop: brokerPartitionAssignment.Partition.RangeStop, }, Filter: sub.ContentConfig.Filter, + Offset: &mq_pb.SubscribeRequest_InitMessage_StartTimestampNs{ + StartTimestampNs: sub.alreadyProcessedTsNs, + }, }, }, }) @@ -68,6 +71,7 @@ func (sub *TopicSubscriber) doProcess() error { if !sub.OnEachMessageFunc(m.Data.Key, m.Data.Value) { return } + sub.alreadyProcessedTsNs = m.Data.TsNs case *mq_pb.SubscribeResponse_Ctrl: if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic { return diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go index 0803b2c79..370f5aa3c 100644 --- a/weed/mq/client/sub_client/subscribe.go +++ b/weed/mq/client/sub_client/subscribe.go @@ -4,17 +4,30 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/util" "io" + "log" + "time" ) // Subscribe subscribes to a topic's specified partitions. // If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker. func (sub *TopicSubscriber) Subscribe() error { + index := -1 util.RetryUntil("subscribe", func() error { + index++ + index = index % len(sub.bootstrapBrokers) // ask balancer for brokers of the topic - if err := sub.doLookup(sub.bootstrapBroker); err != nil { + if err := sub.doLookup(sub.bootstrapBrokers[index]); err != nil { return fmt.Errorf("lookup topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err) } + if len(sub.brokerPartitionAssignments) == 0 { + if sub.waitForMoreMessage { + time.Sleep(1 * time.Second) + return fmt.Errorf("no broker partition assignments") + } else { + return nil + } + } // treat the first broker as the topic leader // connect to the leader broker @@ -25,6 +38,8 @@ func (sub *TopicSubscriber) Subscribe() error { return nil }, func(err error) bool { if err == io.EOF { + log.Printf("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err) + sub.waitForMoreMessage = false return false } return true diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go index 809673de1..9b96b14cb 100644 --- a/weed/mq/client/sub_client/subscriber.go +++ b/weed/mq/client/sub_client/subscriber.go @@ -3,6 +3,7 @@ package sub_client import ( "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "google.golang.org/grpc" + "time" ) type SubscriberConfiguration struct { @@ -19,6 +20,7 @@ type ContentConfiguration struct { Namespace string Topic string Filter string + StartTime time.Time } type OnEachMessageFunc func(key, value []byte) (shouldContinue bool) @@ -30,14 +32,18 @@ type TopicSubscriber struct { brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment OnEachMessageFunc OnEachMessageFunc OnCompletionFunc OnCompletionFunc - bootstrapBroker string + bootstrapBrokers []string + waitForMoreMessage bool + alreadyProcessedTsNs int64 } -func NewTopicSubscriber(bootstrapBroker string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber { +func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber { return &TopicSubscriber{ - SubscriberConfig: subscriber, - ContentConfig: content, - bootstrapBroker: bootstrapBroker, + SubscriberConfig: subscriber, + ContentConfig: content, + bootstrapBrokers: bootstrapBrokers, + waitForMoreMessage: true, + alreadyProcessedTsNs: content.StartTime.UnixNano(), } } @@ -45,6 +51,6 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc sub.OnEachMessageFunc = onEachMessageFn } -func (sub *TopicSubscriber) SetCompletionFunc(onCompeletionFn OnCompletionFunc) { - sub.OnCompletionFunc = onCompeletionFn +func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) { + sub.OnCompletionFunc = onCompletionFn } diff --git a/weed/mq/coordinator/consumer_group.go b/weed/mq/coordinator/consumer_group.go deleted file mode 100644 index e3dec493c..000000000 --- a/weed/mq/coordinator/consumer_group.go +++ /dev/null @@ -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() -} diff --git a/weed/mq/coordinator/coordinator.go b/weed/mq/coordinator/coordinator.go deleted file mode 100644 index e94ac3371..000000000 --- a/weed/mq/coordinator/coordinator.go +++ /dev/null @@ -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] -} diff --git a/weed/mq/balancer/allocate.go b/weed/mq/pub_balancer/allocate.go similarity index 91% rename from weed/mq/balancer/allocate.go rename to weed/mq/pub_balancer/allocate.go index 96a7bcb81..9b2113162 100644 --- a/weed/mq/balancer/allocate.go +++ b/weed/mq/pub_balancer/allocate.go @@ -1,7 +1,8 @@ -package balancer +package pub_balancer import ( cmap "github.com/orcaman/concurrent-map/v2" + "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "math/rand" ) @@ -30,6 +31,7 @@ func allocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p for i, assignment := range assignments { assignment.LeaderBroker = pickedBrokers[i] } + glog.V(0).Infof("allocate topic partitions %d: %v", len(assignments), assignments) return } diff --git a/weed/mq/balancer/allocate_test.go b/weed/mq/pub_balancer/allocate_test.go similarity index 98% rename from weed/mq/balancer/allocate_test.go rename to weed/mq/pub_balancer/allocate_test.go index f5f9d49e3..298b9ebc1 100644 --- a/weed/mq/balancer/allocate_test.go +++ b/weed/mq/pub_balancer/allocate_test.go @@ -1,4 +1,4 @@ -package balancer +package pub_balancer import ( cmap "github.com/orcaman/concurrent-map/v2" diff --git a/weed/mq/pub_balancer/balance.go b/weed/mq/pub_balancer/balance.go new file mode 100644 index 000000000..87fc5739b --- /dev/null +++ b/weed/mq/pub_balancer/balance.go @@ -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 +} diff --git a/weed/mq/pub_balancer/balance_action.go b/weed/mq/pub_balancer/balance_action.go new file mode 100644 index 000000000..c29ec3469 --- /dev/null +++ b/weed/mq/pub_balancer/balance_action.go @@ -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 + +} diff --git a/weed/mq/pub_balancer/balance_action_split.go b/weed/mq/pub_balancer/balance_action_split.go new file mode 100644 index 000000000..6d317ffb9 --- /dev/null +++ b/weed/mq/pub_balancer/balance_action_split.go @@ -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 + +*/ diff --git a/weed/mq/pub_balancer/balance_brokers.go b/weed/mq/pub_balancer/balance_brokers.go new file mode 100644 index 000000000..a6b25b7ca --- /dev/null +++ b/weed/mq/pub_balancer/balance_brokers.go @@ -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, + } +} diff --git a/weed/mq/pub_balancer/balance_brokers_test.go b/weed/mq/pub_balancer/balance_brokers_test.go new file mode 100644 index 000000000..54667d154 --- /dev/null +++ b/weed/mq/pub_balancer/balance_brokers_test.go @@ -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) + } + }) + } +} diff --git a/weed/mq/pub_balancer/balancer.go b/weed/mq/pub_balancer/balancer.go new file mode 100644 index 000000000..988b971af --- /dev/null +++ b/weed/mq/pub_balancer/balancer.go @@ -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) + } +} diff --git a/weed/mq/balancer/balancer.go b/weed/mq/pub_balancer/broker_stats.go similarity index 56% rename from weed/mq/balancer/balancer.go rename to weed/mq/pub_balancer/broker_stats.go index d93cc8de8..461e93c61 100644 --- a/weed/mq/balancer/balancer.go +++ b/weed/mq/pub_balancer/broker_stats.go @@ -1,4 +1,4 @@ -package balancer +package pub_balancer import ( "fmt" @@ -7,37 +7,27 @@ import ( "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] -} - type BrokerStats struct { TopicPartitionCount int32 ConsumerCount int32 CpuUsagePercent int32 - Stats cmap.ConcurrentMap[string, *TopicPartitionStats] + TopicPartitionStats cmap.ConcurrentMap[string, *TopicPartitionStats] // key: topic_partition + Topics []topic.Topic +} +type TopicPartitionStats struct { + topic.TopicPartition + ConsumerCount int32 + IsLeader bool +} + +func NewBrokerStats() *BrokerStats { + return &BrokerStats{ + TopicPartitionStats: cmap.New[*TopicPartitionStats](), + } +} +func (bs *BrokerStats) String() string { + return fmt.Sprintf("BrokerStats{TopicPartitionCount:%d, ConsumerCount:%d, CpuUsagePercent:%d, Stats:%+v}", + bs.TopicPartitionCount, bs.ConsumerCount, bs.CpuUsagePercent, bs.TopicPartitionStats.Items()) } func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) { @@ -45,7 +35,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) { bs.CpuUsagePercent = stats.CpuUsagePercent var consumerCount int32 - currentTopicPartitions := bs.Stats.Items() + currentTopicPartitions := bs.TopicPartitionStats.Items() for _, topicPartitionStats := range stats.Stats { tps := &TopicPartitionStats{ TopicPartition: topic.TopicPartition{ @@ -57,12 +47,12 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) { } consumerCount += topicPartitionStats.ConsumerCount key := tps.TopicPartition.String() - bs.Stats.Set(key, tps) + bs.TopicPartitionStats.Set(key, tps) delete(currentTopicPartitions, key) } // remove the topic partitions that are not in the stats for key := range currentTopicPartitions { - bs.Stats.Remove(key) + bs.TopicPartitionStats.Remove(key) } bs.ConsumerCount = consumerCount @@ -78,28 +68,5 @@ func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Parti IsLeader: true, } key := tps.TopicPartition.String() - bs.Stats.Set(key, tps) -} - -func (bs *BrokerStats) String() string { - return fmt.Sprintf("BrokerStats{TopicPartitionCount:%d, ConsumerCount:%d, CpuUsagePercent:%d, Stats:%+v}", - bs.TopicPartitionCount, bs.ConsumerCount, bs.CpuUsagePercent, bs.Stats.Items()) -} - -type TopicPartitionStats struct { - topic.TopicPartition - ConsumerCount int32 - IsLeader bool -} - -func NewBalancer() *Balancer { - return &Balancer{ - Brokers: cmap.New[*BrokerStats](), - } -} - -func NewBrokerStats() *BrokerStats { - return &BrokerStats{ - Stats: cmap.New[*TopicPartitionStats](), - } + bs.TopicPartitionStats.Set(key, tps) } diff --git a/weed/mq/balancer/lookup.go b/weed/mq/pub_balancer/lookup.go similarity index 60% rename from weed/mq/balancer/lookup.go rename to weed/mq/pub_balancer/lookup.go index d5b78fc45..3e103a650 100644 --- a/weed/mq/balancer/lookup.go +++ b/weed/mq/pub_balancer/lookup.go @@ -1,7 +1,8 @@ -package balancer +package pub_balancer import ( "errors" + "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" ) @@ -9,11 +10,14 @@ var ( ErrNoBroker = errors.New("no broker") ) -func (b *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish bool, partitionCount int32) (assignments []*mq_pb.BrokerPartitionAssignment, err error) { +func (balancer *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish bool, partitionCount int32) (assignments []*mq_pb.BrokerPartitionAssignment, err error) { + if partitionCount == 0 { + partitionCount = 6 + } // find existing topic partition assignments - for brokerStatsItem := range b.Brokers.IterBuffered() { + for brokerStatsItem := range balancer.Brokers.IterBuffered() { broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val - for topicPartitionStatsItem := range brokerStats.Stats.IterBuffered() { + for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() { topicPartitionStat := topicPartitionStatsItem.Val if topicPartitionStat.TopicPartition.Namespace == topic.Namespace && topicPartitionStat.TopicPartition.Name == topic.Name { @@ -30,7 +34,8 @@ func (b *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish b } } } - if len(assignments) > 0 { + if len(assignments) > 0 && len(assignments) == int(partitionCount) || !publish { + glog.V(0).Infof("existing topic partitions %d: %v", len(assignments), assignments) return assignments, nil } @@ -41,8 +46,8 @@ func (b *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish b // if the request is_for_subscribe // return error not found // t := topic.FromPbTopic(request.Topic) - if b.Brokers.IsEmpty() { + if balancer.Brokers.IsEmpty() { return nil, ErrNoBroker } - return allocateTopicPartitions(b.Brokers, partitionCount), nil + return allocateTopicPartitions(balancer.Brokers, partitionCount), nil } diff --git a/weed/mq/pub_balancer/partition_list_broker.go b/weed/mq/pub_balancer/partition_list_broker.go new file mode 100644 index 000000000..7ceb2a9fc --- /dev/null +++ b/weed/mq/pub_balancer/partition_list_broker.go @@ -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 = "" + } + } +} diff --git a/weed/mq/pub_balancer/repair.go b/weed/mq/pub_balancer/repair.go new file mode 100644 index 000000000..0ab1a5ea9 --- /dev/null +++ b/weed/mq/pub_balancer/repair.go @@ -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 +} diff --git a/weed/mq/pub_balancer/repair_test.go b/weed/mq/pub_balancer/repair_test.go new file mode 100644 index 000000000..08465c7e8 --- /dev/null +++ b/weed/mq/pub_balancer/repair_test.go @@ -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) + } + }) + } +} diff --git a/weed/mq/sub_coordinator/consumer_group.go b/weed/mq/sub_coordinator/consumer_group.go new file mode 100644 index 000000000..be06a01f8 --- /dev/null +++ b/weed/mq/sub_coordinator/consumer_group.go @@ -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() { + +} diff --git a/weed/mq/sub_coordinator/coordinator.go b/weed/mq/sub_coordinator/coordinator.go new file mode 100644 index 000000000..f4d65ea5b --- /dev/null +++ b/weed/mq/sub_coordinator/coordinator.go @@ -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) + } +} diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping.go b/weed/mq/sub_coordinator/partition_consumer_mapping.go new file mode 100644 index 000000000..9c80b09c6 --- /dev/null +++ b/weed/mq/sub_coordinator/partition_consumer_mapping.go @@ -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 +} diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping_test.go b/weed/mq/sub_coordinator/partition_consumer_mapping_test.go new file mode 100644 index 000000000..1d3050ef4 --- /dev/null +++ b/weed/mq/sub_coordinator/partition_consumer_mapping_test.go @@ -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) + } + }) + } +} diff --git a/weed/mq/sub_coordinator/partition_list.go b/weed/mq/sub_coordinator/partition_list.go new file mode 100644 index 000000000..ca097f2b3 --- /dev/null +++ b/weed/mq/sub_coordinator/partition_list.go @@ -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 +} diff --git a/weed/mq/topic/local_manager.go b/weed/mq/topic/local_manager.go index 0c54f2bb1..173df090d 100644 --- a/weed/mq/topic/local_manager.go +++ b/weed/mq/topic/local_manager.go @@ -23,10 +23,7 @@ func NewLocalTopicManager() *LocalTopicManager { func (manager *LocalTopicManager) AddTopicPartition(topic Topic, localPartition *LocalPartition) { localTopic, ok := manager.topics.Get(topic.String()) if !ok { - localTopic = &LocalTopic{ - Topic: topic, - Partitions: make([]*LocalPartition, 0), - } + localTopic = NewLocalTopic(topic) } if !manager.topics.SetIfAbsent(topic.String(), localTopic) { localTopic, _ = manager.topics.Get(topic.String()) @@ -59,6 +56,22 @@ func (manager *LocalTopicManager) RemoveTopicPartition(topic Topic, partition Pa return localTopic.removePartition(partition) } +func (manager *LocalTopicManager) ClosePublishers(topic Topic, unixTsNs int64) (removed bool) { + localTopic, ok := manager.topics.Get(topic.String()) + if !ok { + return false + } + return localTopic.closePartitionPublishers(unixTsNs) +} + +func (manager *LocalTopicManager) CloseSubscribers(topic Topic, unixTsNs int64) (removed bool) { + localTopic, ok := manager.topics.Get(topic.String()) + if !ok { + return false + } + return localTopic.closePartitionSubscribers(unixTsNs) +} + func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.BrokerStats { stats := &mq_pb.BrokerStats{ Stats: make(map[string]*mq_pb.TopicPartitionStats), @@ -101,3 +114,11 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br return stats } + +func (manager *LocalTopicManager) WaitUntilNoPublishers(topic Topic) { + localTopic, ok := manager.topics.Get(topic.String()) + if !ok { + return + } + localTopic.WaitUntilNoPublishers() +} diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go index 49b639dfa..aa1274ff5 100644 --- a/weed/mq/topic/local_partition.go +++ b/weed/mq/topic/local_partition.go @@ -11,19 +11,23 @@ import ( type LocalPartition struct { Partition - isLeader bool - FollowerBrokers []pb.ServerAddress - logBuffer *log_buffer.LogBuffer - ConsumerCount int32 + isLeader bool + FollowerBrokers []pb.ServerAddress + logBuffer *log_buffer.LogBuffer + ConsumerCount int32 + StopPublishersCh chan struct{} + Publishers *LocalPartitionPublishers + StopSubscribersCh chan struct{} + Subscribers *LocalPartitionSubscribers } -func NewLocalPartition(topic Topic, partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition { +func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition { return &LocalPartition{ Partition: partition, isLeader: isLeader, FollowerBrokers: followerBrokers, logBuffer: log_buffer.NewLogBuffer( - fmt.Sprintf("%s/%s/%4d-%4d", topic.Namespace, topic.Name, partition.RangeStart, partition.RangeStop), + fmt.Sprintf("%d/%4d-%4d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop), 2*time.Minute, func(startTime, stopTime time.Time, buf []byte) { @@ -32,34 +36,43 @@ func NewLocalPartition(topic Topic, partition Partition, isLeader bool, follower }, ), + Publishers: NewLocalPartitionPublishers(), + Subscribers: NewLocalPartitionSubscribers(), } } type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error -func (p LocalPartition) Publish(message *mq_pb.DataMessage) { +func (p *LocalPartition) Publish(message *mq_pb.DataMessage) { p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano()) } -func (p LocalPartition) Subscribe(clientName string, startReadTime time.Time, eachMessageFn OnEachMessageFn) { - p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, func() bool { - return true - }, eachMessageFn) +func (p *LocalPartition) Subscribe(clientName string, startReadTime time.Time, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) { + p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, onNoMessageFn, eachMessageFn) } func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition { - isLeaer := assignment.LeaderBroker == string(self) - localPartition := &LocalPartition{ - Partition: FromPbPartition(assignment.Partition), - isLeader: isLeaer, - } - if !isLeaer { - return localPartition - } + isLeader := assignment.LeaderBroker == string(self) followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers)) - for i, follower := range assignment.FollowerBrokers { - followers[i] = pb.ServerAddress(follower) + for i, followerBroker := range assignment.FollowerBrokers { + followers[i] = pb.ServerAddress(followerBroker) + } + return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers) +} + +func (p *LocalPartition) closePublishers() { + p.Publishers.SignalShutdown() + close(p.StopPublishersCh) +} +func (p *LocalPartition) closeSubscribers() { + p.Subscribers.SignalShutdown() +} + +func (p *LocalPartition) WaitUntilNoPublishers() { + for { + if p.Publishers.IsEmpty() { + return + } + time.Sleep(113 * time.Millisecond) } - localPartition.FollowerBrokers = followers - return localPartition } diff --git a/weed/mq/topic/local_partition_publishers.go b/weed/mq/topic/local_partition_publishers.go new file mode 100644 index 000000000..367ccce5f --- /dev/null +++ b/weed/mq/topic/local_partition_publishers.go @@ -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 +} diff --git a/weed/mq/topic/local_partition_subscribers.go b/weed/mq/topic/local_partition_subscribers.go new file mode 100644 index 000000000..e177ec7e8 --- /dev/null +++ b/weed/mq/topic/local_partition_subscribers.go @@ -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() + } +} diff --git a/weed/mq/topic/local_topic.go b/weed/mq/topic/local_topic.go index ef3c0e65e..7825d2168 100644 --- a/weed/mq/topic/local_topic.go +++ b/weed/mq/topic/local_topic.go @@ -1,10 +1,19 @@ package topic +import "sync" + type LocalTopic struct { Topic Partitions []*LocalPartition } +func NewLocalTopic(topic Topic) *LocalTopic { + return &LocalTopic{ + Topic: topic, + Partitions: make([]*LocalPartition, 0), + } +} + func (localTopic *LocalTopic) findPartition(partition Partition) *LocalPartition { for _, localPartition := range localTopic.Partitions { if localPartition.Partition.Equals(partition) { @@ -27,3 +36,52 @@ func (localTopic *LocalTopic) removePartition(partition Partition) bool { localTopic.Partitions = append(localTopic.Partitions[:foundPartitionIndex], localTopic.Partitions[foundPartitionIndex+1:]...) return true } + +func (localTopic *LocalTopic) closePartitionPublishers(unixTsNs int64) bool { + var wg sync.WaitGroup + for _, localPartition := range localTopic.Partitions { + if localPartition.UnixTimeNs != unixTsNs { + continue + } + wg.Add(1) + go func(localPartition *LocalPartition) { + defer wg.Done() + localPartition.closePublishers() + }(localPartition) + } + wg.Wait() + return true +} + +func (localTopic *LocalTopic) closePartitionSubscribers(unixTsNs int64) bool { + var wg sync.WaitGroup + for _, localPartition := range localTopic.Partitions { + if localPartition.UnixTimeNs != unixTsNs { + continue + } + wg.Add(1) + go func(localPartition *LocalPartition) { + defer wg.Done() + localPartition.closeSubscribers() + }(localPartition) + } + wg.Wait() + return true +} + +func (localTopic *LocalTopic) WaitUntilNoPublishers() { + for { + var wg sync.WaitGroup + for _, localPartition := range localTopic.Partitions { + wg.Add(1) + go func(localPartition *LocalPartition) { + defer wg.Done() + localPartition.WaitUntilNoPublishers() + }(localPartition) + } + wg.Wait() + if len(localTopic.Partitions) == 0 { + return + } + } +} diff --git a/weed/mq/topic/partition.go b/weed/mq/topic/partition.go index 79c830f13..ca34c2390 100644 --- a/weed/mq/topic/partition.go +++ b/weed/mq/topic/partition.go @@ -8,6 +8,7 @@ type Partition struct { RangeStart int32 RangeStop int32 // exclusive RingSize int32 + UnixTimeNs int64 // in nanoseconds } func (partition Partition) Equals(other Partition) bool { @@ -20,6 +21,9 @@ func (partition Partition) Equals(other Partition) bool { if partition.RingSize != other.RingSize { return false } + if partition.UnixTimeNs != other.UnixTimeNs { + return false + } return true } @@ -28,10 +32,11 @@ func FromPbPartition(partition *mq_pb.Partition) Partition { RangeStart: partition.RangeStart, RangeStop: partition.RangeStop, RingSize: partition.RingSize, + UnixTimeNs: partition.UnixTimeNs, } } -func SplitPartitions(targetCount int32) []*Partition { +func SplitPartitions(targetCount int32, ts int64) []*Partition { partitions := make([]*Partition, 0, targetCount) partitionSize := PartitionCount / targetCount for i := int32(0); i < targetCount; i++ { @@ -43,7 +48,17 @@ func SplitPartitions(targetCount int32) []*Partition { RangeStart: i * partitionSize, RangeStop: partitionStop, RingSize: PartitionCount, + UnixTimeNs: ts, }) } return partitions } + +func (partition Partition) ToPbPartition() *mq_pb.Partition { + return &mq_pb.Partition{ + RangeStart: partition.RangeStart, + RangeStop: partition.RangeStop, + RingSize: partition.RingSize, + UnixTimeNs: partition.UnixTimeNs, + } +} diff --git a/weed/mq/topic/topic.go b/weed/mq/topic/topic.go index 3d457e6f1..6932fcb56 100644 --- a/weed/mq/topic/topic.go +++ b/weed/mq/topic/topic.go @@ -2,9 +2,7 @@ package topic import ( "fmt" - "github.com/seaweedfs/seaweedfs/weed/filer" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" - "time" ) type Topic struct { @@ -25,47 +23,13 @@ func FromPbTopic(topic *mq_pb.Topic) Topic { } } +func (tp Topic) ToPbTopic() *mq_pb.Topic { + return &mq_pb.Topic{ + Namespace: tp.Namespace, + Name: tp.Name, + } +} + func (tp Topic) String() string { return fmt.Sprintf("%s.%s", tp.Namespace, tp.Name) } - -type Segment struct { - Topic Topic - Id int32 - Partition Partition - LastModified time.Time -} - -func FromPbSegment(segment *mq_pb.Segment) *Segment { - return &Segment{ - Topic: Topic{ - Namespace: segment.Namespace, - Name: segment.Topic, - }, - Id: segment.Id, - Partition: Partition{ - RangeStart: segment.Partition.RangeStart, - RangeStop: segment.Partition.RangeStop, - RingSize: segment.Partition.RingSize, - }, - } -} - -func (segment *Segment) ToPbSegment() *mq_pb.Segment { - return &mq_pb.Segment{ - Namespace: string(segment.Topic.Namespace), - Topic: segment.Topic.Name, - Id: segment.Id, - Partition: &mq_pb.Partition{ - RingSize: segment.Partition.RingSize, - RangeStart: segment.Partition.RangeStart, - RangeStop: segment.Partition.RangeStop, - }, - } -} - -func (segment *Segment) DirAndName() (dir string, name string) { - dir = fmt.Sprintf("%s/%s/%s", filer.TopicsDir, segment.Topic.Namespace, segment.Topic.Name) - name = fmt.Sprintf("%4d.segment", segment.Id) - return -} diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto index cc4f962f2..5c309aaf1 100644 --- a/weed/pb/mq.proto +++ b/weed/pb/mq.proto @@ -4,7 +4,7 @@ package messaging_pb; option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"; option java_package = "seaweedfs.mq"; -option java_outer_classname = "MessagQueueProto"; +option java_outer_classname = "MessageQueueProto"; ////////////////////////////////////////////////// @@ -13,50 +13,40 @@ service SeaweedMessaging { // control plane rpc FindBrokerLeader (FindBrokerLeaderRequest) returns (FindBrokerLeaderResponse) { } - rpc AssignSegmentBrokers (AssignSegmentBrokersRequest) returns (AssignSegmentBrokersResponse) { - } - rpc CheckSegmentStatus (CheckSegmentStatusRequest) returns (CheckSegmentStatusResponse) { - } - rpc CheckBrokerLoad (CheckBrokerLoadRequest) returns (CheckBrokerLoadResponse) { - } // control plane for balancer - rpc ConnectToBalancer (stream ConnectToBalancerRequest) returns (stream ConnectToBalancerResponse) { + rpc PublisherToPubBalancer (stream PublisherToPubBalancerRequest) returns (stream PublisherToPubBalancerResponse) { } - rpc DoConfigureTopic (DoConfigureTopicRequest) returns (DoConfigureTopicResponse) { + rpc BalanceTopics (BalanceTopicsRequest) returns (BalanceTopicsResponse) { } + // control plane for topic partitions - rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) { + rpc ListTopics (ListTopicsRequest) returns (ListTopicsResponse) { } rpc ConfigureTopic (ConfigureTopicRequest) returns (ConfigureTopicResponse) { } - rpc ListTopics (ListTopicsRequest) returns (ListTopicsResponse) { - } - // a pub client will call this to get the topic partitions assignment - rpc RequestTopicPartitions (RequestTopicPartitionsRequest) returns (RequestTopicPartitionsResponse) { - } - rpc AssignTopicPartitions (AssignTopicPartitionsRequest) returns (AssignTopicPartitionsResponse) { - } - rpc CheckTopicPartitionsStatus (CheckTopicPartitionsStatusRequest) returns (CheckTopicPartitionsStatusResponse) { + rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) { } - // data plane + // invoked by the balancer, running on each broker + rpc AssignTopicPartitions (AssignTopicPartitionsRequest) returns (AssignTopicPartitionsResponse) { + } + rpc ClosePublishers(ClosePublishersRequest) returns (ClosePublishersResponse) { + } + rpc CloseSubscribers(CloseSubscribersRequest) returns (CloseSubscribersResponse) { + } + + // subscriber connects to broker balancer, which coordinates with the subscribers + rpc SubscriberToSubCoordinator (stream SubscriberToSubCoordinatorRequest) returns (stream SubscriberToSubCoordinatorResponse) { + } + + // data plane for each topic partition rpc Publish (stream PublishRequest) returns (stream PublishResponse) { } rpc Subscribe (SubscribeRequest) returns (stream SubscribeResponse) { } } -////////////////////////////////////////////////// -message SegmentInfo { - Segment segment = 1; - int64 start_ts_ns = 2; - repeated string brokers = 3; - int64 stop_ts_ns = 4; - repeated int32 previous_segments = 5; - repeated int32 next_segments = 6; -} - ////////////////////////////////////////////////// message FindBrokerLeaderRequest { @@ -75,38 +65,7 @@ message Partition { int32 ring_size = 1; int32 range_start = 2; int32 range_stop = 3; -} - -message Segment { - string namespace = 1; - string topic = 2; - int32 id = 3; - Partition partition = 4; -} - -message AssignSegmentBrokersRequest { - Segment segment = 1; -} - -message AssignSegmentBrokersResponse { - repeated string brokers = 1; -} - -message CheckSegmentStatusRequest { - Segment segment = 1; -} - -message CheckSegmentStatusResponse { - bool is_active = 1; -} - -message CheckBrokerLoadRequest { -} - -message CheckBrokerLoadResponse { - int64 message_count = 1; - int64 bytes_count = 2; - + int64 unix_time_ns = 4; } ////////////////////////////////////////////////// @@ -122,7 +81,7 @@ message TopicPartitionStats { } -message ConnectToBalancerRequest { +message PublisherToPubBalancerRequest { message InitMessage { string broker = 1; } @@ -131,8 +90,14 @@ message ConnectToBalancerRequest { BrokerStats stats = 2; } } -message ConnectToBalancerResponse { +message PublisherToPubBalancerResponse { } + +message BalanceTopicsRequest { +} +message BalanceTopicsResponse { +} + ////////////////////////////////////////////////// message ConfigureTopicRequest { Topic topic = 1; @@ -141,12 +106,6 @@ message ConfigureTopicRequest { message ConfigureTopicResponse { repeated BrokerPartitionAssignment broker_partition_assignments = 2; } -message DoConfigureTopicRequest { - Topic topic = 1; - Partition partition = 2; -} -message DoConfigureTopicResponse { -} message ListTopicsRequest { } message ListTopicsResponse { @@ -166,36 +125,49 @@ message BrokerPartitionAssignment { repeated string follower_brokers = 3; } -message RequestTopicPartitionsRequest { - Topic topic = 1; - int32 partition_count = 2; -} -message RequestTopicPartitionsResponse { - repeated BrokerPartitionAssignment broker_partition_assignments = 1; -} - message AssignTopicPartitionsRequest { Topic topic = 1; repeated BrokerPartitionAssignment broker_partition_assignments = 2; bool is_leader = 3; + bool is_draining = 4; } message AssignTopicPartitionsResponse { } -message CheckTopicPartitionsStatusRequest { - string namespace = 1; - string topic = 2; - BrokerPartitionAssignment broker_partition_assignment = 3; - bool should_cancel_if_not_match = 4; +message SubscriberToSubCoordinatorRequest { + message InitMessage { + string consumer_group = 1; + string consumer_instance_id = 2; + Topic topic = 3; + } + message AckMessage { + Partition partition = 1; + int64 ts_ns = 2; + } + oneof message { + InitMessage init = 1; + AckMessage ack = 2; + } } -message CheckTopicPartitionsStatusResponse { - repeated BrokerPartitionAssignment broker_partition_assignments = 1; +message SubscriberToSubCoordinatorResponse { + message AssignedPartition { + Partition partition = 1; + int64 ts_ns = 2; + } + message Assignment { + int64 generation = 1; + repeated AssignedPartition assigned_partitions = 2; + } + oneof message { + Assignment assignment = 1; + } } ////////////////////////////////////////////////// message DataMessage { bytes key = 1; bytes value = 2; + int64 ts_ns = 3; } message PublishRequest { message InitMessage { @@ -212,7 +184,7 @@ message PublishRequest { message PublishResponse { int64 ack_sequence = 1; string error = 2; - string redirect_to_broker = 3; + bool should_close = 3; } message SubscribeRequest { message InitMessage { @@ -246,3 +218,15 @@ message SubscribeResponse { DataMessage data = 2; } } +message ClosePublishersRequest { + Topic topic = 1; + int64 unix_time_ns = 2; +} +message ClosePublishersResponse { +} +message CloseSubscribersRequest { + Topic topic = 1; + int64 unix_time_ns = 2; +} +message CloseSubscribersResponse { +} diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq.pb.go index b06a822ee..e1879d98a 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq.pb.go @@ -20,94 +20,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// //////////////////////////////////////////////// -type SegmentInfo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Segment *Segment `protobuf:"bytes,1,opt,name=segment,proto3" json:"segment,omitempty"` - StartTsNs int64 `protobuf:"varint,2,opt,name=start_ts_ns,json=startTsNs,proto3" json:"start_ts_ns,omitempty"` - Brokers []string `protobuf:"bytes,3,rep,name=brokers,proto3" json:"brokers,omitempty"` - StopTsNs int64 `protobuf:"varint,4,opt,name=stop_ts_ns,json=stopTsNs,proto3" json:"stop_ts_ns,omitempty"` - PreviousSegments []int32 `protobuf:"varint,5,rep,packed,name=previous_segments,json=previousSegments,proto3" json:"previous_segments,omitempty"` - NextSegments []int32 `protobuf:"varint,6,rep,packed,name=next_segments,json=nextSegments,proto3" json:"next_segments,omitempty"` -} - -func (x *SegmentInfo) Reset() { - *x = SegmentInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *SegmentInfo) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*SegmentInfo) ProtoMessage() {} - -func (x *SegmentInfo) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use SegmentInfo.ProtoReflect.Descriptor instead. -func (*SegmentInfo) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{0} -} - -func (x *SegmentInfo) GetSegment() *Segment { - if x != nil { - return x.Segment - } - return nil -} - -func (x *SegmentInfo) GetStartTsNs() int64 { - if x != nil { - return x.StartTsNs - } - return 0 -} - -func (x *SegmentInfo) GetBrokers() []string { - if x != nil { - return x.Brokers - } - return nil -} - -func (x *SegmentInfo) GetStopTsNs() int64 { - if x != nil { - return x.StopTsNs - } - return 0 -} - -func (x *SegmentInfo) GetPreviousSegments() []int32 { - if x != nil { - return x.PreviousSegments - } - return nil -} - -func (x *SegmentInfo) GetNextSegments() []int32 { - if x != nil { - return x.NextSegments - } - return nil -} - type FindBrokerLeaderRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -119,7 +31,7 @@ type FindBrokerLeaderRequest struct { func (x *FindBrokerLeaderRequest) Reset() { *x = FindBrokerLeaderRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[1] + mi := &file_mq_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -132,7 +44,7 @@ func (x *FindBrokerLeaderRequest) String() string { func (*FindBrokerLeaderRequest) ProtoMessage() {} func (x *FindBrokerLeaderRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[1] + mi := &file_mq_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -145,7 +57,7 @@ func (x *FindBrokerLeaderRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use FindBrokerLeaderRequest.ProtoReflect.Descriptor instead. func (*FindBrokerLeaderRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{1} + return file_mq_proto_rawDescGZIP(), []int{0} } func (x *FindBrokerLeaderRequest) GetFilerGroup() string { @@ -166,7 +78,7 @@ type FindBrokerLeaderResponse struct { func (x *FindBrokerLeaderResponse) Reset() { *x = FindBrokerLeaderResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[2] + mi := &file_mq_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -179,7 +91,7 @@ func (x *FindBrokerLeaderResponse) String() string { func (*FindBrokerLeaderResponse) ProtoMessage() {} func (x *FindBrokerLeaderResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[2] + mi := &file_mq_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -192,7 +104,7 @@ func (x *FindBrokerLeaderResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use FindBrokerLeaderResponse.ProtoReflect.Descriptor instead. func (*FindBrokerLeaderResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{2} + return file_mq_proto_rawDescGZIP(), []int{1} } func (x *FindBrokerLeaderResponse) GetBroker() string { @@ -214,7 +126,7 @@ type Topic struct { func (x *Topic) Reset() { *x = Topic{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[3] + mi := &file_mq_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -227,7 +139,7 @@ func (x *Topic) String() string { func (*Topic) ProtoMessage() {} func (x *Topic) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[3] + mi := &file_mq_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -240,7 +152,7 @@ func (x *Topic) ProtoReflect() protoreflect.Message { // Deprecated: Use Topic.ProtoReflect.Descriptor instead. func (*Topic) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{3} + return file_mq_proto_rawDescGZIP(), []int{2} } func (x *Topic) GetNamespace() string { @@ -265,12 +177,13 @@ type Partition struct { RingSize int32 `protobuf:"varint,1,opt,name=ring_size,json=ringSize,proto3" json:"ring_size,omitempty"` RangeStart int32 `protobuf:"varint,2,opt,name=range_start,json=rangeStart,proto3" json:"range_start,omitempty"` RangeStop int32 `protobuf:"varint,3,opt,name=range_stop,json=rangeStop,proto3" json:"range_stop,omitempty"` + UnixTimeNs int64 `protobuf:"varint,4,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` } func (x *Partition) Reset() { *x = Partition{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[4] + mi := &file_mq_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -283,7 +196,7 @@ func (x *Partition) String() string { func (*Partition) ProtoMessage() {} func (x *Partition) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[4] + mi := &file_mq_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -296,7 +209,7 @@ func (x *Partition) ProtoReflect() protoreflect.Message { // Deprecated: Use Partition.ProtoReflect.Descriptor instead. func (*Partition) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{4} + return file_mq_proto_rawDescGZIP(), []int{3} } func (x *Partition) GetRingSize() int32 { @@ -320,354 +233,9 @@ func (x *Partition) GetRangeStop() int32 { return 0 } -type Segment struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` - Topic string `protobuf:"bytes,2,opt,name=topic,proto3" json:"topic,omitempty"` - Id int32 `protobuf:"varint,3,opt,name=id,proto3" json:"id,omitempty"` - Partition *Partition `protobuf:"bytes,4,opt,name=partition,proto3" json:"partition,omitempty"` -} - -func (x *Segment) Reset() { - *x = Segment{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Segment) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Segment) ProtoMessage() {} - -func (x *Segment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use Segment.ProtoReflect.Descriptor instead. -func (*Segment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{5} -} - -func (x *Segment) GetNamespace() string { +func (x *Partition) GetUnixTimeNs() int64 { if x != nil { - return x.Namespace - } - return "" -} - -func (x *Segment) GetTopic() string { - if x != nil { - return x.Topic - } - return "" -} - -func (x *Segment) GetId() int32 { - if x != nil { - return x.Id - } - return 0 -} - -func (x *Segment) GetPartition() *Partition { - if x != nil { - return x.Partition - } - return nil -} - -type AssignSegmentBrokersRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Segment *Segment `protobuf:"bytes,1,opt,name=segment,proto3" json:"segment,omitempty"` -} - -func (x *AssignSegmentBrokersRequest) Reset() { - *x = AssignSegmentBrokersRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *AssignSegmentBrokersRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*AssignSegmentBrokersRequest) ProtoMessage() {} - -func (x *AssignSegmentBrokersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use AssignSegmentBrokersRequest.ProtoReflect.Descriptor instead. -func (*AssignSegmentBrokersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{6} -} - -func (x *AssignSegmentBrokersRequest) GetSegment() *Segment { - if x != nil { - return x.Segment - } - return nil -} - -type AssignSegmentBrokersResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Brokers []string `protobuf:"bytes,1,rep,name=brokers,proto3" json:"brokers,omitempty"` -} - -func (x *AssignSegmentBrokersResponse) Reset() { - *x = AssignSegmentBrokersResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *AssignSegmentBrokersResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*AssignSegmentBrokersResponse) ProtoMessage() {} - -func (x *AssignSegmentBrokersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use AssignSegmentBrokersResponse.ProtoReflect.Descriptor instead. -func (*AssignSegmentBrokersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{7} -} - -func (x *AssignSegmentBrokersResponse) GetBrokers() []string { - if x != nil { - return x.Brokers - } - return nil -} - -type CheckSegmentStatusRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Segment *Segment `protobuf:"bytes,1,opt,name=segment,proto3" json:"segment,omitempty"` -} - -func (x *CheckSegmentStatusRequest) Reset() { - *x = CheckSegmentStatusRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CheckSegmentStatusRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CheckSegmentStatusRequest) ProtoMessage() {} - -func (x *CheckSegmentStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use CheckSegmentStatusRequest.ProtoReflect.Descriptor instead. -func (*CheckSegmentStatusRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{8} -} - -func (x *CheckSegmentStatusRequest) GetSegment() *Segment { - if x != nil { - return x.Segment - } - return nil -} - -type CheckSegmentStatusResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - IsActive bool `protobuf:"varint,1,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"` -} - -func (x *CheckSegmentStatusResponse) Reset() { - *x = CheckSegmentStatusResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CheckSegmentStatusResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CheckSegmentStatusResponse) ProtoMessage() {} - -func (x *CheckSegmentStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use CheckSegmentStatusResponse.ProtoReflect.Descriptor instead. -func (*CheckSegmentStatusResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{9} -} - -func (x *CheckSegmentStatusResponse) GetIsActive() bool { - if x != nil { - return x.IsActive - } - return false -} - -type CheckBrokerLoadRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields -} - -func (x *CheckBrokerLoadRequest) Reset() { - *x = CheckBrokerLoadRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CheckBrokerLoadRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CheckBrokerLoadRequest) ProtoMessage() {} - -func (x *CheckBrokerLoadRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use CheckBrokerLoadRequest.ProtoReflect.Descriptor instead. -func (*CheckBrokerLoadRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{10} -} - -type CheckBrokerLoadResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - MessageCount int64 `protobuf:"varint,1,opt,name=message_count,json=messageCount,proto3" json:"message_count,omitempty"` - BytesCount int64 `protobuf:"varint,2,opt,name=bytes_count,json=bytesCount,proto3" json:"bytes_count,omitempty"` -} - -func (x *CheckBrokerLoadResponse) Reset() { - *x = CheckBrokerLoadResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CheckBrokerLoadResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CheckBrokerLoadResponse) ProtoMessage() {} - -func (x *CheckBrokerLoadResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use CheckBrokerLoadResponse.ProtoReflect.Descriptor instead. -func (*CheckBrokerLoadResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{11} -} - -func (x *CheckBrokerLoadResponse) GetMessageCount() int64 { - if x != nil { - return x.MessageCount - } - return 0 -} - -func (x *CheckBrokerLoadResponse) GetBytesCount() int64 { - if x != nil { - return x.BytesCount + return x.UnixTimeNs } return 0 } @@ -685,7 +253,7 @@ type BrokerStats struct { func (x *BrokerStats) Reset() { *x = BrokerStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -698,7 +266,7 @@ func (x *BrokerStats) String() string { func (*BrokerStats) ProtoMessage() {} func (x *BrokerStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -711,7 +279,7 @@ func (x *BrokerStats) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerStats.ProtoReflect.Descriptor instead. func (*BrokerStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{12} + return file_mq_proto_rawDescGZIP(), []int{4} } func (x *BrokerStats) GetCpuUsagePercent() int32 { @@ -742,7 +310,7 @@ type TopicPartitionStats struct { func (x *TopicPartitionStats) Reset() { *x = TopicPartitionStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -755,7 +323,7 @@ func (x *TopicPartitionStats) String() string { func (*TopicPartitionStats) ProtoMessage() {} func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -768,7 +336,7 @@ func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { // Deprecated: Use TopicPartitionStats.ProtoReflect.Descriptor instead. func (*TopicPartitionStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{13} + return file_mq_proto_rawDescGZIP(), []int{5} } func (x *TopicPartitionStats) GetTopic() *Topic { @@ -799,35 +367,35 @@ func (x *TopicPartitionStats) GetIsLeader() bool { return false } -type ConnectToBalancerRequest struct { +type PublisherToPubBalancerRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields // Types that are assignable to Message: // - // *ConnectToBalancerRequest_Init - // *ConnectToBalancerRequest_Stats - Message isConnectToBalancerRequest_Message `protobuf_oneof:"message"` + // *PublisherToPubBalancerRequest_Init + // *PublisherToPubBalancerRequest_Stats + Message isPublisherToPubBalancerRequest_Message `protobuf_oneof:"message"` } -func (x *ConnectToBalancerRequest) Reset() { - *x = ConnectToBalancerRequest{} +func (x *PublisherToPubBalancerRequest) Reset() { + *x = PublisherToPubBalancerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[14] + mi := &file_mq_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *ConnectToBalancerRequest) String() string { +func (x *PublisherToPubBalancerRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ConnectToBalancerRequest) ProtoMessage() {} +func (*PublisherToPubBalancerRequest) ProtoMessage() {} -func (x *ConnectToBalancerRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[14] +func (x *PublisherToPubBalancerRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -838,71 +406,71 @@ func (x *ConnectToBalancerRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ConnectToBalancerRequest.ProtoReflect.Descriptor instead. -func (*ConnectToBalancerRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{14} +// Deprecated: Use PublisherToPubBalancerRequest.ProtoReflect.Descriptor instead. +func (*PublisherToPubBalancerRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{6} } -func (m *ConnectToBalancerRequest) GetMessage() isConnectToBalancerRequest_Message { +func (m *PublisherToPubBalancerRequest) GetMessage() isPublisherToPubBalancerRequest_Message { if m != nil { return m.Message } return nil } -func (x *ConnectToBalancerRequest) GetInit() *ConnectToBalancerRequest_InitMessage { - if x, ok := x.GetMessage().(*ConnectToBalancerRequest_Init); ok { +func (x *PublisherToPubBalancerRequest) GetInit() *PublisherToPubBalancerRequest_InitMessage { + if x, ok := x.GetMessage().(*PublisherToPubBalancerRequest_Init); ok { return x.Init } return nil } -func (x *ConnectToBalancerRequest) GetStats() *BrokerStats { - if x, ok := x.GetMessage().(*ConnectToBalancerRequest_Stats); ok { +func (x *PublisherToPubBalancerRequest) GetStats() *BrokerStats { + if x, ok := x.GetMessage().(*PublisherToPubBalancerRequest_Stats); ok { return x.Stats } return nil } -type isConnectToBalancerRequest_Message interface { - isConnectToBalancerRequest_Message() +type isPublisherToPubBalancerRequest_Message interface { + isPublisherToPubBalancerRequest_Message() } -type ConnectToBalancerRequest_Init struct { - Init *ConnectToBalancerRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +type PublisherToPubBalancerRequest_Init struct { + Init *PublisherToPubBalancerRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` } -type ConnectToBalancerRequest_Stats struct { +type PublisherToPubBalancerRequest_Stats struct { Stats *BrokerStats `protobuf:"bytes,2,opt,name=stats,proto3,oneof"` } -func (*ConnectToBalancerRequest_Init) isConnectToBalancerRequest_Message() {} +func (*PublisherToPubBalancerRequest_Init) isPublisherToPubBalancerRequest_Message() {} -func (*ConnectToBalancerRequest_Stats) isConnectToBalancerRequest_Message() {} +func (*PublisherToPubBalancerRequest_Stats) isPublisherToPubBalancerRequest_Message() {} -type ConnectToBalancerResponse struct { +type PublisherToPubBalancerResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields } -func (x *ConnectToBalancerResponse) Reset() { - *x = ConnectToBalancerResponse{} +func (x *PublisherToPubBalancerResponse) Reset() { + *x = PublisherToPubBalancerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[15] + mi := &file_mq_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *ConnectToBalancerResponse) String() string { +func (x *PublisherToPubBalancerResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ConnectToBalancerResponse) ProtoMessage() {} +func (*PublisherToPubBalancerResponse) ProtoMessage() {} -func (x *ConnectToBalancerResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[15] +func (x *PublisherToPubBalancerResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -913,9 +481,85 @@ func (x *ConnectToBalancerResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ConnectToBalancerResponse.ProtoReflect.Descriptor instead. -func (*ConnectToBalancerResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{15} +// Deprecated: Use PublisherToPubBalancerResponse.ProtoReflect.Descriptor instead. +func (*PublisherToPubBalancerResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{7} +} + +type BalanceTopicsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *BalanceTopicsRequest) Reset() { + *x = BalanceTopicsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BalanceTopicsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BalanceTopicsRequest) ProtoMessage() {} + +func (x *BalanceTopicsRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BalanceTopicsRequest.ProtoReflect.Descriptor instead. +func (*BalanceTopicsRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{8} +} + +type BalanceTopicsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *BalanceTopicsResponse) Reset() { + *x = BalanceTopicsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BalanceTopicsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BalanceTopicsResponse) ProtoMessage() {} + +func (x *BalanceTopicsResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BalanceTopicsResponse.ProtoReflect.Descriptor instead. +func (*BalanceTopicsResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{9} } // //////////////////////////////////////////////// @@ -931,7 +575,7 @@ type ConfigureTopicRequest struct { func (x *ConfigureTopicRequest) Reset() { *x = ConfigureTopicRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -944,7 +588,7 @@ func (x *ConfigureTopicRequest) String() string { func (*ConfigureTopicRequest) ProtoMessage() {} func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -957,7 +601,7 @@ func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicRequest.ProtoReflect.Descriptor instead. func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{16} + return file_mq_proto_rawDescGZIP(), []int{10} } func (x *ConfigureTopicRequest) GetTopic() *Topic { @@ -985,7 +629,7 @@ type ConfigureTopicResponse struct { func (x *ConfigureTopicResponse) Reset() { *x = ConfigureTopicResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -998,7 +642,7 @@ func (x *ConfigureTopicResponse) String() string { func (*ConfigureTopicResponse) ProtoMessage() {} func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1011,7 +655,7 @@ func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicResponse.ProtoReflect.Descriptor instead. func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{17} + return file_mq_proto_rawDescGZIP(), []int{11} } func (x *ConfigureTopicResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment { @@ -1021,99 +665,6 @@ func (x *ConfigureTopicResponse) GetBrokerPartitionAssignments() []*BrokerPartit return nil } -type DoConfigureTopicRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` -} - -func (x *DoConfigureTopicRequest) Reset() { - *x = DoConfigureTopicRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[18] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *DoConfigureTopicRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*DoConfigureTopicRequest) ProtoMessage() {} - -func (x *DoConfigureTopicRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[18] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use DoConfigureTopicRequest.ProtoReflect.Descriptor instead. -func (*DoConfigureTopicRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{18} -} - -func (x *DoConfigureTopicRequest) GetTopic() *Topic { - if x != nil { - return x.Topic - } - return nil -} - -func (x *DoConfigureTopicRequest) GetPartition() *Partition { - if x != nil { - return x.Partition - } - return nil -} - -type DoConfigureTopicResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields -} - -func (x *DoConfigureTopicResponse) Reset() { - *x = DoConfigureTopicResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[19] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *DoConfigureTopicResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*DoConfigureTopicResponse) ProtoMessage() {} - -func (x *DoConfigureTopicResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[19] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use DoConfigureTopicResponse.ProtoReflect.Descriptor instead. -func (*DoConfigureTopicResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{19} -} - type ListTopicsRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1123,7 +674,7 @@ type ListTopicsRequest struct { func (x *ListTopicsRequest) Reset() { *x = ListTopicsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1136,7 +687,7 @@ func (x *ListTopicsRequest) String() string { func (*ListTopicsRequest) ProtoMessage() {} func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1149,7 +700,7 @@ func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsRequest.ProtoReflect.Descriptor instead. func (*ListTopicsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{20} + return file_mq_proto_rawDescGZIP(), []int{12} } type ListTopicsResponse struct { @@ -1163,7 +714,7 @@ type ListTopicsResponse struct { func (x *ListTopicsResponse) Reset() { *x = ListTopicsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1176,7 +727,7 @@ func (x *ListTopicsResponse) String() string { func (*ListTopicsResponse) ProtoMessage() {} func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1189,7 +740,7 @@ func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsResponse.ProtoReflect.Descriptor instead. func (*ListTopicsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21} + return file_mq_proto_rawDescGZIP(), []int{13} } func (x *ListTopicsResponse) GetTopics() []*Topic { @@ -1211,7 +762,7 @@ type LookupTopicBrokersRequest struct { func (x *LookupTopicBrokersRequest) Reset() { *x = LookupTopicBrokersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[22] + mi := &file_mq_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1224,7 +775,7 @@ func (x *LookupTopicBrokersRequest) String() string { func (*LookupTopicBrokersRequest) ProtoMessage() {} func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[22] + mi := &file_mq_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1237,7 +788,7 @@ func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersRequest.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22} + return file_mq_proto_rawDescGZIP(), []int{14} } func (x *LookupTopicBrokersRequest) GetTopic() *Topic { @@ -1266,7 +817,7 @@ type LookupTopicBrokersResponse struct { func (x *LookupTopicBrokersResponse) Reset() { *x = LookupTopicBrokersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[23] + mi := &file_mq_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1279,7 +830,7 @@ func (x *LookupTopicBrokersResponse) String() string { func (*LookupTopicBrokersResponse) ProtoMessage() {} func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[23] + mi := &file_mq_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1292,7 +843,7 @@ func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersResponse.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{23} + return file_mq_proto_rawDescGZIP(), []int{15} } func (x *LookupTopicBrokersResponse) GetTopic() *Topic { @@ -1322,7 +873,7 @@ type BrokerPartitionAssignment struct { func (x *BrokerPartitionAssignment) Reset() { *x = BrokerPartitionAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[24] + mi := &file_mq_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1335,7 +886,7 @@ func (x *BrokerPartitionAssignment) String() string { func (*BrokerPartitionAssignment) ProtoMessage() {} func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[24] + mi := &file_mq_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1348,7 +899,7 @@ func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerPartitionAssignment.ProtoReflect.Descriptor instead. func (*BrokerPartitionAssignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{24} + return file_mq_proto_rawDescGZIP(), []int{16} } func (x *BrokerPartitionAssignment) GetPartition() *Partition { @@ -1372,108 +923,6 @@ func (x *BrokerPartitionAssignment) GetFollowerBrokers() []string { return nil } -type RequestTopicPartitionsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - PartitionCount int32 `protobuf:"varint,2,opt,name=partition_count,json=partitionCount,proto3" json:"partition_count,omitempty"` -} - -func (x *RequestTopicPartitionsRequest) Reset() { - *x = RequestTopicPartitionsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[25] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *RequestTopicPartitionsRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*RequestTopicPartitionsRequest) ProtoMessage() {} - -func (x *RequestTopicPartitionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[25] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use RequestTopicPartitionsRequest.ProtoReflect.Descriptor instead. -func (*RequestTopicPartitionsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{25} -} - -func (x *RequestTopicPartitionsRequest) GetTopic() *Topic { - if x != nil { - return x.Topic - } - return nil -} - -func (x *RequestTopicPartitionsRequest) GetPartitionCount() int32 { - if x != nil { - return x.PartitionCount - } - return 0 -} - -type RequestTopicPartitionsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,1,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"` -} - -func (x *RequestTopicPartitionsResponse) Reset() { - *x = RequestTopicPartitionsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[26] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *RequestTopicPartitionsResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*RequestTopicPartitionsResponse) ProtoMessage() {} - -func (x *RequestTopicPartitionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[26] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use RequestTopicPartitionsResponse.ProtoReflect.Descriptor instead. -func (*RequestTopicPartitionsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26} -} - -func (x *RequestTopicPartitionsResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment { - if x != nil { - return x.BrokerPartitionAssignments - } - return nil -} - type AssignTopicPartitionsRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1482,12 +931,13 @@ type AssignTopicPartitionsRequest struct { Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"` IsLeader bool `protobuf:"varint,3,opt,name=is_leader,json=isLeader,proto3" json:"is_leader,omitempty"` + IsDraining bool `protobuf:"varint,4,opt,name=is_draining,json=isDraining,proto3" json:"is_draining,omitempty"` } func (x *AssignTopicPartitionsRequest) Reset() { *x = AssignTopicPartitionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1500,7 +950,7 @@ func (x *AssignTopicPartitionsRequest) String() string { func (*AssignTopicPartitionsRequest) ProtoMessage() {} func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1513,7 +963,7 @@ func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsRequest.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27} + return file_mq_proto_rawDescGZIP(), []int{17} } func (x *AssignTopicPartitionsRequest) GetTopic() *Topic { @@ -1537,6 +987,13 @@ func (x *AssignTopicPartitionsRequest) GetIsLeader() bool { return false } +func (x *AssignTopicPartitionsRequest) GetIsDraining() bool { + if x != nil { + return x.IsDraining + } + return false +} + type AssignTopicPartitionsResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1546,7 +1003,7 @@ type AssignTopicPartitionsResponse struct { func (x *AssignTopicPartitionsResponse) Reset() { *x = AssignTopicPartitionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1559,7 +1016,7 @@ func (x *AssignTopicPartitionsResponse) String() string { func (*AssignTopicPartitionsResponse) ProtoMessage() {} func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1572,37 +1029,38 @@ func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsResponse.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{28} + return file_mq_proto_rawDescGZIP(), []int{18} } -type CheckTopicPartitionsStatusRequest struct { +type SubscriberToSubCoordinatorRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` - Topic string `protobuf:"bytes,2,opt,name=topic,proto3" json:"topic,omitempty"` - BrokerPartitionAssignment *BrokerPartitionAssignment `protobuf:"bytes,3,opt,name=broker_partition_assignment,json=brokerPartitionAssignment,proto3" json:"broker_partition_assignment,omitempty"` - ShouldCancelIfNotMatch bool `protobuf:"varint,4,opt,name=should_cancel_if_not_match,json=shouldCancelIfNotMatch,proto3" json:"should_cancel_if_not_match,omitempty"` + // Types that are assignable to Message: + // + // *SubscriberToSubCoordinatorRequest_Init + // *SubscriberToSubCoordinatorRequest_Ack + Message isSubscriberToSubCoordinatorRequest_Message `protobuf_oneof:"message"` } -func (x *CheckTopicPartitionsStatusRequest) Reset() { - *x = CheckTopicPartitionsStatusRequest{} +func (x *SubscriberToSubCoordinatorRequest) Reset() { + *x = SubscriberToSubCoordinatorRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *CheckTopicPartitionsStatusRequest) String() string { +func (x *SubscriberToSubCoordinatorRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CheckTopicPartitionsStatusRequest) ProtoMessage() {} +func (*SubscriberToSubCoordinatorRequest) ProtoMessage() {} -func (x *CheckTopicPartitionsStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[29] +func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1613,64 +1071,76 @@ func (x *CheckTopicPartitionsStatusRequest) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use CheckTopicPartitionsStatusRequest.ProtoReflect.Descriptor instead. -func (*CheckTopicPartitionsStatusRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{29} +// Deprecated: Use SubscriberToSubCoordinatorRequest.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{19} } -func (x *CheckTopicPartitionsStatusRequest) GetNamespace() string { - if x != nil { - return x.Namespace - } - return "" -} - -func (x *CheckTopicPartitionsStatusRequest) GetTopic() string { - if x != nil { - return x.Topic - } - return "" -} - -func (x *CheckTopicPartitionsStatusRequest) GetBrokerPartitionAssignment() *BrokerPartitionAssignment { - if x != nil { - return x.BrokerPartitionAssignment +func (m *SubscriberToSubCoordinatorRequest) GetMessage() isSubscriberToSubCoordinatorRequest_Message { + if m != nil { + return m.Message } return nil } -func (x *CheckTopicPartitionsStatusRequest) GetShouldCancelIfNotMatch() bool { - if x != nil { - return x.ShouldCancelIfNotMatch +func (x *SubscriberToSubCoordinatorRequest) GetInit() *SubscriberToSubCoordinatorRequest_InitMessage { + if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorRequest_Init); ok { + return x.Init } - return false + return nil } -type CheckTopicPartitionsStatusResponse struct { +func (x *SubscriberToSubCoordinatorRequest) GetAck() *SubscriberToSubCoordinatorRequest_AckMessage { + if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorRequest_Ack); ok { + return x.Ack + } + return nil +} + +type isSubscriberToSubCoordinatorRequest_Message interface { + isSubscriberToSubCoordinatorRequest_Message() +} + +type SubscriberToSubCoordinatorRequest_Init struct { + Init *SubscriberToSubCoordinatorRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +} + +type SubscriberToSubCoordinatorRequest_Ack struct { + Ack *SubscriberToSubCoordinatorRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"` +} + +func (*SubscriberToSubCoordinatorRequest_Init) isSubscriberToSubCoordinatorRequest_Message() {} + +func (*SubscriberToSubCoordinatorRequest_Ack) isSubscriberToSubCoordinatorRequest_Message() {} + +type SubscriberToSubCoordinatorResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,1,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"` + // Types that are assignable to Message: + // + // *SubscriberToSubCoordinatorResponse_Assignment_ + Message isSubscriberToSubCoordinatorResponse_Message `protobuf_oneof:"message"` } -func (x *CheckTopicPartitionsStatusResponse) Reset() { - *x = CheckTopicPartitionsStatusResponse{} +func (x *SubscriberToSubCoordinatorResponse) Reset() { + *x = SubscriberToSubCoordinatorResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[30] + mi := &file_mq_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *CheckTopicPartitionsStatusResponse) String() string { +func (x *SubscriberToSubCoordinatorResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CheckTopicPartitionsStatusResponse) ProtoMessage() {} +func (*SubscriberToSubCoordinatorResponse) ProtoMessage() {} -func (x *CheckTopicPartitionsStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[30] +func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1681,18 +1151,36 @@ func (x *CheckTopicPartitionsStatusResponse) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use CheckTopicPartitionsStatusResponse.ProtoReflect.Descriptor instead. -func (*CheckTopicPartitionsStatusResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30} +// Deprecated: Use SubscriberToSubCoordinatorResponse.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{20} } -func (x *CheckTopicPartitionsStatusResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment { - if x != nil { - return x.BrokerPartitionAssignments +func (m *SubscriberToSubCoordinatorResponse) GetMessage() isSubscriberToSubCoordinatorResponse_Message { + if m != nil { + return m.Message } return nil } +func (x *SubscriberToSubCoordinatorResponse) GetAssignment() *SubscriberToSubCoordinatorResponse_Assignment { + if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorResponse_Assignment_); ok { + return x.Assignment + } + return nil +} + +type isSubscriberToSubCoordinatorResponse_Message interface { + isSubscriberToSubCoordinatorResponse_Message() +} + +type SubscriberToSubCoordinatorResponse_Assignment_ struct { + Assignment *SubscriberToSubCoordinatorResponse_Assignment `protobuf:"bytes,1,opt,name=assignment,proto3,oneof"` +} + +func (*SubscriberToSubCoordinatorResponse_Assignment_) isSubscriberToSubCoordinatorResponse_Message() { +} + // //////////////////////////////////////////////// type DataMessage struct { state protoimpl.MessageState @@ -1701,12 +1189,13 @@ type DataMessage struct { Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + TsNs int64 `protobuf:"varint,3,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"` } func (x *DataMessage) Reset() { *x = DataMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1719,7 +1208,7 @@ func (x *DataMessage) String() string { func (*DataMessage) ProtoMessage() {} func (x *DataMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1732,7 +1221,7 @@ func (x *DataMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use DataMessage.ProtoReflect.Descriptor instead. func (*DataMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31} + return file_mq_proto_rawDescGZIP(), []int{21} } func (x *DataMessage) GetKey() []byte { @@ -1749,6 +1238,13 @@ func (x *DataMessage) GetValue() []byte { return nil } +func (x *DataMessage) GetTsNs() int64 { + if x != nil { + return x.TsNs + } + return 0 +} + type PublishRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1765,7 +1261,7 @@ type PublishRequest struct { func (x *PublishRequest) Reset() { *x = PublishRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1778,7 +1274,7 @@ func (x *PublishRequest) String() string { func (*PublishRequest) ProtoMessage() {} func (x *PublishRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1791,7 +1287,7 @@ func (x *PublishRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishRequest.ProtoReflect.Descriptor instead. func (*PublishRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{32} + return file_mq_proto_rawDescGZIP(), []int{22} } func (m *PublishRequest) GetMessage() isPublishRequest_Message { @@ -1843,15 +1339,15 @@ type PublishResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - AckSequence int64 `protobuf:"varint,1,opt,name=ack_sequence,json=ackSequence,proto3" json:"ack_sequence,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - RedirectToBroker string `protobuf:"bytes,3,opt,name=redirect_to_broker,json=redirectToBroker,proto3" json:"redirect_to_broker,omitempty"` + AckSequence int64 `protobuf:"varint,1,opt,name=ack_sequence,json=ackSequence,proto3" json:"ack_sequence,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + ShouldClose bool `protobuf:"varint,3,opt,name=should_close,json=shouldClose,proto3" json:"should_close,omitempty"` } func (x *PublishResponse) Reset() { *x = PublishResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1864,7 +1360,7 @@ func (x *PublishResponse) String() string { func (*PublishResponse) ProtoMessage() {} func (x *PublishResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1877,7 +1373,7 @@ func (x *PublishResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishResponse.ProtoReflect.Descriptor instead. func (*PublishResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{33} + return file_mq_proto_rawDescGZIP(), []int{23} } func (x *PublishResponse) GetAckSequence() int64 { @@ -1894,11 +1390,11 @@ func (x *PublishResponse) GetError() string { return "" } -func (x *PublishResponse) GetRedirectToBroker() string { +func (x *PublishResponse) GetShouldClose() bool { if x != nil { - return x.RedirectToBroker + return x.ShouldClose } - return "" + return false } type SubscribeRequest struct { @@ -1916,7 +1412,7 @@ type SubscribeRequest struct { func (x *SubscribeRequest) Reset() { *x = SubscribeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1929,7 +1425,7 @@ func (x *SubscribeRequest) String() string { func (*SubscribeRequest) ProtoMessage() {} func (x *SubscribeRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1942,7 +1438,7 @@ func (x *SubscribeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeRequest.ProtoReflect.Descriptor instead. func (*SubscribeRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{34} + return file_mq_proto_rawDescGZIP(), []int{24} } func (m *SubscribeRequest) GetMessage() isSubscribeRequest_Message { @@ -1997,7 +1493,7 @@ type SubscribeResponse struct { func (x *SubscribeResponse) Reset() { *x = SubscribeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2010,7 +1506,7 @@ func (x *SubscribeResponse) String() string { func (*SubscribeResponse) ProtoMessage() {} func (x *SubscribeResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2023,7 +1519,7 @@ func (x *SubscribeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeResponse.ProtoReflect.Descriptor instead. func (*SubscribeResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{35} + return file_mq_proto_rawDescGZIP(), []int{25} } func (m *SubscribeResponse) GetMessage() isSubscribeResponse_Message { @@ -2063,31 +1559,32 @@ func (*SubscribeResponse_Ctrl) isSubscribeResponse_Message() {} func (*SubscribeResponse_Data) isSubscribeResponse_Message() {} -type ConnectToBalancerRequest_InitMessage struct { +type ClosePublishersRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Broker string `protobuf:"bytes,1,opt,name=broker,proto3" json:"broker,omitempty"` + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` } -func (x *ConnectToBalancerRequest_InitMessage) Reset() { - *x = ConnectToBalancerRequest_InitMessage{} +func (x *ClosePublishersRequest) Reset() { + *x = ClosePublishersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *ConnectToBalancerRequest_InitMessage) String() string { +func (x *ClosePublishersRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ConnectToBalancerRequest_InitMessage) ProtoMessage() {} +func (*ClosePublishersRequest) ProtoMessage() {} -func (x *ConnectToBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[37] +func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2098,18 +1595,431 @@ func (x *ConnectToBalancerRequest_InitMessage) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use ConnectToBalancerRequest_InitMessage.ProtoReflect.Descriptor instead. -func (*ConnectToBalancerRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{14, 0} +// Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. +func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{26} } -func (x *ConnectToBalancerRequest_InitMessage) GetBroker() string { +func (x *ClosePublishersRequest) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *ClosePublishersRequest) GetUnixTimeNs() int64 { + if x != nil { + return x.UnixTimeNs + } + return 0 +} + +type ClosePublishersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ClosePublishersResponse) Reset() { + *x = ClosePublishersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClosePublishersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClosePublishersResponse) ProtoMessage() {} + +func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. +func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{27} +} + +type CloseSubscribersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` +} + +func (x *CloseSubscribersRequest) Reset() { + *x = CloseSubscribersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseSubscribersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseSubscribersRequest) ProtoMessage() {} + +func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. +func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{28} +} + +func (x *CloseSubscribersRequest) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *CloseSubscribersRequest) GetUnixTimeNs() int64 { + if x != nil { + return x.UnixTimeNs + } + return 0 +} + +type CloseSubscribersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CloseSubscribersResponse) Reset() { + *x = CloseSubscribersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseSubscribersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseSubscribersResponse) ProtoMessage() {} + +func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[29] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. +func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{29} +} + +type PublisherToPubBalancerRequest_InitMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Broker string `protobuf:"bytes,1,opt,name=broker,proto3" json:"broker,omitempty"` +} + +func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { + *x = PublisherToPubBalancerRequest_InitMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublisherToPubBalancerRequest_InitMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} + +func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublisherToPubBalancerRequest_InitMessage.ProtoReflect.Descriptor instead. +func (*PublisherToPubBalancerRequest_InitMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{6, 0} +} + +func (x *PublisherToPubBalancerRequest_InitMessage) GetBroker() string { if x != nil { return x.Broker } return "" } +type SubscriberToSubCoordinatorRequest_InitMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerInstanceId string `protobuf:"bytes,2,opt,name=consumer_instance_id,json=consumerInstanceId,proto3" json:"consumer_instance_id,omitempty"` + Topic *Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` +} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { + *x = SubscriberToSubCoordinatorRequest_InitMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscriberToSubCoordinatorRequest_InitMessage.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorRequest_InitMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{19, 0} +} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroup() string { + if x != nil { + return x.ConsumerGroup + } + return "" +} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerInstanceId() string { + if x != nil { + return x.ConsumerInstanceId + } + return "" +} + +func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +type SubscriberToSubCoordinatorRequest_AckMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + TsNs int64 `protobuf:"varint,2,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"` +} + +func (x *SubscriberToSubCoordinatorRequest_AckMessage) Reset() { + *x = SubscriberToSubCoordinatorRequest_AckMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscriberToSubCoordinatorRequest_AckMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscriberToSubCoordinatorRequest_AckMessage) ProtoMessage() {} + +func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscriberToSubCoordinatorRequest_AckMessage.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorRequest_AckMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{19, 1} +} + +func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetTsNs() int64 { + if x != nil { + return x.TsNs + } + return 0 +} + +type SubscriberToSubCoordinatorResponse_AssignedPartition struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + TsNs int64 `protobuf:"varint,2,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"` +} + +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) Reset() { + *x = SubscriberToSubCoordinatorResponse_AssignedPartition{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoMessage() {} + +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscriberToSubCoordinatorResponse_AssignedPartition.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorResponse_AssignedPartition) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{20, 0} +} + +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetTsNs() int64 { + if x != nil { + return x.TsNs + } + return 0 +} + +type SubscriberToSubCoordinatorResponse_Assignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Generation int64 `protobuf:"varint,1,opt,name=generation,proto3" json:"generation,omitempty"` + AssignedPartitions []*SubscriberToSubCoordinatorResponse_AssignedPartition `protobuf:"bytes,2,rep,name=assigned_partitions,json=assignedPartitions,proto3" json:"assigned_partitions,omitempty"` +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { + *x = SubscriberToSubCoordinatorResponse_Assignment{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[35] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscriberToSubCoordinatorResponse_Assignment.ProtoReflect.Descriptor instead. +func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{20, 1} +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) GetAssignedPartitions() []*SubscriberToSubCoordinatorResponse_AssignedPartition { + if x != nil { + return x.AssignedPartitions + } + return nil +} + type PublishRequest_InitMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2123,7 +2033,7 @@ type PublishRequest_InitMessage struct { func (x *PublishRequest_InitMessage) Reset() { *x = PublishRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2136,7 +2046,7 @@ func (x *PublishRequest_InitMessage) String() string { func (*PublishRequest_InitMessage) ProtoMessage() {} func (x *PublishRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2149,7 +2059,7 @@ func (x *PublishRequest_InitMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishRequest_InitMessage.ProtoReflect.Descriptor instead. func (*PublishRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{32, 0} + return file_mq_proto_rawDescGZIP(), []int{22, 0} } func (x *PublishRequest_InitMessage) GetTopic() *Topic { @@ -2194,7 +2104,7 @@ type SubscribeRequest_InitMessage struct { func (x *SubscribeRequest_InitMessage) Reset() { *x = SubscribeRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2207,7 +2117,7 @@ func (x *SubscribeRequest_InitMessage) String() string { func (*SubscribeRequest_InitMessage) ProtoMessage() {} func (x *SubscribeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2220,7 +2130,7 @@ func (x *SubscribeRequest_InitMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeRequest_InitMessage.ProtoReflect.Descriptor instead. func (*SubscribeRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{34, 0} + return file_mq_proto_rawDescGZIP(), []int{24, 0} } func (x *SubscribeRequest_InitMessage) GetConsumerGroup() string { @@ -2313,7 +2223,7 @@ type SubscribeRequest_AckMessage struct { func (x *SubscribeRequest_AckMessage) Reset() { *x = SubscribeRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2326,7 +2236,7 @@ func (x *SubscribeRequest_AckMessage) String() string { func (*SubscribeRequest_AckMessage) ProtoMessage() {} func (x *SubscribeRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2339,7 +2249,7 @@ func (x *SubscribeRequest_AckMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeRequest_AckMessage.ProtoReflect.Descriptor instead. func (*SubscribeRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{34, 1} + return file_mq_proto_rawDescGZIP(), []int{24, 1} } func (x *SubscribeRequest_AckMessage) GetSequence() int64 { @@ -2362,7 +2272,7 @@ type SubscribeResponse_CtrlMessage struct { func (x *SubscribeResponse_CtrlMessage) Reset() { *x = SubscribeResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2375,7 +2285,7 @@ func (x *SubscribeResponse_CtrlMessage) String() string { func (*SubscribeResponse_CtrlMessage) ProtoMessage() {} func (x *SubscribeResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2388,7 +2298,7 @@ func (x *SubscribeResponse_CtrlMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeResponse_CtrlMessage.ProtoReflect.Descriptor instead. func (*SubscribeResponse_CtrlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{35, 0} + return file_mq_proto_rawDescGZIP(), []int{25, 0} } func (x *SubscribeResponse_CtrlMessage) GetError() string { @@ -2416,374 +2326,327 @@ var File_mq_proto protoreflect.FileDescriptor var file_mq_proto_rawDesc = []byte{ 0x0a, 0x08, 0x6d, 0x71, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x22, 0xe8, 0x01, 0x0a, 0x0b, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2f, 0x0a, 0x07, 0x73, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x52, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1e, 0x0a, 0x0b, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x62, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x73, 0x12, 0x1c, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x74, 0x73, 0x5f, 0x6e, - 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x6f, 0x70, 0x54, 0x73, 0x4e, - 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x05, 0x52, 0x10, 0x70, 0x72, - 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x23, - 0x0a, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, - 0x06, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0c, 0x6e, 0x65, 0x78, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x22, 0x3a, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, - 0x0a, 0x0b, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, - 0x32, 0x0a, 0x18, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x22, 0x39, 0x0a, 0x05, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, - 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x68, - 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x72, - 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, - 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x61, 0x6e, - 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x22, 0x84, 0x01, 0x0a, 0x07, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, - 0x4e, 0x0a, 0x1b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2f, - 0x0a, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x22, - 0x38, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x18, 0x0a, 0x07, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x07, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, 0x4c, 0x0a, 0x19, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2f, 0x0a, 0x07, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x07, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x39, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x61, 0x63, 0x74, 0x69, - 0x76, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x41, 0x63, 0x74, 0x69, - 0x76, 0x65, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x5f, 0x0a, 0x17, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x6f, 0x61, 0x64, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1f, 0x0a, 0x0b, - 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x0a, 0x62, 0x79, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd2, 0x01, - 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x2a, 0x0a, - 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, - 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, 0x75, 0x55, 0x73, 0x61, - 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, - 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, - 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0xbb, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x6f, - 0x75, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x22, 0xc9, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x42, 0x61, - 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x48, 0x0a, - 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x54, 0x6f, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, - 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, - 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x1b, 0x0a, 0x19, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6b, 0x0a, 0x15, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x27, 0x0a, - 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7b, 0x0a, 0x17, - 0x44, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x1a, 0x0a, 0x18, 0x44, 0x6f, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, - 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x6c, 0x0a, - 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x66, 0x6f, 0x72, 0x5f, - 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, - 0x73, 0x46, 0x6f, 0x72, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x22, 0xb2, 0x01, 0x0a, 0x1a, - 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x22, 0x3a, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, + 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x22, 0x32, 0x0a, 0x18, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x22, 0x39, 0x0a, 0x05, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, + 0x1d, 0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x20, + 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, + 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x12, 0x2a, 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, + 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, 0x75, + 0x55, 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x22, 0xa2, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, - 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, - 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, - 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, 0x73, 0x0a, 0x1d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, - 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x8b, 0x01, 0x0a, 0x1e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, - 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xd1, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xbb, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x22, 0xd3, 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, + 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, + 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, + 0x69, 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, + 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, + 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x17, 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6b, 0x0a, 0x15, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x16, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, - 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x1f, 0x0a, 0x1d, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xfc, 0x01, - 0x0a, 0x21, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x67, 0x0a, 0x1b, 0x62, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, + 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, + 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x6c, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, + 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, + 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x46, 0x6f, 0x72, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x22, 0xb2, 0x01, 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, + 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, + 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xa2, 0x01, 0x0a, 0x19, 0x42, + 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, + 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, + 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, + 0xf2, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, + 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x19, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x12, 0x3a, 0x0a, 0x1a, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, - 0x6c, 0x5f, 0x69, 0x66, 0x5f, 0x6e, 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x16, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x43, 0x61, 0x6e, 0x63, - 0x65, 0x6c, 0x49, 0x66, 0x4e, 0x6f, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x8f, 0x01, 0x0a, - 0x22, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x35, - 0x0a, 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3e, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, - 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, - 0x75, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x92, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x69, + 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x72, 0x61, 0x69, + 0x6e, 0x69, 0x6e, 0x67, 0x22, 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xbf, 0x03, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, + 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x04, 0x69, + 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x4e, + 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, + 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0x91, + 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, + 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, 0x0a, 0x14, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x6e, 0x73, + 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x1a, 0x58, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x61, - 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, - 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, - 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x6f, - 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x72, - 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x22, - 0x96, 0x04, 0x0a, 0x10, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, - 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x3d, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, - 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xcb, 0x02, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, - 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0c, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x03, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, - 0x74, 0x12, 0x2e, 0x0a, 0x12, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, - 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, - 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x6f, 0x66, 0x66, - 0x73, 0x65, 0x74, 0x1a, 0x28, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x87, 0x02, 0x0a, 0x11, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, - 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6d, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x42, 0x09, 0x0a, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, 0x03, 0x0a, 0x22, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, + 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, + 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, + 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, + 0x00, 0x52, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x5f, 0x0a, + 0x11, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x1a, 0xa1, + 0x01, 0x0a, 0x0a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1e, 0x0a, + 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x73, 0x0a, + 0x13, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, + 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, + 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x4a, 0x0a, + 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3e, 0x0a, 0x04, + 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, + 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x92, 0x01, 0x0a, 0x0b, 0x49, 0x6e, + 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, + 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x42, 0x09, + 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x6d, 0x0a, 0x0f, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, + 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, + 0x75, 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x96, 0x04, 0x0a, 0x10, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, + 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, + 0x3d, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x74, - 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72, - 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, - 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, - 0x74, 0x61, 0x1a, 0x73, 0x0a, 0x0b, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x65, 0x6e, - 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0d, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x12, 0x25, 0x0a, 0x0f, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45, 0x6e, 0x64, - 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x32, 0xb2, 0x0b, 0x0a, 0x10, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x14, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, - 0x12, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x24, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x6f, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x6f, 0x61, 0x64, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x11, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, - 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x6f, - 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x63, 0x0a, 0x10, 0x44, 0x6f, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x6f, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xcb, + 0x02, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, + 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, + 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0b, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x2e, 0x0a, 0x12, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x73, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x1a, 0x28, 0x0a, 0x0a, + 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, + 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, + 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x22, 0x87, 0x02, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x73, 0x0a, 0x0b, 0x43, + 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x73, 0x45, 0x6e, + 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x25, 0x0a, 0x0f, 0x69, 0x73, 0x5f, + 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x65, 0x0a, 0x16, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, + 0x4e, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, + 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, + 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x1a, 0x0a, 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x32, 0xce, 0x09, 0x0a, 0x10, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x44, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x12, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x65, 0x73, + 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x16, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, + 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, + 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, + 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x5a, 0x0a, 0x0d, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, + 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x22, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x12, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x16, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x12, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x15, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, @@ -2791,30 +2654,43 @@ var file_mq_proto_rawDesc = []byte{ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x07, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x12, 0x1c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1d, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x50, 0x0a, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x12, 0x1e, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0x4e, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65, - 0x65, 0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x42, 0x10, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x51, - 0x75, 0x65, 0x75, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68, 0x75, - 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, - 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, - 0x62, 0x2f, 0x6d, 0x71, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, + 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, + 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, + 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, + 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x07, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x1c, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, + 0x12, 0x50, 0x0a, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x1e, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x30, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, + 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x73, 0x65, 0x61, 0x77, + 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x6d, 0x71, + 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2829,120 +2705,113 @@ func file_mq_proto_rawDescGZIP() []byte { return file_mq_proto_rawDescData } -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 42) +var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 40) var file_mq_proto_goTypes = []interface{}{ - (*SegmentInfo)(nil), // 0: messaging_pb.SegmentInfo - (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest - (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse - (*Topic)(nil), // 3: messaging_pb.Topic - (*Partition)(nil), // 4: messaging_pb.Partition - (*Segment)(nil), // 5: messaging_pb.Segment - (*AssignSegmentBrokersRequest)(nil), // 6: messaging_pb.AssignSegmentBrokersRequest - (*AssignSegmentBrokersResponse)(nil), // 7: messaging_pb.AssignSegmentBrokersResponse - (*CheckSegmentStatusRequest)(nil), // 8: messaging_pb.CheckSegmentStatusRequest - (*CheckSegmentStatusResponse)(nil), // 9: messaging_pb.CheckSegmentStatusResponse - (*CheckBrokerLoadRequest)(nil), // 10: messaging_pb.CheckBrokerLoadRequest - (*CheckBrokerLoadResponse)(nil), // 11: messaging_pb.CheckBrokerLoadResponse - (*BrokerStats)(nil), // 12: messaging_pb.BrokerStats - (*TopicPartitionStats)(nil), // 13: messaging_pb.TopicPartitionStats - (*ConnectToBalancerRequest)(nil), // 14: messaging_pb.ConnectToBalancerRequest - (*ConnectToBalancerResponse)(nil), // 15: messaging_pb.ConnectToBalancerResponse - (*ConfigureTopicRequest)(nil), // 16: messaging_pb.ConfigureTopicRequest - (*ConfigureTopicResponse)(nil), // 17: messaging_pb.ConfigureTopicResponse - (*DoConfigureTopicRequest)(nil), // 18: messaging_pb.DoConfigureTopicRequest - (*DoConfigureTopicResponse)(nil), // 19: messaging_pb.DoConfigureTopicResponse - (*ListTopicsRequest)(nil), // 20: messaging_pb.ListTopicsRequest - (*ListTopicsResponse)(nil), // 21: messaging_pb.ListTopicsResponse - (*LookupTopicBrokersRequest)(nil), // 22: messaging_pb.LookupTopicBrokersRequest - (*LookupTopicBrokersResponse)(nil), // 23: messaging_pb.LookupTopicBrokersResponse - (*BrokerPartitionAssignment)(nil), // 24: messaging_pb.BrokerPartitionAssignment - (*RequestTopicPartitionsRequest)(nil), // 25: messaging_pb.RequestTopicPartitionsRequest - (*RequestTopicPartitionsResponse)(nil), // 26: messaging_pb.RequestTopicPartitionsResponse - (*AssignTopicPartitionsRequest)(nil), // 27: messaging_pb.AssignTopicPartitionsRequest - (*AssignTopicPartitionsResponse)(nil), // 28: messaging_pb.AssignTopicPartitionsResponse - (*CheckTopicPartitionsStatusRequest)(nil), // 29: messaging_pb.CheckTopicPartitionsStatusRequest - (*CheckTopicPartitionsStatusResponse)(nil), // 30: messaging_pb.CheckTopicPartitionsStatusResponse - (*DataMessage)(nil), // 31: messaging_pb.DataMessage - (*PublishRequest)(nil), // 32: messaging_pb.PublishRequest - (*PublishResponse)(nil), // 33: messaging_pb.PublishResponse - (*SubscribeRequest)(nil), // 34: messaging_pb.SubscribeRequest - (*SubscribeResponse)(nil), // 35: messaging_pb.SubscribeResponse - nil, // 36: messaging_pb.BrokerStats.StatsEntry - (*ConnectToBalancerRequest_InitMessage)(nil), // 37: messaging_pb.ConnectToBalancerRequest.InitMessage - (*PublishRequest_InitMessage)(nil), // 38: messaging_pb.PublishRequest.InitMessage - (*SubscribeRequest_InitMessage)(nil), // 39: messaging_pb.SubscribeRequest.InitMessage - (*SubscribeRequest_AckMessage)(nil), // 40: messaging_pb.SubscribeRequest.AckMessage - (*SubscribeResponse_CtrlMessage)(nil), // 41: messaging_pb.SubscribeResponse.CtrlMessage + (*FindBrokerLeaderRequest)(nil), // 0: messaging_pb.FindBrokerLeaderRequest + (*FindBrokerLeaderResponse)(nil), // 1: messaging_pb.FindBrokerLeaderResponse + (*Topic)(nil), // 2: messaging_pb.Topic + (*Partition)(nil), // 3: messaging_pb.Partition + (*BrokerStats)(nil), // 4: messaging_pb.BrokerStats + (*TopicPartitionStats)(nil), // 5: messaging_pb.TopicPartitionStats + (*PublisherToPubBalancerRequest)(nil), // 6: messaging_pb.PublisherToPubBalancerRequest + (*PublisherToPubBalancerResponse)(nil), // 7: messaging_pb.PublisherToPubBalancerResponse + (*BalanceTopicsRequest)(nil), // 8: messaging_pb.BalanceTopicsRequest + (*BalanceTopicsResponse)(nil), // 9: messaging_pb.BalanceTopicsResponse + (*ConfigureTopicRequest)(nil), // 10: messaging_pb.ConfigureTopicRequest + (*ConfigureTopicResponse)(nil), // 11: messaging_pb.ConfigureTopicResponse + (*ListTopicsRequest)(nil), // 12: messaging_pb.ListTopicsRequest + (*ListTopicsResponse)(nil), // 13: messaging_pb.ListTopicsResponse + (*LookupTopicBrokersRequest)(nil), // 14: messaging_pb.LookupTopicBrokersRequest + (*LookupTopicBrokersResponse)(nil), // 15: messaging_pb.LookupTopicBrokersResponse + (*BrokerPartitionAssignment)(nil), // 16: messaging_pb.BrokerPartitionAssignment + (*AssignTopicPartitionsRequest)(nil), // 17: messaging_pb.AssignTopicPartitionsRequest + (*AssignTopicPartitionsResponse)(nil), // 18: messaging_pb.AssignTopicPartitionsResponse + (*SubscriberToSubCoordinatorRequest)(nil), // 19: messaging_pb.SubscriberToSubCoordinatorRequest + (*SubscriberToSubCoordinatorResponse)(nil), // 20: messaging_pb.SubscriberToSubCoordinatorResponse + (*DataMessage)(nil), // 21: messaging_pb.DataMessage + (*PublishRequest)(nil), // 22: messaging_pb.PublishRequest + (*PublishResponse)(nil), // 23: messaging_pb.PublishResponse + (*SubscribeRequest)(nil), // 24: messaging_pb.SubscribeRequest + (*SubscribeResponse)(nil), // 25: messaging_pb.SubscribeResponse + (*ClosePublishersRequest)(nil), // 26: messaging_pb.ClosePublishersRequest + (*ClosePublishersResponse)(nil), // 27: messaging_pb.ClosePublishersResponse + (*CloseSubscribersRequest)(nil), // 28: messaging_pb.CloseSubscribersRequest + (*CloseSubscribersResponse)(nil), // 29: messaging_pb.CloseSubscribersResponse + nil, // 30: messaging_pb.BrokerStats.StatsEntry + (*PublisherToPubBalancerRequest_InitMessage)(nil), // 31: messaging_pb.PublisherToPubBalancerRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 32: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 33: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage + (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 34: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition + (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 35: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + (*PublishRequest_InitMessage)(nil), // 36: messaging_pb.PublishRequest.InitMessage + (*SubscribeRequest_InitMessage)(nil), // 37: messaging_pb.SubscribeRequest.InitMessage + (*SubscribeRequest_AckMessage)(nil), // 38: messaging_pb.SubscribeRequest.AckMessage + (*SubscribeResponse_CtrlMessage)(nil), // 39: messaging_pb.SubscribeResponse.CtrlMessage } var file_mq_proto_depIdxs = []int32{ - 5, // 0: messaging_pb.SegmentInfo.segment:type_name -> messaging_pb.Segment - 4, // 1: messaging_pb.Segment.partition:type_name -> messaging_pb.Partition - 5, // 2: messaging_pb.AssignSegmentBrokersRequest.segment:type_name -> messaging_pb.Segment - 5, // 3: messaging_pb.CheckSegmentStatusRequest.segment:type_name -> messaging_pb.Segment - 36, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry - 3, // 5: messaging_pb.TopicPartitionStats.topic:type_name -> messaging_pb.Topic - 4, // 6: messaging_pb.TopicPartitionStats.partition:type_name -> messaging_pb.Partition - 37, // 7: messaging_pb.ConnectToBalancerRequest.init:type_name -> messaging_pb.ConnectToBalancerRequest.InitMessage - 12, // 8: messaging_pb.ConnectToBalancerRequest.stats:type_name -> messaging_pb.BrokerStats - 3, // 9: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic - 24, // 10: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 3, // 11: messaging_pb.DoConfigureTopicRequest.topic:type_name -> messaging_pb.Topic - 4, // 12: messaging_pb.DoConfigureTopicRequest.partition:type_name -> messaging_pb.Partition - 3, // 13: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic - 3, // 14: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic - 3, // 15: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic - 24, // 16: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 4, // 17: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition - 3, // 18: messaging_pb.RequestTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic - 24, // 19: messaging_pb.RequestTopicPartitionsResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 3, // 20: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic - 24, // 21: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 24, // 22: messaging_pb.CheckTopicPartitionsStatusRequest.broker_partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment - 24, // 23: messaging_pb.CheckTopicPartitionsStatusResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 38, // 24: messaging_pb.PublishRequest.init:type_name -> messaging_pb.PublishRequest.InitMessage - 31, // 25: messaging_pb.PublishRequest.data:type_name -> messaging_pb.DataMessage - 39, // 26: messaging_pb.SubscribeRequest.init:type_name -> messaging_pb.SubscribeRequest.InitMessage - 40, // 27: messaging_pb.SubscribeRequest.ack:type_name -> messaging_pb.SubscribeRequest.AckMessage - 41, // 28: messaging_pb.SubscribeResponse.ctrl:type_name -> messaging_pb.SubscribeResponse.CtrlMessage - 31, // 29: messaging_pb.SubscribeResponse.data:type_name -> messaging_pb.DataMessage - 13, // 30: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 3, // 31: messaging_pb.PublishRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 32: messaging_pb.PublishRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 33: messaging_pb.SubscribeRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 34: messaging_pb.SubscribeRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 1, // 35: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 6, // 36: messaging_pb.SeaweedMessaging.AssignSegmentBrokers:input_type -> messaging_pb.AssignSegmentBrokersRequest - 8, // 37: messaging_pb.SeaweedMessaging.CheckSegmentStatus:input_type -> messaging_pb.CheckSegmentStatusRequest - 10, // 38: messaging_pb.SeaweedMessaging.CheckBrokerLoad:input_type -> messaging_pb.CheckBrokerLoadRequest - 14, // 39: messaging_pb.SeaweedMessaging.ConnectToBalancer:input_type -> messaging_pb.ConnectToBalancerRequest - 18, // 40: messaging_pb.SeaweedMessaging.DoConfigureTopic:input_type -> messaging_pb.DoConfigureTopicRequest - 22, // 41: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 16, // 42: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 20, // 43: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 25, // 44: messaging_pb.SeaweedMessaging.RequestTopicPartitions:input_type -> messaging_pb.RequestTopicPartitionsRequest - 27, // 45: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 29, // 46: messaging_pb.SeaweedMessaging.CheckTopicPartitionsStatus:input_type -> messaging_pb.CheckTopicPartitionsStatusRequest - 32, // 47: messaging_pb.SeaweedMessaging.Publish:input_type -> messaging_pb.PublishRequest - 34, // 48: messaging_pb.SeaweedMessaging.Subscribe:input_type -> messaging_pb.SubscribeRequest - 2, // 49: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 7, // 50: messaging_pb.SeaweedMessaging.AssignSegmentBrokers:output_type -> messaging_pb.AssignSegmentBrokersResponse - 9, // 51: messaging_pb.SeaweedMessaging.CheckSegmentStatus:output_type -> messaging_pb.CheckSegmentStatusResponse - 11, // 52: messaging_pb.SeaweedMessaging.CheckBrokerLoad:output_type -> messaging_pb.CheckBrokerLoadResponse - 15, // 53: messaging_pb.SeaweedMessaging.ConnectToBalancer:output_type -> messaging_pb.ConnectToBalancerResponse - 19, // 54: messaging_pb.SeaweedMessaging.DoConfigureTopic:output_type -> messaging_pb.DoConfigureTopicResponse - 23, // 55: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 17, // 56: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 21, // 57: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 26, // 58: messaging_pb.SeaweedMessaging.RequestTopicPartitions:output_type -> messaging_pb.RequestTopicPartitionsResponse - 28, // 59: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 30, // 60: messaging_pb.SeaweedMessaging.CheckTopicPartitionsStatus:output_type -> messaging_pb.CheckTopicPartitionsStatusResponse - 33, // 61: messaging_pb.SeaweedMessaging.Publish:output_type -> messaging_pb.PublishResponse - 35, // 62: messaging_pb.SeaweedMessaging.Subscribe:output_type -> messaging_pb.SubscribeResponse - 49, // [49:63] is the sub-list for method output_type - 35, // [35:49] is the sub-list for method input_type - 35, // [35:35] is the sub-list for extension type_name - 35, // [35:35] is the sub-list for extension extendee - 0, // [0:35] is the sub-list for field type_name + 30, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 2, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> messaging_pb.Topic + 3, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> messaging_pb.Partition + 31, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 4, // 4: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats + 2, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic + 16, // 6: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 2, // 7: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic + 2, // 8: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic + 2, // 9: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic + 16, // 10: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 3, // 11: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition + 2, // 12: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic + 16, // 13: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 32, // 14: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + 33, // 15: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage + 35, // 16: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + 36, // 17: messaging_pb.PublishRequest.init:type_name -> messaging_pb.PublishRequest.InitMessage + 21, // 18: messaging_pb.PublishRequest.data:type_name -> messaging_pb.DataMessage + 37, // 19: messaging_pb.SubscribeRequest.init:type_name -> messaging_pb.SubscribeRequest.InitMessage + 38, // 20: messaging_pb.SubscribeRequest.ack:type_name -> messaging_pb.SubscribeRequest.AckMessage + 39, // 21: messaging_pb.SubscribeResponse.ctrl:type_name -> messaging_pb.SubscribeResponse.CtrlMessage + 21, // 22: messaging_pb.SubscribeResponse.data:type_name -> messaging_pb.DataMessage + 2, // 23: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic + 2, // 24: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic + 5, // 25: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats + 2, // 26: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 3, // 27: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition + 3, // 28: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition + 34, // 29: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition + 2, // 30: messaging_pb.PublishRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 3, // 31: messaging_pb.PublishRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 2, // 32: messaging_pb.SubscribeRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 3, // 33: messaging_pb.SubscribeRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 0, // 34: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 6, // 35: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 8, // 36: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 12, // 37: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 10, // 38: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 14, // 39: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 17, // 40: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 26, // 41: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 28, // 42: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 19, // 43: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 22, // 44: messaging_pb.SeaweedMessaging.Publish:input_type -> messaging_pb.PublishRequest + 24, // 45: messaging_pb.SeaweedMessaging.Subscribe:input_type -> messaging_pb.SubscribeRequest + 1, // 46: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 7, // 47: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 9, // 48: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 13, // 49: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 11, // 50: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 15, // 51: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 18, // 52: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 27, // 53: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 29, // 54: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 20, // 55: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 23, // 56: messaging_pb.SeaweedMessaging.Publish:output_type -> messaging_pb.PublishResponse + 25, // 57: messaging_pb.SeaweedMessaging.Subscribe:output_type -> messaging_pb.SubscribeResponse + 46, // [46:58] is the sub-list for method output_type + 34, // [34:46] is the sub-list for method input_type + 34, // [34:34] is the sub-list for extension type_name + 34, // [34:34] is the sub-list for extension extendee + 0, // [0:34] is the sub-list for field type_name } func init() { file_mq_proto_init() } @@ -2952,18 +2821,6 @@ func file_mq_proto_init() { } if !protoimpl.UnsafeEnabled { file_mq_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FindBrokerLeaderRequest); i { case 0: return &v.state @@ -2975,7 +2832,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FindBrokerLeaderResponse); i { case 0: return &v.state @@ -2987,7 +2844,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Topic); i { case 0: return &v.state @@ -2999,7 +2856,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Partition); i { case 0: return &v.state @@ -3011,91 +2868,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Segment); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignSegmentBrokersRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignSegmentBrokersResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckSegmentStatusRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckSegmentStatusResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckBrokerLoadRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckBrokerLoadResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BrokerStats); i { case 0: return &v.state @@ -3107,7 +2880,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TopicPartitionStats); i { case 0: return &v.state @@ -3119,8 +2892,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConnectToBalancerRequest); i { + file_mq_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublisherToPubBalancerRequest); i { case 0: return &v.state case 1: @@ -3131,8 +2904,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConnectToBalancerResponse); i { + file_mq_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublisherToPubBalancerResponse); i { case 0: return &v.state case 1: @@ -3143,7 +2916,31 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BalanceTopicsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BalanceTopicsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ConfigureTopicRequest); i { case 0: return &v.state @@ -3155,7 +2952,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ConfigureTopicResponse); i { case 0: return &v.state @@ -3167,31 +2964,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DoConfigureTopicRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DoConfigureTopicResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ListTopicsRequest); i { case 0: return &v.state @@ -3203,7 +2976,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ListTopicsResponse); i { case 0: return &v.state @@ -3215,7 +2988,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupTopicBrokersRequest); i { case 0: return &v.state @@ -3227,7 +3000,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupTopicBrokersResponse); i { case 0: return &v.state @@ -3239,7 +3012,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BrokerPartitionAssignment); i { case 0: return &v.state @@ -3251,31 +3024,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RequestTopicPartitionsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RequestTopicPartitionsResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AssignTopicPartitionsRequest); i { case 0: return &v.state @@ -3287,7 +3036,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AssignTopicPartitionsResponse); i { case 0: return &v.state @@ -3299,8 +3048,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckTopicPartitionsStatusRequest); i { + file_mq_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorRequest); i { case 0: return &v.state case 1: @@ -3311,8 +3060,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckTopicPartitionsStatusResponse); i { + file_mq_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorResponse); i { case 0: return &v.state case 1: @@ -3323,7 +3072,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DataMessage); i { case 0: return &v.state @@ -3335,7 +3084,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishRequest); i { case 0: return &v.state @@ -3347,7 +3096,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishResponse); i { case 0: return &v.state @@ -3359,7 +3108,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeRequest); i { case 0: return &v.state @@ -3371,7 +3120,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeResponse); i { case 0: return &v.state @@ -3383,8 +3132,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConnectToBalancerRequest_InitMessage); i { + file_mq_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ClosePublishersRequest); i { case 0: return &v.state case 1: @@ -3395,7 +3144,103 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ClosePublishersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseSubscribersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseSubscribersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishRequest_InitMessage); i { case 0: return &v.state @@ -3407,7 +3252,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeRequest_InitMessage); i { case 0: return &v.state @@ -3419,7 +3264,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeRequest_AckMessage); i { case 0: return &v.state @@ -3431,7 +3276,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeResponse_CtrlMessage); i { case 0: return &v.state @@ -3444,23 +3289,30 @@ func file_mq_proto_init() { } } } - file_mq_proto_msgTypes[14].OneofWrappers = []interface{}{ - (*ConnectToBalancerRequest_Init)(nil), - (*ConnectToBalancerRequest_Stats)(nil), + file_mq_proto_msgTypes[6].OneofWrappers = []interface{}{ + (*PublisherToPubBalancerRequest_Init)(nil), + (*PublisherToPubBalancerRequest_Stats)(nil), } - file_mq_proto_msgTypes[32].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[19].OneofWrappers = []interface{}{ + (*SubscriberToSubCoordinatorRequest_Init)(nil), + (*SubscriberToSubCoordinatorRequest_Ack)(nil), + } + file_mq_proto_msgTypes[20].OneofWrappers = []interface{}{ + (*SubscriberToSubCoordinatorResponse_Assignment_)(nil), + } + file_mq_proto_msgTypes[22].OneofWrappers = []interface{}{ (*PublishRequest_Init)(nil), (*PublishRequest_Data)(nil), } - file_mq_proto_msgTypes[34].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[24].OneofWrappers = []interface{}{ (*SubscribeRequest_Init)(nil), (*SubscribeRequest_Ack)(nil), } - file_mq_proto_msgTypes[35].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[25].OneofWrappers = []interface{}{ (*SubscribeResponse_Ctrl)(nil), (*SubscribeResponse_Data)(nil), } - file_mq_proto_msgTypes[39].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[37].OneofWrappers = []interface{}{ (*SubscribeRequest_InitMessage_StartOffset)(nil), (*SubscribeRequest_InitMessage_StartTimestampNs)(nil), } @@ -3470,7 +3322,7 @@ func file_mq_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_mq_proto_rawDesc, NumEnums: 0, - NumMessages: 42, + NumMessages: 40, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/pb/mq_pb/mq_grpc.pb.go b/weed/pb/mq_pb/mq_grpc.pb.go index ecbd24978..b8e1387ba 100644 --- a/weed/pb/mq_pb/mq_grpc.pb.go +++ b/weed/pb/mq_pb/mq_grpc.pb.go @@ -20,17 +20,15 @@ const _ = grpc.SupportPackageIsVersion7 const ( SeaweedMessaging_FindBrokerLeader_FullMethodName = "/messaging_pb.SeaweedMessaging/FindBrokerLeader" - SeaweedMessaging_AssignSegmentBrokers_FullMethodName = "/messaging_pb.SeaweedMessaging/AssignSegmentBrokers" - SeaweedMessaging_CheckSegmentStatus_FullMethodName = "/messaging_pb.SeaweedMessaging/CheckSegmentStatus" - SeaweedMessaging_CheckBrokerLoad_FullMethodName = "/messaging_pb.SeaweedMessaging/CheckBrokerLoad" - SeaweedMessaging_ConnectToBalancer_FullMethodName = "/messaging_pb.SeaweedMessaging/ConnectToBalancer" - SeaweedMessaging_DoConfigureTopic_FullMethodName = "/messaging_pb.SeaweedMessaging/DoConfigureTopic" - SeaweedMessaging_LookupTopicBrokers_FullMethodName = "/messaging_pb.SeaweedMessaging/LookupTopicBrokers" - SeaweedMessaging_ConfigureTopic_FullMethodName = "/messaging_pb.SeaweedMessaging/ConfigureTopic" + SeaweedMessaging_PublisherToPubBalancer_FullMethodName = "/messaging_pb.SeaweedMessaging/PublisherToPubBalancer" + SeaweedMessaging_BalanceTopics_FullMethodName = "/messaging_pb.SeaweedMessaging/BalanceTopics" SeaweedMessaging_ListTopics_FullMethodName = "/messaging_pb.SeaweedMessaging/ListTopics" - SeaweedMessaging_RequestTopicPartitions_FullMethodName = "/messaging_pb.SeaweedMessaging/RequestTopicPartitions" + SeaweedMessaging_ConfigureTopic_FullMethodName = "/messaging_pb.SeaweedMessaging/ConfigureTopic" + SeaweedMessaging_LookupTopicBrokers_FullMethodName = "/messaging_pb.SeaweedMessaging/LookupTopicBrokers" SeaweedMessaging_AssignTopicPartitions_FullMethodName = "/messaging_pb.SeaweedMessaging/AssignTopicPartitions" - SeaweedMessaging_CheckTopicPartitionsStatus_FullMethodName = "/messaging_pb.SeaweedMessaging/CheckTopicPartitionsStatus" + SeaweedMessaging_ClosePublishers_FullMethodName = "/messaging_pb.SeaweedMessaging/ClosePublishers" + SeaweedMessaging_CloseSubscribers_FullMethodName = "/messaging_pb.SeaweedMessaging/CloseSubscribers" + SeaweedMessaging_SubscriberToSubCoordinator_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscriberToSubCoordinator" SeaweedMessaging_Publish_FullMethodName = "/messaging_pb.SeaweedMessaging/Publish" SeaweedMessaging_Subscribe_FullMethodName = "/messaging_pb.SeaweedMessaging/Subscribe" ) @@ -41,21 +39,20 @@ const ( type SeaweedMessagingClient interface { // control plane FindBrokerLeader(ctx context.Context, in *FindBrokerLeaderRequest, opts ...grpc.CallOption) (*FindBrokerLeaderResponse, error) - AssignSegmentBrokers(ctx context.Context, in *AssignSegmentBrokersRequest, opts ...grpc.CallOption) (*AssignSegmentBrokersResponse, error) - CheckSegmentStatus(ctx context.Context, in *CheckSegmentStatusRequest, opts ...grpc.CallOption) (*CheckSegmentStatusResponse, error) - CheckBrokerLoad(ctx context.Context, in *CheckBrokerLoadRequest, opts ...grpc.CallOption) (*CheckBrokerLoadResponse, error) // control plane for balancer - ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error) - DoConfigureTopic(ctx context.Context, in *DoConfigureTopicRequest, opts ...grpc.CallOption) (*DoConfigureTopicResponse, error) + PublisherToPubBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublisherToPubBalancerClient, error) + BalanceTopics(ctx context.Context, in *BalanceTopicsRequest, opts ...grpc.CallOption) (*BalanceTopicsResponse, error) // control plane for topic partitions - LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) - ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) ListTopics(ctx context.Context, in *ListTopicsRequest, opts ...grpc.CallOption) (*ListTopicsResponse, error) - // a pub client will call this to get the topic partitions assignment - RequestTopicPartitions(ctx context.Context, in *RequestTopicPartitionsRequest, opts ...grpc.CallOption) (*RequestTopicPartitionsResponse, error) + ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) + LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) + // invoked by the balancer, running on each broker AssignTopicPartitions(ctx context.Context, in *AssignTopicPartitionsRequest, opts ...grpc.CallOption) (*AssignTopicPartitionsResponse, error) - CheckTopicPartitionsStatus(ctx context.Context, in *CheckTopicPartitionsStatusRequest, opts ...grpc.CallOption) (*CheckTopicPartitionsStatusResponse, error) - // data plane + ClosePublishers(ctx context.Context, in *ClosePublishersRequest, opts ...grpc.CallOption) (*ClosePublishersResponse, error) + CloseSubscribers(ctx context.Context, in *CloseSubscribersRequest, opts ...grpc.CallOption) (*CloseSubscribersResponse, error) + // subscriber connects to broker balancer, which coordinates with the subscribers + SubscriberToSubCoordinator(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscriberToSubCoordinatorClient, error) + // data plane for each topic partition Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) } @@ -77,85 +74,40 @@ func (c *seaweedMessagingClient) FindBrokerLeader(ctx context.Context, in *FindB return out, nil } -func (c *seaweedMessagingClient) AssignSegmentBrokers(ctx context.Context, in *AssignSegmentBrokersRequest, opts ...grpc.CallOption) (*AssignSegmentBrokersResponse, error) { - out := new(AssignSegmentBrokersResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_AssignSegmentBrokers_FullMethodName, in, out, opts...) +func (c *seaweedMessagingClient) PublisherToPubBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublisherToPubBalancerClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], SeaweedMessaging_PublisherToPubBalancer_FullMethodName, opts...) if err != nil { return nil, err } - return out, nil -} - -func (c *seaweedMessagingClient) CheckSegmentStatus(ctx context.Context, in *CheckSegmentStatusRequest, opts ...grpc.CallOption) (*CheckSegmentStatusResponse, error) { - out := new(CheckSegmentStatusResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_CheckSegmentStatus_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) CheckBrokerLoad(ctx context.Context, in *CheckBrokerLoadRequest, opts ...grpc.CallOption) (*CheckBrokerLoadResponse, error) { - out := new(CheckBrokerLoadResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_CheckBrokerLoad_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error) { - stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], SeaweedMessaging_ConnectToBalancer_FullMethodName, opts...) - if err != nil { - return nil, err - } - x := &seaweedMessagingConnectToBalancerClient{stream} + x := &seaweedMessagingPublisherToPubBalancerClient{stream} return x, nil } -type SeaweedMessaging_ConnectToBalancerClient interface { - Send(*ConnectToBalancerRequest) error - Recv() (*ConnectToBalancerResponse, error) +type SeaweedMessaging_PublisherToPubBalancerClient interface { + Send(*PublisherToPubBalancerRequest) error + Recv() (*PublisherToPubBalancerResponse, error) grpc.ClientStream } -type seaweedMessagingConnectToBalancerClient struct { +type seaweedMessagingPublisherToPubBalancerClient struct { grpc.ClientStream } -func (x *seaweedMessagingConnectToBalancerClient) Send(m *ConnectToBalancerRequest) error { +func (x *seaweedMessagingPublisherToPubBalancerClient) Send(m *PublisherToPubBalancerRequest) error { return x.ClientStream.SendMsg(m) } -func (x *seaweedMessagingConnectToBalancerClient) Recv() (*ConnectToBalancerResponse, error) { - m := new(ConnectToBalancerResponse) +func (x *seaweedMessagingPublisherToPubBalancerClient) Recv() (*PublisherToPubBalancerResponse, error) { + m := new(PublisherToPubBalancerResponse) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } return m, nil } -func (c *seaweedMessagingClient) DoConfigureTopic(ctx context.Context, in *DoConfigureTopicRequest, opts ...grpc.CallOption) (*DoConfigureTopicResponse, error) { - out := new(DoConfigureTopicResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_DoConfigureTopic_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) { - out := new(LookupTopicBrokersResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_LookupTopicBrokers_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) { - out := new(ConfigureTopicResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_ConfigureTopic_FullMethodName, in, out, opts...) +func (c *seaweedMessagingClient) BalanceTopics(ctx context.Context, in *BalanceTopicsRequest, opts ...grpc.CallOption) (*BalanceTopicsResponse, error) { + out := new(BalanceTopicsResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_BalanceTopics_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -171,9 +123,18 @@ func (c *seaweedMessagingClient) ListTopics(ctx context.Context, in *ListTopicsR return out, nil } -func (c *seaweedMessagingClient) RequestTopicPartitions(ctx context.Context, in *RequestTopicPartitionsRequest, opts ...grpc.CallOption) (*RequestTopicPartitionsResponse, error) { - out := new(RequestTopicPartitionsResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_RequestTopicPartitions_FullMethodName, in, out, opts...) +func (c *seaweedMessagingClient) ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error) { + out := new(ConfigureTopicResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_ConfigureTopic_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingClient) LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) { + out := new(LookupTopicBrokersResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_LookupTopicBrokers_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -189,17 +150,57 @@ func (c *seaweedMessagingClient) AssignTopicPartitions(ctx context.Context, in * return out, nil } -func (c *seaweedMessagingClient) CheckTopicPartitionsStatus(ctx context.Context, in *CheckTopicPartitionsStatusRequest, opts ...grpc.CallOption) (*CheckTopicPartitionsStatusResponse, error) { - out := new(CheckTopicPartitionsStatusResponse) - err := c.cc.Invoke(ctx, SeaweedMessaging_CheckTopicPartitionsStatus_FullMethodName, in, out, opts...) +func (c *seaweedMessagingClient) ClosePublishers(ctx context.Context, in *ClosePublishersRequest, opts ...grpc.CallOption) (*ClosePublishersResponse, error) { + out := new(ClosePublishersResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_ClosePublishers_FullMethodName, in, out, opts...) if err != nil { return nil, err } return out, nil } +func (c *seaweedMessagingClient) CloseSubscribers(ctx context.Context, in *CloseSubscribersRequest, opts ...grpc.CallOption) (*CloseSubscribersResponse, error) { + out := new(CloseSubscribersResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_CloseSubscribers_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingClient) SubscriberToSubCoordinator(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscriberToSubCoordinatorClient, error) { + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], SeaweedMessaging_SubscriberToSubCoordinator_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &seaweedMessagingSubscriberToSubCoordinatorClient{stream} + return x, nil +} + +type SeaweedMessaging_SubscriberToSubCoordinatorClient interface { + Send(*SubscriberToSubCoordinatorRequest) error + Recv() (*SubscriberToSubCoordinatorResponse, error) + grpc.ClientStream +} + +type seaweedMessagingSubscriberToSubCoordinatorClient struct { + grpc.ClientStream +} + +func (x *seaweedMessagingSubscriberToSubCoordinatorClient) Send(m *SubscriberToSubCoordinatorRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *seaweedMessagingSubscriberToSubCoordinatorClient) Recv() (*SubscriberToSubCoordinatorResponse, error) { + m := new(SubscriberToSubCoordinatorResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) { - stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], SeaweedMessaging_Publish_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], SeaweedMessaging_Publish_FullMethodName, opts...) if err != nil { return nil, err } @@ -230,7 +231,7 @@ func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) { } func (c *seaweedMessagingClient) Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) { - stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], SeaweedMessaging_Subscribe_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[3], SeaweedMessaging_Subscribe_FullMethodName, opts...) if err != nil { return nil, err } @@ -267,21 +268,20 @@ func (x *seaweedMessagingSubscribeClient) Recv() (*SubscribeResponse, error) { type SeaweedMessagingServer interface { // control plane FindBrokerLeader(context.Context, *FindBrokerLeaderRequest) (*FindBrokerLeaderResponse, error) - AssignSegmentBrokers(context.Context, *AssignSegmentBrokersRequest) (*AssignSegmentBrokersResponse, error) - CheckSegmentStatus(context.Context, *CheckSegmentStatusRequest) (*CheckSegmentStatusResponse, error) - CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error) // control plane for balancer - ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error - DoConfigureTopic(context.Context, *DoConfigureTopicRequest) (*DoConfigureTopicResponse, error) + PublisherToPubBalancer(SeaweedMessaging_PublisherToPubBalancerServer) error + BalanceTopics(context.Context, *BalanceTopicsRequest) (*BalanceTopicsResponse, error) // control plane for topic partitions - LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) - ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) ListTopics(context.Context, *ListTopicsRequest) (*ListTopicsResponse, error) - // a pub client will call this to get the topic partitions assignment - RequestTopicPartitions(context.Context, *RequestTopicPartitionsRequest) (*RequestTopicPartitionsResponse, error) + ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) + LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) + // invoked by the balancer, running on each broker AssignTopicPartitions(context.Context, *AssignTopicPartitionsRequest) (*AssignTopicPartitionsResponse, error) - CheckTopicPartitionsStatus(context.Context, *CheckTopicPartitionsStatusRequest) (*CheckTopicPartitionsStatusResponse, error) - // data plane + ClosePublishers(context.Context, *ClosePublishersRequest) (*ClosePublishersResponse, error) + CloseSubscribers(context.Context, *CloseSubscribersRequest) (*CloseSubscribersResponse, error) + // subscriber connects to broker balancer, which coordinates with the subscribers + SubscriberToSubCoordinator(SeaweedMessaging_SubscriberToSubCoordinatorServer) error + // data plane for each topic partition Publish(SeaweedMessaging_PublishServer) error Subscribe(*SubscribeRequest, SeaweedMessaging_SubscribeServer) error mustEmbedUnimplementedSeaweedMessagingServer() @@ -294,38 +294,32 @@ type UnimplementedSeaweedMessagingServer struct { func (UnimplementedSeaweedMessagingServer) FindBrokerLeader(context.Context, *FindBrokerLeaderRequest) (*FindBrokerLeaderResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method FindBrokerLeader not implemented") } -func (UnimplementedSeaweedMessagingServer) AssignSegmentBrokers(context.Context, *AssignSegmentBrokersRequest) (*AssignSegmentBrokersResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method AssignSegmentBrokers not implemented") +func (UnimplementedSeaweedMessagingServer) PublisherToPubBalancer(SeaweedMessaging_PublisherToPubBalancerServer) error { + return status.Errorf(codes.Unimplemented, "method PublisherToPubBalancer not implemented") } -func (UnimplementedSeaweedMessagingServer) CheckSegmentStatus(context.Context, *CheckSegmentStatusRequest) (*CheckSegmentStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CheckSegmentStatus not implemented") -} -func (UnimplementedSeaweedMessagingServer) CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CheckBrokerLoad not implemented") -} -func (UnimplementedSeaweedMessagingServer) ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error { - return status.Errorf(codes.Unimplemented, "method ConnectToBalancer not implemented") -} -func (UnimplementedSeaweedMessagingServer) DoConfigureTopic(context.Context, *DoConfigureTopicRequest) (*DoConfigureTopicResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method DoConfigureTopic not implemented") -} -func (UnimplementedSeaweedMessagingServer) LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method LookupTopicBrokers not implemented") -} -func (UnimplementedSeaweedMessagingServer) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ConfigureTopic not implemented") +func (UnimplementedSeaweedMessagingServer) BalanceTopics(context.Context, *BalanceTopicsRequest) (*BalanceTopicsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method BalanceTopics not implemented") } func (UnimplementedSeaweedMessagingServer) ListTopics(context.Context, *ListTopicsRequest) (*ListTopicsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ListTopics not implemented") } -func (UnimplementedSeaweedMessagingServer) RequestTopicPartitions(context.Context, *RequestTopicPartitionsRequest) (*RequestTopicPartitionsResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method RequestTopicPartitions not implemented") +func (UnimplementedSeaweedMessagingServer) ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ConfigureTopic not implemented") +} +func (UnimplementedSeaweedMessagingServer) LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LookupTopicBrokers not implemented") } func (UnimplementedSeaweedMessagingServer) AssignTopicPartitions(context.Context, *AssignTopicPartitionsRequest) (*AssignTopicPartitionsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method AssignTopicPartitions not implemented") } -func (UnimplementedSeaweedMessagingServer) CheckTopicPartitionsStatus(context.Context, *CheckTopicPartitionsStatusRequest) (*CheckTopicPartitionsStatusResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CheckTopicPartitionsStatus not implemented") +func (UnimplementedSeaweedMessagingServer) ClosePublishers(context.Context, *ClosePublishersRequest) (*ClosePublishersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ClosePublishers not implemented") +} +func (UnimplementedSeaweedMessagingServer) CloseSubscribers(context.Context, *CloseSubscribersRequest) (*CloseSubscribersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CloseSubscribers not implemented") +} +func (UnimplementedSeaweedMessagingServer) SubscriberToSubCoordinator(SeaweedMessaging_SubscriberToSubCoordinatorServer) error { + return status.Errorf(codes.Unimplemented, "method SubscriberToSubCoordinator not implemented") } func (UnimplementedSeaweedMessagingServer) Publish(SeaweedMessaging_PublishServer) error { return status.Errorf(codes.Unimplemented, "method Publish not implemented") @@ -364,136 +358,46 @@ func _SeaweedMessaging_FindBrokerLeader_Handler(srv interface{}, ctx context.Con return interceptor(ctx, in, info, handler) } -func _SeaweedMessaging_AssignSegmentBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(AssignSegmentBrokersRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).AssignSegmentBrokers(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: SeaweedMessaging_AssignSegmentBrokers_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).AssignSegmentBrokers(ctx, req.(*AssignSegmentBrokersRequest)) - } - return interceptor(ctx, in, info, handler) +func _SeaweedMessaging_PublisherToPubBalancer_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingServer).PublisherToPubBalancer(&seaweedMessagingPublisherToPubBalancerServer{stream}) } -func _SeaweedMessaging_CheckSegmentStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CheckSegmentStatusRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).CheckSegmentStatus(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: SeaweedMessaging_CheckSegmentStatus_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).CheckSegmentStatus(ctx, req.(*CheckSegmentStatusRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_CheckBrokerLoad_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CheckBrokerLoadRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).CheckBrokerLoad(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: SeaweedMessaging_CheckBrokerLoad_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).CheckBrokerLoad(ctx, req.(*CheckBrokerLoadRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_ConnectToBalancer_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(SeaweedMessagingServer).ConnectToBalancer(&seaweedMessagingConnectToBalancerServer{stream}) -} - -type SeaweedMessaging_ConnectToBalancerServer interface { - Send(*ConnectToBalancerResponse) error - Recv() (*ConnectToBalancerRequest, error) +type SeaweedMessaging_PublisherToPubBalancerServer interface { + Send(*PublisherToPubBalancerResponse) error + Recv() (*PublisherToPubBalancerRequest, error) grpc.ServerStream } -type seaweedMessagingConnectToBalancerServer struct { +type seaweedMessagingPublisherToPubBalancerServer struct { grpc.ServerStream } -func (x *seaweedMessagingConnectToBalancerServer) Send(m *ConnectToBalancerResponse) error { +func (x *seaweedMessagingPublisherToPubBalancerServer) Send(m *PublisherToPubBalancerResponse) error { return x.ServerStream.SendMsg(m) } -func (x *seaweedMessagingConnectToBalancerServer) Recv() (*ConnectToBalancerRequest, error) { - m := new(ConnectToBalancerRequest) +func (x *seaweedMessagingPublisherToPubBalancerServer) Recv() (*PublisherToPubBalancerRequest, error) { + m := new(PublisherToPubBalancerRequest) if err := x.ServerStream.RecvMsg(m); err != nil { return nil, err } return m, nil } -func _SeaweedMessaging_DoConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DoConfigureTopicRequest) +func _SeaweedMessaging_BalanceTopics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(BalanceTopicsRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(SeaweedMessagingServer).DoConfigureTopic(ctx, in) + return srv.(SeaweedMessagingServer).BalanceTopics(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: SeaweedMessaging_DoConfigureTopic_FullMethodName, + FullMethod: SeaweedMessaging_BalanceTopics_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).DoConfigureTopic(ctx, req.(*DoConfigureTopicRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_LookupTopicBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(LookupTopicBrokersRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: SeaweedMessaging_LookupTopicBrokers_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, req.(*LookupTopicBrokersRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ConfigureTopicRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: SeaweedMessaging_ConfigureTopic_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, req.(*ConfigureTopicRequest)) + return srv.(SeaweedMessagingServer).BalanceTopics(ctx, req.(*BalanceTopicsRequest)) } return interceptor(ctx, in, info, handler) } @@ -516,20 +420,38 @@ func _SeaweedMessaging_ListTopics_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } -func _SeaweedMessaging_RequestTopicPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(RequestTopicPartitionsRequest) +func _SeaweedMessaging_ConfigureTopic_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ConfigureTopicRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(SeaweedMessagingServer).RequestTopicPartitions(ctx, in) + return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: SeaweedMessaging_RequestTopicPartitions_FullMethodName, + FullMethod: SeaweedMessaging_ConfigureTopic_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).RequestTopicPartitions(ctx, req.(*RequestTopicPartitionsRequest)) + return srv.(SeaweedMessagingServer).ConfigureTopic(ctx, req.(*ConfigureTopicRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessaging_LookupTopicBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LookupTopicBrokersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessaging_LookupTopicBrokers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, req.(*LookupTopicBrokersRequest)) } return interceptor(ctx, in, info, handler) } @@ -552,24 +474,68 @@ func _SeaweedMessaging_AssignTopicPartitions_Handler(srv interface{}, ctx contex return interceptor(ctx, in, info, handler) } -func _SeaweedMessaging_CheckTopicPartitionsStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CheckTopicPartitionsStatusRequest) +func _SeaweedMessaging_ClosePublishers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ClosePublishersRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(SeaweedMessagingServer).CheckTopicPartitionsStatus(ctx, in) + return srv.(SeaweedMessagingServer).ClosePublishers(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: SeaweedMessaging_CheckTopicPartitionsStatus_FullMethodName, + FullMethod: SeaweedMessaging_ClosePublishers_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(SeaweedMessagingServer).CheckTopicPartitionsStatus(ctx, req.(*CheckTopicPartitionsStatusRequest)) + return srv.(SeaweedMessagingServer).ClosePublishers(ctx, req.(*ClosePublishersRequest)) } return interceptor(ctx, in, info, handler) } +func _SeaweedMessaging_CloseSubscribers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CloseSubscribersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).CloseSubscribers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessaging_CloseSubscribers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).CloseSubscribers(ctx, req.(*CloseSubscribersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessaging_SubscriberToSubCoordinator_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingServer).SubscriberToSubCoordinator(&seaweedMessagingSubscriberToSubCoordinatorServer{stream}) +} + +type SeaweedMessaging_SubscriberToSubCoordinatorServer interface { + Send(*SubscriberToSubCoordinatorResponse) error + Recv() (*SubscriberToSubCoordinatorRequest, error) + grpc.ServerStream +} + +type seaweedMessagingSubscriberToSubCoordinatorServer struct { + grpc.ServerStream +} + +func (x *seaweedMessagingSubscriberToSubCoordinatorServer) Send(m *SubscriberToSubCoordinatorResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *seaweedMessagingSubscriberToSubCoordinatorServer) Recv() (*SubscriberToSubCoordinatorRequest, error) { + m := new(SubscriberToSubCoordinatorRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + func _SeaweedMessaging_Publish_Handler(srv interface{}, stream grpc.ServerStream) error { return srv.(SeaweedMessagingServer).Publish(&seaweedMessagingPublishServer{stream}) } @@ -629,50 +595,44 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ Handler: _SeaweedMessaging_FindBrokerLeader_Handler, }, { - MethodName: "AssignSegmentBrokers", - Handler: _SeaweedMessaging_AssignSegmentBrokers_Handler, - }, - { - MethodName: "CheckSegmentStatus", - Handler: _SeaweedMessaging_CheckSegmentStatus_Handler, - }, - { - MethodName: "CheckBrokerLoad", - Handler: _SeaweedMessaging_CheckBrokerLoad_Handler, - }, - { - MethodName: "DoConfigureTopic", - Handler: _SeaweedMessaging_DoConfigureTopic_Handler, - }, - { - MethodName: "LookupTopicBrokers", - Handler: _SeaweedMessaging_LookupTopicBrokers_Handler, - }, - { - MethodName: "ConfigureTopic", - Handler: _SeaweedMessaging_ConfigureTopic_Handler, + MethodName: "BalanceTopics", + Handler: _SeaweedMessaging_BalanceTopics_Handler, }, { MethodName: "ListTopics", Handler: _SeaweedMessaging_ListTopics_Handler, }, { - MethodName: "RequestTopicPartitions", - Handler: _SeaweedMessaging_RequestTopicPartitions_Handler, + MethodName: "ConfigureTopic", + Handler: _SeaweedMessaging_ConfigureTopic_Handler, + }, + { + MethodName: "LookupTopicBrokers", + Handler: _SeaweedMessaging_LookupTopicBrokers_Handler, }, { MethodName: "AssignTopicPartitions", Handler: _SeaweedMessaging_AssignTopicPartitions_Handler, }, { - MethodName: "CheckTopicPartitionsStatus", - Handler: _SeaweedMessaging_CheckTopicPartitionsStatus_Handler, + MethodName: "ClosePublishers", + Handler: _SeaweedMessaging_ClosePublishers_Handler, + }, + { + MethodName: "CloseSubscribers", + Handler: _SeaweedMessaging_CloseSubscribers_Handler, }, }, Streams: []grpc.StreamDesc{ { - StreamName: "ConnectToBalancer", - Handler: _SeaweedMessaging_ConnectToBalancer_Handler, + StreamName: "PublisherToPubBalancer", + Handler: _SeaweedMessaging_PublisherToPubBalancer_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "SubscriberToSubCoordinator", + Handler: _SeaweedMessaging_SubscriberToSubCoordinator_Handler, ServerStreams: true, ClientStreams: true, }, diff --git a/weed/shell/command_mq_balance.go b/weed/shell/command_mq_balance.go new file mode 100644 index 000000000..dbe218b54 --- /dev/null +++ b/weed/shell/command_mq_balance.go @@ -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 + }) + +} diff --git a/weed/shell/command_mq_topic_create.go b/weed/shell/command_mq_topic_configure.go similarity index 73% rename from weed/shell/command_mq_topic_create.go rename to weed/shell/command_mq_topic_configure.go index 7e1f5d65e..c5721d9d9 100644 --- a/weed/shell/command_mq_topic_create.go +++ b/weed/shell/command_mq_topic_configure.go @@ -11,25 +11,25 @@ import ( ) func init() { - Commands = append(Commands, &commandMqTopicCreate{}) + Commands = append(Commands, &commandMqTopicConfigure{}) } -type commandMqTopicCreate struct { +type commandMqTopicConfigure struct { } -func (c *commandMqTopicCreate) Name() string { - return "mq.topic.create" +func (c *commandMqTopicConfigure) Name() string { + return "mq.topic.configure" } -func (c *commandMqTopicCreate) Help() string { - return `create a topic with a given name +func (c *commandMqTopicConfigure) Help() string { + return `configure a topic with a given name Example: - mq.topic.create -namespace -topic -partition_count + mq.topic.configure -namespace -topic -partition_count ` } -func (c *commandMqTopicCreate) Do(args []string, commandEnv *CommandEnv, writer io.Writer) error { +func (c *commandMqTopicConfigure) Do(args []string, commandEnv *CommandEnv, writer io.Writer) error { // parse parameters mqCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError) diff --git a/weed/shell/command_mq_topic_list.go b/weed/shell/command_mq_topic_list.go index 5fb913a8a..cf6b97e96 100644 --- a/weed/shell/command_mq_topic_list.go +++ b/weed/shell/command_mq_topic_list.go @@ -3,7 +3,7 @@ package shell import ( "context" "fmt" - "github.com/seaweedfs/seaweedfs/weed/mq/balancer" + "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" @@ -52,7 +52,7 @@ func (c *commandMqTopicList) Do(args []string, commandEnv *CommandEnv, writer io func findBrokerBalancer(commandEnv *CommandEnv) (brokerBalancer string, err error) { err = commandEnv.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{ - Name: balancer.LockBrokerBalancer, + Name: pub_balancer.LockBrokerBalancer, }) if err != nil { return err