Browse Source

adding agent

add-message-queue-agent
chrislu 3 weeks ago
parent
commit
11201ac95f
  1. 22
      weed/mq/agent/agent_grpc_pub_session.go
  2. 10
      weed/mq/agent/agent_grpc_publish.go
  3. 57
      weed/mq/agent/agent_grpc_sub_session.go
  4. 75
      weed/mq/agent/agent_grpc_subscribe.go
  5. 20
      weed/mq/agent/agent_server.go
  6. 3
      weed/mq/broker/broker_grpc_assign.go
  7. 3
      weed/mq/broker/broker_grpc_lookup.go
  8. 7
      weed/mq/broker/broker_grpc_sub.go
  9. 63
      weed/mq/client/agent_client/SubscribeSession.go
  10. 2
      weed/mq/client/agent_client/agent_publish.go
  11. 17
      weed/mq/client/agent_client/agent_subscribe.go
  12. 15
      weed/mq/client/agent_client/publish_session.go
  13. 4
      weed/mq/client/cmd/agent_pub_record/publisher_record.go
  14. 23
      weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
  15. 34
      weed/mq/client/cmd/weed_sub_record/subscriber_record.go
  16. 84
      weed/mq/client/sub_client/on_each_partition.go
  17. 18
      weed/mq/client/sub_client/subscribe.go
  18. 21
      weed/mq/client/sub_client/subscriber.go
  19. 3
      weed/mq/pub_balancer/allocate.go
  20. 29
      weed/mq/pub_balancer/allocate_test.go
  21. 3
      weed/mq/pub_balancer/broker_stats.go
  22. 5
      weed/mq/pub_balancer/lookup.go
  23. 4
      weed/mq/pub_balancer/partition_list_broker.go
  24. 3
      weed/mq/pub_balancer/pub_balancer.go
  25. 3
      weed/mq/sub_coordinator/consumer_group.go
  26. 9
      weed/mq/sub_coordinator/sub_coordinator.go
  27. 3
      weed/mq/topic/local_manager.go
  28. 8
      weed/mq/topic/partition.go
  29. 7
      weed/mq/topic/topic.go
  30. 2
      weed/pb/filer_pb/filer.pb.go
  31. 2
      weed/pb/filer_pb/filer_grpc.pb.go
  32. 2
      weed/pb/iam_pb/iam.pb.go
  33. 2
      weed/pb/iam_pb/iam_grpc.pb.go
  34. 2
      weed/pb/master_pb/master.pb.go
  35. 2
      weed/pb/master_pb/master_grpc.pb.go
  36. 2
      weed/pb/mount_pb/mount.pb.go
  37. 2
      weed/pb/mount_pb/mount_grpc.pb.go
  38. 44
      weed/pb/mq_agent.proto
  39. 904
      weed/pb/mq_agent_pb/mq_agent.pb.go
  40. 2
      weed/pb/mq_agent_pb/mq_agent_grpc.pb.go
  41. 75
      weed/pb/mq_broker.proto
  42. 2111
      weed/pb/mq_pb/mq_broker.pb.go
  43. 2
      weed/pb/mq_pb/mq_broker_grpc.pb.go
  44. 33
      weed/pb/mq_schema.proto
  45. 2
      weed/pb/remote_pb/remote.pb.go
  46. 2
      weed/pb/s3_pb/s3.pb.go
  47. 2
      weed/pb/s3_pb/s3_grpc.pb.go
  48. 672
      weed/pb/schema_pb/mq_schema.pb.go
  49. 2
      weed/pb/volume_server_pb/volume_server.pb.go
  50. 2
      weed/pb/volume_server_pb/volume_server_grpc.pb.go
  51. 3
      weed/shell/command_mq_topic_configure.go
  52. 3
      weed/shell/command_mq_topic_desc.go

22
weed/mq/agent/agent_grpc_session.go → weed/mq/agent/agent_grpc_pub_session.go

@ -5,6 +5,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
"log/slog"
"math/rand/v2"
"time"
)
@ -32,8 +33,8 @@ func (a *MessageQueueAgent) StartPublishSession(ctx context.Context, req *mq_age
delete(a.publishers, k)
}
}
a.publishers[SessionId(sessionId)] = &PublisherEntry{
publisher: topicPublisher,
a.publishers[SessionId(sessionId)] = &SessionEntry[*pub_client.TopicPublisher]{
entry: topicPublisher,
}
a.publishersLock.Unlock()
@ -41,3 +42,20 @@ func (a *MessageQueueAgent) StartPublishSession(ctx context.Context, req *mq_age
SessionId: sessionId,
}, nil
}
func (a *MessageQueueAgent) ClosePublishSession(ctx context.Context, req *mq_agent_pb.ClosePublishSessionRequest) (*mq_agent_pb.ClosePublishSessionResponse, error) {
var finishErr string
a.publishersLock.Lock()
publisherEntry, found := a.publishers[SessionId(req.SessionId)]
if found {
if err := publisherEntry.entry.FinishPublish(); err != nil {
finishErr = err.Error()
slog.Warn("failed to finish publish", "error", err)
}
delete(a.publishers, SessionId(req.SessionId))
}
a.publishersLock.Unlock()
return &mq_agent_pb.ClosePublishSessionResponse{
Error: finishErr,
}, nil
}

10
weed/mq/agent/agent_grpc_publish.go

@ -3,7 +3,6 @@ package agent
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
"golang.org/x/exp/slog"
"time"
)
@ -16,18 +15,15 @@ func (a *MessageQueueAgent) PublishRecordRequest(stream mq_agent_pb.SeaweedMessa
publisherEntry, found := a.publishers[SessionId(m.SessionId)]
a.publishersLock.RUnlock()
if !found {
return fmt.Errorf("session id %d not found", m.SessionId)
return fmt.Errorf("publish session id %d not found", m.SessionId)
}
defer func() {
if finishErr := publisherEntry.publisher.FinishPublish(); finishErr != nil {
slog.Warn("failed to finish publish", "error", finishErr)
}
publisherEntry.lastActiveTsNs = time.Now().UnixNano()
}()
publisherEntry.lastActiveTsNs = 0
if m.Value != nil {
if err := publisherEntry.publisher.PublishRecord(m.Key, m.Value); err != nil {
if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil {
return err
}
}
@ -40,7 +36,7 @@ func (a *MessageQueueAgent) PublishRecordRequest(stream mq_agent_pb.SeaweedMessa
if m.Value == nil {
continue
}
if err := publisherEntry.publisher.PublishRecord(m.Key, m.Value); err != nil {
if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil {
return err
}
}

57
weed/mq/agent/agent_grpc_sub_session.go

@ -0,0 +1,57 @@
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
}

75
weed/mq/agent/agent_grpc_subscribe.go

@ -0,0 +1,75 @@
package agent
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
"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"
"google.golang.org/protobuf/proto"
"time"
)
func (a *MessageQueueAgent) SubscribeRecordRequest(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()
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
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
}
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
}
}
})
go func() {
subErr := subscriberEntry.entry.Subscribe()
if subErr != nil {
glog.V(0).Infof("subscriber %d subscribe: %v", m.SessionId, subErr)
if lastErr == nil {
lastErr = subErr
}
}
}()
for {
m, err := stream.Recv()
if err != nil {
return err
}
if m != nil {
subscriberEntry.entry.PartitionOffsetChan <- sub_client.KeyedOffset{
Key: m.AckKey,
Offset: m.AckSequence,
}
}
}
}

20
weed/mq/agent/agent_server.go

@ -2,6 +2,7 @@ package agent
import (
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
"google.golang.org/grpc"
@ -9,8 +10,8 @@ import (
)
type SessionId int64
type PublisherEntry struct {
publisher *pub_client.TopicPublisher
type SessionEntry[T any] struct {
entry T
lastActiveTsNs int64
}
@ -20,11 +21,13 @@ type MessageQueueAgentOptions struct {
type MessageQueueAgent struct {
mq_agent_pb.UnimplementedSeaweedMessagingAgentServer
option *MessageQueueAgentOptions
brokers []pb.ServerAddress
grpcDialOption grpc.DialOption
publishers map[SessionId]*PublisherEntry
publishersLock sync.RWMutex
option *MessageQueueAgentOptions
brokers []pb.ServerAddress
grpcDialOption grpc.DialOption
publishers map[SessionId]*SessionEntry[*pub_client.TopicPublisher]
publishersLock sync.RWMutex
subscribers map[SessionId]*SessionEntry[*sub_client.TopicSubscriber]
subscribersLock sync.RWMutex
}
func NewMessageQueueAgent(option *MessageQueueAgentOptions, grpcDialOption grpc.DialOption) *MessageQueueAgent {
@ -35,7 +38,8 @@ func NewMessageQueueAgent(option *MessageQueueAgentOptions, grpcDialOption grpc.
option: option,
brokers: []pb.ServerAddress{},
grpcDialOption: grpcDialOption,
publishers: make(map[SessionId]*PublisherEntry),
publishers: make(map[SessionId]*SessionEntry[*pub_client.TopicPublisher]),
subscribers: make(map[SessionId]*SessionEntry[*sub_client.TopicSubscriber]),
}
}

3
weed/mq/broker/broker_grpc_assign.go

@ -9,6 +9,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"sync"
)
@ -55,7 +56,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
// called by broker leader to drain existing partitions.
// new/updated partitions will be detected by broker from the filer
func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error {
func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error {
// notify the brokers to create the topic partitions in parallel
var wg sync.WaitGroup
for _, bpa := range assignments {

3
weed/mq/broker/broker_grpc_lookup.go

@ -7,6 +7,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
// LookupTopicBrokers returns the brokers that are serving the topic
@ -54,7 +55,7 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
topicPartitionStat := topicPartitionStatsItem.Val
topic := &mq_pb.Topic{
topic := &schema_pb.Topic{
Namespace: topicPartitionStat.TopicPartition.Namespace,
Name: topicPartitionStat.TopicPartition.Name,
}

7
weed/mq/broker/broker_grpc_sub.go

@ -9,6 +9,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"io"
"time"
@ -54,7 +55,7 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
}()
startPosition := b.getRequestPosition(req.GetInit())
imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().Concurrency))
imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().SlidingWindowSize))
// connect to the follower
var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
@ -212,9 +213,9 @@ func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMess
return
}
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
if offset.StartType == schema_pb.PartitionOffsetStartType_EARLIEST {
startPosition = log_buffer.NewMessagePosition(1, -3)
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
} else if offset.StartType == schema_pb.PartitionOffsetStartType_LATEST {
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
}
return

63
weed/mq/client/agent_client/SubscribeSession.go

@ -0,0 +1,63 @@
package agent_client
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"
)
type SubscribeOption struct {
ConsumerGroup string
ConsumerGroupInstanceId string
Topic topic.Topic
Filter string
MaxSubscribedPartitions int32
PerPartitionConcurrency int32
}
type SubscribeSession struct {
Option *SubscribeOption
stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse]
sessionId int64
}
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())
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{
ConsumerGroup: option.ConsumerGroup,
ConsumerGroupInstanceId: option.ConsumerGroupInstanceId,
Topic: &schema_pb.Topic{
Namespace: option.Topic.Namespace,
Name: option.Topic.Name,
},
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)
}
stream, err := agentClient.SubscribeRecord(context.Background())
if err != nil {
return nil, fmt.Errorf("subscribe record: %v", err)
}
return &SubscribeSession{
Option: option,
stream: stream,
sessionId: resp.SessionId,
}, nil
}

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

@ -5,7 +5,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
func (a *AgentSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
SessionId: a.sessionId,
Key: key,

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

@ -0,0 +1,17 @@
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)
}
}

15
weed/mq/client/agent_client/agent_client.go → weed/mq/client/agent_client/publish_session.go

@ -6,10 +6,11 @@ import (
"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"
)
type AgentSession struct {
type PublishSession struct {
schema *schema.Schema
partitionCount int
publisherName string
@ -17,17 +18,17 @@ type AgentSession struct {
sessionId int64
}
func NewAgentSession(address string, topicSchema *schema.Schema, partitionCount int, publisherName string) (*AgentSession, error) {
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(), address, true, grpc.WithInsecure())
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
if err != nil {
return nil, fmt.Errorf("dial agent server %s: %v", address, err)
return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err)
}
agentClient := mq_agent_pb.NewSeaweedMessagingAgentClient(clientConn)
resp, err := agentClient.StartPublishSession(context.Background(), &mq_agent_pb.StartPublishSessionRequest{
Topic: &mq_agent_pb.Topic{
Topic: &schema_pb.Topic{
Namespace: topicSchema.Namespace,
Name: topicSchema.Name,
},
@ -47,7 +48,7 @@ func NewAgentSession(address string, topicSchema *schema.Schema, partitionCount
return nil, fmt.Errorf("publish record: %v", err)
}
return &AgentSession{
return &PublishSession{
schema: topicSchema,
partitionCount: partitionCount,
publisherName: publisherName,
@ -56,7 +57,7 @@ func NewAgentSession(address string, topicSchema *schema.Schema, partitionCount
}, nil
}
func (a *AgentSession) CloseSession() error {
func (a *PublishSession) CloseSession() error {
if a.schema == nil {
return nil
}

4
weed/mq/client/cmd/agent_pub_record/publisher_record.go

@ -27,7 +27,7 @@ var (
counter int32
)
func doPublish(publisher *agent_client.AgentSession, id int) {
func doPublish(publisher *agent_client.PublishSession, id int) {
startTime := time.Now()
for {
i := atomic.AddInt32(&counter, 1)
@ -100,7 +100,7 @@ func main() {
WithField("field7", schema.TypeBoolean).
RecordTypeEnd()
session, err := agent_client.NewAgentSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName)
session, err := agent_client.NewPublishSession("localhost:16777", schema.NewSchema(*namespace, *t, recordType), *partitionCount, *clientName)
if err != nil {
log.Printf("failed to create session: %v", err)
return

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

@ -6,12 +6,12 @@ import (
"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"
"time"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
)
var (
@ -33,23 +33,24 @@ func main() {
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
MaxPartitionCount: int32(*maxPartitionCount),
PerPartitionConcurrency: int32(*perPartitionConcurrency),
SlidingWindowSize: int32(*perPartitionConcurrency),
}
contentConfig := &sub_client.ContentConfiguration{
Topic: topic.NewTopic(*namespace, *t),
Filter: "",
StartTime: time.Unix(1, 1),
Topic: topic.NewTopic(*namespace, *t),
Filter: "",
}
brokers := strings.Split(*seedBrokers, ",")
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
counter := 0
subscriber.SetEachMessageFunc(func(key, value []byte) error {
counter++
println(string(key), "=>", string(value), counter)
return nil
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() {

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

@ -6,6 +6,7 @@ import (
"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"
@ -60,30 +61,31 @@ func main() {
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
MaxPartitionCount: int32(*maxPartitionCount),
PerPartitionConcurrency: int32(*perPartitionConcurrency),
SlidingWindowSize: int32(*perPartitionConcurrency),
}
contentConfig := &sub_client.ContentConfiguration{
Topic: topic.NewTopic(*namespace, *t),
Filter: "",
StartTime: time.Now().Add(-*timeAgo),
Topic: topic.NewTopic(*namespace, *t),
Filter: "",
// StartTime: time.Now().Add(-*timeAgo),
}
brokers := strings.Split(*seedBrokers, ",")
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
counter := 0
subscriber.SetEachMessageFunc(func(key, value []byte) error {
counter++
record := &schema_pb.RecordValue{}
err := proto.Unmarshal(value, record)
if err != nil {
fmt.Printf("unmarshal record value: %v\n", err)
} else {
fmt.Printf("%s %d: %v\n", string(key), len(value), record)
}
//time.Sleep(1300 * time.Millisecond)
return nil
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() {

84
weed/mq/client/sub_client/on_each_partition.go

@ -6,12 +6,18 @@ import (
"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/util"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"io"
"reflect"
"time"
)
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}) error {
type KeyedOffset struct {
Key []byte
Offset int64
}
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}, onDataMessageFn OnDataMessageFn) error {
// connect to the partition broker
return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
@ -20,31 +26,30 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
return fmt.Errorf("create subscribe client: %v", err)
}
perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency
if perPartitionConcurrency <= 0 {
perPartitionConcurrency = 1
slidingWindowSize := sub.SubscriberConfig.SlidingWindowSize
if slidingWindowSize <= 0 {
slidingWindowSize = 1
}
var stopTsNs int64
if !sub.ContentConfig.StopTime.IsZero() {
stopTsNs = sub.ContentConfig.StopTime.UnixNano()
po := findPartitionOffset(sub.ContentConfig.PartitionOffsets, assigned.Partition)
if po == nil {
po = &schema_pb.PartitionOffset{
Partition: assigned.Partition,
StartTsNs: time.Now().UnixNano(),
StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
}
}
if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Init{
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
PartitionOffset: &mq_pb.PartitionOffset{
Partition: assigned.Partition,
StartTsNs: sub.ContentConfig.StartTime.UnixNano(),
StopTsNs: stopTsNs,
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
},
Filter: sub.ContentConfig.Filter,
FollowerBroker: assigned.FollowerBroker,
Concurrency: perPartitionConcurrency,
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
PartitionOffset: po,
Filter: sub.ContentConfig.Filter,
FollowerBroker: assigned.FollowerBroker,
SlidingWindowSize: slidingWindowSize,
},
},
}); err != nil {
@ -57,24 +62,13 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
defer sub.OnCompletionFunc()
}
type KeyedOffset struct {
Key []byte
Offset int64
}
partitionOffsetChan := make(chan KeyedOffset, 1024)
defer func() {
close(partitionOffsetChan)
}()
executors := util.NewLimitedConcurrentExecutor(int(perPartitionConcurrency))
go func() {
for {
select {
case <-stopCh:
subscribeClient.CloseSend()
return
case ack, ok := <-partitionOffsetChan:
case ack, ok := <-sub.PartitionOffsetChan:
if !ok {
subscribeClient.CloseSend()
return
@ -91,9 +85,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
}
}()
var lastErr error
for lastErr == nil {
for {
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
resp, err := subscribeClient.Recv()
if err != nil {
@ -113,17 +105,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
continue
}
executors.Execute(func() {
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
if processErr == nil {
partitionOffsetChan <- KeyedOffset{
Key: m.Data.Key,
Offset: m.Data.TsNs,
}
} else {
lastErr = processErr
}
})
onDataMessageFn(m)
case *mq_pb.SubscribeMessageResponse_Ctrl:
// glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
@ -132,6 +114,14 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
}
}
return lastErr
})
}
func findPartitionOffset(partitionOffsets []*schema_pb.PartitionOffset, partition *schema_pb.Partition) *schema_pb.PartitionOffset {
for _, po := range partitionOffsets {
if po.Partition == partition {
return po
}
}
return nil
}

18
weed/mq/client/sub_client/subscribe.go

@ -4,6 +4,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"sync"
"time"
)
@ -20,6 +21,7 @@ func (sub *TopicSubscriber) Subscribe() error {
go sub.startProcessors()
// loop forever
// TODO shutdown the subscriber when not needed anymore
sub.doKeepConnectedToSubCoordinator()
return nil
@ -66,7 +68,21 @@ func (sub *TopicSubscriber) startProcessors() {
},
},
}
err := sub.onEachPartition(assigned, stopChan)
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,
}
}
})
}
err := sub.onEachPartition(assigned, stopChan, onDataMessageFn)
if err != nil {
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
} else {

21
weed/mq/client/sub_client/subscriber.go

@ -3,9 +3,9 @@ package sub_client
import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"google.golang.org/grpc"
"sync"
"time"
)
type SubscriberConfiguration struct {
@ -14,16 +14,16 @@ type SubscriberConfiguration struct {
ConsumerGroupInstanceId string
GrpcDialOption grpc.DialOption
MaxPartitionCount int32 // how many partitions to process concurrently
PerPartitionConcurrency int32 // how many messages to process concurrently per partition
SlidingWindowSize int32 // how many messages to process concurrently per partition
}
type ContentConfiguration struct {
Topic topic.Topic
Filter string
StartTime time.Time
StopTime time.Time
Topic topic.Topic
Filter string
PartitionOffsets []*schema_pb.PartitionOffset
}
type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data)
type OnEachMessageFunc func(key, value []byte) (err error)
type OnCompletionFunc func()
@ -32,15 +32,17 @@ type TopicSubscriber struct {
ContentConfig *ContentConfiguration
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
OnDataMessageFnnc OnDataMessageFn
OnEachMessageFunc OnEachMessageFunc
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) *TopicSubscriber {
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
@ -49,6 +51,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
activeProcessors: make(map[topic.Partition]*ProcessorState),
PartitionOffsetChan: partitionOffsetChan,
}
}
@ -56,6 +59,10 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc
sub.OnEachMessageFunc = onEachMessageFn
}
func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) {
sub.OnDataMessageFnnc = fn
}
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
sub.OnCompletionFunc = onCompletionFn
}

3
weed/mq/pub_balancer/allocate.go

@ -4,6 +4,7 @@ import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"math/rand"
"time"
)
@ -14,7 +15,7 @@ func AllocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p
rangeSize := MaxPartitionCount / partitionCount
for i := int32(0); i < partitionCount; i++ {
assignment := &mq_pb.BrokerPartitionAssignment{
Partition: &mq_pb.Partition{
Partition: &schema_pb.Partition{
RingSize: MaxPartitionCount,
RangeStart: int32(i * rangeSize),
RangeStop: int32((i + 1) * rangeSize),

29
weed/mq/pub_balancer/allocate_test.go

@ -4,6 +4,7 @@ import (
"fmt"
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/stretchr/testify/assert"
"testing"
)
@ -29,7 +30,7 @@ func Test_allocateOneBroker(t *testing.T) {
wantAssignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:17777",
Partition: &mq_pb.Partition{
Partition: &schema_pb.Partition{
RingSize: MaxPartitionCount,
RangeStart: 0,
RangeStop: MaxPartitionCount,
@ -95,8 +96,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "",
Partition: &mq_pb.Partition{},
LeaderBroker: "",
Partition: &schema_pb.Partition{},
FollowerBroker: "localhost:2",
},
},
@ -110,8 +111,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
LeaderBroker: "localhost:1",
Partition: &schema_pb.Partition{},
FollowerBroker: "",
},
},
@ -125,8 +126,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
LeaderBroker: "localhost:1",
Partition: &schema_pb.Partition{},
FollowerBroker: "localhost:200",
},
},
@ -140,8 +141,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:100",
Partition: &mq_pb.Partition{},
LeaderBroker: "localhost:100",
Partition: &schema_pb.Partition{},
FollowerBroker: "localhost:200",
},
},
@ -155,8 +156,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
LeaderBroker: "localhost:1",
Partition: &schema_pb.Partition{},
FollowerBroker: "localhost:2",
},
},
@ -171,7 +172,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
Partition: &schema_pb.Partition{},
},
},
},
@ -184,8 +185,8 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
LeaderBroker: "localhost:1",
Partition: &schema_pb.Partition{},
FollowerBroker: "localhost:2",
},
},

3
weed/mq/pub_balancer/broker_stats.go

@ -5,6 +5,7 @@ import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
type BrokerStats struct {
@ -65,7 +66,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
bs.SubscriberCount = subscriberCount
}
func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition, isAdd bool) {
func (bs *BrokerStats) RegisterAssignment(t *schema_pb.Topic, partition *schema_pb.Partition, isAdd bool) {
tps := &TopicPartitionStats{
TopicPartition: topic.TopicPartition{
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},

5
weed/mq/pub_balancer/lookup.go

@ -3,13 +3,14 @@ package pub_balancer
import (
"errors"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
var (
ErrNoBroker = errors.New("no broker")
)
func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
func (balancer *PubBalancer) LookupTopicPartitions(topic *schema_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
// find existing topic partition assignments
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
@ -18,7 +19,7 @@ func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignme
if topicPartitionStat.TopicPartition.Namespace == topic.Namespace &&
topicPartitionStat.TopicPartition.Name == topic.Name {
assignment := &mq_pb.BrokerPartitionAssignment{
Partition: &mq_pb.Partition{
Partition: &schema_pb.Partition{
RingSize: MaxPartitionCount,
RangeStart: topicPartitionStat.RangeStart,
RangeStop: topicPartitionStat.RangeStop,

4
weed/mq/pub_balancer/partition_list_broker.go

@ -2,7 +2,7 @@ package pub_balancer
import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
type PartitionSlotToBroker struct {
@ -24,7 +24,7 @@ func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList {
}
}
func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string, follower string) {
func (ps *PartitionSlotToBrokerList) AddBroker(partition *schema_pb.Partition, broker string, follower string) {
for _, partitionSlot := range ps.PartitionSlots {
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {

3
weed/mq/pub_balancer/pub_balancer.go

@ -4,6 +4,7 @@ import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
const (
@ -32,7 +33,7 @@ type PubBalancer struct {
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
// Collected from all brokers when they connect to the broker leader
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
OnPartitionChange func(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
OnPartitionChange func(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
}
func NewPubBalancer() *PubBalancer {

3
weed/mq/sub_coordinator/consumer_group.go

@ -7,6 +7,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"time"
)
@ -20,7 +21,7 @@ type ConsumerGroup struct {
stopCh chan struct{}
}
func NewConsumerGroup(t *mq_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup {
func NewConsumerGroup(t *schema_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup {
cg := &ConsumerGroup{
topic: topic.FromPbTopic(t),
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),

9
weed/mq/sub_coordinator/sub_coordinator.go

@ -5,6 +5,7 @@ import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/filer_client"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
type TopicConsumerGroups struct {
@ -28,7 +29,7 @@ func NewSubCoordinator() *SubCoordinator {
}
}
func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
func (c *SubCoordinator) GetTopicConsumerGroups(topic *schema_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
topicName := toTopicName(topic)
tcg, _ := c.TopicSubscribers.Get(topicName)
if tcg == nil && createIfMissing {
@ -41,12 +42,12 @@ func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMiss
}
return tcg
}
func (c *SubCoordinator) RemoveTopic(topic *mq_pb.Topic) {
func (c *SubCoordinator) RemoveTopic(topic *schema_pb.Topic) {
topicName := toTopicName(topic)
c.TopicSubscribers.Remove(topicName)
}
func toTopicName(topic *mq_pb.Topic) string {
func toTopicName(topic *schema_pb.Topic) string {
topicName := topic.Namespace + "." + topic.Name
return topicName
}
@ -96,7 +97,7 @@ func (c *SubCoordinator) RemoveSubscriber(initMessage *mq_pb.SubscriberToSubCoor
}
}
func (c *SubCoordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
func (c *SubCoordinator) OnPartitionChange(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
if tcg == nil {
return

3
weed/mq/topic/local_manager.go

@ -3,6 +3,7 @@ package topic
import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/shirou/gopsutil/v3/cpu"
"time"
)
@ -89,7 +90,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
Partition: localPartition.Partition,
}
stats.Stats[topicPartition.TopicPartitionId()] = &mq_pb.TopicPartitionStats{
Topic: &mq_pb.Topic{
Topic: &schema_pb.Topic{
Namespace: string(localTopic.Namespace),
Name: localTopic.Name,
},

8
weed/mq/topic/partition.go

@ -2,7 +2,7 @@ package topic
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"time"
)
@ -40,7 +40,7 @@ func (partition Partition) Equals(other Partition) bool {
return true
}
func FromPbPartition(partition *mq_pb.Partition) Partition {
func FromPbPartition(partition *schema_pb.Partition) Partition {
return Partition{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
@ -67,8 +67,8 @@ func SplitPartitions(targetCount int32, ts int64) []*Partition {
return partitions
}
func (partition Partition) ToPbPartition() *mq_pb.Partition {
return &mq_pb.Partition{
func (partition Partition) ToPbPartition() *schema_pb.Partition {
return &schema_pb.Partition{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
RingSize: partition.RingSize,

7
weed/mq/topic/topic.go

@ -7,6 +7,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
jsonpb "google.golang.org/protobuf/encoding/protojson"
)
@ -21,15 +22,15 @@ func NewTopic(namespace string, name string) Topic {
Name: name,
}
}
func FromPbTopic(topic *mq_pb.Topic) Topic {
func FromPbTopic(topic *schema_pb.Topic) Topic {
return Topic{
Namespace: topic.Namespace,
Name: topic.Name,
}
}
func (t Topic) ToPbTopic() *mq_pb.Topic {
return &mq_pb.Topic{
func (t Topic) ToPbTopic() *schema_pb.Topic {
return &schema_pb.Topic{
Namespace: t.Namespace,
Name: t.Name,
}

2
weed/pb/filer_pb/filer.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: filer.proto
package filer_pb

2
weed/pb/filer_pb/filer_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: filer.proto
package filer_pb

2
weed/pb/iam_pb/iam.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: iam.proto
package iam_pb

2
weed/pb/iam_pb/iam_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: iam.proto
package iam_pb

2
weed/pb/master_pb/master.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: master.proto
package master_pb

2
weed/pb/master_pb/master_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: master.proto
package master_pb

2
weed/pb/mount_pb/mount.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: mount.proto
package mount_pb

2
weed/pb/mount_pb/mount_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: mount.proto
package mount_pb

44
weed/pb/mq_agent.proto

@ -30,40 +30,9 @@ service SeaweedMessagingAgent {
}
//////////////////////////////////////////////////
message Topic {
string namespace = 1;
string name = 2;
}
message Partition {
int32 ring_size = 1;
int32 range_start = 2;
int32 range_stop = 3;
int64 unix_time_ns = 4;
}
message Offset {
Topic topic = 1;
repeated PartitionOffset partition_offsets = 2;
}
enum PartitionOffsetStartType {
EARLIEST = 0;
EARLIEST_IN_MEMORY = 1;
LATEST = 2;
}
message PartitionOffset {
Partition partition = 1;
int64 start_ts_ns = 2;
int64 stop_ts_ns = 3;
PartitionOffsetStartType start_type = 4;
}
//////////////////////////////////////////////////
message StartPublishSessionRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
int32 partition_count = 2;
schema_pb.RecordType record_type = 3;
string publisher_name = 4;
@ -92,12 +61,12 @@ message PublishRecordResponse {
//////////////////////////////////////////////////
message StartSubscribeSessionRequest {
string consumer_group = 1;
string consumer_id = 2;
string client_id = 3;
Topic topic = 4;
PartitionOffset partition_offset = 5;
string consumer_group_instance_id = 2;
schema_pb.Topic topic = 4;
repeated schema_pb.PartitionOffset partition_offsets = 5;
string filter = 6;
int32 concurrency = 8;
int32 max_subscribed_partitions = 8;
int32 sliding_window_size = 9;
}
message StartSubscribeSessionResponse {
string error = 1;
@ -116,7 +85,6 @@ message SubscribeRecordRequest {
bytes ack_key = 3;
}
message SubscribeRecordResponse {
int64 sequence = 1;
bytes key = 2;
schema_pb.RecordValue value = 3;
int64 ts_ns = 4;

904
weed/pb/mq_agent_pb/mq_agent.pb.go
File diff suppressed because it is too large
View File

2
weed/pb/mq_agent_pb/mq_agent_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: mq_agent.proto
package mq_agent_pb

75
weed/pb/mq_broker.proto

@ -64,43 +64,14 @@ message FindBrokerLeaderResponse {
string broker = 1;
}
message Topic {
string namespace = 1;
string name = 2;
}
message Partition {
int32 ring_size = 1;
int32 range_start = 2;
int32 range_stop = 3;
int64 unix_time_ns = 4;
}
message Offset {
Topic topic = 1;
repeated PartitionOffset partition_offsets = 2;
}
enum PartitionOffsetStartType {
EARLIEST = 0;
EARLIEST_IN_MEMORY = 1;
LATEST = 2;
}
message PartitionOffset {
Partition partition = 1;
int64 start_ts_ns = 2;
int64 stop_ts_ns = 3;
PartitionOffsetStartType start_type = 4;
}
//////////////////////////////////////////////////
message BrokerStats {
int32 cpu_usage_percent = 1;
map<string, TopicPartitionStats> stats = 2;
}
message TopicPartitionStats {
Topic topic = 1;
Partition partition = 2;
schema_pb.Topic topic = 1;
schema_pb.Partition partition = 2;
int32 publisher_count = 3;
int32 subscriber_count = 4;
string follower = 5;
@ -126,7 +97,7 @@ message BalanceTopicsResponse {
//////////////////////////////////////////////////
message ConfigureTopicRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
int32 partition_count = 2;
schema_pb.RecordType record_type = 3;
}
@ -137,23 +108,23 @@ message ConfigureTopicResponse {
message ListTopicsRequest {
}
message ListTopicsResponse {
repeated Topic topics = 1;
repeated schema_pb.Topic topics = 1;
}
message LookupTopicBrokersRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
}
message LookupTopicBrokersResponse {
Topic topic = 1;
schema_pb.Topic topic = 1;
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
}
message BrokerPartitionAssignment {
Partition partition = 1;
schema_pb.Partition partition = 1;
string leader_broker = 2;
string follower_broker = 3;
}
message AssignTopicPartitionsRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
bool is_leader = 3;
bool is_draining = 4;
@ -165,7 +136,7 @@ message SubscriberToSubCoordinatorRequest {
message InitMessage {
string consumer_group = 1;
string consumer_group_instance_id = 2;
Topic topic = 3;
schema_pb.Topic topic = 3;
// The consumer group instance will be assigned at most max_partition_count partitions.
// If the number of partitions is less than the sum of max_partition_count,
// the consumer group instance may be assigned partitions less than max_partition_count.
@ -178,10 +149,10 @@ message SubscriberToSubCoordinatorRequest {
int32 rebalance_seconds = 5;
}
message AckUnAssignmentMessage {
Partition partition = 1;
schema_pb.Partition partition = 1;
}
message AckAssignmentMessage {
Partition partition = 1;
schema_pb.Partition partition = 1;
}
oneof message {
InitMessage init = 1;
@ -194,7 +165,7 @@ message SubscriberToSubCoordinatorResponse {
BrokerPartitionAssignment partition_assignment = 1;
}
message UnAssignment {
Partition partition = 1;
schema_pb.Partition partition = 1;
}
oneof message {
Assignment assignment = 1;
@ -215,8 +186,8 @@ message DataMessage {
}
message PublishMessageRequest {
message InitMessage {
Topic topic = 1;
Partition partition = 2;
schema_pb.Topic topic = 1;
schema_pb.Partition partition = 2;
int32 ack_interval = 3;
string follower_broker = 4;
string publisher_name = 5; // for debugging
@ -233,8 +204,8 @@ message PublishMessageResponse {
}
message PublishFollowMeRequest {
message InitMessage {
Topic topic = 1;
Partition partition = 2;
schema_pb.Topic topic = 1;
schema_pb.Partition partition = 2;
}
message FlushMessage {
int64 ts_ns = 1;
@ -256,11 +227,11 @@ message SubscribeMessageRequest {
string consumer_group = 1;
string consumer_id = 2;
string client_id = 3;
Topic topic = 4;
PartitionOffset partition_offset = 5;
schema_pb.Topic topic = 4;
schema_pb.PartitionOffset partition_offset = 5;
string filter = 6;
string follower_broker = 7;
int32 concurrency = 8;
int32 sliding_window_size = 8;
}
message AckMessage {
int64 sequence = 1;
@ -284,8 +255,8 @@ message SubscribeMessageResponse {
}
message SubscribeFollowMeRequest {
message InitMessage {
Topic topic = 1;
Partition partition = 2;
schema_pb.Topic topic = 1;
schema_pb.Partition partition = 2;
string consumer_group = 3;
}
message AckMessage {
@ -303,13 +274,13 @@ message SubscribeFollowMeResponse {
int64 ack_ts_ns = 1;
}
message ClosePublishersRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
int64 unix_time_ns = 2;
}
message ClosePublishersResponse {
}
message CloseSubscribersRequest {
Topic topic = 1;
schema_pb.Topic topic = 1;
int64 unix_time_ns = 2;
}
message CloseSubscribersResponse {

2111
weed/pb/mq_pb/mq_broker.pb.go
File diff suppressed because it is too large
View File

2
weed/pb/mq_pb/mq_broker_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: mq_broker.proto
package mq_pb

33
weed/pb/mq_schema.proto

@ -4,6 +4,39 @@ package schema_pb;
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb";
///////////////////////////
// Topic definition
///////////////////////////
message Topic {
string namespace = 1;
string name = 2;
}
message Partition {
int32 ring_size = 1;
int32 range_start = 2;
int32 range_stop = 3;
int64 unix_time_ns = 4;
}
message Offset {
Topic topic = 1;
repeated PartitionOffset partition_offsets = 2;
}
enum PartitionOffsetStartType {
EARLIEST = 0;
EARLIEST_IN_MEMORY = 1;
LATEST = 2;
}
message PartitionOffset {
Partition partition = 1;
int64 start_ts_ns = 2;
int64 stop_ts_ns = 3;
PartitionOffsetStartType start_type = 4;
}
///////////////////////////
// schema definition
///////////////////////////

2
weed/pb/remote_pb/remote.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: remote.proto
package remote_pb

2
weed/pb/s3_pb/s3.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: s3.proto
package s3_pb

2
weed/pb/s3_pb/s3_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: s3.proto
package s3_pb

672
weed/pb/schema_pb/mq_schema.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: mq_schema.proto
package schema_pb
@ -20,6 +20,55 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
type PartitionOffsetStartType int32
const (
PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0
PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1
PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2
)
// Enum value maps for PartitionOffsetStartType.
var (
PartitionOffsetStartType_name = map[int32]string{
0: "EARLIEST",
1: "EARLIEST_IN_MEMORY",
2: "LATEST",
}
PartitionOffsetStartType_value = map[string]int32{
"EARLIEST": 0,
"EARLIEST_IN_MEMORY": 1,
"LATEST": 2,
}
)
func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType {
p := new(PartitionOffsetStartType)
*p = x
return p
}
func (x PartitionOffsetStartType) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor {
return file_mq_schema_proto_enumTypes[0].Descriptor()
}
func (PartitionOffsetStartType) Type() protoreflect.EnumType {
return &file_mq_schema_proto_enumTypes[0]
}
func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use PartitionOffsetStartType.Descriptor instead.
func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{0}
}
type ScalarType int32
const (
@ -65,11 +114,11 @@ func (x ScalarType) String() string {
}
func (ScalarType) Descriptor() protoreflect.EnumDescriptor {
return file_mq_schema_proto_enumTypes[0].Descriptor()
return file_mq_schema_proto_enumTypes[1].Descriptor()
}
func (ScalarType) Type() protoreflect.EnumType {
return &file_mq_schema_proto_enumTypes[0]
return &file_mq_schema_proto_enumTypes[1]
}
func (x ScalarType) Number() protoreflect.EnumNumber {
@ -78,9 +127,261 @@ func (x ScalarType) Number() protoreflect.EnumNumber {
// Deprecated: Use ScalarType.Descriptor instead.
func (ScalarType) EnumDescriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{1}
}
type Topic struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"`
Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
}
func (x *Topic) Reset() {
*x = Topic{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *Topic) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*Topic) ProtoMessage() {}
func (x *Topic) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[0]
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 Topic.ProtoReflect.Descriptor instead.
func (*Topic) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{0}
}
func (x *Topic) GetNamespace() string {
if x != nil {
return x.Namespace
}
return ""
}
func (x *Topic) GetName() string {
if x != nil {
return x.Name
}
return ""
}
type Partition struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
RingSize int32 `protobuf:"varint,1,opt,name=ring_size,json=ringSize,proto3" json:"ring_size,omitempty"`
RangeStart int32 `protobuf:"varint,2,opt,name=range_start,json=rangeStart,proto3" json:"range_start,omitempty"`
RangeStop int32 `protobuf:"varint,3,opt,name=range_stop,json=rangeStop,proto3" json:"range_stop,omitempty"`
UnixTimeNs int64 `protobuf:"varint,4,opt,name=unix_time_ns,json=unixTimeNs,proto3" json:"unix_time_ns,omitempty"`
}
func (x *Partition) Reset() {
*x = Partition{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *Partition) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*Partition) ProtoMessage() {}
func (x *Partition) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[1]
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 Partition.ProtoReflect.Descriptor instead.
func (*Partition) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{1}
}
func (x *Partition) GetRingSize() int32 {
if x != nil {
return x.RingSize
}
return 0
}
func (x *Partition) GetRangeStart() int32 {
if x != nil {
return x.RangeStart
}
return 0
}
func (x *Partition) GetRangeStop() int32 {
if x != nil {
return x.RangeStop
}
return 0
}
func (x *Partition) GetUnixTimeNs() int64 {
if x != nil {
return x.UnixTimeNs
}
return 0
}
type Offset struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
PartitionOffsets []*PartitionOffset `protobuf:"bytes,2,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"`
}
func (x *Offset) Reset() {
*x = Offset{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *Offset) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*Offset) ProtoMessage() {}
func (x *Offset) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[2]
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 Offset.ProtoReflect.Descriptor instead.
func (*Offset) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{2}
}
func (x *Offset) GetTopic() *Topic {
if x != nil {
return x.Topic
}
return nil
}
func (x *Offset) GetPartitionOffsets() []*PartitionOffset {
if x != nil {
return x.PartitionOffsets
}
return nil
}
type PartitionOffset struct {
state protoimpl.MessageState
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"`
}
func (x *PartitionOffset) Reset() {
*x = PartitionOffset{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[3]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *PartitionOffset) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*PartitionOffset) ProtoMessage() {}
func (x *PartitionOffset) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[3]
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 PartitionOffset.ProtoReflect.Descriptor instead.
func (*PartitionOffset) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{3}
}
func (x *PartitionOffset) GetPartition() *Partition {
if x != nil {
return x.Partition
}
return nil
}
func (x *PartitionOffset) GetStartTsNs() int64 {
if x != nil {
return x.StartTsNs
}
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
@ -92,7 +393,7 @@ type RecordType struct {
func (x *RecordType) Reset() {
*x = RecordType{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[0]
mi := &file_mq_schema_proto_msgTypes[4]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -105,7 +406,7 @@ func (x *RecordType) String() string {
func (*RecordType) ProtoMessage() {}
func (x *RecordType) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[0]
mi := &file_mq_schema_proto_msgTypes[4]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -118,7 +419,7 @@ func (x *RecordType) ProtoReflect() protoreflect.Message {
// Deprecated: Use RecordType.ProtoReflect.Descriptor instead.
func (*RecordType) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{0}
return file_mq_schema_proto_rawDescGZIP(), []int{4}
}
func (x *RecordType) GetFields() []*Field {
@ -143,7 +444,7 @@ type Field struct {
func (x *Field) Reset() {
*x = Field{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[1]
mi := &file_mq_schema_proto_msgTypes[5]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -156,7 +457,7 @@ func (x *Field) String() string {
func (*Field) ProtoMessage() {}
func (x *Field) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[1]
mi := &file_mq_schema_proto_msgTypes[5]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -169,7 +470,7 @@ func (x *Field) ProtoReflect() protoreflect.Message {
// Deprecated: Use Field.ProtoReflect.Descriptor instead.
func (*Field) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{1}
return file_mq_schema_proto_rawDescGZIP(), []int{5}
}
func (x *Field) GetName() string {
@ -223,7 +524,7 @@ type Type struct {
func (x *Type) Reset() {
*x = Type{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[2]
mi := &file_mq_schema_proto_msgTypes[6]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -236,7 +537,7 @@ func (x *Type) String() string {
func (*Type) ProtoMessage() {}
func (x *Type) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[2]
mi := &file_mq_schema_proto_msgTypes[6]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -249,7 +550,7 @@ func (x *Type) ProtoReflect() protoreflect.Message {
// Deprecated: Use Type.ProtoReflect.Descriptor instead.
func (*Type) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{2}
return file_mq_schema_proto_rawDescGZIP(), []int{6}
}
func (m *Type) GetKind() isType_Kind {
@ -313,7 +614,7 @@ type ListType struct {
func (x *ListType) Reset() {
*x = ListType{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[3]
mi := &file_mq_schema_proto_msgTypes[7]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -326,7 +627,7 @@ func (x *ListType) String() string {
func (*ListType) ProtoMessage() {}
func (x *ListType) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[3]
mi := &file_mq_schema_proto_msgTypes[7]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -339,7 +640,7 @@ func (x *ListType) ProtoReflect() protoreflect.Message {
// Deprecated: Use ListType.ProtoReflect.Descriptor instead.
func (*ListType) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{3}
return file_mq_schema_proto_rawDescGZIP(), []int{7}
}
func (x *ListType) GetElementType() *Type {
@ -363,7 +664,7 @@ type RecordValue struct {
func (x *RecordValue) Reset() {
*x = RecordValue{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[4]
mi := &file_mq_schema_proto_msgTypes[8]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -376,7 +677,7 @@ func (x *RecordValue) String() string {
func (*RecordValue) ProtoMessage() {}
func (x *RecordValue) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[4]
mi := &file_mq_schema_proto_msgTypes[8]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -389,7 +690,7 @@ func (x *RecordValue) ProtoReflect() protoreflect.Message {
// Deprecated: Use RecordValue.ProtoReflect.Descriptor instead.
func (*RecordValue) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{4}
return file_mq_schema_proto_rawDescGZIP(), []int{8}
}
func (x *RecordValue) GetFields() map[string]*Value {
@ -421,7 +722,7 @@ type Value struct {
func (x *Value) Reset() {
*x = Value{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[5]
mi := &file_mq_schema_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -434,7 +735,7 @@ func (x *Value) String() string {
func (*Value) ProtoMessage() {}
func (x *Value) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[5]
mi := &file_mq_schema_proto_msgTypes[9]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -447,7 +748,7 @@ func (x *Value) ProtoReflect() protoreflect.Message {
// Deprecated: Use Value.ProtoReflect.Descriptor instead.
func (*Value) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{5}
return file_mq_schema_proto_rawDescGZIP(), []int{9}
}
func (m *Value) GetKind() isValue_Kind {
@ -589,7 +890,7 @@ type ListValue struct {
func (x *ListValue) Reset() {
*x = ListValue{}
if protoimpl.UnsafeEnabled {
mi := &file_mq_schema_proto_msgTypes[6]
mi := &file_mq_schema_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -602,7 +903,7 @@ func (x *ListValue) String() string {
func (*ListValue) ProtoMessage() {}
func (x *ListValue) ProtoReflect() protoreflect.Message {
mi := &file_mq_schema_proto_msgTypes[6]
mi := &file_mq_schema_proto_msgTypes[10]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -615,7 +916,7 @@ func (x *ListValue) ProtoReflect() protoreflect.Message {
// Deprecated: Use ListValue.ProtoReflect.Descriptor instead.
func (*ListValue) Descriptor() ([]byte, []int) {
return file_mq_schema_proto_rawDescGZIP(), []int{6}
return file_mq_schema_proto_rawDescGZIP(), []int{10}
}
func (x *ListValue) GetValues() []*Value {
@ -629,83 +930,121 @@ var File_mq_schema_proto protoreflect.FileDescriptor
var file_mq_schema_proto_rawDesc = []byte{
0x0a, 0x0f, 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x12, 0x09, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 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, 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,
0x6f, 0x12, 0x09, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x39, 0x0a, 0x05,
0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61,
0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70,
0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69,
0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69,
0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72,
0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74,
0x61, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f,
0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74,
0x6f, 0x70, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f,
0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69,
0x6d, 0x65, 0x4e, 0x73, 0x22, 0x79, 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 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, 0x47, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 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, 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,
}
var (
@ -720,36 +1059,45 @@ func file_mq_schema_proto_rawDescGZIP() []byte {
return file_mq_schema_proto_rawDescData
}
var file_mq_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
var file_mq_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 8)
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{
(ScalarType)(0), // 0: schema_pb.ScalarType
(*RecordType)(nil), // 1: schema_pb.RecordType
(*Field)(nil), // 2: schema_pb.Field
(*Type)(nil), // 3: schema_pb.Type
(*ListType)(nil), // 4: schema_pb.ListType
(*RecordValue)(nil), // 5: schema_pb.RecordValue
(*Value)(nil), // 6: schema_pb.Value
(*ListValue)(nil), // 7: schema_pb.ListValue
nil, // 8: schema_pb.RecordValue.FieldsEntry
(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
}
var file_mq_schema_proto_depIdxs = []int32{
2, // 0: schema_pb.RecordType.fields:type_name -> schema_pb.Field
3, // 1: schema_pb.Field.type:type_name -> schema_pb.Type
0, // 2: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType
1, // 3: schema_pb.Type.record_type:type_name -> schema_pb.RecordType
4, // 4: schema_pb.Type.list_type:type_name -> schema_pb.ListType
3, // 5: schema_pb.ListType.element_type:type_name -> schema_pb.Type
8, // 6: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry
7, // 7: schema_pb.Value.list_value:type_name -> schema_pb.ListValue
5, // 8: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue
6, // 9: schema_pb.ListValue.values:type_name -> schema_pb.Value
6, // 10: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value
11, // [11:11] is the sub-list for method output_type
11, // [11:11] is the sub-list for method input_type
11, // [11:11] is the sub-list for extension type_name
11, // [11:11] is the sub-list for extension extendee
0, // [0:11] is the sub-list for field type_name
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
}
func init() { file_mq_schema_proto_init() }
@ -759,7 +1107,7 @@ func file_mq_schema_proto_init() {
}
if !protoimpl.UnsafeEnabled {
file_mq_schema_proto_msgTypes[0].Exporter = func(v any, i int) any {
switch v := v.(*RecordType); i {
switch v := v.(*Topic); i {
case 0:
return &v.state
case 1:
@ -771,7 +1119,7 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[1].Exporter = func(v any, i int) any {
switch v := v.(*Field); i {
switch v := v.(*Partition); i {
case 0:
return &v.state
case 1:
@ -783,7 +1131,7 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[2].Exporter = func(v any, i int) any {
switch v := v.(*Type); i {
switch v := v.(*Offset); i {
case 0:
return &v.state
case 1:
@ -795,7 +1143,7 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[3].Exporter = func(v any, i int) any {
switch v := v.(*ListType); i {
switch v := v.(*PartitionOffset); i {
case 0:
return &v.state
case 1:
@ -807,7 +1155,7 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[4].Exporter = func(v any, i int) any {
switch v := v.(*RecordValue); i {
switch v := v.(*RecordType); i {
case 0:
return &v.state
case 1:
@ -819,7 +1167,7 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[5].Exporter = func(v any, i int) any {
switch v := v.(*Value); i {
switch v := v.(*Field); i {
case 0:
return &v.state
case 1:
@ -831,6 +1179,54 @@ func file_mq_schema_proto_init() {
}
}
file_mq_schema_proto_msgTypes[6].Exporter = func(v any, i int) any {
switch v := v.(*Type); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_mq_schema_proto_msgTypes[7].Exporter = func(v any, i int) any {
switch v := v.(*ListType); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_mq_schema_proto_msgTypes[8].Exporter = func(v any, i int) any {
switch v := v.(*RecordValue); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_mq_schema_proto_msgTypes[9].Exporter = func(v any, i int) any {
switch v := v.(*Value); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_mq_schema_proto_msgTypes[10].Exporter = func(v any, i int) any {
switch v := v.(*ListValue); i {
case 0:
return &v.state
@ -843,12 +1239,12 @@ func file_mq_schema_proto_init() {
}
}
}
file_mq_schema_proto_msgTypes[2].OneofWrappers = []any{
file_mq_schema_proto_msgTypes[6].OneofWrappers = []any{
(*Type_ScalarType)(nil),
(*Type_RecordType)(nil),
(*Type_ListType)(nil),
}
file_mq_schema_proto_msgTypes[5].OneofWrappers = []any{
file_mq_schema_proto_msgTypes[9].OneofWrappers = []any{
(*Value_BoolValue)(nil),
(*Value_Int32Value)(nil),
(*Value_Int64Value)(nil),
@ -864,8 +1260,8 @@ func file_mq_schema_proto_init() {
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_mq_schema_proto_rawDesc,
NumEnums: 1,
NumMessages: 8,
NumEnums: 2,
NumMessages: 12,
NumExtensions: 0,
NumServices: 0,
},

2
weed/pb/volume_server_pb/volume_server.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.34.2
// protoc v5.28.1
// protoc v5.28.3
// source: volume_server.proto
package volume_server_pb

2
weed/pb/volume_server_pb/volume_server_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.5.1
// - protoc v5.28.1
// - protoc v5.28.3
// source: volume_server.proto
package volume_server_pb

3
weed/shell/command_mq_topic_configure.go

@ -7,6 +7,7 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"io"
)
@ -54,7 +55,7 @@ func (c *commandMqTopicConfigure) Do(args []string, commandEnv *CommandEnv, writ
// create topic
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
resp, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Topic: &schema_pb.Topic{
Namespace: *namespace,
Name: *topicName,
},

3
weed/shell/command_mq_topic_desc.go

@ -6,6 +6,7 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"io"
)
@ -46,7 +47,7 @@ func (c *commandMqTopicDescribe) Do(args []string, commandEnv *CommandEnv, write
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
resp, err := client.LookupTopicBrokers(context.Background(), &mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Topic: &schema_pb.Topic{
Namespace: *namespace,
Name: *topicName,
},

Loading…
Cancel
Save