diff options
Diffstat (limited to 'weed')
57 files changed, 3814 insertions, 1574 deletions
diff --git a/weed/command/mq_agent.go b/weed/command/mq_agent.go new file mode 100644 index 000000000..2884c6531 --- /dev/null +++ b/weed/command/mq_agent.go @@ -0,0 +1,74 @@ +package command + +import ( + "github.com/seaweedfs/seaweedfs/weed/mq/agent" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "google.golang.org/grpc/reflection" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/security" + "github.com/seaweedfs/seaweedfs/weed/util" +) + +var ( + mqAgentOptions MessageQueueAgentOptions +) + +type MessageQueueAgentOptions struct { + brokers []pb.ServerAddress + brokersString *string + filerGroup *string + ip *string + port *int +} + +func init() { + cmdMqAgent.Run = runMqAgent // break init cycle + mqAgentOptions.brokersString = cmdMqAgent.Flag.String("broker", "localhost:17777", "comma-separated message queue brokers") + mqAgentOptions.ip = cmdMqAgent.Flag.String("ip", "localhost", "message queue agent host address") + mqAgentOptions.port = cmdMqAgent.Flag.Int("port", 16777, "message queue agent gRPC server port") +} + +var cmdMqAgent = &Command{ + UsageLine: "mq.agent [-port=6377] [-master=<ip:port>]", + Short: "<WIP> start a message queue agent", + Long: `start a message queue agent + + The agent runs on local server to accept gRPC calls to write or read messages. + The messages are sent to message queue brokers. + +`, +} + +func runMqAgent(cmd *Command, args []string) bool { + + util.LoadSecurityConfiguration() + + mqAgentOptions.brokers = pb.ServerAddresses(*mqAgentOptions.brokersString).ToAddresses() + + return mqAgentOptions.startQueueAgent() + +} + +func (mqAgentOpt *MessageQueueAgentOptions) startQueueAgent() bool { + + grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.msg_agent") + + agentServer := agent.NewMessageQueueAgent(&agent.MessageQueueAgentOptions{ + SeedBrokers: mqAgentOpt.brokers, + }, grpcDialOption) + + // start grpc listener + grpcL, _, err := util.NewIpAndLocalListeners(*mqAgentOpt.ip, *mqAgentOpt.port, 0) + if err != nil { + glog.Fatalf("failed to listen on grpc port %d: %v", *mqAgentOpt.port, err) + } + grpcS := pb.NewGrpcServer() + mq_agent_pb.RegisterSeaweedMessagingAgentServer(grpcS, agentServer) + reflection.Register(grpcS) + grpcS.Serve(grpcL) + + return true + +} diff --git a/weed/command/scaffold/security.toml b/weed/command/scaffold/security.toml index 333eb80e1..2efcac354 100644 --- a/weed/command/scaffold/security.toml +++ b/weed/command/scaffold/security.toml @@ -89,6 +89,11 @@ cert = "" key = "" allowed_commonNames = "" # comma-separated SSL certificate common names +[grpc.msg_agent] +cert = "" +key = "" +allowed_commonNames = "" # comma-separated SSL certificate common names + # use this for any place needs a grpc client # i.e., "weed backup|benchmark|filer.copy|filer.replicate|mount|s3|upload" [grpc.client] diff --git a/weed/mq/agent/agent_grpc_pub_session.go b/weed/mq/agent/agent_grpc_pub_session.go new file mode 100644 index 000000000..d5c6d0813 --- /dev/null +++ b/weed/mq/agent/agent_grpc_pub_session.go @@ -0,0 +1,61 @@ +package agent + +import ( + "context" + "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" + "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "log/slog" + "math/rand/v2" + "time" +) + +func (a *MessageQueueAgent) StartPublishSession(ctx context.Context, req *mq_agent_pb.StartPublishSessionRequest) (*mq_agent_pb.StartPublishSessionResponse, error) { + sessionId := rand.Int64() + + topicPublisher := pub_client.NewTopicPublisher( + &pub_client.PublisherConfiguration{ + Topic: topic.NewTopic(req.Topic.Namespace, req.Topic.Name), + PartitionCount: req.PartitionCount, + Brokers: a.brokersList(), + PublisherName: req.PublisherName, + RecordType: req.RecordType, + }) + + a.publishersLock.Lock() + // remove inactive publishers to avoid memory leak + for k, entry := range a.publishers { + if entry.lastActiveTsNs == 0 { + // this is an active session + continue + } + if time.Unix(0, entry.lastActiveTsNs).Add(10 * time.Hour).Before(time.Now()) { + delete(a.publishers, k) + } + } + a.publishers[SessionId(sessionId)] = &SessionEntry[*pub_client.TopicPublisher]{ + entry: topicPublisher, + } + a.publishersLock.Unlock() + + return &mq_agent_pb.StartPublishSessionResponse{ + SessionId: sessionId, + }, nil +} + +func (a *MessageQueueAgent) ClosePublishSession(ctx context.Context, req *mq_agent_pb.ClosePublishSessionRequest) (*mq_agent_pb.ClosePublishSessionResponse, error) { + var finishErr string + a.publishersLock.Lock() + publisherEntry, found := a.publishers[SessionId(req.SessionId)] + if found { + if err := publisherEntry.entry.FinishPublish(); err != nil { + finishErr = err.Error() + slog.Warn("failed to finish publish", "error", err) + } + delete(a.publishers, SessionId(req.SessionId)) + } + a.publishersLock.Unlock() + return &mq_agent_pb.ClosePublishSessionResponse{ + Error: finishErr, + }, nil +} diff --git a/weed/mq/agent/agent_grpc_publish.go b/weed/mq/agent/agent_grpc_publish.go new file mode 100644 index 000000000..485e1d24e --- /dev/null +++ b/weed/mq/agent/agent_grpc_publish.go @@ -0,0 +1,43 @@ +package agent + +import ( + "fmt" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "time" +) + +func (a *MessageQueueAgent) PublishRecordRequest(stream mq_agent_pb.SeaweedMessagingAgent_PublishRecordServer) error { + m, err := stream.Recv() + if err != nil { + return err + } + a.publishersLock.RLock() + publisherEntry, found := a.publishers[SessionId(m.SessionId)] + a.publishersLock.RUnlock() + if !found { + return fmt.Errorf("publish session id %d not found", m.SessionId) + } + defer func() { + publisherEntry.lastActiveTsNs = time.Now().UnixNano() + }() + publisherEntry.lastActiveTsNs = 0 + + if m.Value != nil { + if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil { + return err + } + } + + for { + m, err := stream.Recv() + if err != nil { + return err + } + if m.Value == nil { + continue + } + if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil { + return err + } + } +} diff --git a/weed/mq/agent/agent_grpc_sub_session.go b/weed/mq/agent/agent_grpc_sub_session.go new file mode 100644 index 000000000..17def9ed5 --- /dev/null +++ b/weed/mq/agent/agent_grpc_sub_session.go @@ -0,0 +1,57 @@ +package agent + +import ( + "context" + "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" + "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "math/rand/v2" + "time" +) + +func (a *MessageQueueAgent) StartSubscribeSession(ctx context.Context, req *mq_agent_pb.StartSubscribeSessionRequest) (*mq_agent_pb.StartSubscribeSessionResponse, error) { + sessionId := rand.Int64() + + subscriberConfig := &sub_client.SubscriberConfiguration{ + ConsumerGroup: req.ConsumerGroup, + ConsumerGroupInstanceId: req.ConsumerGroupInstanceId, + GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), + MaxPartitionCount: req.MaxSubscribedPartitions, + SlidingWindowSize: req.SlidingWindowSize, + } + + contentConfig := &sub_client.ContentConfiguration{ + Topic: topic.FromPbTopic(req.Topic), + Filter: req.Filter, + PartitionOffsets: req.PartitionOffsets, + } + + topicSubscriber := sub_client.NewTopicSubscriber( + a.brokersList(), + subscriberConfig, + contentConfig, + make(chan sub_client.KeyedOffset, 1024), + ) + + a.subscribersLock.Lock() + // remove inactive publishers to avoid memory leak + for k, entry := range a.subscribers { + if entry.lastActiveTsNs == 0 { + // this is an active session + continue + } + if time.Unix(0, entry.lastActiveTsNs).Add(10 * time.Hour).Before(time.Now()) { + delete(a.subscribers, k) + } + } + a.subscribers[SessionId(sessionId)] = &SessionEntry[*sub_client.TopicSubscriber]{ + entry: topicSubscriber, + } + a.subscribersLock.Unlock() + + return &mq_agent_pb.StartSubscribeSessionResponse{ + SessionId: sessionId, + }, nil +} diff --git a/weed/mq/agent/agent_grpc_subscribe.go b/weed/mq/agent/agent_grpc_subscribe.go new file mode 100644 index 000000000..feb5bd47c --- /dev/null +++ b/weed/mq/agent/agent_grpc_subscribe.go @@ -0,0 +1,75 @@ +package agent + +import ( + "fmt" + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "google.golang.org/protobuf/proto" + "time" +) + +func (a *MessageQueueAgent) SubscribeRecordRequest(stream mq_agent_pb.SeaweedMessagingAgent_SubscribeRecordServer) error { + // the first message is the subscribe request + // it should only contain the session id + m, err := stream.Recv() + if err != nil { + return err + } + a.subscribersLock.RLock() + subscriberEntry, found := a.subscribers[SessionId(m.SessionId)] + a.subscribersLock.RUnlock() + if !found { + return fmt.Errorf("subscribe session id %d not found", m.SessionId) + } + defer func() { + subscriberEntry.lastActiveTsNs = time.Now().UnixNano() + }() + subscriberEntry.lastActiveTsNs = 0 + + var lastErr error + subscriberEntry.entry.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) { + record := &schema_pb.RecordValue{} + err := proto.Unmarshal(m.Data.Value, record) + if err != nil { + if lastErr == nil { + lastErr = err + } + return + } + if sendErr := stream.Send(&mq_agent_pb.SubscribeRecordResponse{ + Key: m.Data.Key, + Value: record, + TsNs: m.Data.TsNs, + }); sendErr != nil { + if lastErr == nil { + lastErr = sendErr + } + } + }) + + go func() { + subErr := subscriberEntry.entry.Subscribe() + if subErr != nil { + glog.V(0).Infof("subscriber %d subscribe: %v", m.SessionId, subErr) + if lastErr == nil { + lastErr = subErr + } + } + }() + + for { + m, err := stream.Recv() + if err != nil { + return err + } + if m != nil { + subscriberEntry.entry.PartitionOffsetChan <- sub_client.KeyedOffset{ + Key: m.AckKey, + Offset: m.AckSequence, + } + } + } +} diff --git a/weed/mq/agent/agent_server.go b/weed/mq/agent/agent_server.go new file mode 100644 index 000000000..6fc61bbdb --- /dev/null +++ b/weed/mq/agent/agent_server.go @@ -0,0 +1,52 @@ +package agent + +import ( + "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" + "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "google.golang.org/grpc" + "sync" +) + +type SessionId int64 +type SessionEntry[T any] struct { + entry T + lastActiveTsNs int64 +} + +type MessageQueueAgentOptions struct { + SeedBrokers []pb.ServerAddress +} + +type MessageQueueAgent struct { + mq_agent_pb.UnimplementedSeaweedMessagingAgentServer + option *MessageQueueAgentOptions + brokers []pb.ServerAddress + grpcDialOption grpc.DialOption + publishers map[SessionId]*SessionEntry[*pub_client.TopicPublisher] + publishersLock sync.RWMutex + subscribers map[SessionId]*SessionEntry[*sub_client.TopicSubscriber] + subscribersLock sync.RWMutex +} + +func NewMessageQueueAgent(option *MessageQueueAgentOptions, grpcDialOption grpc.DialOption) *MessageQueueAgent { + + // check masters to list all brokers + + return &MessageQueueAgent{ + option: option, + brokers: []pb.ServerAddress{}, + grpcDialOption: grpcDialOption, + publishers: make(map[SessionId]*SessionEntry[*pub_client.TopicPublisher]), + subscribers: make(map[SessionId]*SessionEntry[*sub_client.TopicSubscriber]), + } +} + +func (a *MessageQueueAgent) brokersList() []string { + var brokers []string + for _, broker := range a.brokers { + brokers = append(brokers, broker.String()) + } + return brokers +} diff --git a/weed/mq/broker/broker_grpc_assign.go b/weed/mq/broker/broker_grpc_assign.go index 9a9b34c0b..991208a72 100644 --- a/weed/mq/broker/broker_grpc_assign.go +++ b/weed/mq/broker/broker_grpc_assign.go @@ -9,6 +9,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "sync" ) @@ -55,7 +56,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m // called by broker leader to drain existing partitions. // new/updated partitions will be detected by broker from the filer -func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error { +func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error { // notify the brokers to create the topic partitions in parallel var wg sync.WaitGroup for _, bpa := range assignments { diff --git a/weed/mq/broker/broker_grpc_configure.go b/weed/mq/broker/broker_grpc_configure.go index 361af5c43..6130a8598 100644 --- a/weed/mq/broker/broker_grpc_configure.go +++ b/weed/mq/broker/broker_grpc_configure.go @@ -5,7 +5,6 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer" - "github.com/seaweedfs/seaweedfs/weed/mq/schema" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" @@ -30,9 +29,6 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb. // validate the schema if request.RecordType != nil { - if _, err = schema.NewSchema(request.RecordType); err != nil { - return nil, status.Errorf(codes.InvalidArgument, "invalid record type %+v: %v", request.RecordType, err) - } } t := topic.FromPbTopic(request.Topic) diff --git a/weed/mq/broker/broker_grpc_lookup.go b/weed/mq/broker/broker_grpc_lookup.go index db62fd88a..65a1ffda8 100644 --- a/weed/mq/broker/broker_grpc_lookup.go +++ b/weed/mq/broker/broker_grpc_lookup.go @@ -7,6 +7,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) // LookupTopicBrokers returns the brokers that are serving the topic @@ -54,7 +55,7 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List _, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() { topicPartitionStat := topicPartitionStatsItem.Val - topic := &mq_pb.Topic{ + topic := &schema_pb.Topic{ Namespace: topicPartitionStat.TopicPartition.Namespace, Name: topicPartitionStat.TopicPartition.Name, } diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index 286812a9b..fb8878b2d 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -9,6 +9,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "github.com/seaweedfs/seaweedfs/weed/util/log_buffer" "io" "time" @@ -54,7 +55,7 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs }() startPosition := b.getRequestPosition(req.GetInit()) - imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().Concurrency)) + imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().SlidingWindowSize)) // connect to the follower var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient @@ -212,9 +213,9 @@ func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMess return } - if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST { + if offset.StartType == schema_pb.PartitionOffsetStartType_EARLIEST { startPosition = log_buffer.NewMessagePosition(1, -3) - } else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST { + } else if offset.StartType == schema_pb.PartitionOffsetStartType_LATEST { startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4) } return diff --git a/weed/mq/client/agent_client/SubscribeSession.go b/weed/mq/client/agent_client/SubscribeSession.go new file mode 100644 index 000000000..fc87e17ca --- /dev/null +++ b/weed/mq/client/agent_client/SubscribeSession.go @@ -0,0 +1,63 @@ +package agent_client + +import ( + "context" + "fmt" + "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "google.golang.org/grpc" +) + +type SubscribeOption struct { + ConsumerGroup string + ConsumerGroupInstanceId string + Topic topic.Topic + Filter string + MaxSubscribedPartitions int32 + PerPartitionConcurrency int32 +} + +type SubscribeSession struct { + Option *SubscribeOption + stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse] + sessionId int64 +} + +func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*SubscribeSession, error) { + // call local agent grpc server to create a new session + clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure()) + if err != nil { + return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err) + } + agentClient := mq_agent_pb.NewSeaweedMessagingAgentClient(clientConn) + + resp, err := agentClient.StartSubscribeSession(context.Background(), &mq_agent_pb.StartSubscribeSessionRequest{ + ConsumerGroup: option.ConsumerGroup, + ConsumerGroupInstanceId: option.ConsumerGroupInstanceId, + Topic: &schema_pb.Topic{ + Namespace: option.Topic.Namespace, + Name: option.Topic.Name, + }, + MaxSubscribedPartitions: option.MaxSubscribedPartitions, + Filter: option.Filter, + }) + if err != nil { + return nil, err + } + if resp.Error != "" { + return nil, fmt.Errorf("start subscribe session: %v", resp.Error) + } + + stream, err := agentClient.SubscribeRecord(context.Background()) + if err != nil { + return nil, fmt.Errorf("subscribe record: %v", err) + } + + return &SubscribeSession{ + Option: option, + stream: stream, + sessionId: resp.SessionId, + }, nil +} diff --git a/weed/mq/client/agent_client/agent_publish.go b/weed/mq/client/agent_client/agent_publish.go new file mode 100644 index 000000000..3e16f3b99 --- /dev/null +++ b/weed/mq/client/agent_client/agent_publish.go @@ -0,0 +1,14 @@ +package agent_client + +import ( + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" +) + +func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error { + return a.stream.Send(&mq_agent_pb.PublishRecordRequest{ + SessionId: a.sessionId, + Key: key, + Value: record, + }) +} diff --git a/weed/mq/client/agent_client/agent_subscribe.go b/weed/mq/client/agent_client/agent_subscribe.go new file mode 100644 index 000000000..626a3a123 --- /dev/null +++ b/weed/mq/client/agent_client/agent_subscribe.go @@ -0,0 +1,17 @@ +package agent_client + +import ( + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" +) + +func (a *SubscribeSession) SubscribeMessageRecord( + onEachMessageFn func(key []byte, record *schema_pb.RecordValue), + onCompletionFn func()) error { + for { + resp, err := a.stream.Recv() + if err != nil { + return err + } + onEachMessageFn(resp.Key, resp.Value) + } +} diff --git a/weed/mq/client/agent_client/publish_session.go b/weed/mq/client/agent_client/publish_session.go new file mode 100644 index 000000000..45d46f553 --- /dev/null +++ b/weed/mq/client/agent_client/publish_session.go @@ -0,0 +1,70 @@ +package agent_client + +import ( + "context" + "fmt" + "github.com/seaweedfs/seaweedfs/weed/mq/schema" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "google.golang.org/grpc" +) + +type PublishSession struct { + schema *schema.Schema + partitionCount int + publisherName string + stream grpc.BidiStreamingClient[mq_agent_pb.PublishRecordRequest, mq_agent_pb.PublishRecordResponse] + sessionId int64 +} + +func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitionCount int, publisherName string) (*PublishSession, error) { + + // call local agent grpc server to create a new session + clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure()) + if err != nil { + return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err) + } + agentClient := mq_agent_pb.NewSeaweedMessagingAgentClient(clientConn) + + resp, err := agentClient.StartPublishSession(context.Background(), &mq_agent_pb.StartPublishSessionRequest{ + Topic: &schema_pb.Topic{ + Namespace: topicSchema.Namespace, + Name: topicSchema.Name, + }, + PartitionCount: int32(partitionCount), + RecordType: topicSchema.RecordType, + PublisherName: publisherName, + }) + if err != nil { + return nil, err + } + if resp.Error != "" { + return nil, fmt.Errorf("start publish session: %v", resp.Error) + } + + stream, err := agentClient.PublishRecord(context.Background()) + if err != nil { + return nil, fmt.Errorf("publish record: %v", err) + } + + return &PublishSession{ + schema: topicSchema, + partitionCount: partitionCount, + publisherName: publisherName, + stream: stream, + sessionId: resp.SessionId, + }, nil +} + +func (a *PublishSession) CloseSession() error { + if a.schema == nil { + return nil + } + err := a.stream.CloseSend() + if err != nil { + return fmt.Errorf("close send: %v", err) + } + a.schema = nil + return err +} diff --git a/weed/mq/client/cmd/agent_pub_record/publisher_record.go b/weed/mq/client/cmd/agent_pub_record/publisher_record.go new file mode 100644 index 000000000..eda200e86 --- /dev/null +++ b/weed/mq/client/cmd/agent_pub_record/publisher_record.go @@ -0,0 +1,128 @@ +package main + +import ( + "flag" + "fmt" + "github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client" + "github.com/seaweedfs/seaweedfs/weed/mq/schema" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "log" + "sync" + "sync/atomic" + "time" +) + +var ( + messageCount = flag.Int("n", 1000, "message count") + messageDelay = flag.Duration("d", time.Second, "delay between messages") + concurrency = flag.Int("c", 4, "concurrent publishers") + partitionCount = flag.Int("p", 6, "partition count") + + clientName = flag.String("client", "c1", "client name") + + namespace = flag.String("ns", "test", "namespace") + t = flag.String("t", "test", "t") + seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") + + counter int32 +) + +func doPublish(publisher *agent_client.PublishSession, id int) { + startTime := time.Now() + for { + i := atomic.AddInt32(&counter, 1) + if i > int32(*messageCount) { + break + } + // Simulate publishing a message + myRecord := genMyRecord(int32(i)) + if err := publisher.PublishMessageRecord(myRecord.Key, myRecord.ToRecordValue()); err != nil { + fmt.Println(err) + break + } + if *messageDelay > 0 { + time.Sleep(*messageDelay) + fmt.Printf("sent %+v\n", string(myRecord.Key)) + } + } + elapsed := time.Since(startTime) + log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed) +} + +type MyRecord struct { + Key []byte + Field1 []byte + Field2 string + Field3 int32 + Field4 int64 + Field5 float32 + Field6 float64 + Field7 bool +} + +func genMyRecord(id int32) *MyRecord { + return &MyRecord{ + Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)), + Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)), + Field2: fmt.Sprintf("field2-%s-%d", *clientName, id), + Field3: id, + Field4: int64(id), + Field5: float32(id), + Field6: float64(id), + Field7: id%2 == 0, + } +} + +func (r *MyRecord) ToRecordValue() *schema_pb.RecordValue { + return schema.RecordBegin(). + SetBytes("key", r.Key). + SetBytes("field1", r.Field1). + SetString("field2", r.Field2). + SetInt32("field3", r.Field3). + SetInt64("field4", r.Field4). + SetFloat("field5", r.Field5). + SetDouble("field6", r.Field6). + SetBool("field7", r.Field7). + RecordEnd() +} + +func main() { + flag.Parse() + + recordType := schema.RecordTypeBegin(). + WithField("key", schema.TypeBytes). + WithField("field1", schema.TypeBytes). + WithField("field2", schema.TypeString). + WithField("field3", schema.TypeInt32). + WithField("field4", schema.TypeInt64). + WithField("field5", schema.TypeFloat). + WithField("field6", schema.TypeDouble). + WithField("field7", schema.TypeBoolean). + RecordTypeEnd() + + session, err := agent_client.NewPublishSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName) + if err != nil { + log.Printf("failed to create session: %v", err) + return + } + defer session.CloseSession() + + startTime := time.Now() + + var wg sync.WaitGroup + // Start multiple publishers + for i := 0; i < *concurrency; i++ { + wg.Add(1) + go func(id int) { + defer wg.Done() + doPublish(session, id) + }(i) + } + + // Wait for all publishers to finish + wg.Wait() + elapsed := time.Since(startTime) + + log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds()) + +} diff --git a/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go b/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go index d51fdf7af..f925aa1e1 100644 --- a/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go +++ b/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go @@ -6,12 +6,12 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/util" util_http "github.com/seaweedfs/seaweedfs/weed/util/http" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" "strings" - "time" ) var ( @@ -33,23 +33,24 @@ func main() { ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), MaxPartitionCount: int32(*maxPartitionCount), - PerPartitionConcurrency: int32(*perPartitionConcurrency), + SlidingWindowSize: int32(*perPartitionConcurrency), } contentConfig := &sub_client.ContentConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - Filter: "", - StartTime: time.Unix(1, 1), + Topic: topic.NewTopic(*namespace, *t), + Filter: "", } brokers := strings.Split(*seedBrokers, ",") - subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig) + subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024)) counter := 0 - subscriber.SetEachMessageFunc(func(key, value []byte) error { - counter++ - println(string(key), "=>", string(value), counter) - return nil + executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize)) + subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) { + executors.Execute(func() { + counter++ + println(string(m.Data.Key), "=>", string(m.Data.Value), counter) + }) }) subscriber.SetCompletionFunc(func() { diff --git a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go b/weed/mq/client/cmd/weed_sub_record/subscriber_record.go index 7bdff3715..dee289f19 100644 --- a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go +++ b/weed/mq/client/cmd/weed_sub_record/subscriber_record.go @@ -6,6 +6,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client" "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "github.com/seaweedfs/seaweedfs/weed/util" util_http "github.com/seaweedfs/seaweedfs/weed/util/http" @@ -60,30 +61,31 @@ func main() { ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), MaxPartitionCount: int32(*maxPartitionCount), - PerPartitionConcurrency: int32(*perPartitionConcurrency), + SlidingWindowSize: int32(*perPartitionConcurrency), } contentConfig := &sub_client.ContentConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - Filter: "", - StartTime: time.Now().Add(-*timeAgo), + Topic: topic.NewTopic(*namespace, *t), + Filter: "", + // StartTime: time.Now().Add(-*timeAgo), } brokers := strings.Split(*seedBrokers, ",") - subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig) + subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024)) counter := 0 - subscriber.SetEachMessageFunc(func(key, value []byte) error { - counter++ - record := &schema_pb.RecordValue{} - err := proto.Unmarshal(value, record) - if err != nil { - fmt.Printf("unmarshal record value: %v\n", err) - } else { - fmt.Printf("%s %d: %v\n", string(key), len(value), record) - } - //time.Sleep(1300 * time.Millisecond) - return nil + executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize)) + subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) { + executors.Execute(func() { + counter++ + record := &schema_pb.RecordValue{} + err := proto.Unmarshal(m.Data.Value, record) + if err != nil { + fmt.Printf("unmarshal record value: %v\n", err) + } else { + fmt.Printf("%s %d: %v\n", string(m.Data.Key), len(m.Data.Value), record) + } + }) }) subscriber.SetCompletionFunc(func() { diff --git a/weed/mq/client/sub_client/on_each_partition.go b/weed/mq/client/sub_client/on_each_partition.go index 56cedb32e..fbfcc3c6b 100644 --- a/weed/mq/client/sub_client/on_each_partition.go +++ b/weed/mq/client/sub_client/on_each_partition.go @@ -6,12 +6,18 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" - "github.com/seaweedfs/seaweedfs/weed/util" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "io" "reflect" + "time" ) -func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}) error { +type KeyedOffset struct { + Key []byte + Offset int64 +} + +func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}, onDataMessageFn OnDataMessageFn) error { // connect to the partition broker return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { @@ -20,31 +26,30 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig return fmt.Errorf("create subscribe client: %v", err) } - perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency - if perPartitionConcurrency <= 0 { - perPartitionConcurrency = 1 + slidingWindowSize := sub.SubscriberConfig.SlidingWindowSize + if slidingWindowSize <= 0 { + slidingWindowSize = 1 } - var stopTsNs int64 - if !sub.ContentConfig.StopTime.IsZero() { - stopTsNs = sub.ContentConfig.StopTime.UnixNano() + po := findPartitionOffset(sub.ContentConfig.PartitionOffsets, assigned.Partition) + if po == nil { + po = &schema_pb.PartitionOffset{ + Partition: assigned.Partition, + StartTsNs: time.Now().UnixNano(), + StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY, + } } if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{ Message: &mq_pb.SubscribeMessageRequest_Init{ Init: &mq_pb.SubscribeMessageRequest_InitMessage{ - ConsumerGroup: sub.SubscriberConfig.ConsumerGroup, - ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId, - Topic: sub.ContentConfig.Topic.ToPbTopic(), - PartitionOffset: &mq_pb.PartitionOffset{ - Partition: assigned.Partition, - StartTsNs: sub.ContentConfig.StartTime.UnixNano(), - StopTsNs: stopTsNs, - StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY, - }, - Filter: sub.ContentConfig.Filter, - FollowerBroker: assigned.FollowerBroker, - Concurrency: perPartitionConcurrency, + ConsumerGroup: sub.SubscriberConfig.ConsumerGroup, + ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId, + Topic: sub.ContentConfig.Topic.ToPbTopic(), + PartitionOffset: po, + Filter: sub.ContentConfig.Filter, + FollowerBroker: assigned.FollowerBroker, + SlidingWindowSize: slidingWindowSize, }, }, }); err != nil { @@ -57,24 +62,13 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig defer sub.OnCompletionFunc() } - type KeyedOffset struct { - Key []byte - Offset int64 - } - - partitionOffsetChan := make(chan KeyedOffset, 1024) - defer func() { - close(partitionOffsetChan) - }() - executors := util.NewLimitedConcurrentExecutor(int(perPartitionConcurrency)) - go func() { for { select { case <-stopCh: subscribeClient.CloseSend() return - case ack, ok := <-partitionOffsetChan: + case ack, ok := <-sub.PartitionOffsetChan: if !ok { subscribeClient.CloseSend() return @@ -91,9 +85,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig } }() - var lastErr error - - for lastErr == nil { + for { // glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup) resp, err := subscribeClient.Recv() if err != nil { @@ -113,17 +105,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m)) continue } - executors.Execute(func() { - processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value) - if processErr == nil { - partitionOffsetChan <- KeyedOffset{ - Key: m.Data.Key, - Offset: m.Data.TsNs, - } - } else { - lastErr = processErr - } - }) + onDataMessageFn(m) case *mq_pb.SubscribeMessageResponse_Ctrl: // glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl) if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic { @@ -132,6 +114,14 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig } } - return lastErr }) } + +func findPartitionOffset(partitionOffsets []*schema_pb.PartitionOffset, partition *schema_pb.Partition) *schema_pb.PartitionOffset { + for _, po := range partitionOffsets { + if po.Partition == partition { + return po + } + } + return nil +} diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go index 5669bb348..cf2294891 100644 --- a/weed/mq/client/sub_client/subscribe.go +++ b/weed/mq/client/sub_client/subscribe.go @@ -4,6 +4,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/util" "sync" "time" ) @@ -20,6 +21,7 @@ func (sub *TopicSubscriber) Subscribe() error { go sub.startProcessors() // loop forever + // TODO shutdown the subscriber when not needed anymore sub.doKeepConnectedToSubCoordinator() return nil @@ -66,7 +68,21 @@ func (sub *TopicSubscriber) startProcessors() { }, }, } - err := sub.onEachPartition(assigned, stopChan) + + executors := util.NewLimitedConcurrentExecutor(int(sub.SubscriberConfig.SlidingWindowSize)) + onDataMessageFn := func(m *mq_pb.SubscribeMessageResponse_Data) { + executors.Execute(func() { + processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value) + if processErr == nil { + sub.PartitionOffsetChan <- KeyedOffset{ + Key: m.Data.Key, + Offset: m.Data.TsNs, + } + } + }) + } + + err := sub.onEachPartition(assigned, stopChan, onDataMessageFn) if err != nil { glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err) } else { diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go index 3e5316b67..9a51ce01e 100644 --- a/weed/mq/client/sub_client/subscriber.go +++ b/weed/mq/client/sub_client/subscriber.go @@ -3,9 +3,9 @@ package sub_client import ( "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "google.golang.org/grpc" "sync" - "time" ) type SubscriberConfiguration struct { @@ -14,16 +14,16 @@ type SubscriberConfiguration struct { ConsumerGroupInstanceId string GrpcDialOption grpc.DialOption MaxPartitionCount int32 // how many partitions to process concurrently - PerPartitionConcurrency int32 // how many messages to process concurrently per partition + SlidingWindowSize int32 // how many messages to process concurrently per partition } type ContentConfiguration struct { - Topic topic.Topic - Filter string - StartTime time.Time - StopTime time.Time + Topic topic.Topic + Filter string + PartitionOffsets []*schema_pb.PartitionOffset } +type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data) type OnEachMessageFunc func(key, value []byte) (err error) type OnCompletionFunc func() @@ -32,15 +32,17 @@ type TopicSubscriber struct { ContentConfig *ContentConfiguration brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest + OnDataMessageFnnc OnDataMessageFn OnEachMessageFunc OnEachMessageFunc OnCompletionFunc OnCompletionFunc bootstrapBrokers []string waitForMoreMessage bool activeProcessors map[topic.Partition]*ProcessorState activeProcessorsLock sync.Mutex + PartitionOffsetChan chan KeyedOffset } -func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber { +func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber { return &TopicSubscriber{ SubscriberConfig: subscriber, ContentConfig: content, @@ -49,6 +51,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu bootstrapBrokers: bootstrapBrokers, waitForMoreMessage: true, activeProcessors: make(map[topic.Partition]*ProcessorState), + PartitionOffsetChan: partitionOffsetChan, } } @@ -56,6 +59,10 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc sub.OnEachMessageFunc = onEachMessageFn } +func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) { + sub.OnDataMessageFnnc = fn +} + func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) { sub.OnCompletionFunc = onCompletionFn } diff --git a/weed/mq/pub_balancer/allocate.go b/weed/mq/pub_balancer/allocate.go index ae32188ec..46d423b30 100644 --- a/weed/mq/pub_balancer/allocate.go +++ b/weed/mq/pub_balancer/allocate.go @@ -4,6 +4,7 @@ import ( cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "math/rand" "time" ) @@ -14,7 +15,7 @@ func AllocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p rangeSize := MaxPartitionCount / partitionCount for i := int32(0); i < partitionCount; i++ { assignment := &mq_pb.BrokerPartitionAssignment{ - Partition: &mq_pb.Partition{ + Partition: &schema_pb.Partition{ RingSize: MaxPartitionCount, RangeStart: int32(i * rangeSize), RangeStop: int32((i + 1) * rangeSize), diff --git a/weed/mq/pub_balancer/allocate_test.go b/weed/mq/pub_balancer/allocate_test.go index 63692af0f..fc747634e 100644 --- a/weed/mq/pub_balancer/allocate_test.go +++ b/weed/mq/pub_balancer/allocate_test.go @@ -4,6 +4,7 @@ import ( "fmt" cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "github.com/stretchr/testify/assert" "testing" ) @@ -29,7 +30,7 @@ func Test_allocateOneBroker(t *testing.T) { wantAssignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:17777", - Partition: &mq_pb.Partition{ + Partition: &schema_pb.Partition{ RingSize: MaxPartitionCount, RangeStart: 0, RangeStop: MaxPartitionCount, @@ -96,7 +97,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "localhost:2", }, }, @@ -111,7 +112,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:1", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "", }, }, @@ -126,7 +127,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:1", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "localhost:200", }, }, @@ -141,7 +142,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:100", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "localhost:200", }, }, @@ -156,7 +157,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:1", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "localhost:2", }, }, @@ -171,7 +172,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:1", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, }, }, }, @@ -185,7 +186,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) { assignments: []*mq_pb.BrokerPartitionAssignment{ { LeaderBroker: "localhost:1", - Partition: &mq_pb.Partition{}, + Partition: &schema_pb.Partition{}, FollowerBroker: "localhost:2", }, }, diff --git a/weed/mq/pub_balancer/broker_stats.go b/weed/mq/pub_balancer/broker_stats.go index e72703d5f..da016b7fd 100644 --- a/weed/mq/pub_balancer/broker_stats.go +++ b/weed/mq/pub_balancer/broker_stats.go @@ -5,6 +5,7 @@ import ( cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) type BrokerStats struct { @@ -65,7 +66,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) { bs.SubscriberCount = subscriberCount } -func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition, isAdd bool) { +func (bs *BrokerStats) RegisterAssignment(t *schema_pb.Topic, partition *schema_pb.Partition, isAdd bool) { tps := &TopicPartitionStats{ TopicPartition: topic.TopicPartition{ Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name}, diff --git a/weed/mq/pub_balancer/lookup.go b/weed/mq/pub_balancer/lookup.go index 423b38ecb..5f9c7f32f 100644 --- a/weed/mq/pub_balancer/lookup.go +++ b/weed/mq/pub_balancer/lookup.go @@ -3,13 +3,14 @@ package pub_balancer import ( "errors" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) var ( ErrNoBroker = errors.New("no broker") ) -func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) { +func (balancer *PubBalancer) LookupTopicPartitions(topic *schema_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) { // find existing topic partition assignments for brokerStatsItem := range balancer.Brokers.IterBuffered() { broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val @@ -18,7 +19,7 @@ func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignme if topicPartitionStat.TopicPartition.Namespace == topic.Namespace && topicPartitionStat.TopicPartition.Name == topic.Name { assignment := &mq_pb.BrokerPartitionAssignment{ - Partition: &mq_pb.Partition{ + Partition: &schema_pb.Partition{ RingSize: MaxPartitionCount, RangeStart: topicPartitionStat.RangeStart, RangeStop: topicPartitionStat.RangeStop, diff --git a/weed/mq/pub_balancer/partition_list_broker.go b/weed/mq/pub_balancer/partition_list_broker.go index d084cf74e..34bdfd286 100644 --- a/weed/mq/pub_balancer/partition_list_broker.go +++ b/weed/mq/pub_balancer/partition_list_broker.go @@ -2,7 +2,7 @@ package pub_balancer import ( "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) type PartitionSlotToBroker struct { @@ -24,7 +24,7 @@ func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList { } } -func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string, follower string) { +func (ps *PartitionSlotToBrokerList) AddBroker(partition *schema_pb.Partition, broker string, follower string) { for _, partitionSlot := range ps.PartitionSlots { if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop { if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker { diff --git a/weed/mq/pub_balancer/pub_balancer.go b/weed/mq/pub_balancer/pub_balancer.go index 755cc8f30..9457b76fe 100644 --- a/weed/mq/pub_balancer/pub_balancer.go +++ b/weed/mq/pub_balancer/pub_balancer.go @@ -4,6 +4,7 @@ import ( cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) const ( @@ -32,7 +33,7 @@ type PubBalancer struct { Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address // Collected from all brokers when they connect to the broker leader TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name - OnPartitionChange func(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) + OnPartitionChange func(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) } func NewPubBalancer() *PubBalancer { diff --git a/weed/mq/schema/schema.go b/weed/mq/schema/schema.go index ca31ce534..04d0a7b5e 100644 --- a/weed/mq/schema/schema.go +++ b/weed/mq/schema/schema.go @@ -5,19 +5,24 @@ import ( ) type Schema struct { + Namespace string + Name string + RevisionId uint32 RecordType *schema_pb.RecordType fieldMap map[string]*schema_pb.Field } -func NewSchema(recordType *schema_pb.RecordType) (*Schema, error) { +func NewSchema(namespace string, name string, recordType *schema_pb.RecordType) *Schema { fieldMap := make(map[string]*schema_pb.Field) for _, field := range recordType.Fields { fieldMap[field.Name] = field } return &Schema{ + Namespace: namespace, + Name: name, RecordType: recordType, fieldMap: fieldMap, - }, nil + } } func (s *Schema) GetField(name string) (*schema_pb.Field, bool) { diff --git a/weed/mq/sub_coordinator/consumer_group.go b/weed/mq/sub_coordinator/consumer_group.go index 80a2d5f80..ba94f34b4 100644 --- a/weed/mq/sub_coordinator/consumer_group.go +++ b/weed/mq/sub_coordinator/consumer_group.go @@ -7,6 +7,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "time" ) @@ -20,7 +21,7 @@ type ConsumerGroup struct { stopCh chan struct{} } -func NewConsumerGroup(t *mq_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup { +func NewConsumerGroup(t *schema_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup { cg := &ConsumerGroup{ topic: topic.FromPbTopic(t), ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](), diff --git a/weed/mq/sub_coordinator/sub_coordinator.go b/weed/mq/sub_coordinator/sub_coordinator.go index 47101eee4..a26fb9dc5 100644 --- a/weed/mq/sub_coordinator/sub_coordinator.go +++ b/weed/mq/sub_coordinator/sub_coordinator.go @@ -5,6 +5,7 @@ import ( cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/filer_client" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" ) type TopicConsumerGroups struct { @@ -28,7 +29,7 @@ func NewSubCoordinator() *SubCoordinator { } } -func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups { +func (c *SubCoordinator) GetTopicConsumerGroups(topic *schema_pb.Topic, createIfMissing bool) *TopicConsumerGroups { topicName := toTopicName(topic) tcg, _ := c.TopicSubscribers.Get(topicName) if tcg == nil && createIfMissing { @@ -41,12 +42,12 @@ func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMiss } return tcg } -func (c *SubCoordinator) RemoveTopic(topic *mq_pb.Topic) { +func (c *SubCoordinator) RemoveTopic(topic *schema_pb.Topic) { topicName := toTopicName(topic) c.TopicSubscribers.Remove(topicName) } -func toTopicName(topic *mq_pb.Topic) string { +func toTopicName(topic *schema_pb.Topic) string { topicName := topic.Namespace + "." + topic.Name return topicName } @@ -96,7 +97,7 @@ func (c *SubCoordinator) RemoveSubscriber(initMessage *mq_pb.SubscriberToSubCoor } } -func (c *SubCoordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) { +func (c *SubCoordinator) OnPartitionChange(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) { tcg, _ := c.TopicSubscribers.Get(toTopicName(topic)) if tcg == nil { return diff --git a/weed/mq/topic/local_manager.go b/weed/mq/topic/local_manager.go index 9f273723d..82ee18c4a 100644 --- a/weed/mq/topic/local_manager.go +++ b/weed/mq/topic/local_manager.go @@ -3,6 +3,7 @@ package topic import ( cmap "github.com/orcaman/concurrent-map/v2" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "github.com/shirou/gopsutil/v3/cpu" "time" ) @@ -89,7 +90,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br Partition: localPartition.Partition, } stats.Stats[topicPartition.TopicPartitionId()] = &mq_pb.TopicPartitionStats{ - Topic: &mq_pb.Topic{ + Topic: &schema_pb.Topic{ Namespace: string(localTopic.Namespace), Name: localTopic.Name, }, diff --git a/weed/mq/topic/partition.go b/weed/mq/topic/partition.go index 7edf979b5..cee512ab5 100644 --- a/weed/mq/topic/partition.go +++ b/weed/mq/topic/partition.go @@ -2,7 +2,7 @@ package topic import ( "fmt" - "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "time" ) @@ -40,7 +40,7 @@ func (partition Partition) Equals(other Partition) bool { return true } -func FromPbPartition(partition *mq_pb.Partition) Partition { +func FromPbPartition(partition *schema_pb.Partition) Partition { return Partition{ RangeStart: partition.RangeStart, RangeStop: partition.RangeStop, @@ -67,8 +67,8 @@ func SplitPartitions(targetCount int32, ts int64) []*Partition { return partitions } -func (partition Partition) ToPbPartition() *mq_pb.Partition { - return &mq_pb.Partition{ +func (partition Partition) ToPbPartition() *schema_pb.Partition { + return &schema_pb.Partition{ RangeStart: partition.RangeStart, RangeStop: partition.RangeStop, RingSize: partition.RingSize, diff --git a/weed/mq/topic/topic.go b/weed/mq/topic/topic.go index 5e9012e70..f390c7234 100644 --- a/weed/mq/topic/topic.go +++ b/weed/mq/topic/topic.go @@ -7,6 +7,7 @@ import ( "github.com/seaweedfs/seaweedfs/weed/filer" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" jsonpb "google.golang.org/protobuf/encoding/protojson" ) @@ -21,15 +22,15 @@ func NewTopic(namespace string, name string) Topic { Name: name, } } -func FromPbTopic(topic *mq_pb.Topic) Topic { +func FromPbTopic(topic *schema_pb.Topic) Topic { return Topic{ Namespace: topic.Namespace, Name: topic.Name, } } -func (t Topic) ToPbTopic() *mq_pb.Topic { - return &mq_pb.Topic{ +func (t Topic) ToPbTopic() *schema_pb.Topic { + return &schema_pb.Topic{ Namespace: t.Namespace, Name: t.Name, } diff --git a/weed/pb/Makefile b/weed/pb/Makefile index cc2c064b9..e1ff89dfd 100644 --- a/weed/pb/Makefile +++ b/weed/pb/Makefile @@ -10,8 +10,9 @@ gen: protoc iam.proto --go_out=./iam_pb --go-grpc_out=./iam_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative protoc mount.proto --go_out=./mount_pb --go-grpc_out=./mount_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative protoc s3.proto --go_out=./s3_pb --go-grpc_out=./s3_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative - protoc mq.proto --go_out=./mq_pb --go-grpc_out=./mq_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative - protoc schema.proto --go_out=./schema_pb --go-grpc_out=./schema_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc mq_broker.proto --go_out=./mq_pb --go-grpc_out=./mq_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc mq_schema.proto --go_out=./schema_pb --go-grpc_out=./schema_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative + protoc mq_agent.proto --go_out=./mq_agent_pb --go-grpc_out=./mq_agent_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative # protoc filer.proto --java_out=../../other/java/client/src/main/java cp filer.proto ../../other/java/client/src/main/proto diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index 44aa282a9..cf3976f86 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/weed/pb/filer_pb/filer.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: filer.proto package filer_pb diff --git a/weed/pb/filer_pb/filer_grpc.pb.go b/weed/pb/filer_pb/filer_grpc.pb.go index 88bad1619..aa8ce6fa3 100644 --- a/weed/pb/filer_pb/filer_grpc.pb.go +++ b/weed/pb/filer_pb/filer_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: filer.proto package filer_pb diff --git a/weed/pb/iam_pb/iam.pb.go b/weed/pb/iam_pb/iam.pb.go index f7ee48166..a75416dbf 100644 --- a/weed/pb/iam_pb/iam.pb.go +++ b/weed/pb/iam_pb/iam.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: iam.proto package iam_pb diff --git a/weed/pb/iam_pb/iam_grpc.pb.go b/weed/pb/iam_pb/iam_grpc.pb.go index 47c201f06..ea1e82f10 100644 --- a/weed/pb/iam_pb/iam_grpc.pb.go +++ b/weed/pb/iam_pb/iam_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: iam.proto package iam_pb diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go index 5f87a0281..c4893456a 100644 --- a/weed/pb/master_pb/master.pb.go +++ b/weed/pb/master_pb/master.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: master.proto package master_pb diff --git a/weed/pb/master_pb/master_grpc.pb.go b/weed/pb/master_pb/master_grpc.pb.go index f49429950..7858f4345 100644 --- a/weed/pb/master_pb/master_grpc.pb.go +++ b/weed/pb/master_pb/master_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: master.proto package master_pb diff --git a/weed/pb/mount_pb/mount.pb.go b/weed/pb/mount_pb/mount.pb.go index d5ef3a765..26ff1e096 100644 --- a/weed/pb/mount_pb/mount.pb.go +++ b/weed/pb/mount_pb/mount.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: mount.proto package mount_pb diff --git a/weed/pb/mount_pb/mount_grpc.pb.go b/weed/pb/mount_pb/mount_grpc.pb.go index 58c677eb1..6ed1a347f 100644 --- a/weed/pb/mount_pb/mount_grpc.pb.go +++ b/weed/pb/mount_pb/mount_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: mount.proto package mount_pb diff --git a/weed/pb/mq_agent.proto b/weed/pb/mq_agent.proto new file mode 100644 index 000000000..605f2a31c --- /dev/null +++ b/weed/pb/mq_agent.proto @@ -0,0 +1,95 @@ +syntax = "proto3"; + +package messaging_pb; + +import "mq_schema.proto"; + +option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"; +option java_package = "seaweedfs.mq_agent"; +option java_outer_classname = "MessageQueueAgentProto"; + +////////////////////////////////////////////////// + +service SeaweedMessagingAgent { + + // Publishing + rpc StartPublishSession (StartPublishSessionRequest) returns (StartPublishSessionResponse) { + } + rpc ClosePublishSession (ClosePublishSessionRequest) returns (ClosePublishSessionResponse) { + } + rpc PublishRecord (stream PublishRecordRequest) returns (stream PublishRecordResponse) { + } + + // Subscribing + rpc StartSubscribeSession (StartSubscribeSessionRequest) returns (StartSubscribeSessionResponse) { + } + rpc CloseSubscribeSession (CloseSubscribeSessionRequest) returns (CloseSubscribeSessionResponse) { + } + rpc SubscribeRecord (stream SubscribeRecordRequest) returns (stream SubscribeRecordResponse) { + } + +} + +////////////////////////////////////////////////// +message StartPublishSessionRequest { + schema_pb.Topic topic = 1; + int32 partition_count = 2; + schema_pb.RecordType record_type = 3; + string publisher_name = 4; +} +message StartPublishSessionResponse { + string error = 1; + int64 session_id = 2; +} +message ClosePublishSessionRequest { + int64 session_id = 1; +} +message ClosePublishSessionResponse { + string error = 1; +} + +////////////////////////////////////////////////// +message PublishRecordRequest { + int64 session_id = 1; // session_id is required for the first record + bytes key = 2; + schema_pb.RecordValue value = 3; +} +message PublishRecordResponse { + int64 ack_sequence = 1; + string error = 2; +} +////////////////////////////////////////////////// +message StartSubscribeSessionRequest { + string consumer_group = 1; + string consumer_group_instance_id = 2; + schema_pb.Topic topic = 4; + repeated schema_pb.PartitionOffset partition_offsets = 5; + string filter = 6; + int32 max_subscribed_partitions = 8; + int32 sliding_window_size = 9; +} +message StartSubscribeSessionResponse { + string error = 1; + int64 session_id = 2; +} +message CloseSubscribeSessionRequest { + int64 session_id = 1; +} +message CloseSubscribeSessionResponse { + string error = 1; +} +////////////////////////////////////////////////// +message SubscribeRecordRequest { + int64 session_id = 1; // session_id is required for the first record + int64 ack_sequence = 2; + bytes ack_key = 3; +} +message SubscribeRecordResponse { + bytes key = 2; + schema_pb.RecordValue value = 3; + int64 ts_ns = 4; + string error = 5; + bool is_end_of_stream = 6; + bool is_end_of_topic = 7; +} +////////////////////////////////////////////////// diff --git a/weed/pb/mq_agent_pb/mq_agent.pb.go b/weed/pb/mq_agent_pb/mq_agent.pb.go new file mode 100644 index 000000000..9b280c076 --- /dev/null +++ b/weed/pb/mq_agent_pb/mq_agent.pb.go @@ -0,0 +1,1148 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.34.2 +// protoc v5.28.3 +// source: mq_agent.proto + +package mq_agent_pb + +import ( + schema_pb "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// //////////////////////////////////////////////// +type StartPublishSessionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionCount int32 `protobuf:"varint,2,opt,name=partition_count,json=partitionCount,proto3" json:"partition_count,omitempty"` + RecordType *schema_pb.RecordType `protobuf:"bytes,3,opt,name=record_type,json=recordType,proto3" json:"record_type,omitempty"` + PublisherName string `protobuf:"bytes,4,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"` +} + +func (x *StartPublishSessionRequest) Reset() { + *x = StartPublishSessionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartPublishSessionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartPublishSessionRequest) ProtoMessage() {} + +func (x *StartPublishSessionRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartPublishSessionRequest.ProtoReflect.Descriptor instead. +func (*StartPublishSessionRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{0} +} + +func (x *StartPublishSessionRequest) GetTopic() *schema_pb.Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *StartPublishSessionRequest) GetPartitionCount() int32 { + if x != nil { + return x.PartitionCount + } + return 0 +} + +func (x *StartPublishSessionRequest) GetRecordType() *schema_pb.RecordType { + if x != nil { + return x.RecordType + } + return nil +} + +func (x *StartPublishSessionRequest) GetPublisherName() string { + if x != nil { + return x.PublisherName + } + return "" +} + +type StartPublishSessionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + SessionId int64 `protobuf:"varint,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` +} + +func (x *StartPublishSessionResponse) Reset() { + *x = StartPublishSessionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartPublishSessionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartPublishSessionResponse) ProtoMessage() {} + +func (x *StartPublishSessionResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[1] + 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 StartPublishSessionResponse.ProtoReflect.Descriptor instead. +func (*StartPublishSessionResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{1} +} + +func (x *StartPublishSessionResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +func (x *StartPublishSessionResponse) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +type ClosePublishSessionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` +} + +func (x *ClosePublishSessionRequest) Reset() { + *x = ClosePublishSessionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClosePublishSessionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClosePublishSessionRequest) ProtoMessage() {} + +func (x *ClosePublishSessionRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[2] + 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 ClosePublishSessionRequest.ProtoReflect.Descriptor instead. +func (*ClosePublishSessionRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{2} +} + +func (x *ClosePublishSessionRequest) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +type ClosePublishSessionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *ClosePublishSessionResponse) Reset() { + *x = ClosePublishSessionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClosePublishSessionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClosePublishSessionResponse) ProtoMessage() {} + +func (x *ClosePublishSessionResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[3] + 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 ClosePublishSessionResponse.ProtoReflect.Descriptor instead. +func (*ClosePublishSessionResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{3} +} + +func (x *ClosePublishSessionResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +// //////////////////////////////////////////////// +type PublishRecordRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` // session_id is required for the first record + Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Value *schema_pb.RecordValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *PublishRecordRequest) Reset() { + *x = PublishRecordRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishRecordRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishRecordRequest) ProtoMessage() {} + +func (x *PublishRecordRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[4] + 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 PublishRecordRequest.ProtoReflect.Descriptor instead. +func (*PublishRecordRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{4} +} + +func (x *PublishRecordRequest) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +func (x *PublishRecordRequest) GetKey() []byte { + if x != nil { + return x.Key + } + return nil +} + +func (x *PublishRecordRequest) GetValue() *schema_pb.RecordValue { + if x != nil { + return x.Value + } + return nil +} + +type PublishRecordResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AckSequence int64 `protobuf:"varint,1,opt,name=ack_sequence,json=ackSequence,proto3" json:"ack_sequence,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *PublishRecordResponse) Reset() { + *x = PublishRecordResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishRecordResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishRecordResponse) ProtoMessage() {} + +func (x *PublishRecordResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublishRecordResponse.ProtoReflect.Descriptor instead. +func (*PublishRecordResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{5} +} + +func (x *PublishRecordResponse) GetAckSequence() int64 { + if x != nil { + return x.AckSequence + } + return 0 +} + +func (x *PublishRecordResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +// //////////////////////////////////////////////// +type StartSubscribeSessionRequest 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"` + ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffsets []*schema_pb.PartitionOffset `protobuf:"bytes,5,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"` + Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` + MaxSubscribedPartitions int32 `protobuf:"varint,8,opt,name=max_subscribed_partitions,json=maxSubscribedPartitions,proto3" json:"max_subscribed_partitions,omitempty"` + SlidingWindowSize int32 `protobuf:"varint,9,opt,name=sliding_window_size,json=slidingWindowSize,proto3" json:"sliding_window_size,omitempty"` +} + +func (x *StartSubscribeSessionRequest) Reset() { + *x = StartSubscribeSessionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartSubscribeSessionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartSubscribeSessionRequest) ProtoMessage() {} + +func (x *StartSubscribeSessionRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartSubscribeSessionRequest.ProtoReflect.Descriptor instead. +func (*StartSubscribeSessionRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{6} +} + +func (x *StartSubscribeSessionRequest) GetConsumerGroup() string { + if x != nil { + return x.ConsumerGroup + } + return "" +} + +func (x *StartSubscribeSessionRequest) GetConsumerGroupInstanceId() string { + if x != nil { + return x.ConsumerGroupInstanceId + } + return "" +} + +func (x *StartSubscribeSessionRequest) GetTopic() *schema_pb.Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *StartSubscribeSessionRequest) GetPartitionOffsets() []*schema_pb.PartitionOffset { + if x != nil { + return x.PartitionOffsets + } + return nil +} + +func (x *StartSubscribeSessionRequest) GetFilter() string { + if x != nil { + return x.Filter + } + return "" +} + +func (x *StartSubscribeSessionRequest) GetMaxSubscribedPartitions() int32 { + if x != nil { + return x.MaxSubscribedPartitions + } + return 0 +} + +func (x *StartSubscribeSessionRequest) GetSlidingWindowSize() int32 { + if x != nil { + return x.SlidingWindowSize + } + return 0 +} + +type StartSubscribeSessionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + SessionId int64 `protobuf:"varint,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` +} + +func (x *StartSubscribeSessionResponse) Reset() { + *x = StartSubscribeSessionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartSubscribeSessionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartSubscribeSessionResponse) ProtoMessage() {} + +func (x *StartSubscribeSessionResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartSubscribeSessionResponse.ProtoReflect.Descriptor instead. +func (*StartSubscribeSessionResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{7} +} + +func (x *StartSubscribeSessionResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +func (x *StartSubscribeSessionResponse) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +type CloseSubscribeSessionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` +} + +func (x *CloseSubscribeSessionRequest) Reset() { + *x = CloseSubscribeSessionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseSubscribeSessionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseSubscribeSessionRequest) ProtoMessage() {} + +func (x *CloseSubscribeSessionRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CloseSubscribeSessionRequest.ProtoReflect.Descriptor instead. +func (*CloseSubscribeSessionRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{8} +} + +func (x *CloseSubscribeSessionRequest) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +type CloseSubscribeSessionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *CloseSubscribeSessionResponse) Reset() { + *x = CloseSubscribeSessionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CloseSubscribeSessionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CloseSubscribeSessionResponse) ProtoMessage() {} + +func (x *CloseSubscribeSessionResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CloseSubscribeSessionResponse.ProtoReflect.Descriptor instead. +func (*CloseSubscribeSessionResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{9} +} + +func (x *CloseSubscribeSessionResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +// //////////////////////////////////////////////// +type SubscribeRecordRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` // session_id is required for the first record + AckSequence int64 `protobuf:"varint,2,opt,name=ack_sequence,json=ackSequence,proto3" json:"ack_sequence,omitempty"` + AckKey []byte `protobuf:"bytes,3,opt,name=ack_key,json=ackKey,proto3" json:"ack_key,omitempty"` +} + +func (x *SubscribeRecordRequest) Reset() { + *x = SubscribeRecordRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscribeRecordRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscribeRecordRequest) ProtoMessage() {} + +func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscribeRecordRequest.ProtoReflect.Descriptor instead. +func (*SubscribeRecordRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{10} +} + +func (x *SubscribeRecordRequest) GetSessionId() int64 { + if x != nil { + return x.SessionId + } + return 0 +} + +func (x *SubscribeRecordRequest) GetAckSequence() int64 { + if x != nil { + return x.AckSequence + } + return 0 +} + +func (x *SubscribeRecordRequest) GetAckKey() []byte { + if x != nil { + return x.AckKey + } + return nil +} + +type SubscribeRecordResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"` + Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Value *schema_pb.RecordValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + TsNs int64 `protobuf:"varint,4,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"` + Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` + IsEndOfStream bool `protobuf:"varint,6,opt,name=is_end_of_stream,json=isEndOfStream,proto3" json:"is_end_of_stream,omitempty"` + IsEndOfTopic bool `protobuf:"varint,7,opt,name=is_end_of_topic,json=isEndOfTopic,proto3" json:"is_end_of_topic,omitempty"` +} + +func (x *SubscribeRecordResponse) Reset() { + *x = SubscribeRecordResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_agent_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscribeRecordResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscribeRecordResponse) ProtoMessage() {} + +func (x *SubscribeRecordResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_agent_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscribeRecordResponse.ProtoReflect.Descriptor instead. +func (*SubscribeRecordResponse) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{11} +} + +func (x *SubscribeRecordResponse) GetSequence() int64 { + if x != nil { + return x.Sequence + } + return 0 +} + +func (x *SubscribeRecordResponse) GetKey() []byte { + if x != nil { + return x.Key + } + return nil +} + +func (x *SubscribeRecordResponse) GetValue() *schema_pb.RecordValue { + if x != nil { + return x.Value + } + return nil +} + +func (x *SubscribeRecordResponse) GetTsNs() int64 { + if x != nil { + return x.TsNs + } + return 0 +} + +func (x *SubscribeRecordResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +func (x *SubscribeRecordResponse) GetIsEndOfStream() bool { + if x != nil { + return x.IsEndOfStream + } + return false +} + +func (x *SubscribeRecordResponse) GetIsEndOfTopic() bool { + if x != nil { + return x.IsEndOfTopic + } + return false +} + +var File_mq_agent_proto protoreflect.FileDescriptor + +var file_mq_agent_proto_rawDesc = []byte{ + 0x0a, 0x0e, 0x6d, 0x71, 0x5f, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x1a, 0x0f, + 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, + 0xcc, 0x01, 0x0a, 0x1a, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x26, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, + 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x52, + 0x0a, 0x1b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x53, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 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, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x22, 0x3b, 0x0a, 0x1a, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, + 0x33, 0x0a, 0x1b, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 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, 0x22, 0x75, 0x0a, 0x14, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, + 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2c, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x50, 0x0a, 0x15, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, + 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xf7, 0x02, + 0x0a, 0x1c, 0x53, 0x74, 0x61, 0x72, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 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, 0x3b, 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, + 0x49, 0x64, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x47, 0x0a, 0x11, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, + 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, + 0x74, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x3a, 0x0a, 0x19, 0x6d, + 0x61, 0x78, 0x5f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x64, 0x5f, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x17, + 0x6d, 0x61, 0x78, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x64, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x73, 0x6c, 0x69, 0x64, 0x69, + 0x6e, 0x67, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x73, 0x6c, 0x69, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x54, 0x0a, 0x1d, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 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, 0x1d, + 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x3d, 0x0a, + 0x1c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x35, 0x0a, 0x1d, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 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, 0x22, 0x73, 0x0a, 0x16, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, + 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, + 0x17, 0x0a, 0x07, 0x61, 0x63, 0x6b, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x06, 0x61, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x22, 0xf0, 0x01, 0x0a, 0x17, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 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, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x2c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, + 0x63, 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, + 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, + 0x06, 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, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, + 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x32, 0xa1, 0x05, 0x0a, 0x15, + 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x41, 0x67, 0x65, 0x6e, 0x74, 0x12, 0x6c, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x72, + 0x74, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x13, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, + 0x69, 0x73, 0x68, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x28, 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, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x5e, 0x0a, 0x0d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x63, 0x6f, + 0x72, 0x64, 0x12, 0x22, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, + 0x01, 0x12, 0x72, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x72, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x15, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2a, + 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, 0x53, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 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, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0f, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x24, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, + 0x60, 0x0a, 0x12, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x5f, + 0x61, 0x67, 0x65, 0x6e, 0x74, 0x42, 0x16, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x32, 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, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x5f, 0x70, + 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_mq_agent_proto_rawDescOnce sync.Once + file_mq_agent_proto_rawDescData = file_mq_agent_proto_rawDesc +) + +func file_mq_agent_proto_rawDescGZIP() []byte { + file_mq_agent_proto_rawDescOnce.Do(func() { + file_mq_agent_proto_rawDescData = protoimpl.X.CompressGZIP(file_mq_agent_proto_rawDescData) + }) + return file_mq_agent_proto_rawDescData +} + +var file_mq_agent_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_mq_agent_proto_goTypes = []any{ + (*StartPublishSessionRequest)(nil), // 0: messaging_pb.StartPublishSessionRequest + (*StartPublishSessionResponse)(nil), // 1: messaging_pb.StartPublishSessionResponse + (*ClosePublishSessionRequest)(nil), // 2: messaging_pb.ClosePublishSessionRequest + (*ClosePublishSessionResponse)(nil), // 3: messaging_pb.ClosePublishSessionResponse + (*PublishRecordRequest)(nil), // 4: messaging_pb.PublishRecordRequest + (*PublishRecordResponse)(nil), // 5: messaging_pb.PublishRecordResponse + (*StartSubscribeSessionRequest)(nil), // 6: messaging_pb.StartSubscribeSessionRequest + (*StartSubscribeSessionResponse)(nil), // 7: messaging_pb.StartSubscribeSessionResponse + (*CloseSubscribeSessionRequest)(nil), // 8: messaging_pb.CloseSubscribeSessionRequest + (*CloseSubscribeSessionResponse)(nil), // 9: messaging_pb.CloseSubscribeSessionResponse + (*SubscribeRecordRequest)(nil), // 10: messaging_pb.SubscribeRecordRequest + (*SubscribeRecordResponse)(nil), // 11: messaging_pb.SubscribeRecordResponse + (*schema_pb.Topic)(nil), // 12: schema_pb.Topic + (*schema_pb.RecordType)(nil), // 13: schema_pb.RecordType + (*schema_pb.RecordValue)(nil), // 14: schema_pb.RecordValue + (*schema_pb.PartitionOffset)(nil), // 15: schema_pb.PartitionOffset +} +var file_mq_agent_proto_depIdxs = []int32{ + 12, // 0: messaging_pb.StartPublishSessionRequest.topic:type_name -> schema_pb.Topic + 13, // 1: messaging_pb.StartPublishSessionRequest.record_type:type_name -> schema_pb.RecordType + 14, // 2: messaging_pb.PublishRecordRequest.value:type_name -> schema_pb.RecordValue + 12, // 3: messaging_pb.StartSubscribeSessionRequest.topic:type_name -> schema_pb.Topic + 15, // 4: messaging_pb.StartSubscribeSessionRequest.partition_offsets:type_name -> schema_pb.PartitionOffset + 14, // 5: messaging_pb.SubscribeRecordResponse.value:type_name -> schema_pb.RecordValue + 0, // 6: messaging_pb.SeaweedMessagingAgent.StartPublishSession:input_type -> messaging_pb.StartPublishSessionRequest + 2, // 7: messaging_pb.SeaweedMessagingAgent.ClosePublishSession:input_type -> messaging_pb.ClosePublishSessionRequest + 4, // 8: messaging_pb.SeaweedMessagingAgent.PublishRecord:input_type -> messaging_pb.PublishRecordRequest + 6, // 9: messaging_pb.SeaweedMessagingAgent.StartSubscribeSession:input_type -> messaging_pb.StartSubscribeSessionRequest + 8, // 10: messaging_pb.SeaweedMessagingAgent.CloseSubscribeSession:input_type -> messaging_pb.CloseSubscribeSessionRequest + 10, // 11: messaging_pb.SeaweedMessagingAgent.SubscribeRecord:input_type -> messaging_pb.SubscribeRecordRequest + 1, // 12: messaging_pb.SeaweedMessagingAgent.StartPublishSession:output_type -> messaging_pb.StartPublishSessionResponse + 3, // 13: messaging_pb.SeaweedMessagingAgent.ClosePublishSession:output_type -> messaging_pb.ClosePublishSessionResponse + 5, // 14: messaging_pb.SeaweedMessagingAgent.PublishRecord:output_type -> messaging_pb.PublishRecordResponse + 7, // 15: messaging_pb.SeaweedMessagingAgent.StartSubscribeSession:output_type -> messaging_pb.StartSubscribeSessionResponse + 9, // 16: messaging_pb.SeaweedMessagingAgent.CloseSubscribeSession:output_type -> messaging_pb.CloseSubscribeSessionResponse + 11, // 17: messaging_pb.SeaweedMessagingAgent.SubscribeRecord:output_type -> messaging_pb.SubscribeRecordResponse + 12, // [12:18] is the sub-list for method output_type + 6, // [6:12] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_mq_agent_proto_init() } +func file_mq_agent_proto_init() { + if File_mq_agent_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_mq_agent_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*StartPublishSessionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*StartPublishSessionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*ClosePublishSessionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*ClosePublishSessionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*PublishRecordRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*PublishRecordResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*StartSubscribeSessionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*StartSubscribeSessionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*CloseSubscribeSessionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*CloseSubscribeSessionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*SubscribeRecordRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_agent_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*SubscribeRecordResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_mq_agent_proto_rawDesc, + NumEnums: 0, + NumMessages: 12, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_mq_agent_proto_goTypes, + DependencyIndexes: file_mq_agent_proto_depIdxs, + MessageInfos: file_mq_agent_proto_msgTypes, + }.Build() + File_mq_agent_proto = out.File + file_mq_agent_proto_rawDesc = nil + file_mq_agent_proto_goTypes = nil + file_mq_agent_proto_depIdxs = nil +} diff --git a/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go b/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go new file mode 100644 index 000000000..241064181 --- /dev/null +++ b/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go @@ -0,0 +1,304 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.5.1 +// - protoc v5.28.3 +// source: mq_agent.proto + +package mq_agent_pb + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + SeaweedMessagingAgent_StartPublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/StartPublishSession" + SeaweedMessagingAgent_ClosePublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/ClosePublishSession" + SeaweedMessagingAgent_PublishRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/PublishRecord" + SeaweedMessagingAgent_StartSubscribeSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/StartSubscribeSession" + SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/CloseSubscribeSession" + SeaweedMessagingAgent_SubscribeRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/SubscribeRecord" +) + +// SeaweedMessagingAgentClient is the client API for SeaweedMessagingAgent service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SeaweedMessagingAgentClient interface { + // Publishing + StartPublishSession(ctx context.Context, in *StartPublishSessionRequest, opts ...grpc.CallOption) (*StartPublishSessionResponse, error) + ClosePublishSession(ctx context.Context, in *ClosePublishSessionRequest, opts ...grpc.CallOption) (*ClosePublishSessionResponse, error) + PublishRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse], error) + // Subscribing + StartSubscribeSession(ctx context.Context, in *StartSubscribeSessionRequest, opts ...grpc.CallOption) (*StartSubscribeSessionResponse, error) + CloseSubscribeSession(ctx context.Context, in *CloseSubscribeSessionRequest, opts ...grpc.CallOption) (*CloseSubscribeSessionResponse, error) + SubscribeRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse], error) +} + +type seaweedMessagingAgentClient struct { + cc grpc.ClientConnInterface +} + +func NewSeaweedMessagingAgentClient(cc grpc.ClientConnInterface) SeaweedMessagingAgentClient { + return &seaweedMessagingAgentClient{cc} +} + +func (c *seaweedMessagingAgentClient) StartPublishSession(ctx context.Context, in *StartPublishSessionRequest, opts ...grpc.CallOption) (*StartPublishSessionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(StartPublishSessionResponse) + err := c.cc.Invoke(ctx, SeaweedMessagingAgent_StartPublishSession_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingAgentClient) ClosePublishSession(ctx context.Context, in *ClosePublishSessionRequest, opts ...grpc.CallOption) (*ClosePublishSessionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ClosePublishSessionResponse) + err := c.cc.Invoke(ctx, SeaweedMessagingAgent_ClosePublishSession_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingAgentClient) PublishRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &SeaweedMessagingAgent_ServiceDesc.Streams[0], SeaweedMessagingAgent_PublishRecord_FullMethodName, cOpts...) + if err != nil { + return nil, err + } + x := &grpc.GenericClientStream[PublishRecordRequest, PublishRecordResponse]{ClientStream: stream} + return x, nil +} + +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type SeaweedMessagingAgent_PublishRecordClient = grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse] + +func (c *seaweedMessagingAgentClient) StartSubscribeSession(ctx context.Context, in *StartSubscribeSessionRequest, opts ...grpc.CallOption) (*StartSubscribeSessionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(StartSubscribeSessionResponse) + err := c.cc.Invoke(ctx, SeaweedMessagingAgent_StartSubscribeSession_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingAgentClient) CloseSubscribeSession(ctx context.Context, in *CloseSubscribeSessionRequest, opts ...grpc.CallOption) (*CloseSubscribeSessionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CloseSubscribeSessionResponse) + err := c.cc.Invoke(ctx, SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *seaweedMessagingAgentClient) SubscribeRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &SeaweedMessagingAgent_ServiceDesc.Streams[1], SeaweedMessagingAgent_SubscribeRecord_FullMethodName, cOpts...) + if err != nil { + return nil, err + } + x := &grpc.GenericClientStream[SubscribeRecordRequest, SubscribeRecordResponse]{ClientStream: stream} + return x, nil +} + +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type SeaweedMessagingAgent_SubscribeRecordClient = grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse] + +// SeaweedMessagingAgentServer is the server API for SeaweedMessagingAgent service. +// All implementations must embed UnimplementedSeaweedMessagingAgentServer +// for forward compatibility. +type SeaweedMessagingAgentServer interface { + // Publishing + StartPublishSession(context.Context, *StartPublishSessionRequest) (*StartPublishSessionResponse, error) + ClosePublishSession(context.Context, *ClosePublishSessionRequest) (*ClosePublishSessionResponse, error) + PublishRecord(grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse]) error + // Subscribing + StartSubscribeSession(context.Context, *StartSubscribeSessionRequest) (*StartSubscribeSessionResponse, error) + CloseSubscribeSession(context.Context, *CloseSubscribeSessionRequest) (*CloseSubscribeSessionResponse, error) + SubscribeRecord(grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse]) error + mustEmbedUnimplementedSeaweedMessagingAgentServer() +} + +// UnimplementedSeaweedMessagingAgentServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedSeaweedMessagingAgentServer struct{} + +func (UnimplementedSeaweedMessagingAgentServer) StartPublishSession(context.Context, *StartPublishSessionRequest) (*StartPublishSessionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartPublishSession not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) ClosePublishSession(context.Context, *ClosePublishSessionRequest) (*ClosePublishSessionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ClosePublishSession not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) PublishRecord(grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse]) error { + return status.Errorf(codes.Unimplemented, "method PublishRecord not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) StartSubscribeSession(context.Context, *StartSubscribeSessionRequest) (*StartSubscribeSessionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartSubscribeSession not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) CloseSubscribeSession(context.Context, *CloseSubscribeSessionRequest) (*CloseSubscribeSessionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CloseSubscribeSession not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) SubscribeRecord(grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse]) error { + return status.Errorf(codes.Unimplemented, "method SubscribeRecord not implemented") +} +func (UnimplementedSeaweedMessagingAgentServer) mustEmbedUnimplementedSeaweedMessagingAgentServer() {} +func (UnimplementedSeaweedMessagingAgentServer) testEmbeddedByValue() {} + +// UnsafeSeaweedMessagingAgentServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SeaweedMessagingAgentServer will +// result in compilation errors. +type UnsafeSeaweedMessagingAgentServer interface { + mustEmbedUnimplementedSeaweedMessagingAgentServer() +} + +func RegisterSeaweedMessagingAgentServer(s grpc.ServiceRegistrar, srv SeaweedMessagingAgentServer) { + // If the following call pancis, it indicates UnimplementedSeaweedMessagingAgentServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&SeaweedMessagingAgent_ServiceDesc, srv) +} + +func _SeaweedMessagingAgent_StartPublishSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StartPublishSessionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingAgentServer).StartPublishSession(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessagingAgent_StartPublishSession_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingAgentServer).StartPublishSession(ctx, req.(*StartPublishSessionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessagingAgent_ClosePublishSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ClosePublishSessionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingAgentServer).ClosePublishSession(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessagingAgent_ClosePublishSession_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingAgentServer).ClosePublishSession(ctx, req.(*ClosePublishSessionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessagingAgent_PublishRecord_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingAgentServer).PublishRecord(&grpc.GenericServerStream[PublishRecordRequest, PublishRecordResponse]{ServerStream: stream}) +} + +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type SeaweedMessagingAgent_PublishRecordServer = grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse] + +func _SeaweedMessagingAgent_StartSubscribeSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StartSubscribeSessionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingAgentServer).StartSubscribeSession(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessagingAgent_StartSubscribeSession_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingAgentServer).StartSubscribeSession(ctx, req.(*StartSubscribeSessionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessagingAgent_CloseSubscribeSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CloseSubscribeSessionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SeaweedMessagingAgentServer).CloseSubscribeSession(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SeaweedMessagingAgentServer).CloseSubscribeSession(ctx, req.(*CloseSubscribeSessionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _SeaweedMessagingAgent_SubscribeRecord_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SeaweedMessagingAgentServer).SubscribeRecord(&grpc.GenericServerStream[SubscribeRecordRequest, SubscribeRecordResponse]{ServerStream: stream}) +} + +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type SeaweedMessagingAgent_SubscribeRecordServer = grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse] + +// SeaweedMessagingAgent_ServiceDesc is the grpc.ServiceDesc for SeaweedMessagingAgent service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var SeaweedMessagingAgent_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "messaging_pb.SeaweedMessagingAgent", + HandlerType: (*SeaweedMessagingAgentServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "StartPublishSession", + Handler: _SeaweedMessagingAgent_StartPublishSession_Handler, + }, + { + MethodName: "ClosePublishSession", + Handler: _SeaweedMessagingAgent_ClosePublishSession_Handler, + }, + { + MethodName: "StartSubscribeSession", + Handler: _SeaweedMessagingAgent_StartSubscribeSession_Handler, + }, + { + MethodName: "CloseSubscribeSession", + Handler: _SeaweedMessagingAgent_CloseSubscribeSession_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "PublishRecord", + Handler: _SeaweedMessagingAgent_PublishRecord_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "SubscribeRecord", + Handler: _SeaweedMessagingAgent_SubscribeRecord_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "mq_agent.proto", +} diff --git a/weed/pb/mq.proto b/weed/pb/mq_broker.proto index 23284b767..a1c9912b0 100644 --- a/weed/pb/mq.proto +++ b/weed/pb/mq_broker.proto @@ -2,7 +2,7 @@ syntax = "proto3"; package messaging_pb; -import "schema.proto"; +import "mq_schema.proto"; option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"; option java_package = "seaweedfs.mq"; @@ -64,43 +64,14 @@ message FindBrokerLeaderResponse { string broker = 1; } -message Topic { - string namespace = 1; - string name = 2; -} -message Partition { - int32 ring_size = 1; - int32 range_start = 2; - int32 range_stop = 3; - int64 unix_time_ns = 4; -} - -message Offset { - Topic topic = 1; - repeated PartitionOffset partition_offsets = 2; -} - -enum PartitionOffsetStartType { - EARLIEST = 0; - EARLIEST_IN_MEMORY = 1; - LATEST = 2; -} - -message PartitionOffset { - Partition partition = 1; - int64 start_ts_ns = 2; - int64 stop_ts_ns = 3; - PartitionOffsetStartType start_type = 4; -} - ////////////////////////////////////////////////// message BrokerStats { int32 cpu_usage_percent = 1; map<string, TopicPartitionStats> stats = 2; } message TopicPartitionStats { - Topic topic = 1; - Partition partition = 2; + schema_pb.Topic topic = 1; + schema_pb.Partition partition = 2; int32 publisher_count = 3; int32 subscriber_count = 4; string follower = 5; @@ -126,7 +97,7 @@ message BalanceTopicsResponse { ////////////////////////////////////////////////// message ConfigureTopicRequest { - Topic topic = 1; + schema_pb.Topic topic = 1; int32 partition_count = 2; schema_pb.RecordType record_type = 3; } @@ -137,23 +108,23 @@ message ConfigureTopicResponse { message ListTopicsRequest { } message ListTopicsResponse { - repeated Topic topics = 1; + repeated schema_pb.Topic topics = 1; } message LookupTopicBrokersRequest { - Topic topic = 1; + schema_pb.Topic topic = 1; } message LookupTopicBrokersResponse { - Topic topic = 1; + schema_pb.Topic topic = 1; repeated BrokerPartitionAssignment broker_partition_assignments = 2; } message BrokerPartitionAssignment { - Partition partition = 1; + schema_pb.Partition partition = 1; string leader_broker = 2; string follower_broker = 3; } message AssignTopicPartitionsRequest { - Topic topic = 1; + schema_pb.Topic topic = 1; repeated BrokerPartitionAssignment broker_partition_assignments = 2; bool is_leader = 3; bool is_draining = 4; @@ -165,7 +136,7 @@ message SubscriberToSubCoordinatorRequest { message InitMessage { string consumer_group = 1; string consumer_group_instance_id = 2; - Topic topic = 3; + schema_pb.Topic topic = 3; // The consumer group instance will be assigned at most max_partition_count partitions. // If the number of partitions is less than the sum of max_partition_count, // the consumer group instance may be assigned partitions less than max_partition_count. @@ -178,10 +149,10 @@ message SubscriberToSubCoordinatorRequest { int32 rebalance_seconds = 5; } message AckUnAssignmentMessage { - Partition partition = 1; + schema_pb.Partition partition = 1; } message AckAssignmentMessage { - Partition partition = 1; + schema_pb.Partition partition = 1; } oneof message { InitMessage init = 1; @@ -194,7 +165,7 @@ message SubscriberToSubCoordinatorResponse { BrokerPartitionAssignment partition_assignment = 1; } message UnAssignment { - Partition partition = 1; + schema_pb.Partition partition = 1; } oneof message { Assignment assignment = 1; @@ -215,8 +186,8 @@ message DataMessage { } message PublishMessageRequest { message InitMessage { - Topic topic = 1; - Partition partition = 2; + schema_pb.Topic topic = 1; + schema_pb.Partition partition = 2; int32 ack_interval = 3; string follower_broker = 4; string publisher_name = 5; // for debugging @@ -233,8 +204,8 @@ message PublishMessageResponse { } message PublishFollowMeRequest { message InitMessage { - Topic topic = 1; - Partition partition = 2; + schema_pb.Topic topic = 1; + schema_pb.Partition partition = 2; } message FlushMessage { int64 ts_ns = 1; @@ -256,11 +227,11 @@ message SubscribeMessageRequest { string consumer_group = 1; string consumer_id = 2; string client_id = 3; - Topic topic = 4; - PartitionOffset partition_offset = 5; + schema_pb.Topic topic = 4; + schema_pb.PartitionOffset partition_offset = 5; string filter = 6; string follower_broker = 7; - int32 concurrency = 8; + int32 sliding_window_size = 8; } message AckMessage { int64 sequence = 1; @@ -284,8 +255,8 @@ message SubscribeMessageResponse { } message SubscribeFollowMeRequest { message InitMessage { - Topic topic = 1; - Partition partition = 2; + schema_pb.Topic topic = 1; + schema_pb.Partition partition = 2; string consumer_group = 3; } message AckMessage { @@ -303,13 +274,13 @@ message SubscribeFollowMeResponse { int64 ack_ts_ns = 1; } message ClosePublishersRequest { - Topic topic = 1; + schema_pb.Topic topic = 1; int64 unix_time_ns = 2; } message ClosePublishersResponse { } message CloseSubscribersRequest { - Topic topic = 1; + schema_pb.Topic topic = 1; int64 unix_time_ns = 2; } message CloseSubscribersResponse { diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq_broker.pb.go index 35516378c..56e3500d1 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq_broker.pb.go @@ -1,8 +1,8 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 -// source: mq.proto +// protoc v5.28.3 +// source: mq_broker.proto package mq_pb @@ -21,55 +21,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -type PartitionOffsetStartType int32 - -const ( - PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0 - PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1 - PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2 -) - -// Enum value maps for PartitionOffsetStartType. -var ( - PartitionOffsetStartType_name = map[int32]string{ - 0: "EARLIEST", - 1: "EARLIEST_IN_MEMORY", - 2: "LATEST", - } - PartitionOffsetStartType_value = map[string]int32{ - "EARLIEST": 0, - "EARLIEST_IN_MEMORY": 1, - "LATEST": 2, - } -) - -func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType { - p := new(PartitionOffsetStartType) - *p = x - return p -} - -func (x PartitionOffsetStartType) String() string { - return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) -} - -func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor { - return file_mq_proto_enumTypes[0].Descriptor() -} - -func (PartitionOffsetStartType) Type() protoreflect.EnumType { - return &file_mq_proto_enumTypes[0] -} - -func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber { - return protoreflect.EnumNumber(x) -} - -// Deprecated: Use PartitionOffsetStartType.Descriptor instead. -func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{0} -} - type FindBrokerLeaderRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -81,7 +32,7 @@ type FindBrokerLeaderRequest struct { func (x *FindBrokerLeaderRequest) Reset() { *x = FindBrokerLeaderRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[0] + mi := &file_mq_broker_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -94,7 +45,7 @@ func (x *FindBrokerLeaderRequest) String() string { func (*FindBrokerLeaderRequest) ProtoMessage() {} func (x *FindBrokerLeaderRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[0] + mi := &file_mq_broker_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -107,7 +58,7 @@ func (x *FindBrokerLeaderRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use FindBrokerLeaderRequest.ProtoReflect.Descriptor instead. func (*FindBrokerLeaderRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{0} + return file_mq_broker_proto_rawDescGZIP(), []int{0} } func (x *FindBrokerLeaderRequest) GetFilerGroup() string { @@ -128,7 +79,7 @@ type FindBrokerLeaderResponse struct { func (x *FindBrokerLeaderResponse) Reset() { *x = FindBrokerLeaderResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[1] + mi := &file_mq_broker_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -141,7 +92,7 @@ func (x *FindBrokerLeaderResponse) String() string { func (*FindBrokerLeaderResponse) ProtoMessage() {} func (x *FindBrokerLeaderResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[1] + mi := &file_mq_broker_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -154,7 +105,7 @@ func (x *FindBrokerLeaderResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use FindBrokerLeaderResponse.ProtoReflect.Descriptor instead. func (*FindBrokerLeaderResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{1} + return file_mq_broker_proto_rawDescGZIP(), []int{1} } func (x *FindBrokerLeaderResponse) GetBroker() string { @@ -164,258 +115,6 @@ func (x *FindBrokerLeaderResponse) GetBroker() string { return "" } -type Topic struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` -} - -func (x *Topic) Reset() { - *x = Topic{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Topic) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Topic) ProtoMessage() {} - -func (x *Topic) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[2] - 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 Topic.ProtoReflect.Descriptor instead. -func (*Topic) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{2} -} - -func (x *Topic) GetNamespace() string { - if x != nil { - return x.Namespace - } - return "" -} - -func (x *Topic) GetName() string { - if x != nil { - return x.Name - } - return "" -} - -type Partition struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - RingSize int32 `protobuf:"varint,1,opt,name=ring_size,json=ringSize,proto3" json:"ring_size,omitempty"` - RangeStart int32 `protobuf:"varint,2,opt,name=range_start,json=rangeStart,proto3" json:"range_start,omitempty"` - RangeStop int32 `protobuf:"varint,3,opt,name=range_stop,json=rangeStop,proto3" json:"range_stop,omitempty"` - UnixTimeNs int64 `protobuf:"varint,4,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` -} - -func (x *Partition) Reset() { - *x = Partition{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Partition) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Partition) ProtoMessage() {} - -func (x *Partition) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[3] - 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 Partition.ProtoReflect.Descriptor instead. -func (*Partition) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{3} -} - -func (x *Partition) GetRingSize() int32 { - if x != nil { - return x.RingSize - } - return 0 -} - -func (x *Partition) GetRangeStart() int32 { - if x != nil { - return x.RangeStart - } - return 0 -} - -func (x *Partition) GetRangeStop() int32 { - if x != nil { - return x.RangeStop - } - return 0 -} - -func (x *Partition) GetUnixTimeNs() int64 { - if x != nil { - return x.UnixTimeNs - } - return 0 -} - -type Offset struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - PartitionOffsets []*PartitionOffset `protobuf:"bytes,2,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"` -} - -func (x *Offset) Reset() { - *x = Offset{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *Offset) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*Offset) ProtoMessage() {} - -func (x *Offset) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[4] - 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 Offset.ProtoReflect.Descriptor instead. -func (*Offset) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{4} -} - -func (x *Offset) GetTopic() *Topic { - if x != nil { - return x.Topic - } - return nil -} - -func (x *Offset) GetPartitionOffsets() []*PartitionOffset { - if x != nil { - return x.PartitionOffsets - } - return nil -} - -type PartitionOffset struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` - StartTsNs int64 `protobuf:"varint,2,opt,name=start_ts_ns,json=startTsNs,proto3" json:"start_ts_ns,omitempty"` - StopTsNs int64 `protobuf:"varint,3,opt,name=stop_ts_ns,json=stopTsNs,proto3" json:"stop_ts_ns,omitempty"` - StartType PartitionOffsetStartType `protobuf:"varint,4,opt,name=start_type,json=startType,proto3,enum=messaging_pb.PartitionOffsetStartType" json:"start_type,omitempty"` -} - -func (x *PartitionOffset) Reset() { - *x = PartitionOffset{} - if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *PartitionOffset) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*PartitionOffset) ProtoMessage() {} - -func (x *PartitionOffset) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use PartitionOffset.ProtoReflect.Descriptor instead. -func (*PartitionOffset) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{5} -} - -func (x *PartitionOffset) GetPartition() *Partition { - if x != nil { - return x.Partition - } - return nil -} - -func (x *PartitionOffset) GetStartTsNs() int64 { - if x != nil { - return x.StartTsNs - } - return 0 -} - -func (x *PartitionOffset) GetStopTsNs() int64 { - if x != nil { - return x.StopTsNs - } - return 0 -} - -func (x *PartitionOffset) GetStartType() PartitionOffsetStartType { - if x != nil { - return x.StartType - } - return PartitionOffsetStartType_EARLIEST -} - // //////////////////////////////////////////////// type BrokerStats struct { state protoimpl.MessageState @@ -429,7 +128,7 @@ type BrokerStats struct { func (x *BrokerStats) Reset() { *x = BrokerStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[6] + mi := &file_mq_broker_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -442,7 +141,7 @@ func (x *BrokerStats) String() string { func (*BrokerStats) ProtoMessage() {} func (x *BrokerStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[6] + mi := &file_mq_broker_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -455,7 +154,7 @@ func (x *BrokerStats) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerStats.ProtoReflect.Descriptor instead. func (*BrokerStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{6} + return file_mq_broker_proto_rawDescGZIP(), []int{2} } func (x *BrokerStats) GetCpuUsagePercent() int32 { @@ -477,17 +176,17 @@ type TopicPartitionStats struct { 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"` - PublisherCount int32 `protobuf:"varint,3,opt,name=publisher_count,json=publisherCount,proto3" json:"publisher_count,omitempty"` - SubscriberCount int32 `protobuf:"varint,4,opt,name=subscriber_count,json=subscriberCount,proto3" json:"subscriber_count,omitempty"` - Follower string `protobuf:"bytes,5,opt,name=follower,proto3" json:"follower,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + PublisherCount int32 `protobuf:"varint,3,opt,name=publisher_count,json=publisherCount,proto3" json:"publisher_count,omitempty"` + SubscriberCount int32 `protobuf:"varint,4,opt,name=subscriber_count,json=subscriberCount,proto3" json:"subscriber_count,omitempty"` + Follower string `protobuf:"bytes,5,opt,name=follower,proto3" json:"follower,omitempty"` } func (x *TopicPartitionStats) Reset() { *x = TopicPartitionStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[7] + mi := &file_mq_broker_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -500,7 +199,7 @@ func (x *TopicPartitionStats) String() string { func (*TopicPartitionStats) ProtoMessage() {} func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[7] + mi := &file_mq_broker_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -513,17 +212,17 @@ func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { // Deprecated: Use TopicPartitionStats.ProtoReflect.Descriptor instead. func (*TopicPartitionStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{7} + return file_mq_broker_proto_rawDescGZIP(), []int{3} } -func (x *TopicPartitionStats) GetTopic() *Topic { +func (x *TopicPartitionStats) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } return nil } -func (x *TopicPartitionStats) GetPartition() *Partition { +func (x *TopicPartitionStats) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -566,7 +265,7 @@ type PublisherToPubBalancerRequest struct { func (x *PublisherToPubBalancerRequest) Reset() { *x = PublisherToPubBalancerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[8] + mi := &file_mq_broker_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -579,7 +278,7 @@ func (x *PublisherToPubBalancerRequest) String() string { func (*PublisherToPubBalancerRequest) ProtoMessage() {} func (x *PublisherToPubBalancerRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[8] + mi := &file_mq_broker_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -592,7 +291,7 @@ func (x *PublisherToPubBalancerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PublisherToPubBalancerRequest.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{8} + return file_mq_broker_proto_rawDescGZIP(), []int{4} } func (m *PublisherToPubBalancerRequest) GetMessage() isPublisherToPubBalancerRequest_Message { @@ -641,7 +340,7 @@ type PublisherToPubBalancerResponse struct { func (x *PublisherToPubBalancerResponse) Reset() { *x = PublisherToPubBalancerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[9] + mi := &file_mq_broker_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -654,7 +353,7 @@ func (x *PublisherToPubBalancerResponse) String() string { func (*PublisherToPubBalancerResponse) ProtoMessage() {} func (x *PublisherToPubBalancerResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[9] + mi := &file_mq_broker_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -667,7 +366,7 @@ func (x *PublisherToPubBalancerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PublisherToPubBalancerResponse.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{9} + return file_mq_broker_proto_rawDescGZIP(), []int{5} } type BalanceTopicsRequest struct { @@ -679,7 +378,7 @@ type BalanceTopicsRequest struct { func (x *BalanceTopicsRequest) Reset() { *x = BalanceTopicsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[10] + mi := &file_mq_broker_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -692,7 +391,7 @@ func (x *BalanceTopicsRequest) String() string { func (*BalanceTopicsRequest) ProtoMessage() {} func (x *BalanceTopicsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[10] + mi := &file_mq_broker_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -705,7 +404,7 @@ func (x *BalanceTopicsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BalanceTopicsRequest.ProtoReflect.Descriptor instead. func (*BalanceTopicsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{10} + return file_mq_broker_proto_rawDescGZIP(), []int{6} } type BalanceTopicsResponse struct { @@ -717,7 +416,7 @@ type BalanceTopicsResponse struct { func (x *BalanceTopicsResponse) Reset() { *x = BalanceTopicsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[11] + mi := &file_mq_broker_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -730,7 +429,7 @@ func (x *BalanceTopicsResponse) String() string { func (*BalanceTopicsResponse) ProtoMessage() {} func (x *BalanceTopicsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[11] + mi := &file_mq_broker_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -743,7 +442,7 @@ func (x *BalanceTopicsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use BalanceTopicsResponse.ProtoReflect.Descriptor instead. func (*BalanceTopicsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{11} + return file_mq_broker_proto_rawDescGZIP(), []int{7} } // //////////////////////////////////////////////// @@ -752,7 +451,7 @@ type ConfigureTopicRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` PartitionCount int32 `protobuf:"varint,2,opt,name=partition_count,json=partitionCount,proto3" json:"partition_count,omitempty"` RecordType *schema_pb.RecordType `protobuf:"bytes,3,opt,name=record_type,json=recordType,proto3" json:"record_type,omitempty"` } @@ -760,7 +459,7 @@ type ConfigureTopicRequest struct { func (x *ConfigureTopicRequest) Reset() { *x = ConfigureTopicRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_broker_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -773,7 +472,7 @@ func (x *ConfigureTopicRequest) String() string { func (*ConfigureTopicRequest) ProtoMessage() {} func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_broker_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -786,10 +485,10 @@ func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicRequest.ProtoReflect.Descriptor instead. func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{12} + return file_mq_broker_proto_rawDescGZIP(), []int{8} } -func (x *ConfigureTopicRequest) GetTopic() *Topic { +func (x *ConfigureTopicRequest) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -822,7 +521,7 @@ type ConfigureTopicResponse struct { func (x *ConfigureTopicResponse) Reset() { *x = ConfigureTopicResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_broker_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -835,7 +534,7 @@ func (x *ConfigureTopicResponse) String() string { func (*ConfigureTopicResponse) ProtoMessage() {} func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_broker_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -848,7 +547,7 @@ func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicResponse.ProtoReflect.Descriptor instead. func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{13} + return file_mq_broker_proto_rawDescGZIP(), []int{9} } func (x *ConfigureTopicResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment { @@ -874,7 +573,7 @@ type ListTopicsRequest struct { func (x *ListTopicsRequest) Reset() { *x = ListTopicsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[14] + mi := &file_mq_broker_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -887,7 +586,7 @@ func (x *ListTopicsRequest) String() string { func (*ListTopicsRequest) ProtoMessage() {} func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[14] + mi := &file_mq_broker_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -900,7 +599,7 @@ func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsRequest.ProtoReflect.Descriptor instead. func (*ListTopicsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{14} + return file_mq_broker_proto_rawDescGZIP(), []int{10} } type ListTopicsResponse struct { @@ -908,13 +607,13 @@ type ListTopicsResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topics []*Topic `protobuf:"bytes,1,rep,name=topics,proto3" json:"topics,omitempty"` + Topics []*schema_pb.Topic `protobuf:"bytes,1,rep,name=topics,proto3" json:"topics,omitempty"` } func (x *ListTopicsResponse) Reset() { *x = ListTopicsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[15] + mi := &file_mq_broker_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -927,7 +626,7 @@ func (x *ListTopicsResponse) String() string { func (*ListTopicsResponse) ProtoMessage() {} func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[15] + mi := &file_mq_broker_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -940,10 +639,10 @@ func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsResponse.ProtoReflect.Descriptor instead. func (*ListTopicsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{15} + return file_mq_broker_proto_rawDescGZIP(), []int{11} } -func (x *ListTopicsResponse) GetTopics() []*Topic { +func (x *ListTopicsResponse) GetTopics() []*schema_pb.Topic { if x != nil { return x.Topics } @@ -955,13 +654,13 @@ type LookupTopicBrokersRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` } func (x *LookupTopicBrokersRequest) Reset() { *x = LookupTopicBrokersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_broker_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -974,7 +673,7 @@ func (x *LookupTopicBrokersRequest) String() string { func (*LookupTopicBrokersRequest) ProtoMessage() {} func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_broker_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -987,10 +686,10 @@ func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersRequest.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{16} + return file_mq_broker_proto_rawDescGZIP(), []int{12} } -func (x *LookupTopicBrokersRequest) GetTopic() *Topic { +func (x *LookupTopicBrokersRequest) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -1002,14 +701,14 @@ type LookupTopicBrokersResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"` } func (x *LookupTopicBrokersResponse) Reset() { *x = LookupTopicBrokersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_broker_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1022,7 +721,7 @@ func (x *LookupTopicBrokersResponse) String() string { func (*LookupTopicBrokersResponse) ProtoMessage() {} func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_broker_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1035,10 +734,10 @@ func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersResponse.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{17} + return file_mq_broker_proto_rawDescGZIP(), []int{13} } -func (x *LookupTopicBrokersResponse) GetTopic() *Topic { +func (x *LookupTopicBrokersResponse) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -1057,15 +756,15 @@ type BrokerPartitionAssignment struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` - LeaderBroker string `protobuf:"bytes,2,opt,name=leader_broker,json=leaderBroker,proto3" json:"leader_broker,omitempty"` - FollowerBroker string `protobuf:"bytes,3,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + LeaderBroker string `protobuf:"bytes,2,opt,name=leader_broker,json=leaderBroker,proto3" json:"leader_broker,omitempty"` + FollowerBroker string `protobuf:"bytes,3,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` } func (x *BrokerPartitionAssignment) Reset() { *x = BrokerPartitionAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[18] + mi := &file_mq_broker_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1078,7 +777,7 @@ func (x *BrokerPartitionAssignment) String() string { func (*BrokerPartitionAssignment) ProtoMessage() {} func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[18] + mi := &file_mq_broker_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1091,10 +790,10 @@ func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerPartitionAssignment.ProtoReflect.Descriptor instead. func (*BrokerPartitionAssignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{18} + return file_mq_broker_proto_rawDescGZIP(), []int{14} } -func (x *BrokerPartitionAssignment) GetPartition() *Partition { +func (x *BrokerPartitionAssignment) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -1120,7 +819,7 @@ type AssignTopicPartitionsRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"` IsLeader bool `protobuf:"varint,3,opt,name=is_leader,json=isLeader,proto3" json:"is_leader,omitempty"` IsDraining bool `protobuf:"varint,4,opt,name=is_draining,json=isDraining,proto3" json:"is_draining,omitempty"` @@ -1129,7 +828,7 @@ type AssignTopicPartitionsRequest struct { func (x *AssignTopicPartitionsRequest) Reset() { *x = AssignTopicPartitionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[19] + mi := &file_mq_broker_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1142,7 +841,7 @@ func (x *AssignTopicPartitionsRequest) String() string { func (*AssignTopicPartitionsRequest) ProtoMessage() {} func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[19] + mi := &file_mq_broker_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1155,10 +854,10 @@ func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsRequest.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{19} + return file_mq_broker_proto_rawDescGZIP(), []int{15} } -func (x *AssignTopicPartitionsRequest) GetTopic() *Topic { +func (x *AssignTopicPartitionsRequest) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -1195,7 +894,7 @@ type AssignTopicPartitionsResponse struct { func (x *AssignTopicPartitionsResponse) Reset() { *x = AssignTopicPartitionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_broker_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1208,7 +907,7 @@ func (x *AssignTopicPartitionsResponse) String() string { func (*AssignTopicPartitionsResponse) ProtoMessage() {} func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_broker_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1221,7 +920,7 @@ func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsResponse.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{20} + return file_mq_broker_proto_rawDescGZIP(), []int{16} } type SubscriberToSubCoordinatorRequest struct { @@ -1240,7 +939,7 @@ type SubscriberToSubCoordinatorRequest struct { func (x *SubscriberToSubCoordinatorRequest) Reset() { *x = SubscriberToSubCoordinatorRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_broker_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1253,7 +952,7 @@ func (x *SubscriberToSubCoordinatorRequest) String() string { func (*SubscriberToSubCoordinatorRequest) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_broker_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1266,7 +965,7 @@ func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message // Deprecated: Use SubscriberToSubCoordinatorRequest.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21} + return file_mq_broker_proto_rawDescGZIP(), []int{17} } func (m *SubscriberToSubCoordinatorRequest) GetMessage() isSubscriberToSubCoordinatorRequest_Message { @@ -1336,7 +1035,7 @@ type SubscriberToSubCoordinatorResponse struct { func (x *SubscriberToSubCoordinatorResponse) Reset() { *x = SubscriberToSubCoordinatorResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[22] + mi := &file_mq_broker_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1349,7 +1048,7 @@ func (x *SubscriberToSubCoordinatorResponse) String() string { func (*SubscriberToSubCoordinatorResponse) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[22] + mi := &file_mq_broker_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1362,7 +1061,7 @@ func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message // Deprecated: Use SubscriberToSubCoordinatorResponse.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22} + return file_mq_broker_proto_rawDescGZIP(), []int{18} } func (m *SubscriberToSubCoordinatorResponse) GetMessage() isSubscriberToSubCoordinatorResponse_Message { @@ -1417,7 +1116,7 @@ type ControlMessage struct { func (x *ControlMessage) Reset() { *x = ControlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[23] + mi := &file_mq_broker_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1430,7 +1129,7 @@ func (x *ControlMessage) String() string { func (*ControlMessage) ProtoMessage() {} func (x *ControlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[23] + mi := &file_mq_broker_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1443,7 +1142,7 @@ func (x *ControlMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use ControlMessage.ProtoReflect.Descriptor instead. func (*ControlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{23} + return file_mq_broker_proto_rawDescGZIP(), []int{19} } func (x *ControlMessage) GetIsClose() bool { @@ -1474,7 +1173,7 @@ type DataMessage struct { func (x *DataMessage) Reset() { *x = DataMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[24] + mi := &file_mq_broker_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1487,7 +1186,7 @@ func (x *DataMessage) String() string { func (*DataMessage) ProtoMessage() {} func (x *DataMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[24] + mi := &file_mq_broker_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1500,7 +1199,7 @@ func (x *DataMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use DataMessage.ProtoReflect.Descriptor instead. func (*DataMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{24} + return file_mq_broker_proto_rawDescGZIP(), []int{20} } func (x *DataMessage) GetKey() []byte { @@ -1546,7 +1245,7 @@ type PublishMessageRequest struct { func (x *PublishMessageRequest) Reset() { *x = PublishMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[25] + mi := &file_mq_broker_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1559,7 +1258,7 @@ func (x *PublishMessageRequest) String() string { func (*PublishMessageRequest) ProtoMessage() {} func (x *PublishMessageRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[25] + mi := &file_mq_broker_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1572,7 +1271,7 @@ func (x *PublishMessageRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishMessageRequest.ProtoReflect.Descriptor instead. func (*PublishMessageRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{25} + return file_mq_broker_proto_rawDescGZIP(), []int{21} } func (m *PublishMessageRequest) GetMessage() isPublishMessageRequest_Message { @@ -1625,7 +1324,7 @@ type PublishMessageResponse struct { func (x *PublishMessageResponse) Reset() { *x = PublishMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_broker_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1638,7 +1337,7 @@ func (x *PublishMessageResponse) String() string { func (*PublishMessageResponse) ProtoMessage() {} func (x *PublishMessageResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_broker_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1651,7 +1350,7 @@ func (x *PublishMessageResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishMessageResponse.ProtoReflect.Descriptor instead. func (*PublishMessageResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26} + return file_mq_broker_proto_rawDescGZIP(), []int{22} } func (x *PublishMessageResponse) GetAckSequence() int64 { @@ -1692,7 +1391,7 @@ type PublishFollowMeRequest struct { func (x *PublishFollowMeRequest) Reset() { *x = PublishFollowMeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_broker_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1705,7 +1404,7 @@ func (x *PublishFollowMeRequest) String() string { func (*PublishFollowMeRequest) ProtoMessage() {} func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_broker_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1718,7 +1417,7 @@ func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishFollowMeRequest.ProtoReflect.Descriptor instead. func (*PublishFollowMeRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27} + return file_mq_broker_proto_rawDescGZIP(), []int{23} } func (m *PublishFollowMeRequest) GetMessage() isPublishFollowMeRequest_Message { @@ -1795,7 +1494,7 @@ type PublishFollowMeResponse struct { func (x *PublishFollowMeResponse) Reset() { *x = PublishFollowMeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_broker_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1808,7 +1507,7 @@ func (x *PublishFollowMeResponse) String() string { func (*PublishFollowMeResponse) ProtoMessage() {} func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_broker_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1821,7 +1520,7 @@ func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PublishFollowMeResponse.ProtoReflect.Descriptor instead. func (*PublishFollowMeResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{28} + return file_mq_broker_proto_rawDescGZIP(), []int{24} } func (x *PublishFollowMeResponse) GetAckTsNs() int64 { @@ -1846,7 +1545,7 @@ type SubscribeMessageRequest struct { func (x *SubscribeMessageRequest) Reset() { *x = SubscribeMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_broker_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1859,7 +1558,7 @@ func (x *SubscribeMessageRequest) String() string { func (*SubscribeMessageRequest) ProtoMessage() {} func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_broker_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1872,7 +1571,7 @@ func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMessageRequest.ProtoReflect.Descriptor instead. func (*SubscribeMessageRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{29} + return file_mq_broker_proto_rawDescGZIP(), []int{25} } func (m *SubscribeMessageRequest) GetMessage() isSubscribeMessageRequest_Message { @@ -1927,7 +1626,7 @@ type SubscribeMessageResponse struct { func (x *SubscribeMessageResponse) Reset() { *x = SubscribeMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[30] + mi := &file_mq_broker_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1940,7 +1639,7 @@ func (x *SubscribeMessageResponse) String() string { func (*SubscribeMessageResponse) ProtoMessage() {} func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[30] + mi := &file_mq_broker_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1953,7 +1652,7 @@ func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeMessageResponse.ProtoReflect.Descriptor instead. func (*SubscribeMessageResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30} + return file_mq_broker_proto_rawDescGZIP(), []int{26} } func (m *SubscribeMessageResponse) GetMessage() isSubscribeMessageResponse_Message { @@ -2009,7 +1708,7 @@ type SubscribeFollowMeRequest struct { func (x *SubscribeFollowMeRequest) Reset() { *x = SubscribeFollowMeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_broker_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2022,7 +1721,7 @@ func (x *SubscribeFollowMeRequest) String() string { func (*SubscribeFollowMeRequest) ProtoMessage() {} func (x *SubscribeFollowMeRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_broker_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2035,7 +1734,7 @@ func (x *SubscribeFollowMeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeFollowMeRequest.ProtoReflect.Descriptor instead. func (*SubscribeFollowMeRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31} + return file_mq_broker_proto_rawDescGZIP(), []int{27} } func (m *SubscribeFollowMeRequest) GetMessage() isSubscribeFollowMeRequest_Message { @@ -2099,7 +1798,7 @@ type SubscribeFollowMeResponse struct { func (x *SubscribeFollowMeResponse) Reset() { *x = SubscribeFollowMeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_broker_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2112,7 +1811,7 @@ func (x *SubscribeFollowMeResponse) String() string { func (*SubscribeFollowMeResponse) ProtoMessage() {} func (x *SubscribeFollowMeResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_broker_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2125,7 +1824,7 @@ func (x *SubscribeFollowMeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscribeFollowMeResponse.ProtoReflect.Descriptor instead. func (*SubscribeFollowMeResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{32} + return file_mq_broker_proto_rawDescGZIP(), []int{28} } func (x *SubscribeFollowMeResponse) GetAckTsNs() int64 { @@ -2140,14 +1839,14 @@ type ClosePublishersRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` } func (x *ClosePublishersRequest) Reset() { *x = ClosePublishersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_broker_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2160,7 +1859,7 @@ func (x *ClosePublishersRequest) String() string { func (*ClosePublishersRequest) ProtoMessage() {} func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_broker_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2173,10 +1872,10 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{33} + return file_mq_broker_proto_rawDescGZIP(), []int{29} } -func (x *ClosePublishersRequest) GetTopic() *Topic { +func (x *ClosePublishersRequest) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -2199,7 +1898,7 @@ type ClosePublishersResponse struct { func (x *ClosePublishersResponse) Reset() { *x = ClosePublishersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_broker_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2212,7 +1911,7 @@ func (x *ClosePublishersResponse) String() string { func (*ClosePublishersResponse) ProtoMessage() {} func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_broker_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2225,7 +1924,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{34} + return file_mq_broker_proto_rawDescGZIP(), []int{30} } type CloseSubscribersRequest struct { @@ -2233,14 +1932,14 @@ type CloseSubscribersRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + UnixTimeNs int64 `protobuf:"varint,2,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` } func (x *CloseSubscribersRequest) Reset() { *x = CloseSubscribersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_broker_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2253,7 +1952,7 @@ func (x *CloseSubscribersRequest) String() string { func (*CloseSubscribersRequest) ProtoMessage() {} func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_broker_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2266,10 +1965,10 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{35} + return file_mq_broker_proto_rawDescGZIP(), []int{31} } -func (x *CloseSubscribersRequest) GetTopic() *Topic { +func (x *CloseSubscribersRequest) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -2292,7 +1991,7 @@ type CloseSubscribersResponse struct { func (x *CloseSubscribersResponse) Reset() { *x = CloseSubscribersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[36] + mi := &file_mq_broker_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2305,7 +2004,7 @@ func (x *CloseSubscribersResponse) String() string { func (*CloseSubscribersResponse) ProtoMessage() {} func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[36] + mi := &file_mq_broker_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2318,7 +2017,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{36} + return file_mq_broker_proto_rawDescGZIP(), []int{32} } type PublisherToPubBalancerRequest_InitMessage struct { @@ -2332,7 +2031,7 @@ type PublisherToPubBalancerRequest_InitMessage struct { func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { *x = PublisherToPubBalancerRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_broker_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2345,7 +2044,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string { func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_broker_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2358,7 +2057,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect. // Deprecated: Use PublisherToPubBalancerRequest_InitMessage.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{8, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{4, 0} } func (x *PublisherToPubBalancerRequest_InitMessage) GetBroker() string { @@ -2373,9 +2072,9 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` - ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"` - Topic *Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` + ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` // The consumer group instance will be assigned at most max_partition_count partitions. // If the number of partitions is less than the sum of max_partition_count, // the consumer group instance may be assigned partitions less than max_partition_count. @@ -2391,7 +2090,7 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_broker_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2404,7 +2103,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_broker_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2417,7 +2116,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protorefl // Deprecated: Use SubscriberToSubCoordinatorRequest_InitMessage.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{17, 0} } func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroup() string { @@ -2434,7 +2133,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroupInstance return "" } -func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetTopic() *Topic { +func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } @@ -2460,13 +2159,13 @@ type SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` } func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_broker_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2479,7 +2178,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) String() stri func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[40] + mi := &file_mq_broker_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2492,10 +2191,10 @@ func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect( // Deprecated: Use SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21, 1} + return file_mq_broker_proto_rawDescGZIP(), []int{17, 1} } -func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) GetPartition() *Partition { +func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -2507,13 +2206,13 @@ type SubscriberToSubCoordinatorRequest_AckAssignmentMessage struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` } func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckAssignmentMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_broker_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2526,7 +2225,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) String() string func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[41] + mi := &file_mq_broker_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2539,10 +2238,10 @@ func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect() // Deprecated: Use SubscriberToSubCoordinatorRequest_AckAssignmentMessage.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21, 2} + return file_mq_broker_proto_rawDescGZIP(), []int{17, 2} } -func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) GetPartition() *Partition { +func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -2560,7 +2259,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct { func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_broker_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2573,7 +2272,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[42] + mi := &file_mq_broker_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2586,7 +2285,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protorefl // Deprecated: Use SubscriberToSubCoordinatorResponse_Assignment.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{18, 0} } func (x *SubscriberToSubCoordinatorResponse_Assignment) GetPartitionAssignment() *BrokerPartitionAssignment { @@ -2601,13 +2300,13 @@ type SubscriberToSubCoordinatorResponse_UnAssignment struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` } func (x *SubscriberToSubCoordinatorResponse_UnAssignment) Reset() { *x = SubscriberToSubCoordinatorResponse_UnAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_broker_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2620,7 +2319,7 @@ func (x *SubscriberToSubCoordinatorResponse_UnAssignment) String() string { func (*SubscriberToSubCoordinatorResponse_UnAssignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[43] + mi := &file_mq_broker_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2633,10 +2332,10 @@ func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protore // Deprecated: Use SubscriberToSubCoordinatorResponse_UnAssignment.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse_UnAssignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22, 1} + return file_mq_broker_proto_rawDescGZIP(), []int{18, 1} } -func (x *SubscriberToSubCoordinatorResponse_UnAssignment) GetPartition() *Partition { +func (x *SubscriberToSubCoordinatorResponse_UnAssignment) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -2648,17 +2347,17 @@ type PublishMessageRequest_InitMessage struct { 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"` - AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"` - FollowerBroker string `protobuf:"bytes,4,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` - PublisherName string `protobuf:"bytes,5,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"` // for debugging + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"` + FollowerBroker string `protobuf:"bytes,4,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` + PublisherName string `protobuf:"bytes,5,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"` // for debugging } func (x *PublishMessageRequest_InitMessage) Reset() { *x = PublishMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_broker_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2671,7 +2370,7 @@ func (x *PublishMessageRequest_InitMessage) String() string { func (*PublishMessageRequest_InitMessage) ProtoMessage() {} func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[44] + mi := &file_mq_broker_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2684,17 +2383,17 @@ func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message // Deprecated: Use PublishMessageRequest_InitMessage.ProtoReflect.Descriptor instead. func (*PublishMessageRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{25, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{21, 0} } -func (x *PublishMessageRequest_InitMessage) GetTopic() *Topic { +func (x *PublishMessageRequest_InitMessage) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } return nil } -func (x *PublishMessageRequest_InitMessage) GetPartition() *Partition { +func (x *PublishMessageRequest_InitMessage) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -2727,14 +2426,14 @@ type PublishFollowMeRequest_InitMessage struct { 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"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` } func (x *PublishFollowMeRequest_InitMessage) Reset() { *x = PublishFollowMeRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_broker_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2747,7 +2446,7 @@ func (x *PublishFollowMeRequest_InitMessage) String() string { func (*PublishFollowMeRequest_InitMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[45] + mi := &file_mq_broker_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2760,17 +2459,17 @@ func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message // Deprecated: Use PublishFollowMeRequest_InitMessage.ProtoReflect.Descriptor instead. func (*PublishFollowMeRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{23, 0} } -func (x *PublishFollowMeRequest_InitMessage) GetTopic() *Topic { +func (x *PublishFollowMeRequest_InitMessage) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } return nil } -func (x *PublishFollowMeRequest_InitMessage) GetPartition() *Partition { +func (x *PublishFollowMeRequest_InitMessage) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -2788,7 +2487,7 @@ type PublishFollowMeRequest_FlushMessage struct { func (x *PublishFollowMeRequest_FlushMessage) Reset() { *x = PublishFollowMeRequest_FlushMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[46] + mi := &file_mq_broker_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2801,7 +2500,7 @@ func (x *PublishFollowMeRequest_FlushMessage) String() string { func (*PublishFollowMeRequest_FlushMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[46] + mi := &file_mq_broker_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2814,7 +2513,7 @@ func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Messag // Deprecated: Use PublishFollowMeRequest_FlushMessage.ProtoReflect.Descriptor instead. func (*PublishFollowMeRequest_FlushMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27, 1} + return file_mq_broker_proto_rawDescGZIP(), []int{23, 1} } func (x *PublishFollowMeRequest_FlushMessage) GetTsNs() int64 { @@ -2833,7 +2532,7 @@ type PublishFollowMeRequest_CloseMessage struct { func (x *PublishFollowMeRequest_CloseMessage) Reset() { *x = PublishFollowMeRequest_CloseMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[47] + mi := &file_mq_broker_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2846,7 +2545,7 @@ func (x *PublishFollowMeRequest_CloseMessage) String() string { func (*PublishFollowMeRequest_CloseMessage) ProtoMessage() {} func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[47] + mi := &file_mq_broker_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2859,7 +2558,7 @@ func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Messag // Deprecated: Use PublishFollowMeRequest_CloseMessage.ProtoReflect.Descriptor instead. func (*PublishFollowMeRequest_CloseMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27, 2} + return file_mq_broker_proto_rawDescGZIP(), []int{23, 2} } type SubscribeMessageRequest_InitMessage struct { @@ -2867,20 +2566,20 @@ type SubscribeMessageRequest_InitMessage struct { 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"` - ClientId string `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_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"` - Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` - FollowerBroker string `protobuf:"bytes,7,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` - Concurrency int32 `protobuf:"varint,8,opt,name=concurrency,proto3" json:"concurrency,omitempty"` + 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"` + ClientId string `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffset *schema_pb.PartitionOffset `protobuf:"bytes,5,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"` + Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` + FollowerBroker string `protobuf:"bytes,7,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` + SlidingWindowSize int32 `protobuf:"varint,8,opt,name=sliding_window_size,json=slidingWindowSize,proto3" json:"sliding_window_size,omitempty"` } func (x *SubscribeMessageRequest_InitMessage) Reset() { *x = SubscribeMessageRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[48] + mi := &file_mq_broker_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2893,7 +2592,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string { func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[48] + mi := &file_mq_broker_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2906,7 +2605,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{29, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{25, 0} } func (x *SubscribeMessageRequest_InitMessage) GetConsumerGroup() string { @@ -2930,14 +2629,14 @@ func (x *SubscribeMessageRequest_InitMessage) GetClientId() string { return "" } -func (x *SubscribeMessageRequest_InitMessage) GetTopic() *Topic { +func (x *SubscribeMessageRequest_InitMessage) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } return nil } -func (x *SubscribeMessageRequest_InitMessage) GetPartitionOffset() *PartitionOffset { +func (x *SubscribeMessageRequest_InitMessage) GetPartitionOffset() *schema_pb.PartitionOffset { if x != nil { return x.PartitionOffset } @@ -2958,9 +2657,9 @@ func (x *SubscribeMessageRequest_InitMessage) GetFollowerBroker() string { return "" } -func (x *SubscribeMessageRequest_InitMessage) GetConcurrency() int32 { +func (x *SubscribeMessageRequest_InitMessage) GetSlidingWindowSize() int32 { if x != nil { - return x.Concurrency + return x.SlidingWindowSize } return 0 } @@ -2977,7 +2676,7 @@ type SubscribeMessageRequest_AckMessage struct { func (x *SubscribeMessageRequest_AckMessage) Reset() { *x = SubscribeMessageRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[49] + mi := &file_mq_broker_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2990,7 +2689,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string { func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[49] + mi := &file_mq_broker_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3003,7 +2702,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{29, 1} + return file_mq_broker_proto_rawDescGZIP(), []int{25, 1} } func (x *SubscribeMessageRequest_AckMessage) GetSequence() int64 { @@ -3033,7 +2732,7 @@ type SubscribeMessageResponse_SubscribeCtrlMessage struct { func (x *SubscribeMessageResponse_SubscribeCtrlMessage) Reset() { *x = SubscribeMessageResponse_SubscribeCtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[50] + mi := &file_mq_broker_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3046,7 +2745,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) String() string { func (*SubscribeMessageResponse_SubscribeCtrlMessage) ProtoMessage() {} func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[50] + mi := &file_mq_broker_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3059,7 +2758,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protorefl // Deprecated: Use SubscribeMessageResponse_SubscribeCtrlMessage.ProtoReflect.Descriptor instead. func (*SubscribeMessageResponse_SubscribeCtrlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{30, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{26, 0} } func (x *SubscribeMessageResponse_SubscribeCtrlMessage) GetError() string { @@ -3088,15 +2787,15 @@ type SubscribeFollowMeRequest_InitMessage struct { 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"` - ConsumerGroup string `protobuf:"bytes,3,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *schema_pb.Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + ConsumerGroup string `protobuf:"bytes,3,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` } func (x *SubscribeFollowMeRequest_InitMessage) Reset() { *x = SubscribeFollowMeRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[51] + mi := &file_mq_broker_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3109,7 +2808,7 @@ func (x *SubscribeFollowMeRequest_InitMessage) String() string { func (*SubscribeFollowMeRequest_InitMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[51] + mi := &file_mq_broker_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3122,17 +2821,17 @@ func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Messa // Deprecated: Use SubscribeFollowMeRequest_InitMessage.ProtoReflect.Descriptor instead. func (*SubscribeFollowMeRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31, 0} + return file_mq_broker_proto_rawDescGZIP(), []int{27, 0} } -func (x *SubscribeFollowMeRequest_InitMessage) GetTopic() *Topic { +func (x *SubscribeFollowMeRequest_InitMessage) GetTopic() *schema_pb.Topic { if x != nil { return x.Topic } return nil } -func (x *SubscribeFollowMeRequest_InitMessage) GetPartition() *Partition { +func (x *SubscribeFollowMeRequest_InitMessage) GetPartition() *schema_pb.Partition { if x != nil { return x.Partition } @@ -3157,7 +2856,7 @@ type SubscribeFollowMeRequest_AckMessage struct { func (x *SubscribeFollowMeRequest_AckMessage) Reset() { *x = SubscribeFollowMeRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[52] + mi := &file_mq_broker_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3170,7 +2869,7 @@ func (x *SubscribeFollowMeRequest_AckMessage) String() string { func (*SubscribeFollowMeRequest_AckMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[52] + mi := &file_mq_broker_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3183,7 +2882,7 @@ func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Messag // Deprecated: Use SubscribeFollowMeRequest_AckMessage.ProtoReflect.Descriptor instead. func (*SubscribeFollowMeRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31, 1} + return file_mq_broker_proto_rawDescGZIP(), []int{27, 1} } func (x *SubscribeFollowMeRequest_AckMessage) GetTsNs() int64 { @@ -3202,7 +2901,7 @@ type SubscribeFollowMeRequest_CloseMessage struct { func (x *SubscribeFollowMeRequest_CloseMessage) Reset() { *x = SubscribeFollowMeRequest_CloseMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[53] + mi := &file_mq_broker_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3215,7 +2914,7 @@ func (x *SubscribeFollowMeRequest_CloseMessage) String() string { func (*SubscribeFollowMeRequest_CloseMessage) ProtoMessage() {} func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[53] + mi := &file_mq_broker_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3228,221 +2927,186 @@ func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Mess // Deprecated: Use SubscribeFollowMeRequest_CloseMessage.ProtoReflect.Descriptor instead. func (*SubscribeFollowMeRequest_CloseMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{31, 2} -} - -var File_mq_proto protoreflect.FileDescriptor - -var file_mq_proto_rawDesc = []byte{ - 0x0a, 0x08, 0x6d, 0x71, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x3a, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x47, 0x72, 0x6f, - 0x75, 0x70, 0x22, 0x32, 0x0a, 0x18, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, - 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x22, 0x39, 0x0a, 0x05, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x1b, 0x0a, 0x09, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x08, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x1d, 0x0a, - 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x20, 0x0a, 0x0c, - 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x7f, - 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 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, 0x4a, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 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, 0x10, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x22, - 0xcd, 0x01, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, - 0x73, 0x65, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0b, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x1c, 0x0a, 0x0a, 0x73, 0x74, - 0x6f, 0x70, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, - 0x73, 0x74, 0x6f, 0x70, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x45, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, - 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 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, 0x53, 0x74, 0x61, 0x72, 0x74, - 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, - 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, - 0x2a, 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, 0x72, - 0x63, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, 0x75, 0x55, - 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe7, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, - 0x0a, 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, - 0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x43, 0x6f, 0x75, - 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x22, 0xd3, - 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, - 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, + return file_mq_broker_proto_rawDescGZIP(), []int{27, 2} +} + +var File_mq_broker_proto protoreflect.FileDescriptor + +var file_mq_broker_proto_rawDesc = []byte{ + 0x0a, 0x0f, 0x6d, 0x71, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x1a, + 0x0f, 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0x3a, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x66, + 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0x32, 0x0a, 0x18, + 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x12, 0x2a, 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, + 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, 0x75, + 0x55, 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe1, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x26, 0x0a, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 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, 0x27, 0x0a, 0x0f, 0x70, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x43, 0x6f, 0x75, + 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x73, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x22, 0xd3, 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, - 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, - 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, - 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x17, - 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa3, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x27, 0x0a, 0x0f, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, - 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x22, 0xbb, 0x01, - 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, + 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, + 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, + 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, + 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, + 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x17, 0x0a, 0x15, 0x42, 0x61, 0x6c, + 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0xa0, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x26, 0x0a, 0x05, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x36, 0x0a, + 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x22, 0xbb, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0b, 0x72, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, + 0x79, 0x70, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x3e, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x28, + 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x43, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, + 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, + 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x22, 0xaf, 0x01, + 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x05, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, + 0x9d, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x32, 0x0a, + 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x22, + 0xef, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x4c, - 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x06, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x22, 0x46, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, - 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x22, 0xb2, 0x01, 0x0a, 0x1a, - 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x22, 0xa0, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, - 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, - 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x6f, - 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x22, 0xf2, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, - 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, - 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, - 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, - 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, - 0x44, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x22, 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x82, 0x06, 0x0a, 0x21, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, - 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x51, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, - 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, - 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, - 0x69, 0x74, 0x12, 0x6d, 0x0a, 0x0e, 0x61, 0x63, 0x6b, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, - 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x48, 0x00, 0x52, 0x0d, 0x61, 0x63, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, - 0x74, 0x12, 0x74, 0x0a, 0x11, 0x61, 0x63, 0x6b, 0x5f, 0x75, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, - 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, - 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x61, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0xf9, 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, 0x3b, - 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, - 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2b, 0x0a, 0x11, 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61, - 0x6e, 0x63, 0x65, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x10, 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x1a, 0x4f, 0x0a, 0x16, 0x41, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a, - 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4d, 0x0a, 0x14, 0x41, 0x63, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a, 0x09, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, + 0x67, 0x22, 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0xf9, 0x05, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, + 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x6d, 0x0a, 0x0e, 0x61, + 0x63, 0x6b, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, + 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x63, 0x6b, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x74, 0x0a, 0x11, 0x61, 0x63, + 0x6b, 0x5f, 0x75, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, + 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, + 0x0f, 0x61, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x1a, 0xf6, 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, 0x3b, 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, + 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, + 0x63, 0x65, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2e, 0x0a, 0x13, + 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2b, 0x0a, 0x11, + 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x1a, 0x4c, 0x0a, 0x16, 0x41, 0x63, 0x6b, + 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4a, 0x0a, 0x14, 0x41, 0x63, 0x6b, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, + 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa5, + 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa2, 0x03, 0x0a, 0x22, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, @@ -3464,468 +3128,455 @@ var file_mq_proto_rawDesc = []byte{ 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x13, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x45, 0x0a, 0x0c, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x52, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, - 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63, - 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x75, 0x62, - 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x7c, 0x0a, 0x0b, 0x44, 0x61, - 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x22, 0xff, 0x02, 0x0a, 0x15, 0x50, 0x75, 0x62, + 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x42, 0x0a, 0x0c, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x52, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x7c, 0x0a, 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x13, + 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, + 0x73, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x04, 0x63, 0x74, 0x72, 0x6c, 0x22, 0xf9, 0x02, 0x0a, 0x15, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x45, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 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, 0x12, 0x45, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2f, 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, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, - 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0xe2, 0x01, 0x0a, 0x0b, 0x49, - 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, - 0x61, 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, - 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, - 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, - 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, - 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x74, 0x0a, 0x16, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, - 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, - 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, - 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, - 0x22, 0xd8, 0x03, 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, 0x46, 0x0a, 0x04, 0x69, - 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x2e, - 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, - 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, - 0x64, 0x61, 0x74, 0x61, 0x12, 0x49, 0x0a, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x12, - 0x49, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, - 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, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, 0x6f, 0x0a, 0x0b, 0x49, 0x6e, - 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x23, 0x0a, 0x0c, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, - 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, - 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x35, 0x0a, 0x17, 0x50, - 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x73, - 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73, - 0x4e, 0x73, 0x22, 0xbc, 0x04, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47, - 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 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, 0xca, 0x02, - 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, - 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 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, 0x12, - 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, - 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, - 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, - 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x1a, 0x3a, 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, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x22, 0xa7, 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, 0x51, - 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0x7c, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0xcf, 0x03, 0x0a, 0x18, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, - 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 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, 0x45, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, + 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, + 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, + 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0xdc, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, + 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x49, 0x6e, + 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, + 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x22, 0x74, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, + 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, + 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, 0x75, + 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0xd2, 0x03, 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, 0x46, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x30, 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, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x49, 0x0a, 0x05, 0x66, + 0x6c, 0x75, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, + 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, + 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x12, 0x49, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x2e, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x1a, 0x69, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x23, 0x0a, 0x0c, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a, 0x05, + 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, + 0x73, 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x35, 0x0a, 0x17, + 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, + 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x63, 0x6b, 0x54, + 0x73, 0x4e, 0x73, 0x22, 0xc4, 0x04, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x47, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 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, 0xd2, + 0x02, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, + 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x45, 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, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 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, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x6f, + 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x12, 0x2e, 0x0a, 0x13, 0x73, 0x6c, 0x69, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x77, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x11, 0x73, 0x6c, 0x69, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, + 0x69, 0x7a, 0x65, 0x1a, 0x3a, 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, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, + 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa7, 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, 0x51, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0x7c, 0x0a, 0x14, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0xc9, 0x03, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x48, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 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, 0x12, 0x4b, 0x0a, 0x05, 0x63, 0x6c, 0x6f, - 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 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, 0x45, 0x0a, 0x03, 0x61, + 0x63, 0x6b, 0x18, 0x02, 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, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, - 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, 0x96, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x1a, - 0x21, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a, - 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, - 0x4e, 0x73, 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x37, 0x0a, - 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, - 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, - 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, - 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 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, + 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, + 0x63, 0x6b, 0x12, 0x4b, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, + 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, + 0x90, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, + 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x1a, 0x21, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x04, 0x74, 0x73, 0x4e, 0x73, 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x22, 0x37, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, + 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, + 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x62, 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, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 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, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x63, 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, 0xca, 0x0b, 0x0a, 0x10, 0x53, - 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, - 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, - 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, 0x2b, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, - 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, - 0x5a, 0x0a, 0x0d, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x12, 0x22, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c, - 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x65, 0x73, 0x74, 0x12, 0x26, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, + 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x1a, 0x0a, + 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xca, 0x0b, 0x0a, 0x10, 0x53, 0x65, + 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, + 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, + 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, 0x2b, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x5a, + 0x0a, 0x0d, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, + 0x22, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, + 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c, 0x69, + 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x52, 0x65, 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, + 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, 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, - 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, - 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, - 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, - 0x0a, 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x72, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, - 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, - 0x6f, 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, 0x24, + 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, + 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, + 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, + 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, + 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, - 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 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, 0x67, - 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, + 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, 0x67, 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, 0x28, 0x01, 0x30, 0x01, 0x12, 0x64, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x68, 0x0a, 0x11, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, + 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, + 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, + 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 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, 0x28, 0x01, 0x30, 0x01, 0x12, 0x64, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x68, 0x0a, - 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, - 0x4d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, - 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 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, + 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x28, 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 ( - file_mq_proto_rawDescOnce sync.Once - file_mq_proto_rawDescData = file_mq_proto_rawDesc + file_mq_broker_proto_rawDescOnce sync.Once + file_mq_broker_proto_rawDescData = file_mq_broker_proto_rawDesc ) -func file_mq_proto_rawDescGZIP() []byte { - file_mq_proto_rawDescOnce.Do(func() { - file_mq_proto_rawDescData = protoimpl.X.CompressGZIP(file_mq_proto_rawDescData) +func file_mq_broker_proto_rawDescGZIP() []byte { + file_mq_broker_proto_rawDescOnce.Do(func() { + file_mq_broker_proto_rawDescData = protoimpl.X.CompressGZIP(file_mq_broker_proto_rawDescData) }) - return file_mq_proto_rawDescData -} - -var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 54) -var file_mq_proto_goTypes = []any{ - (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType - (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest - (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse - (*Topic)(nil), // 3: messaging_pb.Topic - (*Partition)(nil), // 4: messaging_pb.Partition - (*Offset)(nil), // 5: messaging_pb.Offset - (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset - (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats - (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats - (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest - (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse - (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest - (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse - (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest - (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse - (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest - (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse - (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest - (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse - (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment - (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest - (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse - (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest - (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse - (*ControlMessage)(nil), // 24: messaging_pb.ControlMessage - (*DataMessage)(nil), // 25: messaging_pb.DataMessage - (*PublishMessageRequest)(nil), // 26: messaging_pb.PublishMessageRequest - (*PublishMessageResponse)(nil), // 27: messaging_pb.PublishMessageResponse - (*PublishFollowMeRequest)(nil), // 28: messaging_pb.PublishFollowMeRequest - (*PublishFollowMeResponse)(nil), // 29: messaging_pb.PublishFollowMeResponse - (*SubscribeMessageRequest)(nil), // 30: messaging_pb.SubscribeMessageRequest - (*SubscribeMessageResponse)(nil), // 31: messaging_pb.SubscribeMessageResponse - (*SubscribeFollowMeRequest)(nil), // 32: messaging_pb.SubscribeFollowMeRequest - (*SubscribeFollowMeResponse)(nil), // 33: messaging_pb.SubscribeFollowMeResponse - (*ClosePublishersRequest)(nil), // 34: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 35: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 36: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 37: messaging_pb.CloseSubscribersResponse - nil, // 38: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 39: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 41: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage - (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 42: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 43: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 44: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment - (*PublishMessageRequest_InitMessage)(nil), // 45: messaging_pb.PublishMessageRequest.InitMessage - (*PublishFollowMeRequest_InitMessage)(nil), // 46: messaging_pb.PublishFollowMeRequest.InitMessage - (*PublishFollowMeRequest_FlushMessage)(nil), // 47: messaging_pb.PublishFollowMeRequest.FlushMessage - (*PublishFollowMeRequest_CloseMessage)(nil), // 48: messaging_pb.PublishFollowMeRequest.CloseMessage - (*SubscribeMessageRequest_InitMessage)(nil), // 49: messaging_pb.SubscribeMessageRequest.InitMessage - (*SubscribeMessageRequest_AckMessage)(nil), // 50: messaging_pb.SubscribeMessageRequest.AckMessage - (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 51: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage - (*SubscribeFollowMeRequest_InitMessage)(nil), // 52: messaging_pb.SubscribeFollowMeRequest.InitMessage - (*SubscribeFollowMeRequest_AckMessage)(nil), // 53: messaging_pb.SubscribeFollowMeRequest.AckMessage - (*SubscribeFollowMeRequest_CloseMessage)(nil), // 54: messaging_pb.SubscribeFollowMeRequest.CloseMessage - (*schema_pb.RecordType)(nil), // 55: schema_pb.RecordType -} -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 - 38, // 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 - 39, // 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 - 55, // 10: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType - 19, // 11: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 55, // 12: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType - 3, // 13: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic - 3, // 14: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic - 3, // 15: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic - 19, // 16: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 4, // 17: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition - 3, // 18: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic - 19, // 19: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 40, // 20: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - 42, // 21: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage - 41, // 22: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage - 43, // 23: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 44, // 24: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment - 24, // 25: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage - 45, // 26: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage - 25, // 27: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage - 46, // 28: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage - 25, // 29: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage - 47, // 30: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage - 48, // 31: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage - 49, // 32: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage - 50, // 33: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage - 51, // 34: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage - 25, // 35: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage - 52, // 36: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage - 53, // 37: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage - 54, // 38: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage - 3, // 39: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic - 3, // 40: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic - 8, // 41: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 3, // 42: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 43: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> messaging_pb.Partition - 4, // 44: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> messaging_pb.Partition - 19, // 45: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment - 4, // 46: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> messaging_pb.Partition - 3, // 47: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 48: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 49: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 50: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 3, // 51: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 6, // 52: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset - 3, // 53: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 4, // 54: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 1, // 55: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 9, // 56: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 11, // 57: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 15, // 58: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 13, // 59: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 17, // 60: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 20, // 61: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 34, // 62: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 36, // 63: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 22, // 64: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 26, // 65: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest - 30, // 66: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest - 28, // 67: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest - 32, // 68: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest - 2, // 69: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 10, // 70: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 12, // 71: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 16, // 72: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 14, // 73: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 18, // 74: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 21, // 75: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 35, // 76: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 37, // 77: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 23, // 78: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 27, // 79: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse - 31, // 80: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse - 29, // 81: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse - 33, // 82: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse - 69, // [69:83] is the sub-list for method output_type - 55, // [55:69] is the sub-list for method input_type - 55, // [55:55] is the sub-list for extension type_name - 55, // [55:55] is the sub-list for extension extendee - 0, // [0:55] is the sub-list for field type_name -} - -func init() { file_mq_proto_init() } -func file_mq_proto_init() { - if File_mq_proto != nil { + return file_mq_broker_proto_rawDescData +} + +var file_mq_broker_proto_msgTypes = make([]protoimpl.MessageInfo, 50) +var file_mq_broker_proto_goTypes = []any{ + (*FindBrokerLeaderRequest)(nil), // 0: messaging_pb.FindBrokerLeaderRequest + (*FindBrokerLeaderResponse)(nil), // 1: messaging_pb.FindBrokerLeaderResponse + (*BrokerStats)(nil), // 2: messaging_pb.BrokerStats + (*TopicPartitionStats)(nil), // 3: messaging_pb.TopicPartitionStats + (*PublisherToPubBalancerRequest)(nil), // 4: messaging_pb.PublisherToPubBalancerRequest + (*PublisherToPubBalancerResponse)(nil), // 5: messaging_pb.PublisherToPubBalancerResponse + (*BalanceTopicsRequest)(nil), // 6: messaging_pb.BalanceTopicsRequest + (*BalanceTopicsResponse)(nil), // 7: messaging_pb.BalanceTopicsResponse + (*ConfigureTopicRequest)(nil), // 8: messaging_pb.ConfigureTopicRequest + (*ConfigureTopicResponse)(nil), // 9: messaging_pb.ConfigureTopicResponse + (*ListTopicsRequest)(nil), // 10: messaging_pb.ListTopicsRequest + (*ListTopicsResponse)(nil), // 11: messaging_pb.ListTopicsResponse + (*LookupTopicBrokersRequest)(nil), // 12: messaging_pb.LookupTopicBrokersRequest + (*LookupTopicBrokersResponse)(nil), // 13: messaging_pb.LookupTopicBrokersResponse + (*BrokerPartitionAssignment)(nil), // 14: messaging_pb.BrokerPartitionAssignment + (*AssignTopicPartitionsRequest)(nil), // 15: messaging_pb.AssignTopicPartitionsRequest + (*AssignTopicPartitionsResponse)(nil), // 16: messaging_pb.AssignTopicPartitionsResponse + (*SubscriberToSubCoordinatorRequest)(nil), // 17: messaging_pb.SubscriberToSubCoordinatorRequest + (*SubscriberToSubCoordinatorResponse)(nil), // 18: messaging_pb.SubscriberToSubCoordinatorResponse + (*ControlMessage)(nil), // 19: messaging_pb.ControlMessage + (*DataMessage)(nil), // 20: messaging_pb.DataMessage + (*PublishMessageRequest)(nil), // 21: messaging_pb.PublishMessageRequest + (*PublishMessageResponse)(nil), // 22: messaging_pb.PublishMessageResponse + (*PublishFollowMeRequest)(nil), // 23: messaging_pb.PublishFollowMeRequest + (*PublishFollowMeResponse)(nil), // 24: messaging_pb.PublishFollowMeResponse + (*SubscribeMessageRequest)(nil), // 25: messaging_pb.SubscribeMessageRequest + (*SubscribeMessageResponse)(nil), // 26: messaging_pb.SubscribeMessageResponse + (*SubscribeFollowMeRequest)(nil), // 27: messaging_pb.SubscribeFollowMeRequest + (*SubscribeFollowMeResponse)(nil), // 28: messaging_pb.SubscribeFollowMeResponse + (*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_AckUnAssignmentMessage)(nil), // 36: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage + (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 37: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage + (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment + (*PublishMessageRequest_InitMessage)(nil), // 40: messaging_pb.PublishMessageRequest.InitMessage + (*PublishFollowMeRequest_InitMessage)(nil), // 41: messaging_pb.PublishFollowMeRequest.InitMessage + (*PublishFollowMeRequest_FlushMessage)(nil), // 42: messaging_pb.PublishFollowMeRequest.FlushMessage + (*PublishFollowMeRequest_CloseMessage)(nil), // 43: messaging_pb.PublishFollowMeRequest.CloseMessage + (*SubscribeMessageRequest_InitMessage)(nil), // 44: messaging_pb.SubscribeMessageRequest.InitMessage + (*SubscribeMessageRequest_AckMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.AckMessage + (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 46: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage + (*SubscribeFollowMeRequest_InitMessage)(nil), // 47: messaging_pb.SubscribeFollowMeRequest.InitMessage + (*SubscribeFollowMeRequest_AckMessage)(nil), // 48: messaging_pb.SubscribeFollowMeRequest.AckMessage + (*SubscribeFollowMeRequest_CloseMessage)(nil), // 49: messaging_pb.SubscribeFollowMeRequest.CloseMessage + (*schema_pb.Topic)(nil), // 50: schema_pb.Topic + (*schema_pb.Partition)(nil), // 51: schema_pb.Partition + (*schema_pb.RecordType)(nil), // 52: schema_pb.RecordType + (*schema_pb.PartitionOffset)(nil), // 53: schema_pb.PartitionOffset +} +var file_mq_broker_proto_depIdxs = []int32{ + 33, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 50, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> schema_pb.Topic + 51, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> schema_pb.Partition + 34, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 2, // 4: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats + 50, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> schema_pb.Topic + 52, // 6: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType + 14, // 7: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 52, // 8: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType + 50, // 9: messaging_pb.ListTopicsResponse.topics:type_name -> schema_pb.Topic + 50, // 10: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> schema_pb.Topic + 50, // 11: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> schema_pb.Topic + 14, // 12: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 51, // 13: messaging_pb.BrokerPartitionAssignment.partition:type_name -> schema_pb.Partition + 50, // 14: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> schema_pb.Topic + 14, // 15: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 35, // 16: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + 37, // 17: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage + 36, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage + 38, // 19: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + 39, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment + 19, // 21: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage + 40, // 22: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 20, // 23: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage + 41, // 24: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage + 20, // 25: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage + 42, // 26: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage + 43, // 27: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage + 44, // 28: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage + 45, // 29: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage + 46, // 30: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage + 20, // 31: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage + 47, // 32: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage + 48, // 33: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage + 49, // 34: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage + 50, // 35: messaging_pb.ClosePublishersRequest.topic:type_name -> schema_pb.Topic + 50, // 36: messaging_pb.CloseSubscribersRequest.topic:type_name -> schema_pb.Topic + 3, // 37: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats + 50, // 38: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> schema_pb.Topic + 51, // 39: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> schema_pb.Partition + 51, // 40: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> schema_pb.Partition + 14, // 41: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment + 51, // 42: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> schema_pb.Partition + 50, // 43: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic + 51, // 44: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> schema_pb.Partition + 50, // 45: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic + 51, // 46: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition + 50, // 47: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic + 53, // 48: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> schema_pb.PartitionOffset + 50, // 49: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic + 51, // 50: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition + 0, // 51: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 4, // 52: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 6, // 53: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 10, // 54: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 8, // 55: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 12, // 56: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 15, // 57: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 29, // 58: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 31, // 59: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 17, // 60: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 21, // 61: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 25, // 62: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 23, // 63: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 27, // 64: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest + 1, // 65: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 5, // 66: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 7, // 67: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 11, // 68: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 9, // 69: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 13, // 70: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 16, // 71: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 30, // 72: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 32, // 73: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 18, // 74: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 22, // 75: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 26, // 76: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 24, // 77: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 28, // 78: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse + 65, // [65:79] is the sub-list for method output_type + 51, // [51:65] is the sub-list for method input_type + 51, // [51:51] is the sub-list for extension type_name + 51, // [51:51] is the sub-list for extension extendee + 0, // [0:51] is the sub-list for field type_name +} + +func init() { file_mq_broker_proto_init() } +func file_mq_broker_proto_init() { + if File_mq_broker_proto != nil { return } if !protoimpl.UnsafeEnabled { - file_mq_proto_msgTypes[0].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*FindBrokerLeaderRequest); i { case 0: return &v.state @@ -3937,7 +3588,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[1].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*FindBrokerLeaderResponse); i { case 0: return &v.state @@ -3949,55 +3600,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*Topic); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*Partition); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*Offset); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[5].Exporter = func(v any, i int) any { - switch v := v.(*PartitionOffset); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_mq_proto_msgTypes[6].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[2].Exporter = func(v any, i int) any { switch v := v.(*BrokerStats); i { case 0: return &v.state @@ -4009,7 +3612,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[7].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[3].Exporter = func(v any, i int) any { switch v := v.(*TopicPartitionStats); i { case 0: return &v.state @@ -4021,7 +3624,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[8].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*PublisherToPubBalancerRequest); i { case 0: return &v.state @@ -4033,7 +3636,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[9].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*PublisherToPubBalancerResponse); i { case 0: return &v.state @@ -4045,7 +3648,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[10].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[6].Exporter = func(v any, i int) any { switch v := v.(*BalanceTopicsRequest); i { case 0: return &v.state @@ -4057,7 +3660,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[11].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[7].Exporter = func(v any, i int) any { switch v := v.(*BalanceTopicsResponse); i { case 0: return &v.state @@ -4069,7 +3672,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[12].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[8].Exporter = func(v any, i int) any { switch v := v.(*ConfigureTopicRequest); i { case 0: return &v.state @@ -4081,7 +3684,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[13].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[9].Exporter = func(v any, i int) any { switch v := v.(*ConfigureTopicResponse); i { case 0: return &v.state @@ -4093,7 +3696,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[14].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[10].Exporter = func(v any, i int) any { switch v := v.(*ListTopicsRequest); i { case 0: return &v.state @@ -4105,7 +3708,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[15].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[11].Exporter = func(v any, i int) any { switch v := v.(*ListTopicsResponse); i { case 0: return &v.state @@ -4117,7 +3720,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[16].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[12].Exporter = func(v any, i int) any { switch v := v.(*LookupTopicBrokersRequest); i { case 0: return &v.state @@ -4129,7 +3732,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[17].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[13].Exporter = func(v any, i int) any { switch v := v.(*LookupTopicBrokersResponse); i { case 0: return &v.state @@ -4141,7 +3744,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[18].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[14].Exporter = func(v any, i int) any { switch v := v.(*BrokerPartitionAssignment); i { case 0: return &v.state @@ -4153,7 +3756,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[19].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[15].Exporter = func(v any, i int) any { switch v := v.(*AssignTopicPartitionsRequest); i { case 0: return &v.state @@ -4165,7 +3768,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[20].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[16].Exporter = func(v any, i int) any { switch v := v.(*AssignTopicPartitionsResponse); i { case 0: return &v.state @@ -4177,7 +3780,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[21].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[17].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorRequest); i { case 0: return &v.state @@ -4189,7 +3792,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[22].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[18].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorResponse); i { case 0: return &v.state @@ -4201,7 +3804,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[23].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[19].Exporter = func(v any, i int) any { switch v := v.(*ControlMessage); i { case 0: return &v.state @@ -4213,7 +3816,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[24].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[20].Exporter = func(v any, i int) any { switch v := v.(*DataMessage); i { case 0: return &v.state @@ -4225,7 +3828,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[25].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[21].Exporter = func(v any, i int) any { switch v := v.(*PublishMessageRequest); i { case 0: return &v.state @@ -4237,7 +3840,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[26].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[22].Exporter = func(v any, i int) any { switch v := v.(*PublishMessageResponse); i { case 0: return &v.state @@ -4249,7 +3852,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[27].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[23].Exporter = func(v any, i int) any { switch v := v.(*PublishFollowMeRequest); i { case 0: return &v.state @@ -4261,7 +3864,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[28].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[24].Exporter = func(v any, i int) any { switch v := v.(*PublishFollowMeResponse); i { case 0: return &v.state @@ -4273,7 +3876,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[29].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[25].Exporter = func(v any, i int) any { switch v := v.(*SubscribeMessageRequest); i { case 0: return &v.state @@ -4285,7 +3888,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[30].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[26].Exporter = func(v any, i int) any { switch v := v.(*SubscribeMessageResponse); i { case 0: return &v.state @@ -4297,7 +3900,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[31].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[27].Exporter = func(v any, i int) any { switch v := v.(*SubscribeFollowMeRequest); i { case 0: return &v.state @@ -4309,7 +3912,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[32].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[28].Exporter = func(v any, i int) any { switch v := v.(*SubscribeFollowMeResponse); i { case 0: return &v.state @@ -4321,7 +3924,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[33].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[29].Exporter = func(v any, i int) any { switch v := v.(*ClosePublishersRequest); i { case 0: return &v.state @@ -4333,7 +3936,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[34].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[30].Exporter = func(v any, i int) any { switch v := v.(*ClosePublishersResponse); i { case 0: return &v.state @@ -4345,7 +3948,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[35].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[31].Exporter = func(v any, i int) any { switch v := v.(*CloseSubscribersRequest); i { case 0: return &v.state @@ -4357,7 +3960,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[36].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[32].Exporter = func(v any, i int) any { switch v := v.(*CloseSubscribersResponse); i { case 0: return &v.state @@ -4369,7 +3972,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[38].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[34].Exporter = func(v any, i int) any { switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { case 0: return &v.state @@ -4381,7 +3984,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[39].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[35].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { case 0: return &v.state @@ -4393,7 +3996,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[40].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[36].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage); i { case 0: return &v.state @@ -4405,7 +4008,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[41].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[37].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorRequest_AckAssignmentMessage); i { case 0: return &v.state @@ -4417,7 +4020,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[42].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[38].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { case 0: return &v.state @@ -4429,7 +4032,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[43].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[39].Exporter = func(v any, i int) any { switch v := v.(*SubscriberToSubCoordinatorResponse_UnAssignment); i { case 0: return &v.state @@ -4441,7 +4044,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[44].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[40].Exporter = func(v any, i int) any { switch v := v.(*PublishMessageRequest_InitMessage); i { case 0: return &v.state @@ -4453,7 +4056,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[45].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[41].Exporter = func(v any, i int) any { switch v := v.(*PublishFollowMeRequest_InitMessage); i { case 0: return &v.state @@ -4465,7 +4068,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[46].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[42].Exporter = func(v any, i int) any { switch v := v.(*PublishFollowMeRequest_FlushMessage); i { case 0: return &v.state @@ -4477,7 +4080,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[47].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[43].Exporter = func(v any, i int) any { switch v := v.(*PublishFollowMeRequest_CloseMessage); i { case 0: return &v.state @@ -4489,7 +4092,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[48].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[44].Exporter = func(v any, i int) any { switch v := v.(*SubscribeMessageRequest_InitMessage); i { case 0: return &v.state @@ -4501,7 +4104,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[49].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[45].Exporter = func(v any, i int) any { switch v := v.(*SubscribeMessageRequest_AckMessage); i { case 0: return &v.state @@ -4513,7 +4116,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[50].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[46].Exporter = func(v any, i int) any { switch v := v.(*SubscribeMessageResponse_SubscribeCtrlMessage); i { case 0: return &v.state @@ -4525,7 +4128,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[51].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[47].Exporter = func(v any, i int) any { switch v := v.(*SubscribeFollowMeRequest_InitMessage); i { case 0: return &v.state @@ -4537,7 +4140,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[52].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[48].Exporter = func(v any, i int) any { switch v := v.(*SubscribeFollowMeRequest_AckMessage); i { case 0: return &v.state @@ -4549,7 +4152,7 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[53].Exporter = func(v any, i int) any { + file_mq_broker_proto_msgTypes[49].Exporter = func(v any, i int) any { switch v := v.(*SubscribeFollowMeRequest_CloseMessage); i { case 0: return &v.state @@ -4562,38 +4165,38 @@ func file_mq_proto_init() { } } } - file_mq_proto_msgTypes[8].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[4].OneofWrappers = []any{ (*PublisherToPubBalancerRequest_Init)(nil), (*PublisherToPubBalancerRequest_Stats)(nil), } - file_mq_proto_msgTypes[21].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[17].OneofWrappers = []any{ (*SubscriberToSubCoordinatorRequest_Init)(nil), (*SubscriberToSubCoordinatorRequest_AckAssignment)(nil), (*SubscriberToSubCoordinatorRequest_AckUnAssignment)(nil), } - file_mq_proto_msgTypes[22].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[18].OneofWrappers = []any{ (*SubscriberToSubCoordinatorResponse_Assignment_)(nil), (*SubscriberToSubCoordinatorResponse_UnAssignment_)(nil), } - file_mq_proto_msgTypes[25].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[21].OneofWrappers = []any{ (*PublishMessageRequest_Init)(nil), (*PublishMessageRequest_Data)(nil), } - file_mq_proto_msgTypes[27].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[23].OneofWrappers = []any{ (*PublishFollowMeRequest_Init)(nil), (*PublishFollowMeRequest_Data)(nil), (*PublishFollowMeRequest_Flush)(nil), (*PublishFollowMeRequest_Close)(nil), } - file_mq_proto_msgTypes[29].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[25].OneofWrappers = []any{ (*SubscribeMessageRequest_Init)(nil), (*SubscribeMessageRequest_Ack)(nil), } - file_mq_proto_msgTypes[30].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[26].OneofWrappers = []any{ (*SubscribeMessageResponse_Ctrl)(nil), (*SubscribeMessageResponse_Data)(nil), } - file_mq_proto_msgTypes[31].OneofWrappers = []any{ + file_mq_broker_proto_msgTypes[27].OneofWrappers = []any{ (*SubscribeFollowMeRequest_Init)(nil), (*SubscribeFollowMeRequest_Ack)(nil), (*SubscribeFollowMeRequest_Close)(nil), @@ -4602,19 +4205,18 @@ func file_mq_proto_init() { out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), - RawDescriptor: file_mq_proto_rawDesc, - NumEnums: 1, - NumMessages: 54, + RawDescriptor: file_mq_broker_proto_rawDesc, + NumEnums: 0, + NumMessages: 50, NumExtensions: 0, NumServices: 1, }, - GoTypes: file_mq_proto_goTypes, - DependencyIndexes: file_mq_proto_depIdxs, - EnumInfos: file_mq_proto_enumTypes, - MessageInfos: file_mq_proto_msgTypes, + GoTypes: file_mq_broker_proto_goTypes, + DependencyIndexes: file_mq_broker_proto_depIdxs, + MessageInfos: file_mq_broker_proto_msgTypes, }.Build() - File_mq_proto = out.File - file_mq_proto_rawDesc = nil - file_mq_proto_goTypes = nil - file_mq_proto_depIdxs = nil + File_mq_broker_proto = out.File + file_mq_broker_proto_rawDesc = nil + file_mq_broker_proto_goTypes = nil + file_mq_broker_proto_depIdxs = nil } diff --git a/weed/pb/mq_pb/mq_grpc.pb.go b/weed/pb/mq_pb/mq_broker_grpc.pb.go index a4789ff4f..44a9e9794 100644 --- a/weed/pb/mq_pb/mq_grpc.pb.go +++ b/weed/pb/mq_pb/mq_broker_grpc.pb.go @@ -1,8 +1,8 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 -// source: mq.proto +// - protoc v5.28.3 +// source: mq_broker.proto package mq_pb @@ -589,5 +589,5 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{ ClientStreams: true, }, }, - Metadata: "mq.proto", + Metadata: "mq_broker.proto", } diff --git a/weed/pb/schema.proto b/weed/pb/mq_schema.proto index 28dbe8c3f..15b68981c 100644 --- a/weed/pb/schema.proto +++ b/weed/pb/mq_schema.proto @@ -5,6 +5,39 @@ package schema_pb; option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"; /////////////////////////// +// Topic definition +/////////////////////////// + +message Topic { + string namespace = 1; + string name = 2; +} +message Partition { + int32 ring_size = 1; + int32 range_start = 2; + int32 range_stop = 3; + int64 unix_time_ns = 4; +} + +message Offset { + Topic topic = 1; + repeated PartitionOffset partition_offsets = 2; +} + +enum PartitionOffsetStartType { + EARLIEST = 0; + EARLIEST_IN_MEMORY = 1; + LATEST = 2; +} + +message PartitionOffset { + Partition partition = 1; + int64 start_ts_ns = 2; + int64 stop_ts_ns = 3; + PartitionOffsetStartType start_type = 4; +} + +/////////////////////////// // schema definition /////////////////////////// diff --git a/weed/pb/remote_pb/remote.pb.go b/weed/pb/remote_pb/remote.pb.go index 94e2f6ed9..d5bed6905 100644 --- a/weed/pb/remote_pb/remote.pb.go +++ b/weed/pb/remote_pb/remote.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: remote.proto package remote_pb diff --git a/weed/pb/s3_pb/s3.pb.go b/weed/pb/s3_pb/s3.pb.go index fad0c1fb8..846155f8d 100644 --- a/weed/pb/s3_pb/s3.pb.go +++ b/weed/pb/s3_pb/s3.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: s3.proto package s3_pb diff --git a/weed/pb/s3_pb/s3_grpc.pb.go b/weed/pb/s3_pb/s3_grpc.pb.go index 054b84307..08c1ae6bf 100644 --- a/weed/pb/s3_pb/s3_grpc.pb.go +++ b/weed/pb/s3_pb/s3_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: s3.proto package s3_pb diff --git a/weed/pb/schema_pb/schema.pb.go b/weed/pb/schema_pb/mq_schema.pb.go index 8ea9bb739..4093c26fb 100644 --- a/weed/pb/schema_pb/schema.pb.go +++ b/weed/pb/schema_pb/mq_schema.pb.go @@ -1,8 +1,8 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 -// source: schema.proto +// protoc v5.28.3 +// source: mq_schema.proto package schema_pb @@ -20,6 +20,55 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +type PartitionOffsetStartType int32 + +const ( + PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0 + PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1 + PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2 +) + +// Enum value maps for PartitionOffsetStartType. +var ( + PartitionOffsetStartType_name = map[int32]string{ + 0: "EARLIEST", + 1: "EARLIEST_IN_MEMORY", + 2: "LATEST", + } + PartitionOffsetStartType_value = map[string]int32{ + "EARLIEST": 0, + "EARLIEST_IN_MEMORY": 1, + "LATEST": 2, + } +) + +func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType { + p := new(PartitionOffsetStartType) + *p = x + return p +} + +func (x PartitionOffsetStartType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor { + return file_mq_schema_proto_enumTypes[0].Descriptor() +} + +func (PartitionOffsetStartType) Type() protoreflect.EnumType { + return &file_mq_schema_proto_enumTypes[0] +} + +func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PartitionOffsetStartType.Descriptor instead. +func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) { + return file_mq_schema_proto_rawDescGZIP(), []int{0} +} + type ScalarType int32 const ( @@ -65,11 +114,11 @@ func (x ScalarType) String() string { } func (ScalarType) Descriptor() protoreflect.EnumDescriptor { - return file_schema_proto_enumTypes[0].Descriptor() + return file_mq_schema_proto_enumTypes[1].Descriptor() } func (ScalarType) Type() protoreflect.EnumType { - return &file_schema_proto_enumTypes[0] + return &file_mq_schema_proto_enumTypes[1] } func (x ScalarType) Number() protoreflect.EnumNumber { @@ -78,7 +127,259 @@ func (x ScalarType) Number() protoreflect.EnumNumber { // Deprecated: Use ScalarType.Descriptor instead. func (ScalarType) EnumDescriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{0} + return file_mq_schema_proto_rawDescGZIP(), []int{1} +} + +type Topic struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *Topic) Reset() { + *x = Topic{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_schema_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Topic) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Topic) ProtoMessage() {} + +func (x *Topic) ProtoReflect() protoreflect.Message { + mi := &file_mq_schema_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Topic.ProtoReflect.Descriptor instead. +func (*Topic) Descriptor() ([]byte, []int) { + return file_mq_schema_proto_rawDescGZIP(), []int{0} +} + +func (x *Topic) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *Topic) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type Partition struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RingSize int32 `protobuf:"varint,1,opt,name=ring_size,json=ringSize,proto3" json:"ring_size,omitempty"` + RangeStart int32 `protobuf:"varint,2,opt,name=range_start,json=rangeStart,proto3" json:"range_start,omitempty"` + RangeStop int32 `protobuf:"varint,3,opt,name=range_stop,json=rangeStop,proto3" json:"range_stop,omitempty"` + UnixTimeNs int64 `protobuf:"varint,4,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"` +} + +func (x *Partition) Reset() { + *x = Partition{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_schema_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Partition) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Partition) ProtoMessage() {} + +func (x *Partition) ProtoReflect() protoreflect.Message { + mi := &file_mq_schema_proto_msgTypes[1] + 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 Partition.ProtoReflect.Descriptor instead. +func (*Partition) Descriptor() ([]byte, []int) { + return file_mq_schema_proto_rawDescGZIP(), []int{1} +} + +func (x *Partition) GetRingSize() int32 { + if x != nil { + return x.RingSize + } + return 0 +} + +func (x *Partition) GetRangeStart() int32 { + if x != nil { + return x.RangeStart + } + return 0 +} + +func (x *Partition) GetRangeStop() int32 { + if x != nil { + return x.RangeStop + } + return 0 +} + +func (x *Partition) GetUnixTimeNs() int64 { + if x != nil { + return x.UnixTimeNs + } + return 0 +} + +type Offset struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffsets []*PartitionOffset `protobuf:"bytes,2,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"` +} + +func (x *Offset) Reset() { + *x = Offset{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_schema_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Offset) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Offset) ProtoMessage() {} + +func (x *Offset) ProtoReflect() protoreflect.Message { + mi := &file_mq_schema_proto_msgTypes[2] + 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 Offset.ProtoReflect.Descriptor instead. +func (*Offset) Descriptor() ([]byte, []int) { + return file_mq_schema_proto_rawDescGZIP(), []int{2} +} + +func (x *Offset) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *Offset) GetPartitionOffsets() []*PartitionOffset { + if x != nil { + return x.PartitionOffsets + } + return nil +} + +type PartitionOffset struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + StartTsNs int64 `protobuf:"varint,2,opt,name=start_ts_ns,json=startTsNs,proto3" json:"start_ts_ns,omitempty"` + StopTsNs int64 `protobuf:"varint,3,opt,name=stop_ts_ns,json=stopTsNs,proto3" json:"stop_ts_ns,omitempty"` + StartType PartitionOffsetStartType `protobuf:"varint,4,opt,name=start_type,json=startType,proto3,enum=schema_pb.PartitionOffsetStartType" json:"start_type,omitempty"` +} + +func (x *PartitionOffset) Reset() { + *x = PartitionOffset{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_schema_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionOffset) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionOffset) ProtoMessage() {} + +func (x *PartitionOffset) ProtoReflect() protoreflect.Message { + mi := &file_mq_schema_proto_msgTypes[3] + 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 PartitionOffset.ProtoReflect.Descriptor instead. +func (*PartitionOffset) Descriptor() ([]byte, []int) { + return file_mq_schema_proto_rawDescGZIP(), []int{3} +} + +func (x *PartitionOffset) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *PartitionOffset) GetStartTsNs() int64 { + if x != nil { + return x.StartTsNs + } + return 0 +} + +func (x *PartitionOffset) GetStopTsNs() int64 { + if x != nil { + return x.StopTsNs + } + return 0 +} + +func (x *PartitionOffset) GetStartType() PartitionOffsetStartType { + if x != nil { + return x.StartType + } + return PartitionOffsetStartType_EARLIEST } type RecordType struct { @@ -92,7 +393,7 @@ type RecordType struct { func (x *RecordType) Reset() { *x = RecordType{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[0] + mi := &file_mq_schema_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -105,7 +406,7 @@ func (x *RecordType) String() string { func (*RecordType) ProtoMessage() {} func (x *RecordType) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[0] + mi := &file_mq_schema_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -118,7 +419,7 @@ func (x *RecordType) ProtoReflect() protoreflect.Message { // Deprecated: Use RecordType.ProtoReflect.Descriptor instead. func (*RecordType) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{0} + return file_mq_schema_proto_rawDescGZIP(), []int{4} } func (x *RecordType) GetFields() []*Field { @@ -143,7 +444,7 @@ type Field struct { func (x *Field) Reset() { *x = Field{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[1] + mi := &file_mq_schema_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -156,7 +457,7 @@ func (x *Field) String() string { func (*Field) ProtoMessage() {} func (x *Field) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[1] + mi := &file_mq_schema_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -169,7 +470,7 @@ func (x *Field) ProtoReflect() protoreflect.Message { // Deprecated: Use Field.ProtoReflect.Descriptor instead. func (*Field) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{1} + return file_mq_schema_proto_rawDescGZIP(), []int{5} } func (x *Field) GetName() string { @@ -223,7 +524,7 @@ type Type struct { func (x *Type) Reset() { *x = Type{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[2] + mi := &file_mq_schema_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -236,7 +537,7 @@ func (x *Type) String() string { func (*Type) ProtoMessage() {} func (x *Type) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[2] + mi := &file_mq_schema_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -249,7 +550,7 @@ func (x *Type) ProtoReflect() protoreflect.Message { // Deprecated: Use Type.ProtoReflect.Descriptor instead. func (*Type) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{2} + return file_mq_schema_proto_rawDescGZIP(), []int{6} } func (m *Type) GetKind() isType_Kind { @@ -313,7 +614,7 @@ type ListType struct { func (x *ListType) Reset() { *x = ListType{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[3] + mi := &file_mq_schema_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -326,7 +627,7 @@ func (x *ListType) String() string { func (*ListType) ProtoMessage() {} func (x *ListType) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[3] + mi := &file_mq_schema_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -339,7 +640,7 @@ func (x *ListType) ProtoReflect() protoreflect.Message { // Deprecated: Use ListType.ProtoReflect.Descriptor instead. func (*ListType) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{3} + return file_mq_schema_proto_rawDescGZIP(), []int{7} } func (x *ListType) GetElementType() *Type { @@ -363,7 +664,7 @@ type RecordValue struct { func (x *RecordValue) Reset() { *x = RecordValue{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[4] + mi := &file_mq_schema_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -376,7 +677,7 @@ func (x *RecordValue) String() string { func (*RecordValue) ProtoMessage() {} func (x *RecordValue) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[4] + mi := &file_mq_schema_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -389,7 +690,7 @@ func (x *RecordValue) ProtoReflect() protoreflect.Message { // Deprecated: Use RecordValue.ProtoReflect.Descriptor instead. func (*RecordValue) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{4} + return file_mq_schema_proto_rawDescGZIP(), []int{8} } func (x *RecordValue) GetFields() map[string]*Value { @@ -421,7 +722,7 @@ type Value struct { func (x *Value) Reset() { *x = Value{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[5] + mi := &file_mq_schema_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -434,7 +735,7 @@ func (x *Value) String() string { func (*Value) ProtoMessage() {} func (x *Value) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[5] + mi := &file_mq_schema_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -447,7 +748,7 @@ func (x *Value) ProtoReflect() protoreflect.Message { // Deprecated: Use Value.ProtoReflect.Descriptor instead. func (*Value) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{5} + return file_mq_schema_proto_rawDescGZIP(), []int{9} } func (m *Value) GetKind() isValue_Kind { @@ -589,7 +890,7 @@ type ListValue struct { func (x *ListValue) Reset() { *x = ListValue{} if protoimpl.UnsafeEnabled { - mi := &file_schema_proto_msgTypes[6] + mi := &file_mq_schema_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -602,7 +903,7 @@ func (x *ListValue) String() string { func (*ListValue) ProtoMessage() {} func (x *ListValue) ProtoReflect() protoreflect.Message { - mi := &file_schema_proto_msgTypes[6] + mi := &file_mq_schema_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -615,7 +916,7 @@ func (x *ListValue) ProtoReflect() protoreflect.Message { // Deprecated: Use ListValue.ProtoReflect.Descriptor instead. func (*ListValue) Descriptor() ([]byte, []int) { - return file_schema_proto_rawDescGZIP(), []int{6} + return file_mq_schema_proto_rawDescGZIP(), []int{10} } func (x *ListValue) GetValues() []*Value { @@ -625,11 +926,44 @@ func (x *ListValue) GetValues() []*Value { return nil } -var File_schema_proto protoreflect.FileDescriptor - -var file_schema_proto_rawDesc = []byte{ - 0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x36, 0x0a, 0x0a, 0x52, 0x65, 0x63, +var File_mq_schema_proto protoreflect.FileDescriptor + +var file_mq_schema_proto_rawDesc = []byte{ + 0x0a, 0x0f, 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x09, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x39, 0x0a, 0x05, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69, + 0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f, + 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, + 0x6f, 0x70, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, + 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, + 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x79, 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x26, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x47, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x10, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x22, + 0xc7, 0x01, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, + 0x73, 0x65, 0x74, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 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, 0x1e, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x1c, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x70, 0x5f, + 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x6f, + 0x70, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x42, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x23, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x36, 0x0a, 0x0a, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, @@ -695,70 +1029,132 @@ var file_schema_proto_rawDesc = []byte{ 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x22, 0x35, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x28, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x2a, 0x5a, 0x0a, - 0x0a, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42, - 0x4f, 0x4f, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01, - 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, - 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, - 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x0a, 0x0a, - 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 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, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 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, 0x2a, 0x5a, 0x0a, 0x0a, 0x53, + 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42, 0x4f, 0x4f, + 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01, 0x12, 0x09, + 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, + 0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x05, + 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x0a, 0x0a, 0x06, 0x53, + 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 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, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( - file_schema_proto_rawDescOnce sync.Once - file_schema_proto_rawDescData = file_schema_proto_rawDesc + file_mq_schema_proto_rawDescOnce sync.Once + file_mq_schema_proto_rawDescData = file_mq_schema_proto_rawDesc ) -func file_schema_proto_rawDescGZIP() []byte { - file_schema_proto_rawDescOnce.Do(func() { - file_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_schema_proto_rawDescData) +func file_mq_schema_proto_rawDescGZIP() []byte { + file_mq_schema_proto_rawDescOnce.Do(func() { + file_mq_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_mq_schema_proto_rawDescData) }) - return file_schema_proto_rawDescData -} - -var file_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 8) -var file_schema_proto_goTypes = []any{ - (ScalarType)(0), // 0: schema_pb.ScalarType - (*RecordType)(nil), // 1: schema_pb.RecordType - (*Field)(nil), // 2: schema_pb.Field - (*Type)(nil), // 3: schema_pb.Type - (*ListType)(nil), // 4: schema_pb.ListType - (*RecordValue)(nil), // 5: schema_pb.RecordValue - (*Value)(nil), // 6: schema_pb.Value - (*ListValue)(nil), // 7: schema_pb.ListValue - nil, // 8: schema_pb.RecordValue.FieldsEntry -} -var file_schema_proto_depIdxs = []int32{ - 2, // 0: schema_pb.RecordType.fields:type_name -> schema_pb.Field - 3, // 1: schema_pb.Field.type:type_name -> schema_pb.Type - 0, // 2: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType - 1, // 3: schema_pb.Type.record_type:type_name -> schema_pb.RecordType - 4, // 4: schema_pb.Type.list_type:type_name -> schema_pb.ListType - 3, // 5: schema_pb.ListType.element_type:type_name -> schema_pb.Type - 8, // 6: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry - 7, // 7: schema_pb.Value.list_value:type_name -> schema_pb.ListValue - 5, // 8: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue - 6, // 9: schema_pb.ListValue.values:type_name -> schema_pb.Value - 6, // 10: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value - 11, // [11:11] is the sub-list for method output_type - 11, // [11:11] is the sub-list for method input_type - 11, // [11:11] is the sub-list for extension type_name - 11, // [11:11] is the sub-list for extension extendee - 0, // [0:11] is the sub-list for field type_name -} - -func init() { file_schema_proto_init() } -func file_schema_proto_init() { - if File_schema_proto != nil { + return file_mq_schema_proto_rawDescData +} + +var file_mq_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_mq_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_mq_schema_proto_goTypes = []any{ + (PartitionOffsetStartType)(0), // 0: schema_pb.PartitionOffsetStartType + (ScalarType)(0), // 1: schema_pb.ScalarType + (*Topic)(nil), // 2: schema_pb.Topic + (*Partition)(nil), // 3: schema_pb.Partition + (*Offset)(nil), // 4: schema_pb.Offset + (*PartitionOffset)(nil), // 5: schema_pb.PartitionOffset + (*RecordType)(nil), // 6: schema_pb.RecordType + (*Field)(nil), // 7: schema_pb.Field + (*Type)(nil), // 8: schema_pb.Type + (*ListType)(nil), // 9: schema_pb.ListType + (*RecordValue)(nil), // 10: schema_pb.RecordValue + (*Value)(nil), // 11: schema_pb.Value + (*ListValue)(nil), // 12: schema_pb.ListValue + nil, // 13: schema_pb.RecordValue.FieldsEntry +} +var file_mq_schema_proto_depIdxs = []int32{ + 2, // 0: schema_pb.Offset.topic:type_name -> schema_pb.Topic + 5, // 1: schema_pb.Offset.partition_offsets:type_name -> schema_pb.PartitionOffset + 3, // 2: schema_pb.PartitionOffset.partition:type_name -> schema_pb.Partition + 0, // 3: schema_pb.PartitionOffset.start_type:type_name -> schema_pb.PartitionOffsetStartType + 7, // 4: schema_pb.RecordType.fields:type_name -> schema_pb.Field + 8, // 5: schema_pb.Field.type:type_name -> schema_pb.Type + 1, // 6: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType + 6, // 7: schema_pb.Type.record_type:type_name -> schema_pb.RecordType + 9, // 8: schema_pb.Type.list_type:type_name -> schema_pb.ListType + 8, // 9: schema_pb.ListType.element_type:type_name -> schema_pb.Type + 13, // 10: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry + 12, // 11: schema_pb.Value.list_value:type_name -> schema_pb.ListValue + 10, // 12: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue + 11, // 13: schema_pb.ListValue.values:type_name -> schema_pb.Value + 11, // 14: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value + 15, // [15:15] is the sub-list for method output_type + 15, // [15:15] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name +} + +func init() { file_mq_schema_proto_init() } +func file_mq_schema_proto_init() { + if File_mq_schema_proto != nil { return } if !protoimpl.UnsafeEnabled { - file_schema_proto_msgTypes[0].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*Topic); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_schema_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Partition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_schema_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*Offset); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_schema_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*PartitionOffset); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_schema_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*RecordType); i { case 0: return &v.state @@ -770,7 +1166,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[1].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*Field); i { case 0: return &v.state @@ -782,7 +1178,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[2].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[6].Exporter = func(v any, i int) any { switch v := v.(*Type); i { case 0: return &v.state @@ -794,7 +1190,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[3].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[7].Exporter = func(v any, i int) any { switch v := v.(*ListType); i { case 0: return &v.state @@ -806,7 +1202,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[4].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[8].Exporter = func(v any, i int) any { switch v := v.(*RecordValue); i { case 0: return &v.state @@ -818,7 +1214,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[5].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[9].Exporter = func(v any, i int) any { switch v := v.(*Value); i { case 0: return &v.state @@ -830,7 +1226,7 @@ func file_schema_proto_init() { return nil } } - file_schema_proto_msgTypes[6].Exporter = func(v any, i int) any { + file_mq_schema_proto_msgTypes[10].Exporter = func(v any, i int) any { switch v := v.(*ListValue); i { case 0: return &v.state @@ -843,12 +1239,12 @@ func file_schema_proto_init() { } } } - file_schema_proto_msgTypes[2].OneofWrappers = []any{ + file_mq_schema_proto_msgTypes[6].OneofWrappers = []any{ (*Type_ScalarType)(nil), (*Type_RecordType)(nil), (*Type_ListType)(nil), } - file_schema_proto_msgTypes[5].OneofWrappers = []any{ + file_mq_schema_proto_msgTypes[9].OneofWrappers = []any{ (*Value_BoolValue)(nil), (*Value_Int32Value)(nil), (*Value_Int64Value)(nil), @@ -863,19 +1259,19 @@ func file_schema_proto_init() { out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), - RawDescriptor: file_schema_proto_rawDesc, - NumEnums: 1, - NumMessages: 8, + RawDescriptor: file_mq_schema_proto_rawDesc, + NumEnums: 2, + NumMessages: 12, NumExtensions: 0, NumServices: 0, }, - GoTypes: file_schema_proto_goTypes, - DependencyIndexes: file_schema_proto_depIdxs, - EnumInfos: file_schema_proto_enumTypes, - MessageInfos: file_schema_proto_msgTypes, + GoTypes: file_mq_schema_proto_goTypes, + DependencyIndexes: file_mq_schema_proto_depIdxs, + EnumInfos: file_mq_schema_proto_enumTypes, + MessageInfos: file_mq_schema_proto_msgTypes, }.Build() - File_schema_proto = out.File - file_schema_proto_rawDesc = nil - file_schema_proto_goTypes = nil - file_schema_proto_depIdxs = nil + File_mq_schema_proto = out.File + file_mq_schema_proto_rawDesc = nil + file_mq_schema_proto_goTypes = nil + file_mq_schema_proto_depIdxs = nil } diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go index 5492e71d4..7ac0fc148 100644 --- a/weed/pb/volume_server_pb/volume_server.pb.go +++ b/weed/pb/volume_server_pb/volume_server.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.34.2 -// protoc v5.28.1 +// protoc v5.28.3 // source: volume_server.proto package volume_server_pb diff --git a/weed/pb/volume_server_pb/volume_server_grpc.pb.go b/weed/pb/volume_server_pb/volume_server_grpc.pb.go index 2cbb45491..19f73cafb 100644 --- a/weed/pb/volume_server_pb/volume_server_grpc.pb.go +++ b/weed/pb/volume_server_pb/volume_server_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.5.1 -// - protoc v5.28.1 +// - protoc v5.28.3 // source: volume_server.proto package volume_server_pb diff --git a/weed/shell/command_mq_topic_configure.go b/weed/shell/command_mq_topic_configure.go index 9342c8604..ab54ee554 100644 --- a/weed/shell/command_mq_topic_configure.go +++ b/weed/shell/command_mq_topic_configure.go @@ -7,6 +7,7 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "io" ) @@ -54,7 +55,7 @@ func (c *commandMqTopicConfigure) Do(args []string, commandEnv *CommandEnv, writ // create topic return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { resp, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{ - Topic: &mq_pb.Topic{ + Topic: &schema_pb.Topic{ Namespace: *namespace, Name: *topicName, }, diff --git a/weed/shell/command_mq_topic_desc.go b/weed/shell/command_mq_topic_desc.go index 8c944271c..9a0d61066 100644 --- a/weed/shell/command_mq_topic_desc.go +++ b/weed/shell/command_mq_topic_desc.go @@ -6,6 +6,7 @@ import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" "io" ) @@ -46,7 +47,7 @@ func (c *commandMqTopicDescribe) Do(args []string, commandEnv *CommandEnv, write return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { resp, err := client.LookupTopicBrokers(context.Background(), &mq_pb.LookupTopicBrokersRequest{ - Topic: &mq_pb.Topic{ + Topic: &schema_pb.Topic{ Namespace: *namespace, Name: *topicName, }, |
