diff options
43 files changed, 1047 insertions, 1585 deletions
diff --git a/.gitignore b/.gitignore index 89f3b8a0c..363310f81 100644 --- a/.gitignore +++ b/.gitignore @@ -87,7 +87,6 @@ other/java/hdfs/dependency-reduced-pom.xml # binary file weed/weed -weed/mq/client/cmd/weed_pub_kv/weed_pub docker/weed # test generated files @@ -95,7 +94,4 @@ weed/*/*.jpg docker/weed_sub docker/weed_pub weed/mq/schema/example.parquet -docker/weed_pub_kv -docker/weed_pub_record -docker/weed_sub_kv -docker/weed_sub_record +docker/agent_sub_record diff --git a/docker/Makefile b/docker/Makefile index b4a61fb01..d4dd70b6b 100644 --- a/docker/Makefile +++ b/docker/Makefile @@ -9,10 +9,8 @@ binary: export SWCOMMIT=$(shell git rev-parse --short HEAD) export SWLDFLAGS="-X github.com/seaweedfs/seaweedfs/weed/util.COMMIT=$(SWCOMMIT)" cd ../weed && CGO_ENABLED=$(cgo) GOOS=linux go build $(options) -tags "$(tags)" -ldflags "-s -w -extldflags -static $(SWLDFLAGS)" && mv weed ../docker/ - cd ../weed/mq/client/cmd/weed_pub_kv && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_pub_kv ../../../../../docker/ - cd ../weed/mq/client/cmd/weed_pub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_pub_record ../../../../../docker/ - cd ../weed/mq/client/cmd/weed_sub_kv && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_sub_kv ../../../../../docker/ - cd ../weed/mq/client/cmd/weed_sub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_sub_record ../../../../../docker/ + cd ../other/mq_client_example/agent_pub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv agent_pub_record ../../../docker/ + cd ../other/mq_client_example/agent_sub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv agent_sub_record ../../../docker/ binary_race: options = -race binary_race: cgo = 1 @@ -97,6 +95,9 @@ s3tests: build s3tests_build brokers: build docker compose -f compose/local-brokers-compose.yml -p seaweedfs up +agent: build + docker compose -f compose/local-mq-test.yml -p seaweedfs up + filer_etcd: build docker stack deploy -c compose/swarm-etcd.yml fs diff --git a/docker/agent_pub_record b/docker/agent_pub_record Binary files differnew file mode 100755 index 000000000..03c1a7879 --- /dev/null +++ b/docker/agent_pub_record diff --git a/docker/compose/local-mq-test.yml b/docker/compose/local-mq-test.yml index d4b071039..fef68cac8 100644 --- a/docker/compose/local-mq-test.yml +++ b/docker/compose/local-mq-test.yml @@ -1,5 +1,3 @@ -version: '3.9' - services: server: image: chrislusf/seaweedfs:local @@ -19,9 +17,16 @@ services: depends_on: server: condition: service_healthy + mq_agent: + image: chrislusf/seaweedfs:local + ports: + - 16777:16777 + command: "mq.agent -broker=mq_broker:17777 -port=16777" + depends_on: + - mq_broker mq_client: image: chrislusf/seaweedfs:local # run a custom command instead of entrypoint command: "ls -al" depends_on: - - mq_broker + - mq_agent diff --git a/weed/mq/client/cmd/agent_pub_record/publisher_record.go b/other/mq_client_example/agent_pub_record/agent_pub_record.go index eda200e86..48e78f530 100644 --- a/weed/mq/client/cmd/agent_pub_record/publisher_record.go +++ b/other/mq_client_example/agent_pub_record/agent_pub_record.go @@ -3,9 +3,9 @@ package main import ( "flag" "fmt" + "github.com/seaweedfs/seaweedfs/other/mq_client_example/example" "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" @@ -20,13 +20,26 @@ var ( 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") + namespace = flag.String("ns", "test", "namespace") + t = flag.String("t", "test", "t") + agent = flag.String("agent", "localhost:16777", "mq agent address") counter int32 ) +func genMyRecord(id int32) *example.MyRecord { + return &example.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 doPublish(publisher *agent_client.PublishSession, id int) { startTime := time.Now() for { @@ -49,58 +62,12 @@ func doPublish(publisher *agent_client.PublishSession, id int) { 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() + recordType := example.MyRecordType() - session, err := agent_client.NewPublishSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName) + session, err := agent_client.NewPublishSession(*agent, schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName) if err != nil { log.Printf("failed to create session: %v", err) return diff --git a/other/mq_client_example/agent_sub_record/agent_sub_record.go b/other/mq_client_example/agent_sub_record/agent_sub_record.go new file mode 100644 index 000000000..21c74a46d --- /dev/null +++ b/other/mq_client_example/agent_sub_record/agent_sub_record.go @@ -0,0 +1,62 @@ +package main + +import ( + "flag" + "fmt" + "github.com/seaweedfs/seaweedfs/other/mq_client_example/example" + "github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client" + "github.com/seaweedfs/seaweedfs/weed/mq/topic" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "github.com/seaweedfs/seaweedfs/weed/util" + "log" + "time" +) + +var ( + namespace = flag.String("ns", "test", "namespace") + t = flag.String("topic", "test", "topic") + agent = flag.String("agent", "localhost:16777", "mq agent address") + maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count") + slidingWindowSize = flag.Int("slidingWindowSize", 1, "per partition concurrency") + timeAgo = flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") + + clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id") +) + +func main() { + flag.Parse() + + // determine the start of the messages + var startTsNs int64 + startType := schema_pb.OffsetType_RESUME_OR_EARLIEST + if *timeAgo > 0 { + startTsNs = time.Now().Add(-*timeAgo).UnixNano() + startType = schema_pb.OffsetType_EXACT_TS_NS + } + + session, err := agent_client.NewSubscribeSession(*agent, &agent_client.SubscribeOption{ + ConsumerGroup: "test", + ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), + Topic: topic.NewTopic(*namespace, *t), + OffsetType: startType, + OffsetTsNs: startTsNs, + Filter: "", + MaxSubscribedPartitions: int32(*maxPartitionCount), + SlidingWindowSize: int32(*slidingWindowSize), + }) + if err != nil { + log.Printf("new subscribe session: %v", err) + return + } + defer session.CloseSession() + + counter := 0 + session.SubscribeMessageRecord(func(key []byte, recordValue *schema_pb.RecordValue) { + counter++ + record := example.FromRecordValue(recordValue) + fmt.Printf("%d %s %v\n", counter, string(key), record.Field2) + }, func() { + log.Printf("done received %d messages", counter) + }) + +} diff --git a/other/mq_client_example/example/my_record.go b/other/mq_client_example/example/my_record.go new file mode 100644 index 000000000..ea6a0e7bd --- /dev/null +++ b/other/mq_client_example/example/my_record.go @@ -0,0 +1,56 @@ +package example + +import ( + "github.com/seaweedfs/seaweedfs/weed/mq/schema" + "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" +) + +type MyRecord struct { + Key []byte + Field1 []byte + Field2 string + Field3 int32 + Field4 int64 + Field5 float32 + Field6 float64 + Field7 bool +} + +func MyRecordType() *schema_pb.RecordType { + return 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() +} + +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 FromRecordValue(recordValue *schema_pb.RecordValue) *MyRecord { + return &MyRecord{ + Key: recordValue.Fields["key"].GetBytesValue(), + Field1: recordValue.Fields["field1"].GetBytesValue(), + Field2: recordValue.Fields["field2"].GetStringValue(), + Field3: recordValue.Fields["field3"].GetInt32Value(), + Field4: recordValue.Fields["field4"].GetInt64Value(), + Field5: recordValue.Fields["field5"].GetFloatValue(), + Field6: recordValue.Fields["field6"].GetDoubleValue(), + Field7: recordValue.Fields["field7"].GetBoolValue(), + } +} diff --git a/weed/Makefile b/weed/Makefile index 7d1a0d2ed..e91673b62 100644 --- a/weed/Makefile +++ b/weed/Makefile @@ -41,6 +41,10 @@ debug_mq: go build -gcflags="all=-N -l" dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 mq.broker +debug_mq_agent: + go build -gcflags="all=-N -l" + dlv --listen=:2346 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 mq.agent -broker=localhost:17777 + debug_filer_copy: go build -gcflags="all=-N -l" dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 filer.backup -filer=localhost:8888 -filerProxy -timeAgo=10h diff --git a/weed/command/command.go b/weed/command/command.go index 9fdf057e7..33cdb12d1 100644 --- a/weed/command/command.go +++ b/weed/command/command.go @@ -32,6 +32,7 @@ var Commands = []*Command{ cmdMaster, cmdMasterFollower, cmdMount, + cmdMqAgent, cmdMqBroker, cmdS3, cmdScaffold, diff --git a/weed/command/mq_agent.go b/weed/command/mq_agent.go index 2884c6531..4a59dcf33 100644 --- a/weed/command/mq_agent.go +++ b/weed/command/mq_agent.go @@ -26,12 +26,12 @@ type MessageQueueAgentOptions struct { 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.ip = cmdMqAgent.Flag.String("ip", "", "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>]", + UsageLine: "mq.agent [-port=16777] [-master=<ip:port>]", Short: "<WIP> start a message queue agent", Long: `start a message queue agent @@ -64,6 +64,7 @@ func (mqAgentOpt *MessageQueueAgentOptions) startQueueAgent() bool { if err != nil { glog.Fatalf("failed to listen on grpc port %d: %v", *mqAgentOpt.port, err) } + glog.Infof("Start Seaweed Message Queue Agent on %s:%d", *mqAgentOpt.ip, *mqAgentOpt.port) grpcS := pb.NewGrpcServer() mq_agent_pb.RegisterSeaweedMessagingAgentServer(grpcS, agentServer) reflection.Register(grpcS) diff --git a/weed/mq/agent/agent_grpc_pub_session.go b/weed/mq/agent/agent_grpc_pub_session.go index d5c6d0813..7cf857da1 100644 --- a/weed/mq/agent/agent_grpc_pub_session.go +++ b/weed/mq/agent/agent_grpc_pub_session.go @@ -7,13 +7,12 @@ import ( "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( + topicPublisher, err := pub_client.NewTopicPublisher( &pub_client.PublisherConfiguration{ Topic: topic.NewTopic(req.Topic.Namespace, req.Topic.Name), PartitionCount: req.PartitionCount, @@ -21,18 +20,11 @@ func (a *MessageQueueAgent) StartPublishSession(ctx context.Context, req *mq_age PublisherName: req.PublisherName, RecordType: req.RecordType, }) + if err != nil { + return nil, err + } 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, } diff --git a/weed/mq/agent/agent_grpc_publish.go b/weed/mq/agent/agent_grpc_publish.go index 485e1d24e..0b666ff6d 100644 --- a/weed/mq/agent/agent_grpc_publish.go +++ b/weed/mq/agent/agent_grpc_publish.go @@ -3,24 +3,25 @@ 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 { +func (a *MessageQueueAgent) PublishRecord(stream mq_agent_pb.SeaweedMessagingAgent_PublishRecordServer) error { m, err := stream.Recv() if err != nil { return err } + sessionId := SessionId(m.SessionId) a.publishersLock.RLock() - publisherEntry, found := a.publishers[SessionId(m.SessionId)] + publisherEntry, found := a.publishers[sessionId] a.publishersLock.RUnlock() if !found { - return fmt.Errorf("publish session id %d not found", m.SessionId) + return fmt.Errorf("publish session id %d not found", sessionId) } defer func() { - publisherEntry.lastActiveTsNs = time.Now().UnixNano() + a.publishersLock.Lock() + delete(a.publishers, sessionId) + a.publishersLock.Unlock() }() - publisherEntry.lastActiveTsNs = 0 if m.Value != nil { if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil { @@ -29,7 +30,7 @@ func (a *MessageQueueAgent) PublishRecordRequest(stream mq_agent_pb.SeaweedMessa } for { - m, err := stream.Recv() + m, err = stream.Recv() if err != nil { return err } diff --git a/weed/mq/agent/agent_grpc_sub_session.go b/weed/mq/agent/agent_grpc_sub_session.go deleted file mode 100644 index 17def9ed5..000000000 --- a/weed/mq/agent/agent_grpc_sub_session.go +++ /dev/null @@ -1,57 +0,0 @@ -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 index feb5bd47c..87baa466c 100644 --- a/weed/mq/agent/agent_grpc_subscribe.go +++ b/weed/mq/agent/agent_grpc_subscribe.go @@ -1,59 +1,59 @@ package agent import ( - "fmt" + "context" "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_agent_pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" + "github.com/seaweedfs/seaweedfs/weed/util" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/protobuf/proto" - "time" ) -func (a *MessageQueueAgent) SubscribeRecordRequest(stream mq_agent_pb.SeaweedMessagingAgent_SubscribeRecordServer) error { +func (a *MessageQueueAgent) SubscribeRecord(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() + initMessage, 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 + + subscriber := a.handleInitSubscribeRecordRequest(stream.Context(), initMessage.Init) 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 + executors := util.NewLimitedConcurrentExecutor(int(subscriber.SubscriberConfig.SlidingWindowSize)) + subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) { + executors.Execute(func() { + record := &schema_pb.RecordValue{} + err := proto.Unmarshal(m.Data.Value, record) + if err != nil { + glog.V(0).Infof("unmarshal record value: %v", err) + if lastErr == nil { + lastErr = err + } + return } - 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 + if sendErr := stream.Send(&mq_agent_pb.SubscribeRecordResponse{ + Key: m.Data.Key, + Value: record, + TsNs: m.Data.TsNs, + }); sendErr != nil { + glog.V(0).Infof("send record: %v", sendErr) + if lastErr == nil { + lastErr = sendErr + } } - } + }) }) go func() { - subErr := subscriberEntry.entry.Subscribe() + subErr := subscriber.Subscribe() if subErr != nil { - glog.V(0).Infof("subscriber %d subscribe: %v", m.SessionId, subErr) + glog.V(0).Infof("subscriber %s subscribe: %v", subscriber.SubscriberConfig.String(), subErr) if lastErr == nil { lastErr = subErr } @@ -63,13 +63,43 @@ func (a *MessageQueueAgent) SubscribeRecordRequest(stream mq_agent_pb.SeaweedMes for { m, err := stream.Recv() if err != nil { + glog.V(0).Infof("subscriber %s receive: %v", subscriber.SubscriberConfig.String(), err) return err } if m != nil { - subscriberEntry.entry.PartitionOffsetChan <- sub_client.KeyedOffset{ + subscriber.PartitionOffsetChan <- sub_client.KeyedOffset{ Key: m.AckKey, Offset: m.AckSequence, } } } } + +func (a *MessageQueueAgent) handleInitSubscribeRecordRequest(ctx context.Context, req *mq_agent_pb.SubscribeRecordRequest_InitSubscribeRecordRequest) *sub_client.TopicSubscriber { + + 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, + OffsetType: req.OffsetType, + OffsetTsNs: req.OffsetTsNs, + } + + topicSubscriber := sub_client.NewTopicSubscriber( + ctx, + a.brokersList(), + subscriberConfig, + contentConfig, + make(chan sub_client.KeyedOffset, 1024), + ) + + return topicSubscriber +} diff --git a/weed/mq/agent/agent_server.go b/weed/mq/agent/agent_server.go index 6fc61bbdb..f1d6ec679 100644 --- a/weed/mq/agent/agent_server.go +++ b/weed/mq/agent/agent_server.go @@ -11,8 +11,7 @@ import ( type SessionId int64 type SessionEntry[T any] struct { - entry T - lastActiveTsNs int64 + entry T } type MessageQueueAgentOptions struct { @@ -32,11 +31,15 @@ type MessageQueueAgent struct { func NewMessageQueueAgent(option *MessageQueueAgentOptions, grpcDialOption grpc.DialOption) *MessageQueueAgent { - // check masters to list all brokers + // initialize brokers which may change later + var brokers []pb.ServerAddress + for _, broker := range option.SeedBrokers { + brokers = append(brokers, broker) + } return &MessageQueueAgent{ option: option, - brokers: []pb.ServerAddress{}, + brokers: brokers, grpcDialOption: grpcDialOption, publishers: make(map[SessionId]*SessionEntry[*pub_client.TopicPublisher]), subscribers: make(map[SessionId]*SessionEntry[*sub_client.TopicSubscriber]), diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index 31d2a8082..9cdbe8325 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/weed/mq/broker/broker_grpc_sub.go @@ -68,7 +68,9 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs } else { defer func() { println("closing SubscribeFollowMe connection", follower) - subscribeFollowMeStream.CloseSend() + if subscribeFollowMeStream != nil { + subscribeFollowMeStream.CloseSend() + } // followerGrpcConnection.Close() }() followerClient := mq_pb.NewSeaweedMessagingClient(followerGrpcConnection) @@ -142,7 +144,9 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs Close: &mq_pb.SubscribeFollowMeRequest_CloseMessage{}, }, }); err != nil { - glog.Errorf("Error sending close to follower: %v", err) + if err != io.EOF { + glog.Errorf("Error sending close to follower: %v", err) + } } } }() @@ -178,6 +182,19 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs for imt.IsInflight(logEntry.Key) { time.Sleep(137 * time.Millisecond) + // Check if the client has disconnected by monitoring the context + select { + case <-ctx.Done(): + err := ctx.Err() + if err == context.Canceled { + // Client disconnected + return false, nil + } + glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err) + return false, nil + default: + // Continue processing the request + } } if logEntry.Key != nil { imt.EnflightMessage(logEntry.Key, logEntry.TsNs) @@ -204,20 +221,35 @@ func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMess return } offset := initMessage.GetPartitionOffset() - if offset.StartTsNs != 0 { + offsetType := initMessage.OffsetType + + // reset to earliest or latest + if offsetType == schema_pb.OffsetType_RESET_TO_EARLIEST { + startPosition = log_buffer.NewMessagePosition(1, -3) + return + } + if offsetType == schema_pb.OffsetType_RESET_TO_LATEST { + startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4) + return + } + + // use the exact timestamp + if offsetType == schema_pb.OffsetType_EXACT_TS_NS { startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2) return } + + // try to resume if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil { glog.V(0).Infof("resume from saved offset %v %v %v: %v", initMessage.Topic, initMessage.PartitionOffset.Partition, initMessage.ConsumerGroup, storedOffset) startPosition = log_buffer.NewMessagePosition(storedOffset, -2) return } - if offset.StartType == schema_pb.PartitionOffsetStartType_EARLIEST { - startPosition = log_buffer.NewMessagePosition(1, -3) - } else if offset.StartType == schema_pb.PartitionOffsetStartType_LATEST { - startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4) + if offsetType == schema_pb.OffsetType_RESUME_OR_EARLIEST { + startPosition = log_buffer.NewMessagePosition(1, -5) + } else if offsetType == schema_pb.OffsetType_RESUME_OR_LATEST { + startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -6) } return } diff --git a/weed/mq/client/agent_client/agent_publish.go b/weed/mq/client/agent_client/agent_publish.go deleted file mode 100644 index 3e16f3b99..000000000 --- a/weed/mq/client/agent_client/agent_publish.go +++ /dev/null @@ -1,14 +0,0 @@ -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 deleted file mode 100644 index 626a3a123..000000000 --- a/weed/mq/client/agent_client/agent_subscribe.go +++ /dev/null @@ -1,17 +0,0 @@ -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 index 45d46f553..c12d345a1 100644 --- a/weed/mq/client/agent_client/publish_session.go +++ b/weed/mq/client/agent_client/publish_session.go @@ -4,10 +4,10 @@ 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" + "google.golang.org/grpc/credentials/insecure" ) type PublishSession struct { @@ -15,13 +15,12 @@ type PublishSession struct { 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()) + clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err) } @@ -48,12 +47,17 @@ func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitio return nil, fmt.Errorf("publish record: %v", err) } + if err = stream.Send(&mq_agent_pb.PublishRecordRequest{ + SessionId: resp.SessionId, + }); err != nil { + return nil, fmt.Errorf("send session id: %v", err) + } + return &PublishSession{ schema: topicSchema, partitionCount: partitionCount, publisherName: publisherName, stream: stream, - sessionId: resp.SessionId, }, nil } @@ -68,3 +72,10 @@ func (a *PublishSession) CloseSession() error { a.schema = nil return err } + +func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error { + return a.stream.Send(&mq_agent_pb.PublishRecordRequest{ + Key: key, + Value: record, + }) +} diff --git a/weed/mq/client/agent_client/SubscribeSession.go b/weed/mq/client/agent_client/subscribe_session.go index fc87e17ca..397450a29 100644 --- a/weed/mq/client/agent_client/SubscribeSession.go +++ b/weed/mq/client/agent_client/subscribe_session.go @@ -4,50 +4,48 @@ 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" + "google.golang.org/grpc/credentials/insecure" ) type SubscribeOption struct { ConsumerGroup string ConsumerGroupInstanceId string Topic topic.Topic + OffsetType schema_pb.OffsetType + OffsetTsNs int64 Filter string MaxSubscribedPartitions int32 - PerPartitionConcurrency int32 + SlidingWindowSize int32 } type SubscribeSession struct { - Option *SubscribeOption - stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse] - sessionId int64 + Option *SubscribeOption + stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse] } 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()) + clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials())) 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{ + initRequest := &mq_agent_pb.SubscribeRecordRequest_InitSubscribeRecordRequest{ ConsumerGroup: option.ConsumerGroup, ConsumerGroupInstanceId: option.ConsumerGroupInstanceId, Topic: &schema_pb.Topic{ Namespace: option.Topic.Namespace, Name: option.Topic.Name, }, + OffsetType: option.OffsetType, + OffsetTsNs: option.OffsetTsNs, 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) + SlidingWindowSize: option.SlidingWindowSize, } stream, err := agentClient.SubscribeRecord(context.Background()) @@ -55,9 +53,35 @@ func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*Subscri return nil, fmt.Errorf("subscribe record: %v", err) } + if err = stream.Send(&mq_agent_pb.SubscribeRecordRequest{ + Init: initRequest, + }); err != nil { + return nil, fmt.Errorf("send session id: %v", err) + } + return &SubscribeSession{ - Option: option, - stream: stream, - sessionId: resp.SessionId, + Option: option, + stream: stream, }, nil } + +func (s *SubscribeSession) CloseSession() error { + err := s.stream.CloseSend() + return err +} + +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) + } + if onCompletionFn != nil { + onCompletionFn() + } + return nil +} diff --git a/weed/mq/client/cmd/weed_pub_kv/publisher_kv.go b/weed/mq/client/cmd/weed_pub_kv/publisher_kv.go deleted file mode 100644 index b4d07ae02..000000000 --- a/weed/mq/client/cmd/weed_pub_kv/publisher_kv.go +++ /dev/null @@ -1,78 +0,0 @@ -package main - -import ( - "flag" - "fmt" - "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" - "github.com/seaweedfs/seaweedfs/weed/mq/topic" - util_http "github.com/seaweedfs/seaweedfs/weed/util/http" - "log" - "strings" - "sync" - "time" -) - -var ( - messageCount = flag.Int("n", 1000, "message count") - 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") -) - -func doPublish(publisher *pub_client.TopicPublisher, id int) { - startTime := time.Now() - for i := 0; i < *messageCount / *concurrency; i++ { - // Simulate publishing a message - key := []byte(fmt.Sprintf("key-%s-%d-%d", *clientName, id, i)) - value := []byte(fmt.Sprintf("value-%s-%d-%d", *clientName, id, i)) - if err := publisher.Publish(key, value); err != nil { - fmt.Println(err) - break - } - time.Sleep(time.Second) - // println("Published", string(key), string(value)) - } - if err := publisher.FinishPublish(); err != nil { - fmt.Println(err) - } - elapsed := time.Since(startTime) - log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed) -} - -func main() { - flag.Parse() - util_http.InitGlobalHttpClient() - - config := &pub_client.PublisherConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - PartitionCount: int32(*partitionCount), - Brokers: strings.Split(*seedBrokers, ","), - PublisherName: *clientName, - } - publisher := pub_client.NewTopicPublisher(config) - - 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(publisher, id) - }(i) - } - - // Wait for all publishers to finish - wg.Wait() - elapsed := time.Since(startTime) - publisher.Shutdown() - - log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds()) - -} diff --git a/weed/mq/client/cmd/weed_pub_record/publisher_record.go b/weed/mq/client/cmd/weed_pub_record/publisher_record.go deleted file mode 100644 index 9b28200bc..000000000 --- a/weed/mq/client/cmd/weed_pub_record/publisher_record.go +++ /dev/null @@ -1,138 +0,0 @@ -package main - -import ( - "flag" - "fmt" - "github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client" - "github.com/seaweedfs/seaweedfs/weed/mq/schema" - "github.com/seaweedfs/seaweedfs/weed/mq/topic" - "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" - util_http "github.com/seaweedfs/seaweedfs/weed/util/http" - "log" - "strings" - "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 *pub_client.TopicPublisher, 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.PublishRecord(myRecord.Key, myRecord.ToRecordValue()); err != nil { - fmt.Println(err) - break - } - if *messageDelay > 0 { - time.Sleep(*messageDelay) - fmt.Printf("sent %+v\n", string(myRecord.Key)) - } - } - if err := publisher.FinishPublish(); err != nil { - fmt.Println(err) - } - 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() - util_http.InitGlobalHttpClient() - - 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() - - config := &pub_client.PublisherConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - PartitionCount: int32(*partitionCount), - Brokers: strings.Split(*seedBrokers, ","), - PublisherName: *clientName, - RecordType: recordType, - } - publisher := pub_client.NewTopicPublisher(config) - - 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(publisher, id) - }(i) - } - - // Wait for all publishers to finish - wg.Wait() - elapsed := time.Since(startTime) - publisher.Shutdown() - - 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 deleted file mode 100644 index f925aa1e1..000000000 --- a/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go +++ /dev/null @@ -1,64 +0,0 @@ -package main - -import ( - "flag" - "fmt" - "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" -) - -var ( - namespace = flag.String("ns", "test", "namespace") - t = flag.String("topic", "test", "topic") - seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") - maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count") - perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency") - - clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id") -) - -func main() { - flag.Parse() - util_http.InitGlobalHttpClient() - - subscriberConfig := &sub_client.SubscriberConfiguration{ - ConsumerGroup: "test", - ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), - GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), - MaxPartitionCount: int32(*maxPartitionCount), - SlidingWindowSize: int32(*perPartitionConcurrency), - } - - contentConfig := &sub_client.ContentConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - Filter: "", - } - - brokers := strings.Split(*seedBrokers, ",") - subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024)) - - counter := 0 - 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() { - glog.V(0).Infof("done received %d messages", counter) - }) - - if err := subscriber.Subscribe(); err != nil { - fmt.Println(err) - } - -} diff --git a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go b/weed/mq/client/cmd/weed_sub_record/subscriber_record.go deleted file mode 100644 index dee289f19..000000000 --- a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go +++ /dev/null @@ -1,99 +0,0 @@ -package main - -import ( - "flag" - "fmt" - "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" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/protobuf/proto" - "strings" - "time" -) - -var ( - namespace = flag.String("ns", "test", "namespace") - t = flag.String("topic", "test", "topic") - seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers") - maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count") - perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency") - timeAgo = flag.Duration("timeAgo", 1*time.Hour, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") - - clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id") -) - -type MyRecord struct { - Key []byte - Field1 []byte - Field2 string - Field3 int32 - Field4 int64 - Field5 float32 - Field6 float64 - Field7 bool -} - -func FromSchemaRecordValue(recordValue *schema_pb.RecordValue) *MyRecord { - return &MyRecord{ - Key: recordValue.Fields["key"].GetBytesValue(), - Field1: recordValue.Fields["field1"].GetBytesValue(), - Field2: recordValue.Fields["field2"].GetStringValue(), - Field3: recordValue.Fields["field3"].GetInt32Value(), - Field4: recordValue.Fields["field4"].GetInt64Value(), - Field5: recordValue.Fields["field5"].GetFloatValue(), - Field6: recordValue.Fields["field6"].GetDoubleValue(), - Field7: recordValue.Fields["field7"].GetBoolValue(), - } -} - -func main() { - flag.Parse() - util_http.InitGlobalHttpClient() - - subscriberConfig := &sub_client.SubscriberConfiguration{ - ConsumerGroup: "test", - ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId), - GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()), - MaxPartitionCount: int32(*maxPartitionCount), - SlidingWindowSize: int32(*perPartitionConcurrency), - } - - contentConfig := &sub_client.ContentConfiguration{ - Topic: topic.NewTopic(*namespace, *t), - Filter: "", - // StartTime: time.Now().Add(-*timeAgo), - } - - brokers := strings.Split(*seedBrokers, ",") - subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024)) - - counter := 0 - 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() { - glog.V(0).Infof("done received %d messages", counter) - }) - - if err := subscriber.Subscribe(); err != nil { - fmt.Println(err) - } - -} diff --git a/weed/mq/client/pub_client/publisher.go b/weed/mq/client/pub_client/publisher.go index 04c2ebe08..f95d00602 100644 --- a/weed/mq/client/pub_client/publisher.go +++ b/weed/mq/client/pub_client/publisher.go @@ -34,8 +34,8 @@ type TopicPublisher struct { jobs []*EachPartitionPublishJob } -func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher { - tp := &TopicPublisher{ +func NewTopicPublisher(config *PublisherConfiguration) (tp *TopicPublisher, err error) { + tp = &TopicPublisher{ partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int { return int(a - b) }), @@ -46,7 +46,7 @@ func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher { wg := sync.WaitGroup{} wg.Add(1) go func() { - if err := tp.startSchedulerThread(&wg); err != nil { + if err = tp.startSchedulerThread(&wg); err != nil { log.Println(err) return } @@ -54,7 +54,7 @@ func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher { wg.Wait() - return tp + return } func (p *TopicPublisher) Shutdown() error { diff --git a/weed/mq/client/pub_client/scheduler.go b/weed/mq/client/pub_client/scheduler.go index df2270b2c..a768fa7f8 100644 --- a/weed/mq/client/pub_client/scheduler.go +++ b/weed/mq/client/pub_client/scheduler.go @@ -7,7 +7,9 @@ import ( "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/util/buffered_queue" + "google.golang.org/grpc" "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/status" "log" "sort" @@ -33,6 +35,7 @@ type EachPartitionPublishJob struct { func (p *TopicPublisher) startSchedulerThread(wg *sync.WaitGroup) error { if err := p.doConfigureTopic(); err != nil { + wg.Done() return fmt.Errorf("configure topic %s: %v", p.config.Topic, err) } @@ -111,6 +114,7 @@ func (p *TopicPublisher) onEachAssignments(generation int, assignments []*mq_pb. go func(job *EachPartitionPublishJob) { defer job.wg.Done() if err := p.doPublishToPartition(job); err != nil { + log.Printf("publish to %s partition %v: %v", p.config.Topic, job.Partition, err) errChan <- EachPartitionError{assignment, err, generation} } }(job) @@ -126,7 +130,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro log.Printf("connecting to %v for topic partition %+v", job.LeaderBroker, job.Partition) - grpcConnection, err := pb.GrpcDial(context.Background(), job.LeaderBroker, true, p.grpcDialOption) + grpcConnection, err := grpc.NewClient(job.LeaderBroker, grpc.WithTransportCredentials(insecure.NewCredentials()), p.grpcDialOption) if err != nil { return fmt.Errorf("dial broker %s: %v", job.LeaderBroker, err) } @@ -225,7 +229,7 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro func (p *TopicPublisher) doConfigureTopic() (err error) { if len(p.config.Brokers) == 0 { - return fmt.Errorf("no bootstrap brokers") + return fmt.Errorf("topic configuring found no bootstrap brokers") } var lastErr error for _, brokerAddress := range p.config.Brokers { @@ -256,7 +260,7 @@ func (p *TopicPublisher) doConfigureTopic() (err error) { func (p *TopicPublisher) doLookupTopicPartitions() (assignments []*mq_pb.BrokerPartitionAssignment, err error) { if len(p.config.Brokers) == 0 { - return nil, fmt.Errorf("no bootstrap brokers") + return nil, fmt.Errorf("lookup found no bootstrap brokers") } var lastErr error for _, brokerAddress := range p.config.Brokers { diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go index d05ddb960..e88aaca2f 100644 --- a/weed/mq/client/sub_client/connect_to_sub_coordinator.go +++ b/weed/mq/client/sub_client/connect_to_sub_coordinator.go @@ -1,7 +1,6 @@ package sub_client import ( - "context" "github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/pb" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" @@ -12,10 +11,17 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() { waitTime := 1 * time.Second for { for _, broker := range sub.bootstrapBrokers { + + select { + case <-sub.ctx.Done(): + return + default: + } + // lookup topic brokers var brokerLeader string err := pb.WithBrokerGrpcClient(false, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { - resp, err := client.FindBrokerLeader(context.Background(), &mq_pb.FindBrokerLeaderRequest{}) + resp, err := client.FindBrokerLeader(sub.ctx, &mq_pb.FindBrokerLeaderRequest{}) if err != nil { return err } @@ -30,10 +36,8 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() { // connect to the balancer pb.WithBrokerGrpcClient(true, brokerLeader, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stream, err := client.SubscriberToSubCoordinator(ctx) + stream, err := client.SubscriberToSubCoordinator(sub.ctx) if err != nil { glog.V(0).Infof("subscriber %s: %v", sub.ContentConfig.Topic, err) return err @@ -58,6 +62,13 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() { go func() { for reply := range sub.brokerPartitionAssignmentAckChan { + + select { + case <-sub.ctx.Done(): + return + default: + } + glog.V(0).Infof("subscriber instance %s ack %+v", sub.SubscriberConfig.ConsumerGroupInstanceId, reply) if err := stream.Send(reply); err != nil { glog.V(0).Infof("subscriber %s reply: %v", sub.ContentConfig.Topic, err) @@ -73,6 +84,13 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() { glog.V(0).Infof("subscriber %s receive: %v", sub.ContentConfig.Topic, err) return err } + + select { + case <-sub.ctx.Done(): + return nil + default: + } + sub.brokerPartitionAssignmentChan <- resp glog.V(0).Infof("Received assignment: %+v", resp) } diff --git a/weed/mq/client/sub_client/on_each_partition.go b/weed/mq/client/sub_client/on_each_partition.go index fbfcc3c6b..14a38cfa8 100644 --- a/weed/mq/client/sub_client/on_each_partition.go +++ b/weed/mq/client/sub_client/on_each_partition.go @@ -2,14 +2,13 @@ package sub_client import ( "context" + "errors" "fmt" "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/pb/schema_pb" "io" - "reflect" - "time" ) type KeyedOffset struct { @@ -35,8 +34,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig if po == nil { po = &schema_pb.PartitionOffset{ Partition: assigned.Partition, - StartTsNs: time.Now().UnixNano(), - StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY, + StartTsNs: sub.ContentConfig.OffsetTsNs, } } @@ -47,6 +45,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId, Topic: sub.ContentConfig.Topic.ToPbTopic(), PartitionOffset: po, + OffsetType: sub.ContentConfig.OffsetType, Filter: sub.ContentConfig.Filter, FollowerBroker: assigned.FollowerBroker, SlidingWindowSize: slidingWindowSize, @@ -65,6 +64,9 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig go func() { for { select { + case <-sub.ctx.Done(): + subscribeClient.CloseSend() + return case <-stopCh: subscribeClient.CloseSend() return @@ -86,15 +88,27 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig }() for { - // glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup) + // glog.V(0).Infof("subscriber %s/%s waiting for message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup) resp, err := subscribeClient.Recv() if err != nil { + if errors.Is(err, io.EOF) { + return nil + } return fmt.Errorf("subscribe recv: %v", err) } if resp.Message == nil { glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup) continue } + + select { + case <-sub.ctx.Done(): + return nil + case <-stopCh: + return nil + default: + } + switch m := resp.Message.(type) { case *mq_pb.SubscribeMessageResponse_Data: if m.Data.Ctrl != nil { @@ -102,7 +116,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig continue } if len(m.Data.Key) == 0 { - fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m)) + // fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m)) continue } onDataMessageFn(m) diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go index cf2294891..d4dea3852 100644 --- a/weed/mq/client/sub_client/subscribe.go +++ b/weed/mq/client/sub_client/subscribe.go @@ -72,12 +72,12 @@ func (sub *TopicSubscriber) startProcessors() { 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, - } + if sub.OnDataMessageFunc != nil { + sub.OnDataMessageFunc(m) + } + sub.PartitionOffsetChan <- KeyedOffset{ + Key: m.Data.Key, + Offset: m.Data.TsNs, } }) } diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go index 9a51ce01e..ec15d998e 100644 --- a/weed/mq/client/sub_client/subscriber.go +++ b/weed/mq/client/sub_client/subscriber.go @@ -1,6 +1,7 @@ package sub_client import ( + "context" "github.com/seaweedfs/seaweedfs/weed/mq/topic" "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb" "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" @@ -17,50 +18,50 @@ type SubscriberConfiguration struct { SlidingWindowSize int32 // how many messages to process concurrently per partition } +func (s *SubscriberConfiguration) String() string { + return "ClientId: " + s.ClientId + ", ConsumerGroup: " + s.ConsumerGroup + ", ConsumerGroupInstanceId: " + s.ConsumerGroupInstanceId +} + type ContentConfiguration struct { Topic topic.Topic Filter string PartitionOffsets []*schema_pb.PartitionOffset + OffsetType schema_pb.OffsetType + OffsetTsNs int64 } type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data) -type OnEachMessageFunc func(key, value []byte) (err error) type OnCompletionFunc func() type TopicSubscriber struct { + ctx context.Context SubscriberConfig *SubscriberConfiguration ContentConfig *ContentConfiguration brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest - OnDataMessageFnnc OnDataMessageFn - OnEachMessageFunc OnEachMessageFunc + OnDataMessageFunc OnDataMessageFn 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, partitionOffsetChan chan KeyedOffset) *TopicSubscriber { +func NewTopicSubscriber(ctx context.Context, bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber { return &TopicSubscriber{ + ctx: ctx, SubscriberConfig: subscriber, ContentConfig: content, brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024), brokerPartitionAssignmentAckChan: make(chan *mq_pb.SubscriberToSubCoordinatorRequest, 1024), bootstrapBrokers: bootstrapBrokers, - waitForMoreMessage: true, activeProcessors: make(map[topic.Partition]*ProcessorState), PartitionOffsetChan: partitionOffsetChan, } } -func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc) { - sub.OnEachMessageFunc = onEachMessageFn -} - func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) { - sub.OnDataMessageFnnc = fn + sub.OnDataMessageFunc = fn } func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) { diff --git a/weed/mq/logstore/read_log_from_disk.go b/weed/mq/logstore/read_log_from_disk.go index c3c679b87..71ba58c1f 100644 --- a/weed/mq/logstore/read_log_from_disk.go +++ b/weed/mq/logstore/read_log_from_disk.go @@ -36,7 +36,7 @@ func GenLogOnDiskReadFunc(filerClient filer_pb.FilerClient, t topic.Topic, p top err = fmt.Errorf("unexpected unmarshal mq_pb.Message: %v", err) return } - if logEntry.TsNs < starTsNs { + if logEntry.TsNs <= starTsNs { pos += 4 + int(size) continue } diff --git a/weed/mq/logstore/read_parquet_to_log.go b/weed/mq/logstore/read_parquet_to_log.go index f55d5e3b7..4811d6c80 100644 --- a/weed/mq/logstore/read_parquet_to_log.go +++ b/weed/mq/logstore/read_parquet_to_log.go @@ -73,7 +73,7 @@ func GenParquetReadFunc(filerClient filer_pb.FilerClient, t topic.Topic, p topic return processedTsNs, fmt.Errorf("ToRecordValue failed: %v", err) } processedTsNs = recordValue.Fields[SW_COLUMN_NAME_TS].GetInt64Value() - if processedTsNs < starTsNs { + if processedTsNs <= starTsNs { continue } if stopTsNs != 0 && processedTsNs >= stopTsNs { diff --git a/weed/mq/sub_coordinator/market.go b/weed/mq/sub_coordinator/market.go index cce6be4e1..df07edfd5 100644 --- a/weed/mq/sub_coordinator/market.go +++ b/weed/mq/sub_coordinator/market.go @@ -20,7 +20,7 @@ Trigger rebalance when: * A new consumer group instance is added * Some partitions are unassigned from a consumer group instance. -If multiple reblance requests are received, after a certain period, the market will only process the latest request. +If multiple rebalance requests are received, after a certain period, the market will only process the latest request. However, if the number of unassigned partition is increased to exactly the total number of partitions, and total partitions are less than or equal to the sum of the max partition count of all consumer group instances, diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go index e32fc2398..d1433775a 100644 --- a/weed/mq/topic/local_partition.go +++ b/weed/mq/topic/local_partition.go @@ -89,12 +89,16 @@ func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.M return nil } - startPosition = processedPosition + if processedPosition.Time.UnixNano() != 0 { + startPosition = processedPosition + } processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn) if isDone { return nil } - startPosition = processedPosition + if processedPosition.Time.UnixNano() != 0 { + startPosition = processedPosition + } if readInMemoryLogErr == log_buffer.ResumeFromDiskError { continue diff --git a/weed/pb/mq_agent.proto b/weed/pb/mq_agent.proto index 605f2a31c..91f5a4cfc 100644 --- a/weed/pb/mq_agent.proto +++ b/weed/pb/mq_agent.proto @@ -21,10 +21,6 @@ service SeaweedMessagingAgent { } // Subscribing - rpc StartSubscribeSession (StartSubscribeSessionRequest) returns (StartSubscribeSessionResponse) { - } - rpc CloseSubscribeSession (CloseSubscribeSessionRequest) returns (CloseSubscribeSessionResponse) { - } rpc SubscribeRecord (stream SubscribeRecordRequest) returns (stream SubscribeRecordResponse) { } @@ -59,28 +55,19 @@ message PublishRecordResponse { 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 + message InitSubscribeRecordRequest { + string consumer_group = 1; + string consumer_group_instance_id = 2; + schema_pb.Topic topic = 4; + repeated schema_pb.PartitionOffset partition_offsets = 5; + schema_pb.OffsetType offset_type = 6; + int64 offset_ts_ns = 7; + string filter = 10; + int32 max_subscribed_partitions = 11; + int32 sliding_window_size = 12; + } + InitSubscribeRecordRequest init = 1; int64 ack_sequence = 2; bytes ack_key = 3; } diff --git a/weed/pb/mq_agent_pb/mq_agent.pb.go b/weed/pb/mq_agent_pb/mq_agent.pb.go index 9b280c076..fc6b9a00b 100644 --- a/weed/pb/mq_agent_pb/mq_agent.pb.go +++ b/weed/pb/mq_agent_pb/mq_agent.pb.go @@ -362,22 +362,18 @@ func (x *PublishRecordResponse) GetError() string { } // //////////////////////////////////////////////// -type StartSubscribeSessionRequest struct { +type SubscribeRecordRequest 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"` + Init *SubscribeRecordRequest_InitSubscribeRecordRequest `protobuf:"bytes,1,opt,name=init,proto3" json:"init,omitempty"` + 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 *StartSubscribeSessionRequest) Reset() { - *x = StartSubscribeSessionRequest{} +func (x *SubscribeRecordRequest) Reset() { + *x = SubscribeRecordRequest{} if protoimpl.UnsafeEnabled { mi := &file_mq_agent_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -385,13 +381,13 @@ func (x *StartSubscribeSessionRequest) Reset() { } } -func (x *StartSubscribeSessionRequest) String() string { +func (x *SubscribeRecordRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*StartSubscribeSessionRequest) ProtoMessage() {} +func (*SubscribeRecordRequest) ProtoMessage() {} -func (x *StartSubscribeSessionRequest) ProtoReflect() protoreflect.Message { +func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message { mi := &file_mq_agent_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -403,71 +399,47 @@ func (x *StartSubscribeSessionRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StartSubscribeSessionRequest.ProtoReflect.Descriptor instead. -func (*StartSubscribeSessionRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use SubscribeRecordRequest.ProtoReflect.Descriptor instead. +func (*SubscribeRecordRequest) 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 { +func (x *SubscribeRecordRequest) GetInit() *SubscribeRecordRequest_InitSubscribeRecordRequest { if x != nil { - return x.ConsumerGroupInstanceId - } - return "" -} - -func (x *StartSubscribeSessionRequest) GetTopic() *schema_pb.Topic { - if x != nil { - return x.Topic + return x.Init } 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 { +func (x *SubscribeRecordRequest) GetAckSequence() int64 { if x != nil { - return x.MaxSubscribedPartitions + return x.AckSequence } return 0 } -func (x *StartSubscribeSessionRequest) GetSlidingWindowSize() int32 { +func (x *SubscribeRecordRequest) GetAckKey() []byte { if x != nil { - return x.SlidingWindowSize + return x.AckKey } - return 0 + return nil } -type StartSubscribeSessionResponse struct { +type SubscribeRecordResponse 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"` + 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 *StartSubscribeSessionResponse) Reset() { - *x = StartSubscribeSessionResponse{} +func (x *SubscribeRecordResponse) Reset() { + *x = SubscribeRecordResponse{} if protoimpl.UnsafeEnabled { mi := &file_mq_agent_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -475,13 +447,13 @@ func (x *StartSubscribeSessionResponse) Reset() { } } -func (x *StartSubscribeSessionResponse) String() string { +func (x *SubscribeRecordResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*StartSubscribeSessionResponse) ProtoMessage() {} +func (*SubscribeRecordResponse) ProtoMessage() {} -func (x *StartSubscribeSessionResponse) ProtoReflect() protoreflect.Message { +func (x *SubscribeRecordResponse) ProtoReflect() protoreflect.Message { mi := &file_mq_agent_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -493,147 +465,86 @@ func (x *StartSubscribeSessionResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StartSubscribeSessionResponse.ProtoReflect.Descriptor instead. -func (*StartSubscribeSessionResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use SubscribeRecordResponse.ProtoReflect.Descriptor instead. +func (*SubscribeRecordResponse) Descriptor() ([]byte, []int) { return file_mq_agent_proto_rawDescGZIP(), []int{7} } -func (x *StartSubscribeSessionResponse) GetError() string { +func (x *SubscribeRecordResponse) GetKey() []byte { if x != nil { - return x.Error + return x.Key } - return "" + return nil } -func (x *StartSubscribeSessionResponse) GetSessionId() int64 { +func (x *SubscribeRecordResponse) GetValue() *schema_pb.RecordValue { 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 x.Value } - return mi.MessageOf(x) -} - -// Deprecated: Use CloseSubscribeSessionRequest.ProtoReflect.Descriptor instead. -func (*CloseSubscribeSessionRequest) Descriptor() ([]byte, []int) { - return file_mq_agent_proto_rawDescGZIP(), []int{8} + return nil } -func (x *CloseSubscribeSessionRequest) GetSessionId() int64 { +func (x *SubscribeRecordResponse) GetTsNs() int64 { if x != nil { - return x.SessionId + return x.TsNs } 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 *SubscribeRecordResponse) GetError() string { + if x != nil { + return x.Error } + return "" } -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 +func (x *SubscribeRecordResponse) GetIsEndOfStream() bool { + if x != nil { + return x.IsEndOfStream } - return mi.MessageOf(x) -} - -// Deprecated: Use CloseSubscribeSessionResponse.ProtoReflect.Descriptor instead. -func (*CloseSubscribeSessionResponse) Descriptor() ([]byte, []int) { - return file_mq_agent_proto_rawDescGZIP(), []int{9} + return false } -func (x *CloseSubscribeSessionResponse) GetError() string { +func (x *SubscribeRecordResponse) GetIsEndOfTopic() bool { if x != nil { - return x.Error + return x.IsEndOfTopic } - return "" + return false } -// //////////////////////////////////////////////// -type SubscribeRecordRequest struct { +type SubscribeRecordRequest_InitSubscribeRecordRequest 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"` + 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"` + OffsetType schema_pb.OffsetType `protobuf:"varint,6,opt,name=offset_type,json=offsetType,proto3,enum=schema_pb.OffsetType" json:"offset_type,omitempty"` + OffsetTsNs int64 `protobuf:"varint,7,opt,name=offset_ts_ns,json=offsetTsNs,proto3" json:"offset_ts_ns,omitempty"` + Filter string `protobuf:"bytes,10,opt,name=filter,proto3" json:"filter,omitempty"` + MaxSubscribedPartitions int32 `protobuf:"varint,11,opt,name=max_subscribed_partitions,json=maxSubscribedPartitions,proto3" json:"max_subscribed_partitions,omitempty"` + SlidingWindowSize int32 `protobuf:"varint,12,opt,name=sliding_window_size,json=slidingWindowSize,proto3" json:"sliding_window_size,omitempty"` } -func (x *SubscribeRecordRequest) Reset() { - *x = SubscribeRecordRequest{} +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) Reset() { + *x = SubscribeRecordRequest_InitSubscribeRecordRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_agent_proto_msgTypes[10] + mi := &file_mq_agent_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeRecordRequest) String() string { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeRecordRequest) ProtoMessage() {} +func (*SubscribeRecordRequest_InitSubscribeRecordRequest) ProtoMessage() {} -func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_agent_proto_msgTypes[10] +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) 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 { @@ -644,125 +555,72 @@ func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeRecordRequest.ProtoReflect.Descriptor instead. -func (*SubscribeRecordRequest) Descriptor() ([]byte, []int) { - return file_mq_agent_proto_rawDescGZIP(), []int{10} +// Deprecated: Use SubscribeRecordRequest_InitSubscribeRecordRequest.ProtoReflect.Descriptor instead. +func (*SubscribeRecordRequest_InitSubscribeRecordRequest) Descriptor() ([]byte, []int) { + return file_mq_agent_proto_rawDescGZIP(), []int{6, 0} } -func (x *SubscribeRecordRequest) GetSessionId() int64 { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetConsumerGroup() string { if x != nil { - return x.SessionId + return x.ConsumerGroup } - return 0 + return "" } -func (x *SubscribeRecordRequest) GetAckSequence() int64 { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetConsumerGroupInstanceId() string { if x != nil { - return x.AckSequence + return x.ConsumerGroupInstanceId } - return 0 + return "" } -func (x *SubscribeRecordRequest) GetAckKey() []byte { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetTopic() *schema_pb.Topic { if x != nil { - return x.AckKey + return x.Topic } 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 { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetPartitionOffsets() []*schema_pb.PartitionOffset { if x != nil { - return x.Sequence - } - return 0 -} - -func (x *SubscribeRecordResponse) GetKey() []byte { - if x != nil { - return x.Key + return x.PartitionOffsets } return nil } -func (x *SubscribeRecordResponse) GetValue() *schema_pb.RecordValue { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetOffsetType() schema_pb.OffsetType { if x != nil { - return x.Value + return x.OffsetType } - return nil + return schema_pb.OffsetType(0) } -func (x *SubscribeRecordResponse) GetTsNs() int64 { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetOffsetTsNs() int64 { if x != nil { - return x.TsNs + return x.OffsetTsNs } return 0 } -func (x *SubscribeRecordResponse) GetError() string { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetFilter() string { if x != nil { - return x.Error + return x.Filter } return "" } -func (x *SubscribeRecordResponse) GetIsEndOfStream() bool { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetMaxSubscribedPartitions() int32 { if x != nil { - return x.IsEndOfStream + return x.MaxSubscribedPartitions } - return false + return 0 } -func (x *SubscribeRecordResponse) GetIsEndOfTopic() bool { +func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetSlidingWindowSize() int32 { if x != nil { - return x.IsEndOfTopic + return x.SlidingWindowSize } - return false + return 0 } var File_mq_agent_proto protoreflect.FileDescriptor @@ -808,115 +666,95 @@ var file_mq_agent_proto_rawDesc = []byte{ 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, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xfb, 0x04, + 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, 0x53, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 0x2e, 0x49, 0x6e, 0x69, + 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 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, 0x1a, 0xcf, 0x03, 0x0a, 0x1a, 0x49, 0x6e, + 0x69, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, + 0x64, 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, 0x36, 0x0a, + 0x0b, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x4f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x6f, 0x66, 0x66, 0x73, 0x65, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, + 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6f, 0x66, 0x66, + 0x73, 0x65, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x18, 0x0a, 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, 0x0b, 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, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x73, 0x6c, 0x69, 0x64, 0x69, 0x6e, + 0x67, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x69, 0x7a, 0x65, 0x22, 0xd4, 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, 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, 0xb9, 0x03, 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, 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 ( @@ -931,49 +769,45 @@ func file_mq_agent_proto_rawDescGZIP() []byte { return file_mq_agent_proto_rawDescData } -var file_mq_agent_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_mq_agent_proto_msgTypes = make([]protoimpl.MessageInfo, 9) 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 + (*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 + (*SubscribeRecordRequest)(nil), // 6: messaging_pb.SubscribeRecordRequest + (*SubscribeRecordResponse)(nil), // 7: messaging_pb.SubscribeRecordResponse + (*SubscribeRecordRequest_InitSubscribeRecordRequest)(nil), // 8: messaging_pb.SubscribeRecordRequest.InitSubscribeRecordRequest + (*schema_pb.Topic)(nil), // 9: schema_pb.Topic + (*schema_pb.RecordType)(nil), // 10: schema_pb.RecordType + (*schema_pb.RecordValue)(nil), // 11: schema_pb.RecordValue + (*schema_pb.PartitionOffset)(nil), // 12: schema_pb.PartitionOffset + (schema_pb.OffsetType)(0), // 13: schema_pb.OffsetType } 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 + 9, // 0: messaging_pb.StartPublishSessionRequest.topic:type_name -> schema_pb.Topic + 10, // 1: messaging_pb.StartPublishSessionRequest.record_type:type_name -> schema_pb.RecordType + 11, // 2: messaging_pb.PublishRecordRequest.value:type_name -> schema_pb.RecordValue + 8, // 3: messaging_pb.SubscribeRecordRequest.init:type_name -> messaging_pb.SubscribeRecordRequest.InitSubscribeRecordRequest + 11, // 4: messaging_pb.SubscribeRecordResponse.value:type_name -> schema_pb.RecordValue + 9, // 5: messaging_pb.SubscribeRecordRequest.InitSubscribeRecordRequest.topic:type_name -> schema_pb.Topic + 12, // 6: messaging_pb.SubscribeRecordRequest.InitSubscribeRecordRequest.partition_offsets:type_name -> schema_pb.PartitionOffset + 13, // 7: messaging_pb.SubscribeRecordRequest.InitSubscribeRecordRequest.offset_type:type_name -> schema_pb.OffsetType + 0, // 8: messaging_pb.SeaweedMessagingAgent.StartPublishSession:input_type -> messaging_pb.StartPublishSessionRequest + 2, // 9: messaging_pb.SeaweedMessagingAgent.ClosePublishSession:input_type -> messaging_pb.ClosePublishSessionRequest + 4, // 10: messaging_pb.SeaweedMessagingAgent.PublishRecord:input_type -> messaging_pb.PublishRecordRequest + 6, // 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 + 7, // 15: messaging_pb.SeaweedMessagingAgent.SubscribeRecord:output_type -> messaging_pb.SubscribeRecordResponse + 12, // [12:16] is the sub-list for method output_type + 8, // [8:12] is the sub-list for method input_type + 8, // [8:8] is the sub-list for extension type_name + 8, // [8:8] is the sub-list for extension extendee + 0, // [0:8] is the sub-list for field type_name } func init() { file_mq_agent_proto_init() } @@ -1055,7 +889,7 @@ func file_mq_agent_proto_init() { } } file_mq_agent_proto_msgTypes[6].Exporter = func(v any, i int) any { - switch v := v.(*StartSubscribeSessionRequest); i { + switch v := v.(*SubscribeRecordRequest); i { case 0: return &v.state case 1: @@ -1067,7 +901,7 @@ func file_mq_agent_proto_init() { } } file_mq_agent_proto_msgTypes[7].Exporter = func(v any, i int) any { - switch v := v.(*StartSubscribeSessionResponse); i { + switch v := v.(*SubscribeRecordResponse); i { case 0: return &v.state case 1: @@ -1079,43 +913,7 @@ func file_mq_agent_proto_init() { } } 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 { + switch v := v.(*SubscribeRecordRequest_InitSubscribeRecordRequest); i { case 0: return &v.state case 1: @@ -1133,7 +931,7 @@ func file_mq_agent_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_mq_agent_proto_rawDesc, NumEnums: 0, - NumMessages: 12, + NumMessages: 9, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go b/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go index 241064181..5db13506e 100644 --- a/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go +++ b/weed/pb/mq_agent_pb/mq_agent_grpc.pb.go @@ -19,12 +19,10 @@ import ( 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" + SeaweedMessagingAgent_StartPublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/StartPublishSession" + SeaweedMessagingAgent_ClosePublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/ClosePublishSession" + SeaweedMessagingAgent_PublishRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/PublishRecord" + SeaweedMessagingAgent_SubscribeRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/SubscribeRecord" ) // SeaweedMessagingAgentClient is the client API for SeaweedMessagingAgent service. @@ -36,8 +34,6 @@ type SeaweedMessagingAgentClient interface { 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) } @@ -82,26 +78,6 @@ func (c *seaweedMessagingAgentClient) PublishRecord(ctx context.Context, opts .. // 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...) @@ -124,8 +100,6 @@ type SeaweedMessagingAgentServer interface { 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() } @@ -146,12 +120,6 @@ func (UnimplementedSeaweedMessagingAgentServer) ClosePublishSession(context.Cont 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") } @@ -219,42 +187,6 @@ func _SeaweedMessagingAgent_PublishRecord_Handler(srv interface{}, stream grpc.S // 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}) } @@ -277,14 +209,6 @@ var SeaweedMessagingAgent_ServiceDesc = grpc.ServiceDesc{ MethodName: "ClosePublishSession", Handler: _SeaweedMessagingAgent_ClosePublishSession_Handler, }, - { - MethodName: "StartSubscribeSession", - Handler: _SeaweedMessagingAgent_StartSubscribeSession_Handler, - }, - { - MethodName: "CloseSubscribeSession", - Handler: _SeaweedMessagingAgent_CloseSubscribeSession_Handler, - }, }, Streams: []grpc.StreamDesc{ { diff --git a/weed/pb/mq_broker.proto b/weed/pb/mq_broker.proto index a1c9912b0..f02ae30ea 100644 --- a/weed/pb/mq_broker.proto +++ b/weed/pb/mq_broker.proto @@ -229,9 +229,10 @@ message SubscribeMessageRequest { string client_id = 3; schema_pb.Topic topic = 4; schema_pb.PartitionOffset partition_offset = 5; - string filter = 6; - string follower_broker = 7; - int32 sliding_window_size = 8; + schema_pb.OffsetType offset_type = 6; + string filter = 10; + string follower_broker = 11; + int32 sliding_window_size = 12; } message AckMessage { int64 sequence = 1; diff --git a/weed/pb/mq_pb/mq_broker.pb.go b/weed/pb/mq_pb/mq_broker.pb.go index 56e3500d1..e7e18147d 100644 --- a/weed/pb/mq_pb/mq_broker.pb.go +++ b/weed/pb/mq_pb/mq_broker.pb.go @@ -2571,9 +2571,10 @@ type SubscribeMessageRequest_InitMessage struct { 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"` + OffsetType schema_pb.OffsetType `protobuf:"varint,6,opt,name=offset_type,json=offsetType,proto3,enum=schema_pb.OffsetType" json:"offset_type,omitempty"` + Filter string `protobuf:"bytes,10,opt,name=filter,proto3" json:"filter,omitempty"` + FollowerBroker string `protobuf:"bytes,11,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"` + SlidingWindowSize int32 `protobuf:"varint,12,opt,name=sliding_window_size,json=slidingWindowSize,proto3" json:"sliding_window_size,omitempty"` } func (x *SubscribeMessageRequest_InitMessage) Reset() { @@ -2643,6 +2644,13 @@ func (x *SubscribeMessageRequest_InitMessage) GetPartitionOffset() *schema_pb.Pa return nil } +func (x *SubscribeMessageRequest_InitMessage) GetOffsetType() schema_pb.OffsetType { + if x != nil { + return x.OffsetType + } + return schema_pb.OffsetType(0) +} + func (x *SubscribeMessageRequest_InitMessage) GetFilter() string { if x != nil { return x.Filter @@ -3210,7 +3218,7 @@ var file_mq_broker_proto_rawDesc = []byte{ 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, + 0x73, 0x4e, 0x73, 0x22, 0xfc, 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, @@ -3220,8 +3228,8 @@ var file_mq_broker_proto_rawDesc = []byte{ 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, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0x8a, + 0x03, 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, @@ -3235,183 +3243,187 @@ var file_mq_broker_proto_rawDesc = []byte{ 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, + 0x65, 0x74, 0x12, 0x36, 0x0a, 0x0b, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x70, 0x62, 0x2e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, + 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x18, 0x0a, 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, 0x0b, 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, 0x0c, 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, + 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, 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, 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, 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, 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, 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, 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, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, - 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x12, - 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, - 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, - 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x15, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, - 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, 0x24, - 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, - 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, - 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, - 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, - 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, - 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, - 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 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, + 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, 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, 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, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x12, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x72, 0x0a, 0x15, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, + 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, + 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, + 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, + 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, - 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, + 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, 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, 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, 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 ( @@ -3482,6 +3494,7 @@ var file_mq_broker_proto_goTypes = []any{ (*schema_pb.Partition)(nil), // 51: schema_pb.Partition (*schema_pb.RecordType)(nil), // 52: schema_pb.RecordType (*schema_pb.PartitionOffset)(nil), // 53: schema_pb.PartitionOffset + (schema_pb.OffsetType)(0), // 54: schema_pb.OffsetType } var file_mq_broker_proto_depIdxs = []int32{ 33, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry @@ -3533,41 +3546,42 @@ var file_mq_broker_proto_depIdxs = []int32{ 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 + 54, // 49: messaging_pb.SubscribeMessageRequest.InitMessage.offset_type:type_name -> schema_pb.OffsetType + 50, // 50: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic + 51, // 51: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition + 0, // 52: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 4, // 53: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 6, // 54: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 10, // 55: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 8, // 56: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 12, // 57: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 15, // 58: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 29, // 59: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 31, // 60: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 17, // 61: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 21, // 62: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 25, // 63: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 23, // 64: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 27, // 65: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest + 1, // 66: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 5, // 67: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 7, // 68: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 11, // 69: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 9, // 70: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 13, // 71: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 16, // 72: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 30, // 73: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 32, // 74: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 18, // 75: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 22, // 76: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 26, // 77: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 24, // 78: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 28, // 79: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse + 66, // [66:80] is the sub-list for method output_type + 52, // [52:66] is the sub-list for method input_type + 52, // [52:52] is the sub-list for extension type_name + 52, // [52:52] is the sub-list for extension extendee + 0, // [0:52] is the sub-list for field type_name } func init() { file_mq_broker_proto_init() } diff --git a/weed/pb/mq_schema.proto b/weed/pb/mq_schema.proto index 15b68981c..e2196c5fc 100644 --- a/weed/pb/mq_schema.proto +++ b/weed/pb/mq_schema.proto @@ -24,17 +24,17 @@ message Offset { repeated PartitionOffset partition_offsets = 2; } -enum PartitionOffsetStartType { - EARLIEST = 0; - EARLIEST_IN_MEMORY = 1; - LATEST = 2; +enum OffsetType { + RESUME_OR_EARLIEST = 0; + RESET_TO_EARLIEST = 5; + EXACT_TS_NS = 10; + RESET_TO_LATEST = 15; + RESUME_OR_LATEST = 20; } message PartitionOffset { Partition partition = 1; int64 start_ts_ns = 2; - int64 stop_ts_ns = 3; - PartitionOffsetStartType start_type = 4; } /////////////////////////// diff --git a/weed/pb/schema_pb/mq_schema.pb.go b/weed/pb/schema_pb/mq_schema.pb.go index 4093c26fb..c3ef746bf 100644 --- a/weed/pb/schema_pb/mq_schema.pb.go +++ b/weed/pb/schema_pb/mq_schema.pb.go @@ -20,52 +20,58 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -type PartitionOffsetStartType int32 +type OffsetType int32 const ( - PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0 - PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1 - PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2 + OffsetType_RESET_TO_EARLIEST OffsetType = 0 + OffsetType_RESET_TO_LATEST OffsetType = 2 + OffsetType_EXACT_TS_NS OffsetType = 3 + OffsetType_RESUME_OR_EARLIEST OffsetType = 4 + OffsetType_RESUME_OR_LATEST OffsetType = 5 ) -// Enum value maps for PartitionOffsetStartType. +// Enum value maps for OffsetType. 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, + OffsetType_name = map[int32]string{ + 0: "RESET_TO_EARLIEST", + 2: "RESET_TO_LATEST", + 3: "EXACT_TS_NS", + 4: "RESUME_OR_EARLIEST", + 5: "RESUME_OR_LATEST", + } + OffsetType_value = map[string]int32{ + "RESET_TO_EARLIEST": 0, + "RESET_TO_LATEST": 2, + "EXACT_TS_NS": 3, + "RESUME_OR_EARLIEST": 4, + "RESUME_OR_LATEST": 5, } ) -func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType { - p := new(PartitionOffsetStartType) +func (x OffsetType) Enum() *OffsetType { + p := new(OffsetType) *p = x return p } -func (x PartitionOffsetStartType) String() string { +func (x OffsetType) String() string { return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) } -func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor { +func (OffsetType) Descriptor() protoreflect.EnumDescriptor { return file_mq_schema_proto_enumTypes[0].Descriptor() } -func (PartitionOffsetStartType) Type() protoreflect.EnumType { +func (OffsetType) Type() protoreflect.EnumType { return &file_mq_schema_proto_enumTypes[0] } -func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber { +func (x OffsetType) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use PartitionOffsetStartType.Descriptor instead. -func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) { +// Deprecated: Use OffsetType.Descriptor instead. +func (OffsetType) EnumDescriptor() ([]byte, []int) { return file_mq_schema_proto_rawDescGZIP(), []int{0} } @@ -316,10 +322,8 @@ type PartitionOffset struct { 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"` + 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"` } func (x *PartitionOffset) Reset() { @@ -368,20 +372,6 @@ func (x *PartitionOffset) GetStartTsNs() int64 { 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 { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -951,100 +941,96 @@ var file_mq_schema_proto_rawDesc = []byte{ 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, - 0x73, 0x22, 0xa3, 0x01, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, - 0x1f, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x12, 0x23, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, - 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x65, - 0x61, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x52, 0x65, - 0x70, 0x65, 0x61, 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x71, - 0x75, 0x69, 0x72, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x22, 0xb6, 0x01, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x38, 0x0a, 0x0b, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a, - 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x38, 0x0a, 0x0b, 0x72, 0x65, - 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 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, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, - 0x54, 0x79, 0x70, 0x65, 0x12, 0x32, 0x0a, 0x09, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x08, - 0x6c, 0x69, 0x73, 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, - 0x22, 0x3e, 0x0a, 0x08, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x32, 0x0a, 0x0c, - 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, - 0x22, 0x96, 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x3a, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x22, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, - 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, - 0x46, 0x69, 0x65, 0x6c, 0x64, 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, 0x26, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfa, 0x02, 0x0a, 0x05, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x69, 0x6e, 0x74, 0x33, 0x32, 0x5f, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x74, - 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x69, 0x6e, 0x74, 0x36, 0x34, - 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0a, - 0x69, 0x6e, 0x74, 0x36, 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x66, 0x6c, - 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x48, - 0x00, 0x52, 0x0a, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, - 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, - 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x79, 0x74, 0x65, 0x73, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, - 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x35, 0x0a, 0x0a, 0x6c, 0x69, - 0x73, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, - 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x09, 0x6c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x0f, 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, 0x48, - 0x00, 0x52, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x06, - 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, 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, + 0x65, 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, 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, 0x73, 0x22, 0xa3, + 0x01, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x23, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x52, 0x65, 0x70, 0x65, 0x61, + 0x74, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, + 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x69, 0x72, 0x65, 0x64, 0x22, 0xb6, 0x01, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x38, 0x0a, + 0x0b, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x53, + 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x63, 0x61, + 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x38, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 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, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x32, 0x0a, 0x09, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x08, 0x6c, 0x69, 0x73, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x22, 0x3e, 0x0a, + 0x08, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x32, 0x0a, 0x0c, 0x65, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x96, 0x01, + 0x0a, 0x0b, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3a, 0x0a, + 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, 0x46, 0x69, 0x65, + 0x6c, 0x64, 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, 0x26, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfa, 0x02, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x69, 0x6e, 0x74, 0x33, 0x32, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x74, 0x33, 0x32, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x74, + 0x36, 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x66, 0x6c, 0x6f, 0x61, 0x74, + 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, 0x0a, + 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, + 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, + 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, + 0x21, 0x0a, 0x0b, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x79, 0x74, 0x65, 0x73, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, + 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x35, 0x0a, 0x0a, 0x6c, 0x69, 0x73, 0x74, 0x5f, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x48, 0x00, 0x52, 0x09, 0x6c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3b, + 0x0a, 0x0c, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0f, + 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, 0x48, 0x00, 0x52, 0x0b, + 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x06, 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, 0x77, 0x0a, 0x0a, 0x4f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x52, 0x45, 0x53, 0x45, + 0x54, 0x5f, 0x54, 0x4f, 0x5f, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, + 0x13, 0x0a, 0x0f, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x5f, 0x4c, 0x41, 0x54, 0x45, + 0x53, 0x54, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x58, 0x41, 0x43, 0x54, 0x5f, 0x54, 0x53, + 0x5f, 0x4e, 0x53, 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x52, 0x45, 0x53, 0x55, 0x4d, 0x45, 0x5f, + 0x4f, 0x52, 0x5f, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x04, 0x12, 0x14, 0x0a, + 0x10, 0x52, 0x45, 0x53, 0x55, 0x4d, 0x45, 0x5f, 0x4f, 0x52, 0x5f, 0x4c, 0x41, 0x54, 0x45, 0x53, + 0x54, 0x10, 0x05, 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 ( @@ -1062,42 +1048,41 @@ func file_mq_schema_proto_rawDescGZIP() []byte { 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 + (OffsetType)(0), // 0: schema_pb.OffsetType + (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 + 7, // 3: schema_pb.RecordType.fields:type_name -> schema_pb.Field + 8, // 4: schema_pb.Field.type:type_name -> schema_pb.Type + 1, // 5: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType + 6, // 6: schema_pb.Type.record_type:type_name -> schema_pb.RecordType + 9, // 7: schema_pb.Type.list_type:type_name -> schema_pb.ListType + 8, // 8: schema_pb.ListType.element_type:type_name -> schema_pb.Type + 13, // 9: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry + 12, // 10: schema_pb.Value.list_value:type_name -> schema_pb.ListValue + 10, // 11: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue + 11, // 12: schema_pb.ListValue.values:type_name -> schema_pb.Value + 11, // 13: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value + 14, // [14:14] is the sub-list for method output_type + 14, // [14:14] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name } func init() { file_mq_schema_proto_init() } diff --git a/weed/util/log_buffer/log_buffer.go b/weed/util/log_buffer/log_buffer.go index 30498f92d..fb1f8dc2f 100644 --- a/weed/util/log_buffer/log_buffer.go +++ b/weed/util/log_buffer/log_buffer.go @@ -46,7 +46,7 @@ type LogBuffer struct { isStopping *atomic.Bool isAllFlushed bool flushChan chan *dataToFlush - LastTsNs int64 + LastTsNs atomic.Int64 sync.RWMutex } @@ -95,12 +95,12 @@ func (logBuffer *LogBuffer) AddDataToBuffer(partitionKey, data []byte, processin } else { ts = time.Unix(0, processingTsNs) } - if logBuffer.LastTsNs >= processingTsNs { + if logBuffer.LastTsNs.Load() >= processingTsNs { // this is unlikely to happen, but just in case - processingTsNs = logBuffer.LastTsNs + 1 + processingTsNs = logBuffer.LastTsNs.Add(1) ts = time.Unix(0, processingTsNs) } - logBuffer.LastTsNs = processingTsNs + logBuffer.LastTsNs.Store(processingTsNs) logEntry := &filer_pb.LogEntry{ TsNs: processingTsNs, PartitionKeyHash: util.HashToInt32(partitionKey), diff --git a/weed/util/log_buffer/log_read.go b/weed/util/log_buffer/log_read.go index 0d044fc14..cf83de1e5 100644 --- a/weed/util/log_buffer/log_read.go +++ b/weed/util/log_buffer/log_read.go @@ -66,17 +66,10 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition isDone = true return } - logBuffer.RLock() - lastTsNs := logBuffer.LastTsNs - logBuffer.RUnlock() - loopTsNs := lastTsNs // make a copy + lastTsNs := logBuffer.LastTsNs.Load() - for lastTsNs == loopTsNs { + for lastTsNs == logBuffer.LastTsNs.Load() { if waitForDataFn() { - // Update loopTsNs and loop again - logBuffer.RLock() - loopTsNs = logBuffer.LastTsNs - logBuffer.RUnlock() continue } else { isDone = true |
