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