From 6f75a0af55811fcfd584f4bf136cdc42db9cff5a Mon Sep 17 00:00:00 2001 From: chrislu Date: Sun, 10 Mar 2024 14:34:28 -0700 Subject: [PATCH] ask follower to follow --- weed/mq/broker/broker_grpc_pub.go | 34 +- weed/mq/broker/broker_grpc_pub_follow.go | 78 ++ weed/mq/broker/broker_grpc_sub.go | 194 ++- .../broker_topic_partition_read_write.go | 3 + weed/mq/topic/local_partition.go | 18 +- weed/mq/topic/local_topic.go | 1 + weed/pb/mq.proto | 39 + weed/pb/mq_pb/mq.pb.go | 1200 +++++++++++++---- weed/pb/mq_pb/mq_grpc.pb.go | 103 ++ weed/util/log_buffer/log_buffer.go | 1 + weed/util/log_buffer/log_read.go | 7 +- 11 files changed, 1362 insertions(+), 316 deletions(-) create mode 100644 weed/mq/broker/broker_grpc_pub_follow.go diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go index 17d01f620..8c46ea99d 100644 --- a/weed/mq/broker/broker_grpc_pub.go +++ b/weed/mq/broker/broker_grpc_pub.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "google.golang.org/grpc/peer" "io" @@ -59,6 +60,21 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis return stream.Send(response) } ackInterval = int(initMessage.AckInterval) + for _, follower := range initMessage.FollowerBrokers { + followErr := b.withBrokerClient(false, pb.ServerAddress(follower), func(client mq_pb.SeaweedMessagingClient) error { + _, err := client.PublishFollowMe(context.Background(), &mq_pb.PublishFollowMeRequest{ + Topic: initMessage.Topic, + Partition: initMessage.Partition, + BrokerSelf: string(b.option.BrokerAddress()), + }) + return err + }) + if followErr != nil { + response.Error = fmt.Sprintf("follower %v failed: %v", follower, followErr) + glog.Errorf("follower %v failed: %v", follower, followErr) + return stream.Send(response) + } + } stream.Send(response) } else { response.Error = fmt.Sprintf("missing init message") @@ -86,21 +102,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis glog.V(0).Infof("topic %v partition %v published %d messges.", initMessage.Topic, initMessage.Partition, ackSequence) }() go func() { - for { - select { - case resp := <-respChan: - if resp != nil { - if err := stream.Send(resp); err != nil { - glog.Errorf("Error sending response %v: %v", resp, err) - } - } else { - return - } - case <-localTopicPartition.StopPublishersCh: - respChan <- &mq_pb.PublishMessageResponse{ - AckSequence: ackSequence, - ShouldClose: true, - } + for resp := range respChan { + if err := stream.Send(resp); err != nil { + glog.Errorf("Error sending response %v: %v", resp, err) } } }() diff --git a/weed/mq/broker/broker_grpc_pub_follow.go b/weed/mq/broker/broker_grpc_pub_follow.go new file mode 100644 index 000000000..e74d7025f --- /dev/null +++ b/weed/mq/broker/broker_grpc_pub_follow.go @@ -0,0 +1,78 @@ +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" + "time" +) + +func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error){ + glog.V(0).Infof("PublishFollowMe %v", request) + 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-%d", 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) + return err + } + + b.doFollowInMemoryMessage(context.Background(), subscribeClient) + + return nil + }) + return &mq_pb.PublishFollowMeResponse{}, nil +} + +func (b *MessageQueueBroker) doFollowInMemoryMessage(c context.Context, 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 to %d", m.Ctrl.FollowerChangedToId) + return + } + } + } + } +} diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index c6dde6f4e..3280be2c0 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -8,6 +8,7 @@ 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" ) @@ -69,15 +70,7 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest var startPosition log_buffer.MessagePosition if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil { - offset := req.GetInit().GetPartitionOffset() - if offset.StartTsNs != 0 { - startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2) - } - if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST { - startPosition = log_buffer.NewMessagePosition(1, -3) - } else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST { - startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4) - } + startPosition = getRequestPosition(req.GetInit().GetPartitionOffset()) } return localTopicPartition.Subscribe(clientName, startPosition, func() bool { @@ -85,10 +78,10 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest return false } sleepIntervalCount++ - if sleepIntervalCount > 10 { - sleepIntervalCount = 10 + if sleepIntervalCount > 32 { + sleepIntervalCount = 32 } - time.Sleep(time.Duration(sleepIntervalCount) * 337 * time.Millisecond) + time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond) // Check if the client has disconnected by monitoring the context select { @@ -116,7 +109,7 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest TsNs: logEntry.TsNs, }, }}); err != nil { - glog.Errorf("Error sending setup response: %v", err) + glog.Errorf("Error sending data: %v", err) return false, err } @@ -124,3 +117,178 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest return false, nil }) } + +func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer.MessagePosition) { + if offset.StartTsNs != 0 { + startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2) + } + if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST { + startPosition = log_buffer.NewMessagePosition(1, -3) + } else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST { + startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4) + } + 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) + isConnected := true + sleepIntervalCount := 0 + + var counter int64 + defer func() { + isConnected = false + glog.V(0).Infof("FollowInMemoryMessages %s on %v %v disconnected, sent %d", clientName, t, partition, counter) + }() + + 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 { + if !isConnected { + return false + } + sleepIntervalCount++ + if sleepIntervalCount > 32 { + sleepIntervalCount = 32 + } + time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond) + + 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) { + // reset the sleep interval count + sleepIntervalCount = 0 + + // 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/mq/broker/broker_topic_partition_read_write.go b/weed/mq/broker/broker_topic_partition_read_write.go index 4ebb62000..a058d8da5 100644 --- a/weed/mq/broker/broker_topic_partition_read_write.go +++ b/weed/mq/broker/broker_topic_partition_read_write.go @@ -11,6 +11,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/util/log_buffer" "google.golang.org/protobuf/proto" "math" + "sync/atomic" "time" ) @@ -38,6 +39,8 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par break } } + + atomic.StoreInt64(&logBuffer.LastFlushTsNs, stopTime.UnixNano()) } } diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go index 0947d259b..062f3f4bd 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" + "sync/atomic" "time" ) @@ -15,10 +16,9 @@ type LocalPartition struct { FollowerBrokers []pb.ServerAddress LogBuffer *log_buffer.LogBuffer ConsumerCount int32 - StopPublishersCh chan struct{} Publishers *LocalPartitionPublishers - StopSubscribersCh chan struct{} Subscribers *LocalPartitionSubscribers + FollowerId int32 } var TIME_FORMAT = "2006-01-02-15-04-05" @@ -58,6 +58,9 @@ func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.M startPosition = processedPosition processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn) + if isDone { + return nil + } startPosition = processedPosition if readInMemoryLogErr == log_buffer.ResumeFromDiskError { @@ -67,9 +70,6 @@ func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.M glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr) return readInMemoryLogErr } - if isDone { - return nil - } } } @@ -96,7 +96,6 @@ func FromPbBrokerPartitionAssignment(self pb.ServerAddress, partition Partition, func (p *LocalPartition) closePublishers() { p.Publishers.SignalShutdown() - close(p.StopPublishersCh) } func (p *LocalPartition) closeSubscribers() { p.Subscribers.SignalShutdown() @@ -118,3 +117,10 @@ func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) { } return } + +func (p *LocalPartition) Shutdown() { + p.closePublishers() + p.closeSubscribers() + p.LogBuffer.ShutdownLogBuffer() + atomic.StoreInt32(&p.FollowerId, 0) +} diff --git a/weed/mq/topic/local_topic.go b/weed/mq/topic/local_topic.go index 7825d2168..8ab2a0db5 100644 --- a/weed/mq/topic/local_topic.go +++ b/weed/mq/topic/local_topic.go @@ -27,6 +27,7 @@ func (localTopic *LocalTopic) removePartition(partition Partition) bool { for i, localPartition := range localTopic.Partitions { if localPartition.Partition.Equals(partition) { foundPartitionIndex = i + localPartition.Shutdown() break } } diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto index 4e9690537..369f82fb3 100644 --- a/weed/pb/mq.proto +++ b/weed/pb/mq.proto @@ -45,6 +45,11 @@ service SeaweedMessaging { } rpc SubscribeMessage (SubscribeMessageRequest) returns (stream SubscribeMessageResponse) { } + // The lead broker asks a follower broker to follow itself + rpc PublishFollowMe (PublishFollowMeRequest) returns (PublishFollowMeResponse) { + } + rpc FollowInMemoryMessages (FollowInMemoryMessagesRequest) returns (stream FollowInMemoryMessagesResponse) { + } } ////////////////////////////////////////////////// @@ -205,6 +210,14 @@ message PublishMessageResponse { string error = 2; bool should_close = 3; } +message PublishFollowMeRequest { + Topic topic = 1; + Partition partition = 2; + string broker_self = 3; +} +message PublishFollowMeResponse { + string error = 1; +} message SubscribeMessageRequest { message InitMessage { string consumer_group = 1; @@ -233,6 +246,32 @@ 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 cf9056c0e..ed9f5c002 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq.pb.go @@ -1564,6 +1564,116 @@ func (x *PublishMessageResponse) GetShouldClose() bool { return false } +type PublishFollowMeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + BrokerSelf string `protobuf:"bytes,3,opt,name=broker_self,json=brokerSelf,proto3" json:"broker_self,omitempty"` +} + +func (x *PublishFollowMeRequest) Reset() { + *x = PublishFollowMeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishFollowMeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishFollowMeRequest) ProtoMessage() {} + +func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublishFollowMeRequest.ProtoReflect.Descriptor instead. +func (*PublishFollowMeRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{26} +} + +func (x *PublishFollowMeRequest) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *PublishFollowMeRequest) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *PublishFollowMeRequest) GetBrokerSelf() string { + if x != nil { + return x.BrokerSelf + } + return "" +} + +type PublishFollowMeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *PublishFollowMeResponse) Reset() { + *x = PublishFollowMeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishFollowMeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishFollowMeResponse) ProtoMessage() {} + +func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublishFollowMeResponse.ProtoReflect.Descriptor instead. +func (*PublishFollowMeResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{27} +} + +func (x *PublishFollowMeResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + type SubscribeMessageRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1579,7 +1689,7 @@ type SubscribeMessageRequest struct { func (x *SubscribeMessageRequest) Reset() { *x = SubscribeMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1592,7 +1702,7 @@ func (x *SubscribeMessageRequest) String() string { func (*SubscribeMessageRequest) ProtoMessage() {} func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1605,7 +1715,7 @@ func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMessageRequest.ProtoReflect.Descriptor instead. func (*SubscribeMessageRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26} + return file_mq_proto_rawDescGZIP(), []int{28} } func (m *SubscribeMessageRequest) GetMessage() isSubscribeMessageRequest_Message { @@ -1660,7 +1770,7 @@ type SubscribeMessageResponse struct { func (x *SubscribeMessageResponse) Reset() { *x = SubscribeMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1673,7 +1783,7 @@ func (x *SubscribeMessageResponse) String() string { func (*SubscribeMessageResponse) ProtoMessage() {} func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1686,7 +1796,7 @@ func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMessageResponse.ProtoReflect.Descriptor instead. func (*SubscribeMessageResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27} + return file_mq_proto_rawDescGZIP(), []int{29} } func (m *SubscribeMessageResponse) GetMessage() isSubscribeMessageResponse_Message { @@ -1726,6 +1836,168 @@ 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 @@ -1738,7 +2010,7 @@ type ClosePublishersRequest struct { func (x *ClosePublishersRequest) Reset() { *x = ClosePublishersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1751,7 +2023,7 @@ func (x *ClosePublishersRequest) String() string { func (*ClosePublishersRequest) ProtoMessage() {} func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1764,7 +2036,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{28} + return file_mq_proto_rawDescGZIP(), []int{32} } func (x *ClosePublishersRequest) GetTopic() *Topic { @@ -1790,7 +2062,7 @@ type ClosePublishersResponse struct { func (x *ClosePublishersResponse) Reset() { *x = ClosePublishersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1803,7 +2075,7 @@ func (x *ClosePublishersResponse) String() string { func (*ClosePublishersResponse) ProtoMessage() {} func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1816,7 +2088,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{29} + return file_mq_proto_rawDescGZIP(), []int{33} } type CloseSubscribersRequest struct { @@ -1831,7 +2103,7 @@ type CloseSubscribersRequest struct { func (x *CloseSubscribersRequest) Reset() { *x = CloseSubscribersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[30] + mi := &file_mq_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1844,7 +2116,7 @@ func (x *CloseSubscribersRequest) String() string { func (*CloseSubscribersRequest) ProtoMessage() {} func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[30] + mi := &file_mq_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1857,7 +2129,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30} + return file_mq_proto_rawDescGZIP(), []int{34} } func (x *CloseSubscribersRequest) GetTopic() *Topic { @@ -1883,7 +2155,7 @@ type CloseSubscribersResponse struct { func (x *CloseSubscribersResponse) Reset() { *x = CloseSubscribersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1896,7 +2168,7 @@ func (x *CloseSubscribersResponse) String() string { func (*CloseSubscribersResponse) ProtoMessage() {} func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1909,7 +2181,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31} + return file_mq_proto_rawDescGZIP(), []int{35} } type PublisherToPubBalancerRequest_InitMessage struct { @@ -1923,7 +2195,7 @@ type PublisherToPubBalancerRequest_InitMessage struct { func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { *x = PublisherToPubBalancerRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1936,7 +2208,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string { func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1972,7 +2244,7 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1985,7 +2257,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2034,7 +2306,7 @@ type SubscriberToSubCoordinatorRequest_AckMessage struct { func (x *SubscriberToSubCoordinatorRequest_AckMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2047,7 +2319,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) String() string { func (*SubscriberToSubCoordinatorRequest_AckMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2090,7 +2362,7 @@ type SubscriberToSubCoordinatorResponse_AssignedPartition struct { func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) Reset() { *x = SubscriberToSubCoordinatorResponse_AssignedPartition{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[36] + mi := &file_mq_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2103,7 +2375,7 @@ func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) String() string { func (*SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[36] + mi := &file_mq_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2152,7 +2424,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct { func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2165,7 +2437,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2209,7 +2481,7 @@ type PublishMessageRequest_InitMessage struct { func (x *PublishMessageRequest_InitMessage) Reset() { *x = PublishMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2222,7 +2494,7 @@ func (x *PublishMessageRequest_InitMessage) String() string { func (*PublishMessageRequest_InitMessage) ProtoMessage() {} func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2282,7 +2554,7 @@ type SubscribeMessageRequest_InitMessage struct { func (x *SubscribeMessageRequest_InitMessage) Reset() { *x = SubscribeMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2295,7 +2567,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string { func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2308,7 +2580,7 @@ func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Messag // Deprecated: Use SubscribeMessageRequest_InitMessage.ProtoReflect.Descriptor instead. func (*SubscribeMessageRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26, 0} + return file_mq_proto_rawDescGZIP(), []int{28, 0} } func (x *SubscribeMessageRequest_InitMessage) GetConsumerGroup() string { @@ -2364,7 +2636,7 @@ type SubscribeMessageRequest_AckMessage struct { func (x *SubscribeMessageRequest_AckMessage) Reset() { *x = SubscribeMessageRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2377,7 +2649,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string { func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2390,7 +2662,7 @@ func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message // Deprecated: Use SubscribeMessageRequest_AckMessage.ProtoReflect.Descriptor instead. func (*SubscribeMessageRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26, 1} + return file_mq_proto_rawDescGZIP(), []int{28, 1} } func (x *SubscribeMessageRequest_AckMessage) GetSequence() int64 { @@ -2413,7 +2685,7 @@ type SubscribeMessageResponse_CtrlMessage struct { func (x *SubscribeMessageResponse_CtrlMessage) Reset() { *x = SubscribeMessageResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2426,7 +2698,7 @@ func (x *SubscribeMessageResponse_CtrlMessage) String() string { func (*SubscribeMessageResponse_CtrlMessage) ProtoMessage() {} func (x *SubscribeMessageResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2439,7 +2711,7 @@ func (x *SubscribeMessageResponse_CtrlMessage) ProtoReflect() protoreflect.Messa // Deprecated: Use SubscribeMessageResponse_CtrlMessage.ProtoReflect.Descriptor instead. func (*SubscribeMessageResponse_CtrlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27, 0} + return file_mq_proto_rawDescGZIP(), []int{29, 0} } func (x *SubscribeMessageResponse_CtrlMessage) GetError() string { @@ -2463,6 +2735,187 @@ 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{ @@ -2707,161 +3160,235 @@ var file_mq_proto_rawDesc = []byte{ 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, 0x75, 0x6c, - 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0xdf, 0x03, 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, 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, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x44, 0x0a, 0x03, 0x61, - 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 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, 0x67, - 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, - 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x12, 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, 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, + 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x73, 0x65, + 0x6c, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x53, 0x65, 0x6c, 0x66, 0x22, 0x2f, 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, 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, 0x72, 0x6f, 0x72, 0x22, 0xdf, 0x03, 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, 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, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x44, 0x0a, 0x03, 0x61, 0x63, + 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 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, 0x67, 0x65, + 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 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, 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, + 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, 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, 0xf8, 0x09, 0x0a, 0x10, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, - 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, - 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 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, + 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, 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, + 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, 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, 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, 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, } var ( @@ -2877,7 +3404,7 @@ func file_mq_proto_rawDescGZIP() []byte { } var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 42) +var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 49) var file_mq_proto_goTypes = []interface{}{ (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest @@ -2906,32 +3433,39 @@ var file_mq_proto_goTypes = []interface{}{ (*DataMessage)(nil), // 24: messaging_pb.DataMessage (*PublishMessageRequest)(nil), // 25: messaging_pb.PublishMessageRequest (*PublishMessageResponse)(nil), // 26: messaging_pb.PublishMessageResponse - (*SubscribeMessageRequest)(nil), // 27: messaging_pb.SubscribeMessageRequest - (*SubscribeMessageResponse)(nil), // 28: messaging_pb.SubscribeMessageResponse - (*ClosePublishersRequest)(nil), // 29: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 30: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 31: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 32: messaging_pb.CloseSubscribersResponse - nil, // 33: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 34: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 35: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 36: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 37: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*PublishMessageRequest_InitMessage)(nil), // 39: messaging_pb.PublishMessageRequest.InitMessage - (*SubscribeMessageRequest_InitMessage)(nil), // 40: messaging_pb.SubscribeMessageRequest.InitMessage - (*SubscribeMessageRequest_AckMessage)(nil), // 41: messaging_pb.SubscribeMessageRequest.AckMessage - (*SubscribeMessageResponse_CtrlMessage)(nil), // 42: messaging_pb.SubscribeMessageResponse.CtrlMessage + (*PublishFollowMeRequest)(nil), // 27: messaging_pb.PublishFollowMeRequest + (*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 } 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 - 33, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 37, // 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 - 34, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 38, // 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 @@ -2942,55 +3476,67 @@ 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 - 35, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - 36, // 19: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - 38, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 39, // 21: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 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 24, // 22: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage - 40, // 23: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage - 41, // 24: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage - 42, // 25: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage - 24, // 26: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage - 3, // 27: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic - 3, // 28: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic - 8, // 29: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 3, // 30: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 31: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition - 4, // 32: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition - 37, // 33: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - 3, // 34: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 35: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 36: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 6, // 37: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset - 1, // 38: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 9, // 39: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 11, // 40: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 15, // 41: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 13, // 42: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 17, // 43: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 20, // 44: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 29, // 45: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 31, // 46: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 22, // 47: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 25, // 48: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest - 27, // 49: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest - 2, // 50: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 10, // 51: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 12, // 52: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 16, // 53: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 14, // 54: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 18, // 55: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 21, // 56: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 30, // 57: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 32, // 58: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 23, // 59: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 26, // 60: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse - 28, // 61: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse - 50, // [50:62] is the sub-list for method output_type - 38, // [38:50] is the sub-list for method input_type - 38, // [38:38] is the sub-list for extension type_name - 38, // [38:38] is the sub-list for extension extendee - 0, // [0:38] is the sub-list for field type_name + 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 + 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 } func init() { file_mq_proto_init() } @@ -3312,7 +3858,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMessageRequest); i { + switch v := v.(*PublishFollowMeRequest); i { case 0: return &v.state case 1: @@ -3324,7 +3870,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMessageResponse); i { + switch v := v.(*PublishFollowMeResponse); i { case 0: return &v.state case 1: @@ -3336,7 +3882,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClosePublishersRequest); i { + switch v := v.(*SubscribeMessageRequest); i { case 0: return &v.state case 1: @@ -3348,7 +3894,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClosePublishersResponse); i { + switch v := v.(*SubscribeMessageResponse); i { case 0: return &v.state case 1: @@ -3360,7 +3906,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseSubscribersRequest); i { + switch v := v.(*FollowInMemoryMessagesRequest); i { case 0: return &v.state case 1: @@ -3372,7 +3918,19 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseSubscribersResponse); i { + 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 case 1: @@ -3384,7 +3942,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { + switch v := v.(*ClosePublishersResponse); i { case 0: return &v.state case 1: @@ -3396,7 +3954,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { + switch v := v.(*CloseSubscribersRequest); i { case 0: return &v.state case 1: @@ -3408,19 +3966,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { + switch v := v.(*CloseSubscribersResponse); i { case 0: return &v.state case 1: @@ -3432,7 +3978,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { + switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { case 0: return &v.state case 1: @@ -3444,7 +3990,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublishMessageRequest_InitMessage); i { + switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { case 0: return &v.state case 1: @@ -3456,7 +4002,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMessageRequest_InitMessage); i { + switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i { case 0: return &v.state case 1: @@ -3468,7 +4014,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeMessageRequest_AckMessage); i { + switch v := v.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { case 0: return &v.state case 1: @@ -3480,6 +4026,54 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublishMessageRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscribeMessageRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscribeMessageRequest_AckMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscribeMessageResponse_CtrlMessage); i { case 0: return &v.state @@ -3491,6 +4085,42 @@ 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), @@ -3507,21 +4137,29 @@ func file_mq_proto_init() { (*PublishMessageRequest_Init)(nil), (*PublishMessageRequest_Data)(nil), } - file_mq_proto_msgTypes[26].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[28].OneofWrappers = []interface{}{ (*SubscribeMessageRequest_Init)(nil), (*SubscribeMessageRequest_Ack)(nil), } - file_mq_proto_msgTypes[27].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[29].OneofWrappers = []interface{}{ (*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: 42, + NumMessages: 49, 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 c2ddcdc62..64518d605 100644 --- a/weed/pb/mq_pb/mq_grpc.pb.go +++ b/weed/pb/mq_pb/mq_grpc.pb.go @@ -31,6 +31,8 @@ const ( SeaweedMessaging_SubscriberToSubCoordinator_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscriberToSubCoordinator" 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. @@ -55,6 +57,9 @@ type SeaweedMessagingClient interface { // data plane for each topic partition PublishMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishMessageClient, error) 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 { @@ -262,6 +267,47 @@ func (x *seaweedMessagingSubscribeMessageClient) Recv() (*SubscribeMessageRespon return m, nil } +func (c *seaweedMessagingClient) PublishFollowMe(ctx context.Context, in *PublishFollowMeRequest, opts ...grpc.CallOption) (*PublishFollowMeResponse, error) { + out := new(PublishFollowMeResponse) + err := c.cc.Invoke(ctx, SeaweedMessaging_PublishFollowMe_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + 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 @@ -284,6 +330,9 @@ type SeaweedMessagingServer interface { // data plane for each topic partition PublishMessage(SeaweedMessaging_PublishMessageServer) error 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() } @@ -327,6 +376,12 @@ func (UnimplementedSeaweedMessagingServer) PublishMessage(SeaweedMessaging_Publi func (UnimplementedSeaweedMessagingServer) SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error { return status.Errorf(codes.Unimplemented, "method SubscribeMessage not implemented") } +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. @@ -583,6 +638,45 @@ func (x *seaweedMessagingSubscribeMessageServer) Send(m *SubscribeMessageRespons return x.ServerStream.SendMsg(m) } +func _SeaweedMessaging_PublishFollowMe_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PublishFollowMeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingServer).PublishFollowMe(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessaging_PublishFollowMe_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingServer).PublishFollowMe(ctx, req.(*PublishFollowMeRequest)) + } + 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) @@ -622,6 +716,10 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ MethodName: "CloseSubscribers", Handler: _SeaweedMessaging_CloseSubscribers_Handler, }, + { + MethodName: "PublishFollowMe", + Handler: _SeaweedMessaging_PublishFollowMe_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -647,6 +745,11 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ Handler: _SeaweedMessaging_SubscribeMessage_Handler, ServerStreams: true, }, + { + StreamName: "FollowInMemoryMessages", + Handler: _SeaweedMessaging_FollowInMemoryMessages_Handler, + ServerStreams: true, + }, }, Metadata: "mq.proto", } diff --git a/weed/util/log_buffer/log_buffer.go b/weed/util/log_buffer/log_buffer.go index bf787eb79..7709a43e9 100644 --- a/weed/util/log_buffer/log_buffer.go +++ b/weed/util/log_buffer/log_buffer.go @@ -27,6 +27,7 @@ type LogFlushFuncType func(logBuffer *LogBuffer, startTime, stopTime time.Time, type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error) type LogBuffer struct { + LastFlushTsNs int64 name string prevBuffers *SealedBuffers buf []byte diff --git a/weed/util/log_buffer/log_read.go b/weed/util/log_buffer/log_read.go index 5529a6691..d7ca47155 100644 --- a/weed/util/log_buffer/log_read.go +++ b/weed/util/log_buffer/log_read.go @@ -57,7 +57,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition if bytesBuf != nil { readSize = bytesBuf.Len() } - glog.V(0).Infof("%s ReadFromBuffer at %v batch:%d, read bytes:%v batch:%d", readerName, lastReadPosition, lastReadPosition.BatchIndex, readSize, batchIndex) + glog.V(0).Infof("%s ReadFromBuffer at %v batch %d. Read bytes %v batch %d", readerName, lastReadPosition, lastReadPosition.BatchIndex, readSize, batchIndex) if bytesBuf == nil { if batchIndex >= 0 { lastReadPosition = NewMessagePosition(lastReadPosition.UnixNano(), batchIndex) @@ -72,6 +72,10 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition isDone = true return } + if logBuffer.IsStopping() { + isDone = true + return + } } buf := bytesBuf.Bytes() @@ -107,6 +111,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition return } if isDone { + glog.V(0).Infof("LoopProcessLogData2: %s process log entry %d", readerName, batchSize+1) return }