diff --git a/weed/mq/broker/broker_grpc_pub_follow.go b/weed/mq/broker/broker_grpc_pub_follow.go index f0c4a26ca..f6a06f0b8 100644 --- a/weed/mq/broker/broker_grpc_pub_follow.go +++ b/weed/mq/broker/broker_grpc_pub_follow.go @@ -2,97 +2,11 @@ package broker import ( "context" - "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" - "io" - "math/rand" - "sync" - "time" ) func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error) { glog.V(0).Infof("PublishFollowMe %v", request) - var wg sync.WaitGroup - wg.Add(1) - var ret error - go b.withBrokerClient(true, pb.ServerAddress(request.BrokerSelf), func(client mq_pb.SeaweedMessagingClient) error { - followerId := rand.Int31() - subscribeClient, err := client.FollowInMemoryMessages(context.Background(), &mq_pb.FollowInMemoryMessagesRequest{ - Message: &mq_pb.FollowInMemoryMessagesRequest_Init{ - Init: &mq_pb.FollowInMemoryMessagesRequest_InitMessage{ - ConsumerGroup: string(b.option.BrokerAddress()), - ConsumerId: fmt.Sprintf("followMe@%s-%d", b.option.BrokerAddress(), followerId), - FollowerId: followerId, - Topic: request.Topic, - PartitionOffset: &mq_pb.PartitionOffset{ - Partition: request.Partition, - StartTsNs: 0, - StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY, - }, - }, - }, - }) - - if err != nil { - glog.Errorf("FollowInMemoryMessages error: %v", err) - ret = err - return err - } - - // receive first hello message - resp, err := subscribeClient.Recv() - if err != nil { - return fmt.Errorf("FollowInMemoryMessages recv first message error: %v", err) - } - if resp == nil { - glog.V(0).Infof("doFollowInMemoryMessage recv first message nil response") - return io.ErrUnexpectedEOF - } - wg.Done() - - b.doFollowInMemoryMessage(context.Background(), followerId, subscribeClient) - - return nil - }) - wg.Wait() - return &mq_pb.PublishFollowMeResponse{}, ret -} - -func (b *MessageQueueBroker) doFollowInMemoryMessage(c context.Context, followerId int32, client mq_pb.SeaweedMessaging_FollowInMemoryMessagesClient) { - for { - resp, err := client.Recv() - if err != nil { - if err != io.EOF { - glog.V(0).Infof("doFollowInMemoryMessage error: %v", err) - } - return - } - if resp == nil { - glog.V(0).Infof("doFollowInMemoryMessage nil response") - return - } - if resp.Message != nil { - // process ctrl message or data message - switch m := resp.Message.(type) { - case *mq_pb.FollowInMemoryMessagesResponse_Data: - // process data message - print("d") - case *mq_pb.FollowInMemoryMessagesResponse_Ctrl: - // process ctrl message - if m.Ctrl.FlushedSequence > 0 { - flushTime := time.Unix(0, m.Ctrl.FlushedSequence) - glog.V(0).Infof("doFollowInMemoryMessage flushTime: %v", flushTime) - } - if m.Ctrl.FollowerChangedToId != 0 { - // follower changed - glog.V(0).Infof("doFollowInMemoryMessage follower changed from %d to %d", followerId, m.Ctrl.FollowerChangedToId) - return - } - default: - glog.V(0).Infof("doFollowInMemoryMessage unknown message type: %v", m) - } - } - } + return &mq_pb.PublishFollowMeResponse{}, nil } diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index 846342596..6033ad43c 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -8,7 +8,6 @@ import ( "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/util/log_buffer" - "sync/atomic" "time" ) @@ -129,167 +128,3 @@ func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer } return } - -func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMessagesRequest, stream mq_pb.SeaweedMessaging_FollowInMemoryMessagesServer) (err error) { - ctx := stream.Context() - clientName := req.GetInit().ConsumerId - - t := topic.FromPbTopic(req.GetInit().Topic) - partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition()) - - glog.V(0).Infof("FollowInMemoryMessages %s on %v %v connected", clientName, t, partition) - - waitIntervalCount := 0 - - 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 > 32 { - waitIntervalCount = 32 - } - time.Sleep(time.Duration(waitIntervalCount) * 137 * 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("FollowInMemoryMessages %s disconnected: %v", clientName, err) - return nil - default: - // Continue processing the request - } - } - - // set the current follower id - followerId := req.GetInit().FollowerId - atomic.StoreInt32(&localTopicPartition.FollowerId, followerId) - - glog.V(0).Infof("FollowInMemoryMessages %s connected on %v %v", clientName, t, partition) - - var counter int64 - defer func() { - glog.V(0).Infof("FollowInMemoryMessages %s on %v %v disconnected, sent %d", clientName, t, partition, counter) - }() - - // send first hello message - // to indicate the follower is connected - stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ - Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{}, - }, - }) - - var startPosition log_buffer.MessagePosition - if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil { - startPosition = getRequestPosition(req.GetInit().GetPartitionOffset()) - } - - var prevFlushTsNs int64 - - _, _, err = localTopicPartition.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, func() bool { - // wait for the log buffer to be ready - localTopicPartition.ListenersLock.Lock() - atomic.AddInt64(&localTopicPartition.ListenersWaits, 1) - localTopicPartition.ListenersCond.Wait() - atomic.AddInt64(&localTopicPartition.ListenersWaits, -1) - localTopicPartition.ListenersLock.Unlock() - - if localTopicPartition.LogBuffer.IsStopping() { - newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId) - glog.V(0).Infof("FollowInMemoryMessages1 %s on %v %v follower id changed to %d", clientName, t, partition, newFollowerId) - stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ - Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{ - FollowerChangedToId: newFollowerId, - }, - }, - }) - return false - } - - // 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 - } - - // send the last flushed sequence - flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs) - if flushTsNs != prevFlushTsNs { - prevFlushTsNs = flushTsNs - stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ - Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{ - FlushedSequence: flushTsNs, - }, - }, - }) - } - - return true - }, func(logEntry *filer_pb.LogEntry) (bool, error) { - - // check the follower id - newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId) - if newFollowerId != followerId { - glog.V(0).Infof("FollowInMemoryMessages2 %s on %v %v follower id %d changed to %d", clientName, t, partition, followerId, newFollowerId) - stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ - Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{ - FollowerChangedToId: newFollowerId, - }, - }, - }) - return true, nil - } - - // send the last flushed sequence - flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs) - if flushTsNs != prevFlushTsNs { - prevFlushTsNs = flushTsNs - stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{ - Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{ - FlushedSequence: flushTsNs, - }, - }, - }) - } - - // send the log entry - if err := stream.Send(&mq_pb.FollowInMemoryMessagesResponse{ - Message: &mq_pb.FollowInMemoryMessagesResponse_Data{ - Data: &mq_pb.DataMessage{ - Key: logEntry.Key, - Value: logEntry.Data, - TsNs: logEntry.TsNs, - }, - }}); err != nil { - glog.Errorf("Error sending setup response: %v", err) - return false, err - } - - counter++ - return false, nil - }) - - return err -} diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto index 369f82fb3..12c9ccc93 100644 --- a/weed/pb/mq.proto +++ b/weed/pb/mq.proto @@ -48,8 +48,6 @@ service SeaweedMessaging { // The lead broker asks a follower broker to follow itself rpc PublishFollowMe (PublishFollowMeRequest) returns (PublishFollowMeResponse) { } - rpc FollowInMemoryMessages (FollowInMemoryMessagesRequest) returns (stream FollowInMemoryMessagesResponse) { - } } ////////////////////////////////////////////////// @@ -246,32 +244,6 @@ message SubscribeMessageResponse { DataMessage data = 2; } } -message FollowInMemoryMessagesRequest { - message InitMessage { - string consumer_group = 1; - string consumer_id = 2; - int32 follower_id = 3; - Topic topic = 4; - PartitionOffset partition_offset = 5; - } - message AckMessage { - int64 sequence = 1; - } - oneof message { - InitMessage init = 1; - AckMessage ack = 2; - } -} -message FollowInMemoryMessagesResponse { - message CtrlMessage { - int64 flushed_sequence = 1; - int32 follower_changed_to_id = 2; - } - oneof message { - CtrlMessage ctrl = 1; - DataMessage data = 2; - } -} message ClosePublishersRequest { Topic topic = 1; int64 unix_time_ns = 2; diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq.pb.go index ed9f5c002..91e7e42b0 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq.pb.go @@ -1836,168 +1836,6 @@ func (*SubscribeMessageResponse_Ctrl) isSubscribeMessageResponse_Message() {} func (*SubscribeMessageResponse_Data) isSubscribeMessageResponse_Message() {} -type FollowInMemoryMessagesRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Message: - // - // *FollowInMemoryMessagesRequest_Init - // *FollowInMemoryMessagesRequest_Ack - Message isFollowInMemoryMessagesRequest_Message `protobuf_oneof:"message"` -} - -func (x *FollowInMemoryMessagesRequest) Reset() { - *x = FollowInMemoryMessagesRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[30] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *FollowInMemoryMessagesRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*FollowInMemoryMessagesRequest) ProtoMessage() {} - -func (x *FollowInMemoryMessagesRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[30] - 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 FollowInMemoryMessagesRequest.ProtoReflect.Descriptor instead. -func (*FollowInMemoryMessagesRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30} -} - -func (m *FollowInMemoryMessagesRequest) GetMessage() isFollowInMemoryMessagesRequest_Message { - if m != nil { - return m.Message - } - return nil -} - -func (x *FollowInMemoryMessagesRequest) GetInit() *FollowInMemoryMessagesRequest_InitMessage { - if x, ok := x.GetMessage().(*FollowInMemoryMessagesRequest_Init); ok { - return x.Init - } - return nil -} - -func (x *FollowInMemoryMessagesRequest) GetAck() *FollowInMemoryMessagesRequest_AckMessage { - if x, ok := x.GetMessage().(*FollowInMemoryMessagesRequest_Ack); ok { - return x.Ack - } - return nil -} - -type isFollowInMemoryMessagesRequest_Message interface { - isFollowInMemoryMessagesRequest_Message() -} - -type FollowInMemoryMessagesRequest_Init struct { - Init *FollowInMemoryMessagesRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` -} - -type FollowInMemoryMessagesRequest_Ack struct { - Ack *FollowInMemoryMessagesRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"` -} - -func (*FollowInMemoryMessagesRequest_Init) isFollowInMemoryMessagesRequest_Message() {} - -func (*FollowInMemoryMessagesRequest_Ack) isFollowInMemoryMessagesRequest_Message() {} - -type FollowInMemoryMessagesResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Message: - // - // *FollowInMemoryMessagesResponse_Ctrl - // *FollowInMemoryMessagesResponse_Data - Message isFollowInMemoryMessagesResponse_Message `protobuf_oneof:"message"` -} - -func (x *FollowInMemoryMessagesResponse) Reset() { - *x = FollowInMemoryMessagesResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[31] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *FollowInMemoryMessagesResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*FollowInMemoryMessagesResponse) ProtoMessage() {} - -func (x *FollowInMemoryMessagesResponse) 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 FollowInMemoryMessagesResponse.ProtoReflect.Descriptor instead. -func (*FollowInMemoryMessagesResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31} -} - -func (m *FollowInMemoryMessagesResponse) GetMessage() isFollowInMemoryMessagesResponse_Message { - if m != nil { - return m.Message - } - return nil -} - -func (x *FollowInMemoryMessagesResponse) GetCtrl() *FollowInMemoryMessagesResponse_CtrlMessage { - if x, ok := x.GetMessage().(*FollowInMemoryMessagesResponse_Ctrl); ok { - return x.Ctrl - } - return nil -} - -func (x *FollowInMemoryMessagesResponse) GetData() *DataMessage { - if x, ok := x.GetMessage().(*FollowInMemoryMessagesResponse_Data); ok { - return x.Data - } - return nil -} - -type isFollowInMemoryMessagesResponse_Message interface { - isFollowInMemoryMessagesResponse_Message() -} - -type FollowInMemoryMessagesResponse_Ctrl struct { - Ctrl *FollowInMemoryMessagesResponse_CtrlMessage `protobuf:"bytes,1,opt,name=ctrl,proto3,oneof"` -} - -type FollowInMemoryMessagesResponse_Data struct { - Data *DataMessage `protobuf:"bytes,2,opt,name=data,proto3,oneof"` -} - -func (*FollowInMemoryMessagesResponse_Ctrl) isFollowInMemoryMessagesResponse_Message() {} - -func (*FollowInMemoryMessagesResponse_Data) isFollowInMemoryMessagesResponse_Message() {} - type ClosePublishersRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2010,7 +1848,7 @@ type ClosePublishersRequest struct { func (x *ClosePublishersRequest) Reset() { *x = ClosePublishersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2023,7 +1861,7 @@ func (x *ClosePublishersRequest) String() string { func (*ClosePublishersRequest) ProtoMessage() {} func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2036,7 +1874,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{32} + return file_mq_proto_rawDescGZIP(), []int{30} } func (x *ClosePublishersRequest) GetTopic() *Topic { @@ -2062,7 +1900,7 @@ type ClosePublishersResponse struct { func (x *ClosePublishersResponse) Reset() { *x = ClosePublishersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2075,7 +1913,7 @@ func (x *ClosePublishersResponse) String() string { func (*ClosePublishersResponse) ProtoMessage() {} func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2088,7 +1926,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{33} + return file_mq_proto_rawDescGZIP(), []int{31} } type CloseSubscribersRequest struct { @@ -2103,7 +1941,7 @@ type CloseSubscribersRequest struct { func (x *CloseSubscribersRequest) Reset() { *x = CloseSubscribersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2116,7 +1954,7 @@ func (x *CloseSubscribersRequest) String() string { func (*CloseSubscribersRequest) ProtoMessage() {} func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2129,7 +1967,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{34} + return file_mq_proto_rawDescGZIP(), []int{32} } func (x *CloseSubscribersRequest) GetTopic() *Topic { @@ -2155,7 +1993,7 @@ type CloseSubscribersResponse struct { func (x *CloseSubscribersResponse) Reset() { *x = CloseSubscribersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2168,7 +2006,7 @@ func (x *CloseSubscribersResponse) String() string { func (*CloseSubscribersResponse) ProtoMessage() {} func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2181,7 +2019,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{35} + return file_mq_proto_rawDescGZIP(), []int{33} } type PublisherToPubBalancerRequest_InitMessage struct { @@ -2195,7 +2033,7 @@ type PublisherToPubBalancerRequest_InitMessage struct { func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { *x = PublisherToPubBalancerRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2208,7 +2046,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string { func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2244,7 +2082,7 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_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) } @@ -2257,7 +2095,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_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 { @@ -2306,7 +2144,7 @@ type SubscriberToSubCoordinatorRequest_AckMessage struct { func (x *SubscriberToSubCoordinatorRequest_AckMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckMessage{} 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) } @@ -2319,7 +2157,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) String() string { func (*SubscriberToSubCoordinatorRequest_AckMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckMessage) 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 { @@ -2362,7 +2200,7 @@ type SubscriberToSubCoordinatorResponse_AssignedPartition struct { func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) Reset() { *x = SubscriberToSubCoordinatorResponse_AssignedPartition{} 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) } @@ -2375,7 +2213,7 @@ func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) String() string { func (*SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) 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 { @@ -2424,7 +2262,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct { func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} 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) } @@ -2437,7 +2275,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) 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 { @@ -2481,7 +2319,7 @@ type PublishMessageRequest_InitMessage struct { func (x *PublishMessageRequest_InitMessage) Reset() { *x = PublishMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2494,7 +2332,7 @@ func (x *PublishMessageRequest_InitMessage) String() string { func (*PublishMessageRequest_InitMessage) ProtoMessage() {} func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2554,7 +2392,7 @@ type SubscribeMessageRequest_InitMessage struct { func (x *SubscribeMessageRequest_InitMessage) Reset() { *x = SubscribeMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2567,7 +2405,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string { func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2636,7 +2474,7 @@ type SubscribeMessageRequest_AckMessage struct { func (x *SubscribeMessageRequest_AckMessage) Reset() { *x = SubscribeMessageRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2649,7 +2487,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string { func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2685,7 +2523,7 @@ type SubscribeMessageResponse_CtrlMessage struct { func (x *SubscribeMessageResponse_CtrlMessage) Reset() { *x = SubscribeMessageResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2698,7 +2536,7 @@ func (x *SubscribeMessageResponse_CtrlMessage) String() string { func (*SubscribeMessageResponse_CtrlMessage) ProtoMessage() {} func (x *SubscribeMessageResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2735,187 +2573,6 @@ func (x *SubscribeMessageResponse_CtrlMessage) GetIsEndOfTopic() bool { return false } -type FollowInMemoryMessagesRequest_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"` - ConsumerId string `protobuf:"bytes,2,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` - FollowerId int32 `protobuf:"varint,3,opt,name=follower_id,json=followerId,proto3" json:"follower_id,omitempty"` - 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"` -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) Reset() { - *x = FollowInMemoryMessagesRequest_InitMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[46] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*FollowInMemoryMessagesRequest_InitMessage) ProtoMessage() {} - -func (x *FollowInMemoryMessagesRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[46] - 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 FollowInMemoryMessagesRequest_InitMessage.ProtoReflect.Descriptor instead. -func (*FollowInMemoryMessagesRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30, 0} -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) GetConsumerGroup() string { - if x != nil { - return x.ConsumerGroup - } - return "" -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) GetConsumerId() string { - if x != nil { - return x.ConsumerId - } - return "" -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) GetFollowerId() int32 { - if x != nil { - return x.FollowerId - } - return 0 -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) GetTopic() *Topic { - if x != nil { - return x.Topic - } - return nil -} - -func (x *FollowInMemoryMessagesRequest_InitMessage) GetPartitionOffset() *PartitionOffset { - if x != nil { - return x.PartitionOffset - } - return nil -} - -type FollowInMemoryMessagesRequest_AckMessage struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"` -} - -func (x *FollowInMemoryMessagesRequest_AckMessage) Reset() { - *x = FollowInMemoryMessagesRequest_AckMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[47] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *FollowInMemoryMessagesRequest_AckMessage) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*FollowInMemoryMessagesRequest_AckMessage) ProtoMessage() {} - -func (x *FollowInMemoryMessagesRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[47] - 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 FollowInMemoryMessagesRequest_AckMessage.ProtoReflect.Descriptor instead. -func (*FollowInMemoryMessagesRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30, 1} -} - -func (x *FollowInMemoryMessagesRequest_AckMessage) GetSequence() int64 { - if x != nil { - return x.Sequence - } - return 0 -} - -type FollowInMemoryMessagesResponse_CtrlMessage struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - FlushedSequence int64 `protobuf:"varint,1,opt,name=flushed_sequence,json=flushedSequence,proto3" json:"flushed_sequence,omitempty"` - FollowerChangedToId int32 `protobuf:"varint,2,opt,name=follower_changed_to_id,json=followerChangedToId,proto3" json:"follower_changed_to_id,omitempty"` -} - -func (x *FollowInMemoryMessagesResponse_CtrlMessage) Reset() { - *x = FollowInMemoryMessagesResponse_CtrlMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[48] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *FollowInMemoryMessagesResponse_CtrlMessage) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*FollowInMemoryMessagesResponse_CtrlMessage) ProtoMessage() {} - -func (x *FollowInMemoryMessagesResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[48] - 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 FollowInMemoryMessagesResponse_CtrlMessage.ProtoReflect.Descriptor instead. -func (*FollowInMemoryMessagesResponse_CtrlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31, 0} -} - -func (x *FollowInMemoryMessagesResponse_CtrlMessage) GetFlushedSequence() int64 { - if x != nil { - return x.FlushedSequence - } - return 0 -} - -func (x *FollowInMemoryMessagesResponse_CtrlMessage) GetFollowerChangedToId() int32 { - if x != nil { - return x.FollowerChangedToId - } - return 0 -} - var File_mq_proto protoreflect.FileDescriptor var file_mq_proto_rawDesc = []byte{ @@ -3221,174 +2878,119 @@ var file_mq_proto_rawDesc = []byte{ 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, - 0xdd, 0x03, 0x0a, 0x1d, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, - 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 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, 0x46, - 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 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, 0x4a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, - 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x73, 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, 0xeb, 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, 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, 0x1f, 0x0a, 0x0b, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, - 0x77, 0x65, 0x72, 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, 0x48, 0x0a, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 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, 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, - 0x9b, 0x02, 0x0a, 0x1e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, - 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x38, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 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, 0x6d, 0x0a, 0x0b, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x73, 0x65, - 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x66, 0x6c, - 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x33, 0x0a, - 0x16, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, - 0x64, 0x5f, 0x74, 0x6f, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x66, - 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x54, 0x6f, - 0x49, 0x64, 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, 0xd3, 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, 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, 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, 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, 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, 0x60, 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, 0x12, 0x77, - 0x0a, 0x16, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, - 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, - 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 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, + 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, 0xda, 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, 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, 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, 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, 0x60, + 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, + 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 ( @@ -3404,7 +3006,7 @@ func file_mq_proto_rawDescGZIP() []byte { } var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 49) +var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 44) var file_mq_proto_goTypes = []interface{}{ (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest @@ -3437,35 +3039,30 @@ var file_mq_proto_goTypes = []interface{}{ (*PublishFollowMeResponse)(nil), // 28: messaging_pb.PublishFollowMeResponse (*SubscribeMessageRequest)(nil), // 29: messaging_pb.SubscribeMessageRequest (*SubscribeMessageResponse)(nil), // 30: messaging_pb.SubscribeMessageResponse - (*FollowInMemoryMessagesRequest)(nil), // 31: messaging_pb.FollowInMemoryMessagesRequest - (*FollowInMemoryMessagesResponse)(nil), // 32: messaging_pb.FollowInMemoryMessagesResponse - (*ClosePublishersRequest)(nil), // 33: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 34: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 35: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 36: messaging_pb.CloseSubscribersResponse - nil, // 37: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 38: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 41: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 42: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*PublishMessageRequest_InitMessage)(nil), // 43: messaging_pb.PublishMessageRequest.InitMessage - (*SubscribeMessageRequest_InitMessage)(nil), // 44: messaging_pb.SubscribeMessageRequest.InitMessage - (*SubscribeMessageRequest_AckMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.AckMessage - (*SubscribeMessageResponse_CtrlMessage)(nil), // 46: messaging_pb.SubscribeMessageResponse.CtrlMessage - (*FollowInMemoryMessagesRequest_InitMessage)(nil), // 47: messaging_pb.FollowInMemoryMessagesRequest.InitMessage - (*FollowInMemoryMessagesRequest_AckMessage)(nil), // 48: messaging_pb.FollowInMemoryMessagesRequest.AckMessage - (*FollowInMemoryMessagesResponse_CtrlMessage)(nil), // 49: messaging_pb.FollowInMemoryMessagesResponse.CtrlMessage + (*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 + (*SubscribeMessageRequest_InitMessage)(nil), // 42: messaging_pb.SubscribeMessageRequest.InitMessage + (*SubscribeMessageRequest_AckMessage)(nil), // 43: messaging_pb.SubscribeMessageRequest.AckMessage + (*SubscribeMessageResponse_CtrlMessage)(nil), // 44: messaging_pb.SubscribeMessageResponse.CtrlMessage } var file_mq_proto_depIdxs = []int32{ 3, // 0: messaging_pb.Offset.topic:type_name -> messaging_pb.Topic 6, // 1: messaging_pb.Offset.partition_offsets:type_name -> messaging_pb.PartitionOffset 4, // 2: messaging_pb.PartitionOffset.partition:type_name -> messaging_pb.Partition 0, // 3: messaging_pb.PartitionOffset.start_type:type_name -> messaging_pb.PartitionOffsetStartType - 37, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 35, // 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 - 38, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 36, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage 7, // 8: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats 3, // 9: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic 19, // 10: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment @@ -3476,67 +3073,59 @@ var file_mq_proto_depIdxs = []int32{ 4, // 15: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition 3, // 16: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic 19, // 17: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 39, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - 40, // 19: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - 42, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 43, // 21: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 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 24, // 22: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage 3, // 23: messaging_pb.PublishFollowMeRequest.topic:type_name -> messaging_pb.Topic 4, // 24: messaging_pb.PublishFollowMeRequest.partition:type_name -> messaging_pb.Partition - 44, // 25: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage - 45, // 26: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage - 46, // 27: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage + 42, // 25: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage + 43, // 26: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage + 44, // 27: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage 24, // 28: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage - 47, // 29: messaging_pb.FollowInMemoryMessagesRequest.init:type_name -> messaging_pb.FollowInMemoryMessagesRequest.InitMessage - 48, // 30: messaging_pb.FollowInMemoryMessagesRequest.ack:type_name -> messaging_pb.FollowInMemoryMessagesRequest.AckMessage - 49, // 31: messaging_pb.FollowInMemoryMessagesResponse.ctrl:type_name -> messaging_pb.FollowInMemoryMessagesResponse.CtrlMessage - 24, // 32: messaging_pb.FollowInMemoryMessagesResponse.data:type_name -> messaging_pb.DataMessage - 3, // 33: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic - 3, // 34: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic - 8, // 35: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 3, // 36: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 37: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition - 4, // 38: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition - 41, // 39: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - 3, // 40: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 41: messaging_pb.PublishMessageRequest.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 - 3, // 44: messaging_pb.FollowInMemoryMessagesRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 6, // 45: messaging_pb.FollowInMemoryMessagesRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset - 1, // 46: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 9, // 47: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 11, // 48: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 15, // 49: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 13, // 50: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 17, // 51: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 20, // 52: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 33, // 53: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 35, // 54: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 22, // 55: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 25, // 56: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest - 29, // 57: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest - 27, // 58: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest - 31, // 59: messaging_pb.SeaweedMessaging.FollowInMemoryMessages:input_type -> messaging_pb.FollowInMemoryMessagesRequest - 2, // 60: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 10, // 61: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 12, // 62: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 16, // 63: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 14, // 64: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 18, // 65: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 21, // 66: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 34, // 67: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 36, // 68: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 23, // 69: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 26, // 70: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse - 30, // 71: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse - 28, // 72: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse - 32, // 73: messaging_pb.SeaweedMessaging.FollowInMemoryMessages:output_type -> messaging_pb.FollowInMemoryMessagesResponse - 60, // [60:74] is the sub-list for method output_type - 46, // [46:60] is the sub-list for method input_type - 46, // [46:46] is the sub-list for extension type_name - 46, // [46:46] is the sub-list for extension extendee - 0, // [0:46] is the sub-list for field type_name + 3, // 29: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic + 3, // 30: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic + 8, // 31: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats + 3, // 32: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 33: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition + 4, // 34: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition + 39, // 35: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition + 3, // 36: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 37: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 3, // 38: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 6, // 39: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset + 1, // 40: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 9, // 41: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 11, // 42: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 15, // 43: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 13, // 44: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 17, // 45: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 20, // 46: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 31, // 47: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 33, // 48: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 22, // 49: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 25, // 50: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 29, // 51: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 27, // 52: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 2, // 53: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 10, // 54: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 12, // 55: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 16, // 56: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 14, // 57: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 18, // 58: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 21, // 59: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 32, // 60: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 34, // 61: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 23, // 62: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 26, // 63: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 30, // 64: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 28, // 65: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 53, // [53:66] is the sub-list for method output_type + 40, // [40:53] is the sub-list for method input_type + 40, // [40:40] is the sub-list for extension type_name + 40, // [40:40] is the sub-list for extension extendee + 0, // [0:40] is the sub-list for field type_name } func init() { file_mq_proto_init() } @@ -3906,30 +3495,6 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FollowInMemoryMessagesRequest); 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.(*FollowInMemoryMessagesResponse); 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.(*ClosePublishersRequest); i { case 0: return &v.state @@ -3941,7 +3506,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClosePublishersResponse); i { case 0: return &v.state @@ -3953,7 +3518,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseSubscribersRequest); i { case 0: return &v.state @@ -3965,7 +3530,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseSubscribersResponse); i { case 0: return &v.state @@ -3977,7 +3542,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { case 0: return &v.state @@ -3989,7 +3554,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { case 0: return &v.state @@ -4001,7 +3566,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.(*SubscriberToSubCoordinatorRequest_AckMessage); i { case 0: return &v.state @@ -4013,7 +3578,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.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { case 0: return &v.state @@ -4025,7 +3590,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.(*SubscriberToSubCoordinatorResponse_Assignment); i { case 0: return &v.state @@ -4037,7 +3602,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PublishMessageRequest_InitMessage); i { case 0: return &v.state @@ -4049,7 +3614,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageRequest_InitMessage); i { case 0: return &v.state @@ -4061,7 +3626,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageRequest_AckMessage); i { case 0: return &v.state @@ -4073,7 +3638,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageResponse_CtrlMessage); i { case 0: return &v.state @@ -4085,42 +3650,6 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FollowInMemoryMessagesRequest_InitMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FollowInMemoryMessagesRequest_AckMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FollowInMemoryMessagesResponse_CtrlMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } } file_mq_proto_msgTypes[8].OneofWrappers = []interface{}{ (*PublisherToPubBalancerRequest_Init)(nil), @@ -4145,21 +3674,13 @@ func file_mq_proto_init() { (*SubscribeMessageResponse_Ctrl)(nil), (*SubscribeMessageResponse_Data)(nil), } - file_mq_proto_msgTypes[30].OneofWrappers = []interface{}{ - (*FollowInMemoryMessagesRequest_Init)(nil), - (*FollowInMemoryMessagesRequest_Ack)(nil), - } - file_mq_proto_msgTypes[31].OneofWrappers = []interface{}{ - (*FollowInMemoryMessagesResponse_Ctrl)(nil), - (*FollowInMemoryMessagesResponse_Data)(nil), - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_mq_proto_rawDesc, NumEnums: 1, - NumMessages: 49, + NumMessages: 44, 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 64518d605..356c5c3d4 100644 --- a/weed/pb/mq_pb/mq_grpc.pb.go +++ b/weed/pb/mq_pb/mq_grpc.pb.go @@ -32,7 +32,6 @@ const ( SeaweedMessaging_PublishMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishMessage" SeaweedMessaging_SubscribeMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeMessage" SeaweedMessaging_PublishFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishFollowMe" - SeaweedMessaging_FollowInMemoryMessages_FullMethodName = "/messaging_pb.SeaweedMessaging/FollowInMemoryMessages" ) // SeaweedMessagingClient is the client API for SeaweedMessaging service. @@ -59,7 +58,6 @@ type SeaweedMessagingClient interface { SubscribeMessage(ctx context.Context, in *SubscribeMessageRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error) // The lead broker asks a follower broker to follow itself PublishFollowMe(ctx context.Context, in *PublishFollowMeRequest, opts ...grpc.CallOption) (*PublishFollowMeResponse, error) - FollowInMemoryMessages(ctx context.Context, in *FollowInMemoryMessagesRequest, opts ...grpc.CallOption) (SeaweedMessaging_FollowInMemoryMessagesClient, error) } type seaweedMessagingClient struct { @@ -276,38 +274,6 @@ func (c *seaweedMessagingClient) PublishFollowMe(ctx context.Context, in *Publis return out, nil } -func (c *seaweedMessagingClient) FollowInMemoryMessages(ctx context.Context, in *FollowInMemoryMessagesRequest, opts ...grpc.CallOption) (SeaweedMessaging_FollowInMemoryMessagesClient, error) { - stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[4], SeaweedMessaging_FollowInMemoryMessages_FullMethodName, opts...) - if err != nil { - return nil, err - } - x := &seaweedMessagingFollowInMemoryMessagesClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - return x, nil -} - -type SeaweedMessaging_FollowInMemoryMessagesClient interface { - Recv() (*FollowInMemoryMessagesResponse, error) - grpc.ClientStream -} - -type seaweedMessagingFollowInMemoryMessagesClient struct { - grpc.ClientStream -} - -func (x *seaweedMessagingFollowInMemoryMessagesClient) Recv() (*FollowInMemoryMessagesResponse, error) { - m := new(FollowInMemoryMessagesResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - // SeaweedMessagingServer is the server API for SeaweedMessaging service. // All implementations must embed UnimplementedSeaweedMessagingServer // for forward compatibility @@ -332,7 +298,6 @@ type SeaweedMessagingServer interface { SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error // The lead broker asks a follower broker to follow itself PublishFollowMe(context.Context, *PublishFollowMeRequest) (*PublishFollowMeResponse, error) - FollowInMemoryMessages(*FollowInMemoryMessagesRequest, SeaweedMessaging_FollowInMemoryMessagesServer) error mustEmbedUnimplementedSeaweedMessagingServer() } @@ -379,9 +344,6 @@ func (UnimplementedSeaweedMessagingServer) SubscribeMessage(*SubscribeMessageReq func (UnimplementedSeaweedMessagingServer) PublishFollowMe(context.Context, *PublishFollowMeRequest) (*PublishFollowMeResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method PublishFollowMe not implemented") } -func (UnimplementedSeaweedMessagingServer) FollowInMemoryMessages(*FollowInMemoryMessagesRequest, SeaweedMessaging_FollowInMemoryMessagesServer) error { - return status.Errorf(codes.Unimplemented, "method FollowInMemoryMessages not implemented") -} func (UnimplementedSeaweedMessagingServer) mustEmbedUnimplementedSeaweedMessagingServer() {} // UnsafeSeaweedMessagingServer may be embedded to opt out of forward compatibility for this service. @@ -656,27 +618,6 @@ func _SeaweedMessaging_PublishFollowMe_Handler(srv interface{}, ctx context.Cont return interceptor(ctx, in, info, handler) } -func _SeaweedMessaging_FollowInMemoryMessages_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(FollowInMemoryMessagesRequest) - if err := stream.RecvMsg(m); err != nil { - return err - } - return srv.(SeaweedMessagingServer).FollowInMemoryMessages(m, &seaweedMessagingFollowInMemoryMessagesServer{stream}) -} - -type SeaweedMessaging_FollowInMemoryMessagesServer interface { - Send(*FollowInMemoryMessagesResponse) error - grpc.ServerStream -} - -type seaweedMessagingFollowInMemoryMessagesServer struct { - grpc.ServerStream -} - -func (x *seaweedMessagingFollowInMemoryMessagesServer) Send(m *FollowInMemoryMessagesResponse) error { - return x.ServerStream.SendMsg(m) -} - // SeaweedMessaging_ServiceDesc is the grpc.ServiceDesc for SeaweedMessaging service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -745,11 +686,6 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ Handler: _SeaweedMessaging_SubscribeMessage_Handler, ServerStreams: true, }, - { - StreamName: "FollowInMemoryMessages", - Handler: _SeaweedMessaging_FollowInMemoryMessages_Handler, - ServerStreams: true, - }, }, Metadata: "mq.proto", }