aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-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