aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--weed/command/mq_agent.go74
-rw-r--r--weed/command/scaffold/security.toml5
-rw-r--r--weed/mq/agent/agent_grpc_pub_session.go61
-rw-r--r--weed/mq/agent/agent_grpc_publish.go43
-rw-r--r--weed/mq/agent/agent_grpc_sub_session.go57
-rw-r--r--weed/mq/agent/agent_grpc_subscribe.go75
-rw-r--r--weed/mq/agent/agent_server.go52
-rw-r--r--weed/mq/broker/broker_grpc_assign.go3
-rw-r--r--weed/mq/broker/broker_grpc_configure.go4
-rw-r--r--weed/mq/broker/broker_grpc_lookup.go3
-rw-r--r--weed/mq/broker/broker_grpc_sub.go7
-rw-r--r--weed/mq/client/agent_client/SubscribeSession.go63
-rw-r--r--weed/mq/client/agent_client/agent_publish.go14
-rw-r--r--weed/mq/client/agent_client/agent_subscribe.go17
-rw-r--r--weed/mq/client/agent_client/publish_session.go70
-rw-r--r--weed/mq/client/cmd/agent_pub_record/publisher_record.go128
-rw-r--r--weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go21
-rw-r--r--weed/mq/client/cmd/weed_sub_record/subscriber_record.go34
-rw-r--r--weed/mq/client/sub_client/on_each_partition.go84
-rw-r--r--weed/mq/client/sub_client/subscribe.go18
-rw-r--r--weed/mq/client/sub_client/subscriber.go21
-rw-r--r--weed/mq/pub_balancer/allocate.go3
-rw-r--r--weed/mq/pub_balancer/allocate_test.go17
-rw-r--r--weed/mq/pub_balancer/broker_stats.go3
-rw-r--r--weed/mq/pub_balancer/lookup.go5
-rw-r--r--weed/mq/pub_balancer/partition_list_broker.go4
-rw-r--r--weed/mq/pub_balancer/pub_balancer.go3
-rw-r--r--weed/mq/schema/schema.go9
-rw-r--r--weed/mq/sub_coordinator/consumer_group.go3
-rw-r--r--weed/mq/sub_coordinator/sub_coordinator.go9
-rw-r--r--weed/mq/topic/local_manager.go3
-rw-r--r--weed/mq/topic/partition.go8
-rw-r--r--weed/mq/topic/topic.go7
-rw-r--r--weed/pb/Makefile5
-rw-r--r--weed/pb/filer_pb/filer.pb.go2
-rw-r--r--weed/pb/filer_pb/filer_grpc.pb.go2
-rw-r--r--weed/pb/iam_pb/iam.pb.go2
-rw-r--r--weed/pb/iam_pb/iam_grpc.pb.go2
-rw-r--r--weed/pb/master_pb/master.pb.go2
-rw-r--r--weed/pb/master_pb/master_grpc.pb.go2
-rw-r--r--weed/pb/mount_pb/mount.pb.go2
-rw-r--r--weed/pb/mount_pb/mount_grpc.pb.go2
-rw-r--r--weed/pb/mq_agent.proto95
-rw-r--r--weed/pb/mq_agent_pb/mq_agent.pb.go1148
-rw-r--r--weed/pb/mq_agent_pb/mq_agent_grpc.pb.go304
-rw-r--r--weed/pb/mq_broker.proto (renamed from weed/pb/mq.proto)77
-rw-r--r--weed/pb/mq_pb/mq_broker.pb.go (renamed from weed/pb/mq_pb/mq.pb.go)2154
-rw-r--r--weed/pb/mq_pb/mq_broker_grpc.pb.go (renamed from weed/pb/mq_pb/mq_grpc.pb.go)6
-rw-r--r--weed/pb/mq_schema.proto (renamed from weed/pb/schema.proto)33
-rw-r--r--weed/pb/remote_pb/remote.pb.go2
-rw-r--r--weed/pb/s3_pb/s3.pb.go2
-rw-r--r--weed/pb/s3_pb/s3_grpc.pb.go2
-rw-r--r--weed/pb/schema_pb/mq_schema.pb.go (renamed from weed/pb/schema_pb/schema.pb.go)606
-rw-r--r--weed/pb/volume_server_pb/volume_server.pb.go2
-rw-r--r--weed/pb/volume_server_pb/volume_server_grpc.pb.go2
-rw-r--r--weed/shell/command_mq_topic_configure.go3
-rw-r--r--weed/shell/command_mq_topic_desc.go3
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,
},