Browse Source

Accumulated changes for message queue (#6600)

* rename

* set agent address

* refactor

* add agent sub

* pub messages

* grpc new client

* can publish records via agent

* send init message with session id

* fmt

* check cancelled request while waiting

* use sessionId

* handle possible nil stream

* subscriber process messages

* separate debug port

* use atomic int64

* less logs

* minor

* skip io.EOF

* rename

* remove unused

* use saved offsets

* do not reuse session, since always session id is new after restart

remove last active ts from SessionEntry

* simplify printing

* purge unused

* just proxy the subscription, skipping the session step

* adjust offset types

* subscribe offset type and possible value

* start after the known tsns

* avoid wrongly set startPosition

* move

* remove

* refactor

* typo

* fix

* fix changed path
dependabot/go_modules/github.com/parquet-go/parquet-go-0.25.0
Chris Lu 2 days ago
committed by GitHub
parent
commit
02773a6107
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 6
      .gitignore
  2. 9
      docker/Makefile
  3. BIN
      docker/agent_pub_record
  4. 11
      docker/compose/local-mq-test.yml
  5. 71
      other/mq_client_example/agent_pub_record/agent_pub_record.go
  6. 62
      other/mq_client_example/agent_sub_record/agent_sub_record.go
  7. 56
      other/mq_client_example/example/my_record.go
  8. 4
      weed/Makefile
  9. 1
      weed/command/command.go
  10. 5
      weed/command/mq_agent.go
  11. 16
      weed/mq/agent/agent_grpc_pub_session.go
  12. 15
      weed/mq/agent/agent_grpc_publish.go
  13. 57
      weed/mq/agent/agent_grpc_sub_session.go
  14. 96
      weed/mq/agent/agent_grpc_subscribe.go
  15. 11
      weed/mq/agent/agent_server.go
  16. 46
      weed/mq/broker/broker_grpc_sub.go
  17. 14
      weed/mq/client/agent_client/agent_publish.go
  18. 17
      weed/mq/client/agent_client/agent_subscribe.go
  19. 19
      weed/mq/client/agent_client/publish_session.go
  20. 56
      weed/mq/client/agent_client/subscribe_session.go
  21. 78
      weed/mq/client/cmd/weed_pub_kv/publisher_kv.go
  22. 138
      weed/mq/client/cmd/weed_pub_record/publisher_record.go
  23. 64
      weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
  24. 99
      weed/mq/client/cmd/weed_sub_record/subscriber_record.go
  25. 8
      weed/mq/client/pub_client/publisher.go
  26. 10
      weed/mq/client/pub_client/scheduler.go
  27. 28
      weed/mq/client/sub_client/connect_to_sub_coordinator.go
  28. 26
      weed/mq/client/sub_client/on_each_partition.go
  29. 12
      weed/mq/client/sub_client/subscribe.go
  30. 23
      weed/mq/client/sub_client/subscriber.go
  31. 2
      weed/mq/logstore/read_log_from_disk.go
  32. 2
      weed/mq/logstore/read_parquet_to_log.go
  33. 2
      weed/mq/sub_coordinator/market.go
  34. 8
      weed/mq/topic/local_partition.go
  35. 37
      weed/pb/mq_agent.proto
  36. 642
      weed/pb/mq_agent_pb/mq_agent.pb.go
  37. 84
      weed/pb/mq_agent_pb/mq_agent_grpc.pb.go
  38. 7
      weed/pb/mq_broker.proto
  39. 444
      weed/pb/mq_pb/mq_broker.pb.go
  40. 12
      weed/pb/mq_schema.proto
  41. 315
      weed/pb/schema_pb/mq_schema.pb.go
  42. 8
      weed/util/log_buffer/log_buffer.go
  43. 11
      weed/util/log_buffer/log_read.go

6
.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

9
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

BIN
docker/agent_pub_record

11
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

71
weed/mq/client/cmd/agent_pub_record/publisher_record.go → 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

62
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)
})
}

56
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(),
}
}

4
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

1
weed/command/command.go

@ -32,6 +32,7 @@ var Commands = []*Command{
cmdMaster,
cmdMasterFollower,
cmdMount,
cmdMqAgent,
cmdMqBroker,
cmdS3,
cmdScaffold,

5
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)

16
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,
}

15
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
}

57
weed/mq/agent/agent_grpc_sub_session.go

@ -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
}

96
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
}

11
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]),

46
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
}

14
weed/mq/client/agent_client/agent_publish.go

@ -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,
})
}

17
weed/mq/client/agent_client/agent_subscribe.go

@ -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)
}
}

19
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,
})
}

56
weed/mq/client/agent_client/SubscribeSession.go → 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
}

78
weed/mq/client/cmd/weed_pub_kv/publisher_kv.go

@ -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())
}

138
weed/mq/client/cmd/weed_pub_record/publisher_record.go

@ -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())
}

64
weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go

@ -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)
}
}

99
weed/mq/client/cmd/weed_sub_record/subscriber_record.go

@ -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)
}
}

8
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 {

10
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 {

28
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)
}

26
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)

12
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,
}
})
}

23
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) {

2
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
}

2
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 {

2
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,

8
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

37
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;
}

642
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,
},

84
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{
{

7
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;

444
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() }

12
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;
}
///////////////////////////

315
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() }

8
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),

11
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

Loading…
Cancel
Save