mirror of
https://github.com/chrislusf/seaweedfs
synced 2025-05-24 07:00:13 +02:00
Accumulated changes for message queue (#6600)
* rename * set agent address * refactor * add agent sub * pub messages * grpc new client * can publish records via agent * send init message with session id * fmt * check cancelled request while waiting * use sessionId * handle possible nil stream * subscriber process messages * separate debug port * use atomic int64 * less logs * minor * skip io.EOF * rename * remove unused * use saved offsets * do not reuse session, since always session id is new after restart remove last active ts from SessionEntry * simplify printing * purge unused * just proxy the subscription, skipping the session step * adjust offset types * subscribe offset type and possible value * start after the known tsns * avoid wrongly set startPosition * move * remove * refactor * typo * fix * fix changed path
This commit is contained in:
parent
14cb8a24c6
commit
02773a6107
43 changed files with 1086 additions and 1624 deletions
6
.gitignore
vendored
6
.gitignore
vendored
|
@ -87,7 +87,6 @@ other/java/hdfs/dependency-reduced-pom.xml
|
|||
|
||||
# binary file
|
||||
weed/weed
|
||||
weed/mq/client/cmd/weed_pub_kv/weed_pub
|
||||
docker/weed
|
||||
|
||||
# test generated files
|
||||
|
@ -95,7 +94,4 @@ weed/*/*.jpg
|
|||
docker/weed_sub
|
||||
docker/weed_pub
|
||||
weed/mq/schema/example.parquet
|
||||
docker/weed_pub_kv
|
||||
docker/weed_pub_record
|
||||
docker/weed_sub_kv
|
||||
docker/weed_sub_record
|
||||
docker/agent_sub_record
|
||||
|
|
|
@ -9,10 +9,8 @@ binary:
|
|||
export SWCOMMIT=$(shell git rev-parse --short HEAD)
|
||||
export SWLDFLAGS="-X github.com/seaweedfs/seaweedfs/weed/util.COMMIT=$(SWCOMMIT)"
|
||||
cd ../weed && CGO_ENABLED=$(cgo) GOOS=linux go build $(options) -tags "$(tags)" -ldflags "-s -w -extldflags -static $(SWLDFLAGS)" && mv weed ../docker/
|
||||
cd ../weed/mq/client/cmd/weed_pub_kv && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_pub_kv ../../../../../docker/
|
||||
cd ../weed/mq/client/cmd/weed_pub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_pub_record ../../../../../docker/
|
||||
cd ../weed/mq/client/cmd/weed_sub_kv && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_sub_kv ../../../../../docker/
|
||||
cd ../weed/mq/client/cmd/weed_sub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_sub_record ../../../../../docker/
|
||||
cd ../other/mq_client_example/agent_pub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv agent_pub_record ../../../docker/
|
||||
cd ../other/mq_client_example/agent_sub_record && CGO_ENABLED=$(cgo) GOOS=linux go build && mv agent_sub_record ../../../docker/
|
||||
|
||||
binary_race: options = -race
|
||||
binary_race: cgo = 1
|
||||
|
@ -97,6 +95,9 @@ s3tests: build s3tests_build
|
|||
brokers: build
|
||||
docker compose -f compose/local-brokers-compose.yml -p seaweedfs up
|
||||
|
||||
agent: build
|
||||
docker compose -f compose/local-mq-test.yml -p seaweedfs up
|
||||
|
||||
filer_etcd: build
|
||||
docker stack deploy -c compose/swarm-etcd.yml fs
|
||||
|
||||
|
|
BIN
docker/agent_pub_record
Executable file
BIN
docker/agent_pub_record
Executable file
Binary file not shown.
|
@ -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
|
||||
|
|
|
@ -3,9 +3,9 @@ package main
|
|||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/other/mq_client_example/example"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"log"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
@ -20,13 +20,26 @@ var (
|
|||
|
||||
clientName = flag.String("client", "c1", "client name")
|
||||
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("t", "test", "t")
|
||||
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("t", "test", "t")
|
||||
agent = flag.String("agent", "localhost:16777", "mq agent address")
|
||||
|
||||
counter int32
|
||||
)
|
||||
|
||||
func genMyRecord(id int32) *example.MyRecord {
|
||||
return &example.MyRecord{
|
||||
Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)),
|
||||
Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)),
|
||||
Field2: fmt.Sprintf("field2-%s-%d", *clientName, id),
|
||||
Field3: id,
|
||||
Field4: int64(id),
|
||||
Field5: float32(id),
|
||||
Field6: float64(id),
|
||||
Field7: id%2 == 0,
|
||||
}
|
||||
}
|
||||
|
||||
func doPublish(publisher *agent_client.PublishSession, id int) {
|
||||
startTime := time.Now()
|
||||
for {
|
||||
|
@ -49,58 +62,12 @@ func doPublish(publisher *agent_client.PublishSession, id int) {
|
|||
log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed)
|
||||
}
|
||||
|
||||
type MyRecord struct {
|
||||
Key []byte
|
||||
Field1 []byte
|
||||
Field2 string
|
||||
Field3 int32
|
||||
Field4 int64
|
||||
Field5 float32
|
||||
Field6 float64
|
||||
Field7 bool
|
||||
}
|
||||
|
||||
func genMyRecord(id int32) *MyRecord {
|
||||
return &MyRecord{
|
||||
Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)),
|
||||
Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)),
|
||||
Field2: fmt.Sprintf("field2-%s-%d", *clientName, id),
|
||||
Field3: id,
|
||||
Field4: int64(id),
|
||||
Field5: float32(id),
|
||||
Field6: float64(id),
|
||||
Field7: id%2 == 0,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *MyRecord) ToRecordValue() *schema_pb.RecordValue {
|
||||
return schema.RecordBegin().
|
||||
SetBytes("key", r.Key).
|
||||
SetBytes("field1", r.Field1).
|
||||
SetString("field2", r.Field2).
|
||||
SetInt32("field3", r.Field3).
|
||||
SetInt64("field4", r.Field4).
|
||||
SetFloat("field5", r.Field5).
|
||||
SetDouble("field6", r.Field6).
|
||||
SetBool("field7", r.Field7).
|
||||
RecordEnd()
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
|
||||
recordType := schema.RecordTypeBegin().
|
||||
WithField("key", schema.TypeBytes).
|
||||
WithField("field1", schema.TypeBytes).
|
||||
WithField("field2", schema.TypeString).
|
||||
WithField("field3", schema.TypeInt32).
|
||||
WithField("field4", schema.TypeInt64).
|
||||
WithField("field5", schema.TypeFloat).
|
||||
WithField("field6", schema.TypeDouble).
|
||||
WithField("field7", schema.TypeBoolean).
|
||||
RecordTypeEnd()
|
||||
recordType := example.MyRecordType()
|
||||
|
||||
session, err := agent_client.NewPublishSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName)
|
||||
session, err := agent_client.NewPublishSession(*agent, schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName)
|
||||
if err != nil {
|
||||
log.Printf("failed to create session: %v", err)
|
||||
return
|
62
other/mq_client_example/agent_sub_record/agent_sub_record.go
Normal file
62
other/mq_client_example/agent_sub_record/agent_sub_record.go
Normal file
|
@ -0,0 +1,62 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/other/mq_client_example/example"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"log"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
namespace = flag.String("ns", "test", "namespace")
|
||||
t = flag.String("topic", "test", "topic")
|
||||
agent = flag.String("agent", "localhost:16777", "mq agent address")
|
||||
maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count")
|
||||
slidingWindowSize = flag.Int("slidingWindowSize", 1, "per partition concurrency")
|
||||
timeAgo = flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"")
|
||||
|
||||
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
|
||||
)
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
|
||||
// determine the start of the messages
|
||||
var startTsNs int64
|
||||
startType := schema_pb.OffsetType_RESUME_OR_EARLIEST
|
||||
if *timeAgo > 0 {
|
||||
startTsNs = time.Now().Add(-*timeAgo).UnixNano()
|
||||
startType = schema_pb.OffsetType_EXACT_TS_NS
|
||||
}
|
||||
|
||||
session, err := agent_client.NewSubscribeSession(*agent, &agent_client.SubscribeOption{
|
||||
ConsumerGroup: "test",
|
||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
OffsetType: startType,
|
||||
OffsetTsNs: startTsNs,
|
||||
Filter: "",
|
||||
MaxSubscribedPartitions: int32(*maxPartitionCount),
|
||||
SlidingWindowSize: int32(*slidingWindowSize),
|
||||
})
|
||||
if err != nil {
|
||||
log.Printf("new subscribe session: %v", err)
|
||||
return
|
||||
}
|
||||
defer session.CloseSession()
|
||||
|
||||
counter := 0
|
||||
session.SubscribeMessageRecord(func(key []byte, recordValue *schema_pb.RecordValue) {
|
||||
counter++
|
||||
record := example.FromRecordValue(recordValue)
|
||||
fmt.Printf("%d %s %v\n", counter, string(key), record.Field2)
|
||||
}, func() {
|
||||
log.Printf("done received %d messages", counter)
|
||||
})
|
||||
|
||||
}
|
56
other/mq_client_example/example/my_record.go
Normal file
56
other/mq_client_example/example/my_record.go
Normal file
|
@ -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(),
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -32,6 +32,7 @@ var Commands = []*Command{
|
|||
cmdMaster,
|
||||
cmdMasterFollower,
|
||||
cmdMount,
|
||||
cmdMqAgent,
|
||||
cmdMqBroker,
|
||||
cmdS3,
|
||||
cmdScaffold,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -1,59 +1,59 @@
|
|||
package agent
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"context"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/protobuf/proto"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (a *MessageQueueAgent) SubscribeRecordRequest(stream mq_agent_pb.SeaweedMessagingAgent_SubscribeRecordServer) error {
|
||||
func (a *MessageQueueAgent) SubscribeRecord(stream mq_agent_pb.SeaweedMessagingAgent_SubscribeRecordServer) error {
|
||||
// the first message is the subscribe request
|
||||
// it should only contain the session id
|
||||
m, err := stream.Recv()
|
||||
initMessage, err := stream.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
a.subscribersLock.RLock()
|
||||
subscriberEntry, found := a.subscribers[SessionId(m.SessionId)]
|
||||
a.subscribersLock.RUnlock()
|
||||
if !found {
|
||||
return fmt.Errorf("subscribe session id %d not found", m.SessionId)
|
||||
}
|
||||
defer func() {
|
||||
subscriberEntry.lastActiveTsNs = time.Now().UnixNano()
|
||||
}()
|
||||
subscriberEntry.lastActiveTsNs = 0
|
||||
|
||||
subscriber := a.handleInitSubscribeRecordRequest(stream.Context(), initMessage.Init)
|
||||
|
||||
var lastErr error
|
||||
subscriberEntry.entry.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
record := &schema_pb.RecordValue{}
|
||||
err := proto.Unmarshal(m.Data.Value, record)
|
||||
if err != nil {
|
||||
if lastErr == nil {
|
||||
lastErr = err
|
||||
executors := util.NewLimitedConcurrentExecutor(int(subscriber.SubscriberConfig.SlidingWindowSize))
|
||||
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
record := &schema_pb.RecordValue{}
|
||||
err := proto.Unmarshal(m.Data.Value, record)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("unmarshal record value: %v", err)
|
||||
if lastErr == nil {
|
||||
lastErr = err
|
||||
}
|
||||
return
|
||||
}
|
||||
return
|
||||
}
|
||||
if sendErr := stream.Send(&mq_agent_pb.SubscribeRecordResponse{
|
||||
Key: m.Data.Key,
|
||||
Value: record,
|
||||
TsNs: m.Data.TsNs,
|
||||
}); sendErr != nil {
|
||||
if lastErr == nil {
|
||||
lastErr = sendErr
|
||||
if sendErr := stream.Send(&mq_agent_pb.SubscribeRecordResponse{
|
||||
Key: m.Data.Key,
|
||||
Value: record,
|
||||
TsNs: m.Data.TsNs,
|
||||
}); sendErr != nil {
|
||||
glog.V(0).Infof("send record: %v", sendErr)
|
||||
if lastErr == nil {
|
||||
lastErr = sendErr
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
go func() {
|
||||
subErr := subscriberEntry.entry.Subscribe()
|
||||
subErr := subscriber.Subscribe()
|
||||
if subErr != nil {
|
||||
glog.V(0).Infof("subscriber %d subscribe: %v", m.SessionId, subErr)
|
||||
glog.V(0).Infof("subscriber %s subscribe: %v", subscriber.SubscriberConfig.String(), subErr)
|
||||
if lastErr == nil {
|
||||
lastErr = subErr
|
||||
}
|
||||
|
@ -63,13 +63,43 @@ func (a *MessageQueueAgent) SubscribeRecordRequest(stream mq_agent_pb.SeaweedMes
|
|||
for {
|
||||
m, err := stream.Recv()
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s receive: %v", subscriber.SubscriberConfig.String(), err)
|
||||
return err
|
||||
}
|
||||
if m != nil {
|
||||
subscriberEntry.entry.PartitionOffsetChan <- sub_client.KeyedOffset{
|
||||
subscriber.PartitionOffsetChan <- sub_client.KeyedOffset{
|
||||
Key: m.AckKey,
|
||||
Offset: m.AckSequence,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (a *MessageQueueAgent) handleInitSubscribeRecordRequest(ctx context.Context, req *mq_agent_pb.SubscribeRecordRequest_InitSubscribeRecordRequest) *sub_client.TopicSubscriber {
|
||||
|
||||
subscriberConfig := &sub_client.SubscriberConfiguration{
|
||||
ConsumerGroup: req.ConsumerGroup,
|
||||
ConsumerGroupInstanceId: req.ConsumerGroupInstanceId,
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
MaxPartitionCount: req.MaxSubscribedPartitions,
|
||||
SlidingWindowSize: req.SlidingWindowSize,
|
||||
}
|
||||
|
||||
contentConfig := &sub_client.ContentConfiguration{
|
||||
Topic: topic.FromPbTopic(req.Topic),
|
||||
Filter: req.Filter,
|
||||
PartitionOffsets: req.PartitionOffsets,
|
||||
OffsetType: req.OffsetType,
|
||||
OffsetTsNs: req.OffsetTsNs,
|
||||
}
|
||||
|
||||
topicSubscriber := sub_client.NewTopicSubscriber(
|
||||
ctx,
|
||||
a.brokersList(),
|
||||
subscriberConfig,
|
||||
contentConfig,
|
||||
make(chan sub_client.KeyedOffset, 1024),
|
||||
)
|
||||
|
||||
return topicSubscriber
|
||||
}
|
||||
|
|
|
@ -11,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]),
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
})
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
})
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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())
|
||||
|
||||
}
|
|
@ -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())
|
||||
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -361,266 +361,21 @@ func (x *PublishRecordResponse) GetError() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
// ////////////////////////////////////////////////
|
||||
type StartSubscribeSessionRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"`
|
||||
ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"`
|
||||
Topic *schema_pb.Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"`
|
||||
PartitionOffsets []*schema_pb.PartitionOffset `protobuf:"bytes,5,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"`
|
||||
Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"`
|
||||
MaxSubscribedPartitions int32 `protobuf:"varint,8,opt,name=max_subscribed_partitions,json=maxSubscribedPartitions,proto3" json:"max_subscribed_partitions,omitempty"`
|
||||
SlidingWindowSize int32 `protobuf:"varint,9,opt,name=sliding_window_size,json=slidingWindowSize,proto3" json:"sliding_window_size,omitempty"`
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) Reset() {
|
||||
*x = StartSubscribeSessionRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*StartSubscribeSessionRequest) ProtoMessage() {}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use StartSubscribeSessionRequest.ProtoReflect.Descriptor instead.
|
||||
func (*StartSubscribeSessionRequest) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{6}
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetConsumerGroup() string {
|
||||
if x != nil {
|
||||
return x.ConsumerGroup
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetConsumerGroupInstanceId() string {
|
||||
if x != nil {
|
||||
return x.ConsumerGroupInstanceId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetTopic() *schema_pb.Topic {
|
||||
if x != nil {
|
||||
return x.Topic
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetPartitionOffsets() []*schema_pb.PartitionOffset {
|
||||
if x != nil {
|
||||
return x.PartitionOffsets
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetFilter() string {
|
||||
if x != nil {
|
||||
return x.Filter
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetMaxSubscribedPartitions() int32 {
|
||||
if x != nil {
|
||||
return x.MaxSubscribedPartitions
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionRequest) GetSlidingWindowSize() int32 {
|
||||
if x != nil {
|
||||
return x.SlidingWindowSize
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type StartSubscribeSessionResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
|
||||
SessionId int64 `protobuf:"varint,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionResponse) Reset() {
|
||||
*x = StartSubscribeSessionResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*StartSubscribeSessionResponse) ProtoMessage() {}
|
||||
|
||||
func (x *StartSubscribeSessionResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use StartSubscribeSessionResponse.ProtoReflect.Descriptor instead.
|
||||
func (*StartSubscribeSessionResponse) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{7}
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionResponse) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StartSubscribeSessionResponse) GetSessionId() int64 {
|
||||
if x != nil {
|
||||
return x.SessionId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type CloseSubscribeSessionRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionRequest) Reset() {
|
||||
*x = CloseSubscribeSessionRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*CloseSubscribeSessionRequest) ProtoMessage() {}
|
||||
|
||||
func (x *CloseSubscribeSessionRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use CloseSubscribeSessionRequest.ProtoReflect.Descriptor instead.
|
||||
func (*CloseSubscribeSessionRequest) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{8}
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionRequest) GetSessionId() int64 {
|
||||
if x != nil {
|
||||
return x.SessionId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type CloseSubscribeSessionResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionResponse) Reset() {
|
||||
*x = CloseSubscribeSessionResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*CloseSubscribeSessionResponse) ProtoMessage() {}
|
||||
|
||||
func (x *CloseSubscribeSessionResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use CloseSubscribeSessionResponse.ProtoReflect.Descriptor instead.
|
||||
func (*CloseSubscribeSessionResponse) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{9}
|
||||
}
|
||||
|
||||
func (x *CloseSubscribeSessionResponse) GetError() string {
|
||||
if x != nil {
|
||||
return x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// ////////////////////////////////////////////////
|
||||
type SubscribeRecordRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
SessionId int64 `protobuf:"varint,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` // session_id is required for the first record
|
||||
AckSequence int64 `protobuf:"varint,2,opt,name=ack_sequence,json=ackSequence,proto3" json:"ack_sequence,omitempty"`
|
||||
AckKey []byte `protobuf:"bytes,3,opt,name=ack_key,json=ackKey,proto3" json:"ack_key,omitempty"`
|
||||
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 *SubscribeRecordRequest) Reset() {
|
||||
*x = SubscribeRecordRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[10]
|
||||
mi := &file_mq_agent_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -633,7 +388,7 @@ func (x *SubscribeRecordRequest) String() string {
|
|||
func (*SubscribeRecordRequest) ProtoMessage() {}
|
||||
|
||||
func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[10]
|
||||
mi := &file_mq_agent_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -646,14 +401,14 @@ func (x *SubscribeRecordRequest) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use SubscribeRecordRequest.ProtoReflect.Descriptor instead.
|
||||
func (*SubscribeRecordRequest) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{10}
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{6}
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest) GetSessionId() int64 {
|
||||
func (x *SubscribeRecordRequest) GetInit() *SubscribeRecordRequest_InitSubscribeRecordRequest {
|
||||
if x != nil {
|
||||
return x.SessionId
|
||||
return x.Init
|
||||
}
|
||||
return 0
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest) GetAckSequence() int64 {
|
||||
|
@ -675,7 +430,6 @@ type SubscribeRecordResponse struct {
|
|||
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"`
|
||||
|
@ -687,7 +441,7 @@ type SubscribeRecordResponse struct {
|
|||
func (x *SubscribeRecordResponse) Reset() {
|
||||
*x = SubscribeRecordResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[11]
|
||||
mi := &file_mq_agent_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
@ -700,7 +454,7 @@ func (x *SubscribeRecordResponse) String() string {
|
|||
func (*SubscribeRecordResponse) ProtoMessage() {}
|
||||
|
||||
func (x *SubscribeRecordResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_mq_agent_proto_msgTypes[11]
|
||||
mi := &file_mq_agent_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
@ -713,14 +467,7 @@ func (x *SubscribeRecordResponse) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use SubscribeRecordResponse.ProtoReflect.Descriptor instead.
|
||||
func (*SubscribeRecordResponse) Descriptor() ([]byte, []int) {
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{11}
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordResponse) GetSequence() int64 {
|
||||
if x != nil {
|
||||
return x.Sequence
|
||||
}
|
||||
return 0
|
||||
return file_mq_agent_proto_rawDescGZIP(), []int{7}
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordResponse) GetKey() []byte {
|
||||
|
@ -765,6 +512,117 @@ func (x *SubscribeRecordResponse) GetIsEndOfTopic() bool {
|
|||
return false
|
||||
}
|
||||
|
||||
type SubscribeRecordRequest_InitSubscribeRecordRequest 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"`
|
||||
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_InitSubscribeRecordRequest) Reset() {
|
||||
*x = SubscribeRecordRequest_InitSubscribeRecordRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_mq_agent_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*SubscribeRecordRequest_InitSubscribeRecordRequest) ProtoMessage() {}
|
||||
|
||||
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 {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// 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_InitSubscribeRecordRequest) GetConsumerGroup() string {
|
||||
if x != nil {
|
||||
return x.ConsumerGroup
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetConsumerGroupInstanceId() string {
|
||||
if x != nil {
|
||||
return x.ConsumerGroupInstanceId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetTopic() *schema_pb.Topic {
|
||||
if x != nil {
|
||||
return x.Topic
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetPartitionOffsets() []*schema_pb.PartitionOffset {
|
||||
if x != nil {
|
||||
return x.PartitionOffsets
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetOffsetType() schema_pb.OffsetType {
|
||||
if x != nil {
|
||||
return x.OffsetType
|
||||
}
|
||||
return schema_pb.OffsetType(0)
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetOffsetTsNs() int64 {
|
||||
if x != nil {
|
||||
return x.OffsetTsNs
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetFilter() string {
|
||||
if x != nil {
|
||||
return x.Filter
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetMaxSubscribedPartitions() int32 {
|
||||
if x != nil {
|
||||
return x.MaxSubscribedPartitions
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *SubscribeRecordRequest_InitSubscribeRecordRequest) GetSlidingWindowSize() int32 {
|
||||
if x != nil {
|
||||
return x.SlidingWindowSize
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
var File_mq_agent_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_mq_agent_proto_rawDesc = []byte{
|
||||
|
@ -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,54 +889,6 @@ func file_mq_agent_proto_init() {
|
|||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*StartSubscribeSessionRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[7].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*StartSubscribeSessionResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[8].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*CloseSubscribeSessionRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[9].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*CloseSubscribeSessionResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[10].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*SubscribeRecordRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
|
@ -1114,7 +900,7 @@ func file_mq_agent_proto_init() {
|
|||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[11].Exporter = func(v any, i int) any {
|
||||
file_mq_agent_proto_msgTypes[7].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*SubscribeRecordResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
|
@ -1126,6 +912,18 @@ func file_mq_agent_proto_init() {
|
|||
return nil
|
||||
}
|
||||
}
|
||||
file_mq_agent_proto_msgTypes[8].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*SubscribeRecordRequest_InitSubscribeRecordRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -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{
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
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,
|
||||
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,
|
||||
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, 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,
|
||||
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, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65,
|
||||
0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51,
|
||||
0x75, 0x65, 0x75, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68, 0x75,
|
||||
0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f,
|
||||
0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70,
|
||||
0x62, 0x2f, 0x6d, 0x71, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 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, 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() }
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
///////////////////////////
|
||||
|
|
|
@ -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",
|
||||
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",
|
||||
}
|
||||
PartitionOffsetStartType_value = map[string]int32{
|
||||
"EARLIEST": 0,
|
||||
"EARLIEST_IN_MEMORY": 1,
|
||||
"LATEST": 2,
|
||||
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() }
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -66,17 +66,10 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition
|
|||
isDone = true
|
||||
return
|
||||
}
|
||||
logBuffer.RLock()
|
||||
lastTsNs := logBuffer.LastTsNs
|
||||
logBuffer.RUnlock()
|
||||
loopTsNs := lastTsNs // make a copy
|
||||
lastTsNs := logBuffer.LastTsNs.Load()
|
||||
|
||||
for lastTsNs == loopTsNs {
|
||||
for lastTsNs == logBuffer.LastTsNs.Load() {
|
||||
if waitForDataFn() {
|
||||
// Update loopTsNs and loop again
|
||||
logBuffer.RLock()
|
||||
loopTsNs = logBuffer.LastTsNs
|
||||
logBuffer.RUnlock()
|
||||
continue
|
||||
} else {
|
||||
isDone = true
|
||||
|
|
Loading…
Add table
Reference in a new issue