From 205d2285f3414477ce9ad57197d7d960b8373e07 Mon Sep 17 00:00:00 2001 From: chrislu Date: Sun, 24 Mar 2024 12:57:09 -0700 Subject: [PATCH] setup follower by publisher * the subscriber would getOrGen a local partition and wait * the publisher would getOrGen a local partition. If localPartition follower is not setup, and init message has follower info, it would create followers based on init.Messages. --- weed/mq/broker/broker_grpc_pub.go | 40 +- weed/mq/broker/broker_grpc_sub.go | 44 +- weed/mq/broker/broker_grpc_sub_coordinator.go | 9 +- .../mq/broker/broker_topic_conf_read_write.go | 10 +- .../sub_client/connect_to_sub_coordinator.go | 21 +- weed/mq/sub_coordinator/consumer_group.go | 8 +- weed/mq/topic/local_partition.go | 4 + weed/pb/mq.proto | 8 +- weed/pb/mq_pb/mq.pb.go | 629 ++++++++---------- 9 files changed, 339 insertions(+), 434 deletions(-) diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go index 673a7b1cf..72ddf30f3 100644 --- a/weed/mq/broker/broker_grpc_pub.go +++ b/weed/mq/broker/broker_grpc_pub.go @@ -7,7 +7,6 @@ import ( "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" "google.golang.org/grpc/peer" "io" "math/rand" @@ -41,7 +40,6 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis // 3. write to the filer var localTopicPartition *topic.LocalPartition - var isGenerated bool req, err := stream.Recv() if err != nil { return err @@ -58,7 +56,13 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis // get or generate a local partition t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition) - localTopicPartition, isGenerated, err = b.GetOrGenLocalPartition(t, p) + conf, readConfErr := b.readTopicConfFromFiler(t) + if readConfErr != nil { + response.Error = fmt.Sprintf("topic %v not found: %v", initMessage.Topic, readConfErr) + glog.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr) + return stream.Send(response) + } + localTopicPartition, _, err = b.GetOrGenLocalPartition(t, p, conf) if err != nil { 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) @@ -67,25 +71,23 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis ackInterval = int(initMessage.AckInterval) // connect to follower brokers - var followerStream mq_pb.SeaweedMessaging_PublishFollowMeClient - var grpcConnection *grpc.ClientConn - if isGenerated && len(initMessage.FollowerBrokers) > 0 { + if localTopicPartition.FollowerStream == nil && len(initMessage.FollowerBrokers) > 0 { follower := initMessage.FollowerBrokers[0] ctx := stream.Context() - grpcConnection, err = pb.GrpcDial(ctx, follower, true, b.grpcDialOption) + localTopicPartition.GrpcConnection, err = pb.GrpcDial(ctx, follower, true, b.grpcDialOption) if err != nil { response.Error = fmt.Sprintf("fail to dial %s: %v", follower, err) glog.Errorf("fail to dial %s: %v", follower, err) return stream.Send(response) } - followerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection) - followerStream, err = followerClient.PublishFollowMe(ctx) + followerClient := mq_pb.NewSeaweedMessagingClient(localTopicPartition.GrpcConnection) + localTopicPartition.FollowerStream, err = followerClient.PublishFollowMe(ctx) if err != nil { response.Error = fmt.Sprintf("fail to create publish client: %v", err) glog.Errorf("fail to create publish client: %v", err) return stream.Send(response) } - if err = followerStream.Send(&mq_pb.PublishFollowMeRequest{ + if err = localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{ Message: &mq_pb.PublishFollowMeRequest_Init{ Init: &mq_pb.PublishFollowMeRequest_InitMessage{ Topic: initMessage.Topic, @@ -104,7 +106,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis ackCounter := 0 var ackSequence int64 defer func() { - if followerStream == nil { + if localTopicPartition.FollowerStream == nil { // remove the publisher localTopicPartition.Publishers.RemovePublisher(clientName) glog.V(0).Infof("topic %v partition %v published %d messges Publisher:%d Subscriber:%d", initMessage.Topic, initMessage.Partition, ackSequence, localTopicPartition.Publishers.Size(), localTopicPartition.Subscribers.Size()) @@ -114,7 +116,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis } }() - if followerStream != nil { + if localTopicPartition.FollowerStream != nil { go func() { defer func() { println("stop receiving ack from follower") @@ -126,11 +128,11 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis b.localTopicManager.RemoveTopicPartition(t, p) } println("closing grpcConnection to follower") - grpcConnection.Close() + localTopicPartition.GrpcConnection.Close() }() for { - ack, err := followerStream.Recv() + ack, err := localTopicPartition.FollowerStream.Recv() if err != nil { glog.Errorf("Error receiving response: %v", err) return @@ -153,7 +155,7 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis var receivedSequence, acknowledgedSequence int64 defer func() { - if followerStream != nil { + if localTopicPartition.FollowerStream != nil { //if err := followerStream.CloseSend(); err != nil { // glog.Errorf("Error closing follower stream: %v", err) //} @@ -193,9 +195,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis receivedSequence = dataMessage.TsNs // maybe send to the follower - if followerStream != nil { + if localTopicPartition.FollowerStream != nil { println("recv", string(dataMessage.Key), dataMessage.TsNs) - if followErr := followerStream.Send(&mq_pb.PublishFollowMeRequest{ + if followErr := localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{ Message: &mq_pb.PublishFollowMeRequest_Data{ Data: dataMessage, }, @@ -218,9 +220,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis } } - if followerStream != nil { + if localTopicPartition.FollowerStream != nil { // send close to the follower - if followErr := followerStream.Send(&mq_pb.PublishFollowMeRequest{ + if followErr := localTopicPartition.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{ Message: &mq_pb.PublishFollowMeRequest_Close{ Close: &mq_pb.PublishFollowMeRequest_CloseMessage{}, }, diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index 6033ad43c..724a34f7e 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -16,40 +16,28 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest ctx := stream.Context() clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId) + initMessage := req.GetInit() + if initMessage == nil { + glog.Errorf("missing init message") + return fmt.Errorf("missing init message") + } + t := topic.FromPbTopic(req.GetInit().Topic) partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition()) glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition) - waitIntervalCount := 0 - + // get or generate a local partition var localTopicPartition *topic.LocalPartition - for localTopicPartition == nil { - localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition) - if err != nil { - glog.V(1).Infof("topic %v partition %v not setup", t, partition) - } - if localTopicPartition != nil { - break - } - waitIntervalCount++ - if waitIntervalCount > 10 { - waitIntervalCount = 10 - } - time.Sleep(time.Duration(waitIntervalCount) * 337 * 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 nil - } - glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err) - return nil - default: - // Continue processing the request - } + conf, readConfErr := b.readTopicConfFromFiler(t) + if readConfErr != nil { + glog.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr) + return fmt.Errorf("topic %v not found: %v", initMessage.Topic, readConfErr) + } + localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition, conf) + if err != nil { + glog.Errorf("topic %v partition %v not setup", initMessage.Topic, partition) + return fmt.Errorf("topic %v partition %v not setup", initMessage.Topic, partition) } localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber()) diff --git a/weed/mq/broker/broker_grpc_sub_coordinator.go b/weed/mq/broker/broker_grpc_sub_coordinator.go index 89c221af5..3fd97f1c2 100644 --- a/weed/mq/broker/broker_grpc_sub_coordinator.go +++ b/weed/mq/broker/broker_grpc_sub_coordinator.go @@ -39,18 +39,11 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess go func() { // try to load the partition assignment from filer if conf, err := b.readTopicConfFromFiler(topic.FromPbTopic(initMessage.Topic)); err == nil { - assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(conf.BrokerPartitionAssignments)) - for i, assignment := range conf.BrokerPartitionAssignments { - assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{ - Partition: assignment.Partition, - Broker: assignment.LeaderBroker, - } - } // send partition assignment to subscriber cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{ Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{ Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{ - AssignedPartitions: assignedPartitions, + PartitionAssignments: conf.BrokerPartitionAssignments, }, }, } diff --git a/weed/mq/broker/broker_topic_conf_read_write.go b/weed/mq/broker/broker_topic_conf_read_write.go index 35d95c0e4..b5d47e1c9 100644 --- a/weed/mq/broker/broker_topic_conf_read_write.go +++ b/weed/mq/broker/broker_topic_conf_read_write.go @@ -56,12 +56,12 @@ func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb. return conf, nil } -func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) { +func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) { b.accessLock.Lock() defer b.accessLock.Unlock() if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil { - localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition) + localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition, conf) if err != nil { return nil, false, err } @@ -69,12 +69,8 @@ func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition top return localPartition, isGenerated, nil } -func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) { +func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) { self := b.option.BrokerAddress() - conf, err := b.readTopicConfFromFiler(t) - if err != nil { - return nil, isGenerated, err - } for _, assignment := range conf.BrokerPartitionAssignments { if assignment.LeaderBroker == string(self) && partition.Equals(topic.FromPbPartition(assignment.Partition)) { localPartition = topic.FromPbBrokerPartitionAssignment(b.option.BrokerAddress(), partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition)) diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go index 4cc3c8ff2..b0b533e42 100644 --- a/weed/mq/client/sub_client/connect_to_sub_coordinator.go +++ b/weed/mq/client/sub_client/connect_to_sub_coordinator.go @@ -91,26 +91,26 @@ func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCo var wg sync.WaitGroup semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit) - for _, assigned := range assignment.AssignedPartitions { + for _, assigned := range assignment.PartitionAssignments { wg.Add(1) semaphore <- struct{}{} - go func(partition *mq_pb.Partition, broker string) { + go func(assigned *mq_pb.BrokerPartitionAssignment) { defer wg.Done() defer func() { <-semaphore }() - glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker) - err := sub.onEachPartition(partition, broker) + glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker) + err := sub.onEachPartition(assigned) if err != nil { - glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err) + glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err) } - }(assigned.Partition, assigned.Broker) + }(assigned) } wg.Wait() } -func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker string) error { +func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment) error { // connect to the partition broker - return pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { + return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { subscribeClient, err := client.SubscribeMessage(context.Background(), &mq_pb.SubscribeMessageRequest{ Message: &mq_pb.SubscribeMessageRequest_Init{ Init: &mq_pb.SubscribeMessageRequest_InitMessage{ @@ -118,11 +118,12 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId, Topic: sub.ContentConfig.Topic.ToPbTopic(), PartitionOffset: &mq_pb.PartitionOffset{ - Partition: partition, + Partition: assigned.Partition, StartTsNs: sub.alreadyProcessedTsNs, StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY, }, Filter: sub.ContentConfig.Filter, + FollowerBrokers: assigned.FollowerBrokers, }, }, }) @@ -131,7 +132,7 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s return fmt.Errorf("create subscribe client: %v", err) } - glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker) + glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker) if sub.OnCompletionFunc != nil { defer sub.OnCompletionFunc() diff --git a/weed/mq/sub_coordinator/consumer_group.go b/weed/mq/sub_coordinator/consumer_group.go index f897fe2b3..d24a38d8a 100644 --- a/weed/mq/sub_coordinator/consumer_group.go +++ b/weed/mq/sub_coordinator/consumer_group.go @@ -103,22 +103,22 @@ func (cg *ConsumerGroup) RebalanceConsumberGroupInstances(knownPartitionSlotToBr partitionSlots = make([]*PartitionSlotToConsumerInstance, 0) } consumerGroupInstance.Partitions = ToPartitions(partitionSlotToBrokerList.RingSize, partitionSlots) - assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(partitionSlots)) + assignedPartitions := make([]*mq_pb.BrokerPartitionAssignment, len(partitionSlots)) for i, partitionSlot := range partitionSlots { - assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{ + assignedPartitions[i] = &mq_pb.BrokerPartitionAssignment{ Partition: &mq_pb.Partition{ RangeStop: partitionSlot.RangeStop, RangeStart: partitionSlot.RangeStart, RingSize: partitionSlotToBrokerList.RingSize, UnixTimeNs: partitionSlot.UnixTimeNs, }, - Broker: partitionSlot.Broker, + LeaderBroker: partitionSlot.Broker, } } response := &mq_pb.SubscriberToSubCoordinatorResponse{ Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{ Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{ - AssignedPartitions: assignedPartitions, + PartitionAssignments: assignedPartitions, }, }, } diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go index d72c86fcd..a25786280 100644 --- a/weed/mq/topic/local_partition.go +++ b/weed/mq/topic/local_partition.go @@ -6,6 +6,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/util/log_buffer" + "google.golang.org/grpc" "sync" "sync/atomic" "time" @@ -26,6 +27,9 @@ type LocalPartition struct { Publishers *LocalPartitionPublishers Subscribers *LocalPartitionSubscribers FollowerId int32 + + FollowerStream mq_pb.SeaweedMessaging_PublishFollowMeClient + GrpcConnection *grpc.ClientConn } var TIME_FORMAT = "2006-01-02-15-04-05" diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto index b9a32651c..45253059f 100644 --- a/weed/pb/mq.proto +++ b/weed/pb/mq.proto @@ -170,14 +170,9 @@ message SubscriberToSubCoordinatorRequest { } } message SubscriberToSubCoordinatorResponse { - message AssignedPartition { - Partition partition = 1; - int64 ts_ns = 2; - string broker = 3; - } message Assignment { int64 generation = 1; - repeated AssignedPartition assigned_partitions = 2; + repeated BrokerPartitionAssignment partition_assignments = 2; } oneof message { Assignment assignment = 1; @@ -236,6 +231,7 @@ message SubscribeMessageRequest { Topic topic = 4; PartitionOffset partition_offset = 5; string filter = 6; + repeated string follower_brokers = 7; } message AckMessage { int64 sequence = 1; diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq.pb.go index 1d316b937..b5182e6af 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq.pb.go @@ -2233,82 +2233,19 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetTsNs() int64 { 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"` - Broker string `protobuf:"bytes,3,opt,name=broker,proto3" json:"broker,omitempty"` -} - -func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) Reset() { - *x = SubscriberToSubCoordinatorResponse_AssignedPartition{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[38] - 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[38] - 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{22, 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 -} - -func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetBroker() string { - if x != nil { - return x.Broker - } - return "" -} - 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"` + Generation int64 `protobuf:"varint,1,opt,name=generation,proto3" json:"generation,omitempty"` + PartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=partition_assignments,json=partitionAssignments,proto3" json:"partition_assignments,omitempty"` } func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2321,7 +2258,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2334,7 +2271,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protorefl // Deprecated: Use SubscriberToSubCoordinatorResponse_Assignment.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22, 1} + return file_mq_proto_rawDescGZIP(), []int{22, 0} } func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 { @@ -2344,9 +2281,9 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 { return 0 } -func (x *SubscriberToSubCoordinatorResponse_Assignment) GetAssignedPartitions() []*SubscriberToSubCoordinatorResponse_AssignedPartition { +func (x *SubscriberToSubCoordinatorResponse_Assignment) GetPartitionAssignments() []*BrokerPartitionAssignment { if x != nil { - return x.AssignedPartitions + return x.PartitionAssignments } return nil } @@ -2365,7 +2302,7 @@ type PublishMessageRequest_InitMessage struct { func (x *PublishMessageRequest_InitMessage) Reset() { *x = PublishMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2378,7 +2315,7 @@ func (x *PublishMessageRequest_InitMessage) String() string { func (*PublishMessageRequest_InitMessage) ProtoMessage() {} func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2434,7 +2371,7 @@ type PublishFollowMeRequest_InitMessage struct { func (x *PublishFollowMeRequest_InitMessage) Reset() { *x = PublishFollowMeRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2447,7 +2384,7 @@ func (x *PublishFollowMeRequest_InitMessage) String() string { func (*PublishFollowMeRequest_InitMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2488,7 +2425,7 @@ type PublishFollowMeRequest_FlushMessage struct { func (x *PublishFollowMeRequest_FlushMessage) Reset() { *x = PublishFollowMeRequest_FlushMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2501,7 +2438,7 @@ func (x *PublishFollowMeRequest_FlushMessage) String() string { func (*PublishFollowMeRequest_FlushMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2533,7 +2470,7 @@ type PublishFollowMeRequest_CloseMessage struct { func (x *PublishFollowMeRequest_CloseMessage) Reset() { *x = PublishFollowMeRequest_CloseMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2546,7 +2483,7 @@ func (x *PublishFollowMeRequest_CloseMessage) String() string { func (*PublishFollowMeRequest_CloseMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2573,12 +2510,13 @@ type SubscribeMessageRequest_InitMessage struct { Topic *Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` PartitionOffset *PartitionOffset `protobuf:"bytes,5,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"` Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` + FollowerBrokers []string `protobuf:"bytes,7,rep,name=follower_brokers,json=followerBrokers,proto3" json:"follower_brokers,omitempty"` } func (x *SubscribeMessageRequest_InitMessage) Reset() { *x = SubscribeMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2591,7 +2529,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string { func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2649,6 +2587,13 @@ func (x *SubscribeMessageRequest_InitMessage) GetFilter() string { return "" } +func (x *SubscribeMessageRequest_InitMessage) GetFollowerBrokers() []string { + if x != nil { + return x.FollowerBrokers + } + return nil +} + type SubscribeMessageRequest_AckMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2660,7 +2605,7 @@ type SubscribeMessageRequest_AckMessage struct { func (x *SubscribeMessageRequest_AckMessage) Reset() { *x = SubscribeMessageRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2673,7 +2618,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string { func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2709,7 +2654,7 @@ type SubscribeMessageResponse_CtrlMessage struct { func (x *SubscribeMessageResponse_CtrlMessage) Reset() { *x = SubscribeMessageResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[46] + mi := &file_mq_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2722,7 +2667,7 @@ func (x *SubscribeMessageResponse_CtrlMessage) String() string { func (*SubscribeMessageResponse_CtrlMessage) ProtoMessage() {} func (x *SubscribeMessageResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[46] + mi := &file_mq_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2940,7 +2885,7 @@ var file_mq_proto_rawDesc = []byte{ 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, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xab, 0x03, + 0x4e, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9b, 0x02, 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, @@ -2949,24 +2894,15 @@ var file_mq_proto_rawDesc = []byte{ 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, 0x77, 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, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 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, + 0x65, 0x6e, 0x74, 0x1a, 0x8a, 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, 0x5c, 0x0a, 0x15, 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, 0x14, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 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, @@ -3036,7 +2972,7 @@ var file_mq_proto_rawDesc = []byte{ 0x65, 0x22, 0x35, 0x0a, 0x17, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0xdf, 0x03, 0x0a, 0x17, 0x53, 0x75, 0x62, + 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x8a, 0x04, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, @@ -3047,7 +2983,7 @@ var file_mq_proto_rawDesc = []byte{ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 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, 0xff, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x63, 0x6b, 0x1a, 0xaa, 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, @@ -3063,140 +2999,143 @@ var file_mq_proto_rawDesc = []byte{ 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, - 0x69, 0x6c, 0x74, 0x65, 0x72, 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, 0x95, 0x02, 0x0a, 0x18, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 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, 0x2a, 0x4c, 0x0a, 0x18, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, - 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, - 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x5f, 0x49, - 0x4e, 0x5f, 0x4d, 0x45, 0x4d, 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x41, - 0x54, 0x45, 0x53, 0x54, 0x10, 0x02, 0x32, 0xde, 0x0a, 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, 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, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, + 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, + 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, 0x95, 0x02, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 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, 0x2a, 0x4c, 0x0a, 0x18, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, + 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, + 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, + 0x02, 0x32, 0xde, 0x0a, 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, 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, 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, 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, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x52, 0x65, 0x71, 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, 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, + 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, 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, + 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, + 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, 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, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 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, 0x61, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x10, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, - 0x01, 0x12, 0x64, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, - 0x6f, 0x77, 0x4d, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, - 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x28, 0x01, 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, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x10, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x64, 0x0a, 0x0f, + 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, + 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, + 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, + 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 ( @@ -3212,56 +3151,55 @@ func file_mq_proto_rawDescGZIP() []byte { } var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 47) +var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 46) var file_mq_proto_goTypes = []interface{}{ - (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType - (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest - (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse - (*Topic)(nil), // 3: messaging_pb.Topic - (*Partition)(nil), // 4: messaging_pb.Partition - (*Offset)(nil), // 5: messaging_pb.Offset - (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset - (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats - (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats - (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest - (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse - (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest - (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse - (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest - (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse - (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest - (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse - (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest - (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse - (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment - (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest - (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse - (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest - (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse - (*DataMessage)(nil), // 24: messaging_pb.DataMessage - (*PublishMessageRequest)(nil), // 25: messaging_pb.PublishMessageRequest - (*PublishMessageResponse)(nil), // 26: messaging_pb.PublishMessageResponse - (*PublishFollowMeRequest)(nil), // 27: messaging_pb.PublishFollowMeRequest - (*PublishFollowMeResponse)(nil), // 28: messaging_pb.PublishFollowMeResponse - (*SubscribeMessageRequest)(nil), // 29: messaging_pb.SubscribeMessageRequest - (*SubscribeMessageResponse)(nil), // 30: messaging_pb.SubscribeMessageResponse - (*ClosePublishersRequest)(nil), // 31: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 32: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 33: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 34: messaging_pb.CloseSubscribersResponse - nil, // 35: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 36: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 37: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*PublishMessageRequest_InitMessage)(nil), // 41: messaging_pb.PublishMessageRequest.InitMessage - (*PublishFollowMeRequest_InitMessage)(nil), // 42: messaging_pb.PublishFollowMeRequest.InitMessage - (*PublishFollowMeRequest_FlushMessage)(nil), // 43: messaging_pb.PublishFollowMeRequest.FlushMessage - (*PublishFollowMeRequest_CloseMessage)(nil), // 44: messaging_pb.PublishFollowMeRequest.CloseMessage - (*SubscribeMessageRequest_InitMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.InitMessage - (*SubscribeMessageRequest_AckMessage)(nil), // 46: messaging_pb.SubscribeMessageRequest.AckMessage - (*SubscribeMessageResponse_CtrlMessage)(nil), // 47: messaging_pb.SubscribeMessageResponse.CtrlMessage + (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType + (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest + (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse + (*Topic)(nil), // 3: messaging_pb.Topic + (*Partition)(nil), // 4: messaging_pb.Partition + (*Offset)(nil), // 5: messaging_pb.Offset + (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset + (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats + (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats + (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest + (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse + (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest + (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse + (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest + (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse + (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest + (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse + (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest + (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse + (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment + (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest + (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse + (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest + (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse + (*DataMessage)(nil), // 24: messaging_pb.DataMessage + (*PublishMessageRequest)(nil), // 25: messaging_pb.PublishMessageRequest + (*PublishMessageResponse)(nil), // 26: messaging_pb.PublishMessageResponse + (*PublishFollowMeRequest)(nil), // 27: messaging_pb.PublishFollowMeRequest + (*PublishFollowMeResponse)(nil), // 28: messaging_pb.PublishFollowMeResponse + (*SubscribeMessageRequest)(nil), // 29: messaging_pb.SubscribeMessageRequest + (*SubscribeMessageResponse)(nil), // 30: messaging_pb.SubscribeMessageResponse + (*ClosePublishersRequest)(nil), // 31: messaging_pb.ClosePublishersRequest + (*ClosePublishersResponse)(nil), // 32: messaging_pb.ClosePublishersResponse + (*CloseSubscribersRequest)(nil), // 33: messaging_pb.CloseSubscribersRequest + (*CloseSubscribersResponse)(nil), // 34: messaging_pb.CloseSubscribersResponse + nil, // 35: messaging_pb.BrokerStats.StatsEntry + (*PublisherToPubBalancerRequest_InitMessage)(nil), // 36: messaging_pb.PublisherToPubBalancerRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 37: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage + (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + (*PublishMessageRequest_InitMessage)(nil), // 40: messaging_pb.PublishMessageRequest.InitMessage + (*PublishFollowMeRequest_InitMessage)(nil), // 41: messaging_pb.PublishFollowMeRequest.InitMessage + (*PublishFollowMeRequest_FlushMessage)(nil), // 42: messaging_pb.PublishFollowMeRequest.FlushMessage + (*PublishFollowMeRequest_CloseMessage)(nil), // 43: messaging_pb.PublishFollowMeRequest.CloseMessage + (*SubscribeMessageRequest_InitMessage)(nil), // 44: messaging_pb.SubscribeMessageRequest.InitMessage + (*SubscribeMessageRequest_AckMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.AckMessage + (*SubscribeMessageResponse_CtrlMessage)(nil), // 46: messaging_pb.SubscribeMessageResponse.CtrlMessage } var file_mq_proto_depIdxs = []int32{ 3, // 0: messaging_pb.Offset.topic:type_name -> messaging_pb.Topic @@ -3284,61 +3222,60 @@ var file_mq_proto_depIdxs = []int32{ 19, // 17: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment 37, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage 38, // 19: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - 40, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 41, // 21: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 39, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + 40, // 21: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage 24, // 22: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage - 42, // 23: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage + 41, // 23: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage 24, // 24: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage - 43, // 25: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage - 44, // 26: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage - 45, // 27: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage - 46, // 28: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage - 47, // 29: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage + 42, // 25: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage + 43, // 26: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage + 44, // 27: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage + 45, // 28: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage + 46, // 29: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage 24, // 30: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage 3, // 31: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic 3, // 32: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic 8, // 33: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats 3, // 34: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic 4, // 35: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition - 4, // 36: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition - 39, // 37: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - 3, // 38: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 39: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 40: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 41: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 42: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 6, // 43: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset - 1, // 44: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 9, // 45: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 11, // 46: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 15, // 47: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 13, // 48: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 17, // 49: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 20, // 50: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 31, // 51: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 33, // 52: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 22, // 53: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 25, // 54: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest - 29, // 55: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest - 27, // 56: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest - 2, // 57: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 10, // 58: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 12, // 59: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 16, // 60: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 14, // 61: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 18, // 62: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 21, // 63: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 32, // 64: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 34, // 65: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 23, // 66: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 26, // 67: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse - 30, // 68: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse - 28, // 69: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse - 57, // [57:70] is the sub-list for method output_type - 44, // [44:57] is the sub-list for method input_type - 44, // [44:44] is the sub-list for extension type_name - 44, // [44:44] is the sub-list for extension extendee - 0, // [0:44] is the sub-list for field type_name + 19, // 36: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 3, // 37: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 38: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 3, // 39: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 40: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 3, // 41: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 6, // 42: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset + 1, // 43: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 9, // 44: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 11, // 45: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 15, // 46: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 13, // 47: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 17, // 48: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 20, // 49: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 31, // 50: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 33, // 51: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 22, // 52: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 25, // 53: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 29, // 54: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 27, // 55: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 2, // 56: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 10, // 57: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 12, // 58: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 16, // 59: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 14, // 60: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 18, // 61: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 21, // 62: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 32, // 63: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 34, // 64: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 23, // 65: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 26, // 66: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 30, // 67: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 28, // 68: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 56, // [56:69] is the sub-list for method output_type + 43, // [43:56] is the sub-list for method input_type + 43, // [43:43] is the sub-list for extension type_name + 43, // [43:43] is the sub-list for extension extendee + 0, // [0:43] is the sub-list for field type_name } func init() { file_mq_proto_init() } @@ -3792,18 +3729,6 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[38].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[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { case 0: return &v.state @@ -3815,7 +3740,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishMessageRequest_InitMessage); i { case 0: return &v.state @@ -3827,7 +3752,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishFollowMeRequest_InitMessage); i { case 0: return &v.state @@ -3839,7 +3764,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishFollowMeRequest_FlushMessage); i { case 0: return &v.state @@ -3851,7 +3776,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishFollowMeRequest_CloseMessage); i { case 0: return &v.state @@ -3863,7 +3788,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageRequest_InitMessage); i { case 0: return &v.state @@ -3875,7 +3800,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageRequest_AckMessage); i { case 0: return &v.state @@ -3887,7 +3812,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageResponse_CtrlMessage); i { case 0: return &v.state @@ -3935,7 +3860,7 @@ func file_mq_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_mq_proto_rawDesc, NumEnums: 1, - NumMessages: 47, + NumMessages: 46, NumExtensions: 0, NumServices: 1, },