mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-11-09 12:37:26 +08:00
Add message queue agent (#6463)
* scaffold message queue agent * adjust proto, add mq_agent * add agent client implementation * remove unused function * agent publish server implementation * adding agent
This commit is contained in:
74
weed/command/mq_agent.go
Normal file
74
weed/command/mq_agent.go
Normal file
@@ -0,0 +1,74 @@
|
|||||||
|
package command
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/agent"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||||
|
"google.golang.org/grpc/reflection"
|
||||||
|
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/security"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
mqAgentOptions MessageQueueAgentOptions
|
||||||
|
)
|
||||||
|
|
||||||
|
type MessageQueueAgentOptions struct {
|
||||||
|
brokers []pb.ServerAddress
|
||||||
|
brokersString *string
|
||||||
|
filerGroup *string
|
||||||
|
ip *string
|
||||||
|
port *int
|
||||||
|
}
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
cmdMqAgent.Run = runMqAgent // break init cycle
|
||||||
|
mqAgentOptions.brokersString = cmdMqAgent.Flag.String("broker", "localhost:17777", "comma-separated message queue brokers")
|
||||||
|
mqAgentOptions.ip = cmdMqAgent.Flag.String("ip", "localhost", "message queue agent host address")
|
||||||
|
mqAgentOptions.port = cmdMqAgent.Flag.Int("port", 16777, "message queue agent gRPC server port")
|
||||||
|
}
|
||||||
|
|
||||||
|
var cmdMqAgent = &Command{
|
||||||
|
UsageLine: "mq.agent [-port=6377] [-master=<ip:port>]",
|
||||||
|
Short: "<WIP> start a message queue agent",
|
||||||
|
Long: `start a message queue agent
|
||||||
|
|
||||||
|
The agent runs on local server to accept gRPC calls to write or read messages.
|
||||||
|
The messages are sent to message queue brokers.
|
||||||
|
|
||||||
|
`,
|
||||||
|
}
|
||||||
|
|
||||||
|
func runMqAgent(cmd *Command, args []string) bool {
|
||||||
|
|
||||||
|
util.LoadSecurityConfiguration()
|
||||||
|
|
||||||
|
mqAgentOptions.brokers = pb.ServerAddresses(*mqAgentOptions.brokersString).ToAddresses()
|
||||||
|
|
||||||
|
return mqAgentOptions.startQueueAgent()
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
func (mqAgentOpt *MessageQueueAgentOptions) startQueueAgent() bool {
|
||||||
|
|
||||||
|
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.msg_agent")
|
||||||
|
|
||||||
|
agentServer := agent.NewMessageQueueAgent(&agent.MessageQueueAgentOptions{
|
||||||
|
SeedBrokers: mqAgentOpt.brokers,
|
||||||
|
}, grpcDialOption)
|
||||||
|
|
||||||
|
// start grpc listener
|
||||||
|
grpcL, _, err := util.NewIpAndLocalListeners(*mqAgentOpt.ip, *mqAgentOpt.port, 0)
|
||||||
|
if err != nil {
|
||||||
|
glog.Fatalf("failed to listen on grpc port %d: %v", *mqAgentOpt.port, err)
|
||||||
|
}
|
||||||
|
grpcS := pb.NewGrpcServer()
|
||||||
|
mq_agent_pb.RegisterSeaweedMessagingAgentServer(grpcS, agentServer)
|
||||||
|
reflection.Register(grpcS)
|
||||||
|
grpcS.Serve(grpcL)
|
||||||
|
|
||||||
|
return true
|
||||||
|
|
||||||
|
}
|
||||||
@@ -89,6 +89,11 @@ cert = ""
|
|||||||
key = ""
|
key = ""
|
||||||
allowed_commonNames = "" # comma-separated SSL certificate common names
|
allowed_commonNames = "" # comma-separated SSL certificate common names
|
||||||
|
|
||||||
|
[grpc.msg_agent]
|
||||||
|
cert = ""
|
||||||
|
key = ""
|
||||||
|
allowed_commonNames = "" # comma-separated SSL certificate common names
|
||||||
|
|
||||||
# use this for any place needs a grpc client
|
# use this for any place needs a grpc client
|
||||||
# i.e., "weed backup|benchmark|filer.copy|filer.replicate|mount|s3|upload"
|
# i.e., "weed backup|benchmark|filer.copy|filer.replicate|mount|s3|upload"
|
||||||
[grpc.client]
|
[grpc.client]
|
||||||
|
|||||||
61
weed/mq/agent/agent_grpc_pub_session.go
Normal file
61
weed/mq/agent/agent_grpc_pub_session.go
Normal file
@@ -0,0 +1,61 @@
|
|||||||
|
package agent
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"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"
|
||||||
|
)
|
||||||
|
|
||||||
|
func (a *MessageQueueAgent) StartPublishSession(ctx context.Context, req *mq_agent_pb.StartPublishSessionRequest) (*mq_agent_pb.StartPublishSessionResponse, error) {
|
||||||
|
sessionId := rand.Int64()
|
||||||
|
|
||||||
|
topicPublisher := pub_client.NewTopicPublisher(
|
||||||
|
&pub_client.PublisherConfiguration{
|
||||||
|
Topic: topic.NewTopic(req.Topic.Namespace, req.Topic.Name),
|
||||||
|
PartitionCount: req.PartitionCount,
|
||||||
|
Brokers: a.brokersList(),
|
||||||
|
PublisherName: req.PublisherName,
|
||||||
|
RecordType: req.RecordType,
|
||||||
|
})
|
||||||
|
|
||||||
|
a.publishersLock.Lock()
|
||||||
|
// remove inactive publishers to avoid memory leak
|
||||||
|
for k, entry := range a.publishers {
|
||||||
|
if entry.lastActiveTsNs == 0 {
|
||||||
|
// this is an active session
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if time.Unix(0, entry.lastActiveTsNs).Add(10 * time.Hour).Before(time.Now()) {
|
||||||
|
delete(a.publishers, k)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
a.publishers[SessionId(sessionId)] = &SessionEntry[*pub_client.TopicPublisher]{
|
||||||
|
entry: topicPublisher,
|
||||||
|
}
|
||||||
|
a.publishersLock.Unlock()
|
||||||
|
|
||||||
|
return &mq_agent_pb.StartPublishSessionResponse{
|
||||||
|
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
|
||||||
|
}
|
||||||
43
weed/mq/agent/agent_grpc_publish.go
Normal file
43
weed/mq/agent/agent_grpc_publish.go
Normal file
@@ -0,0 +1,43 @@
|
|||||||
|
package agent
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
func (a *MessageQueueAgent) PublishRecordRequest(stream mq_agent_pb.SeaweedMessagingAgent_PublishRecordServer) error {
|
||||||
|
m, err := stream.Recv()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
a.publishersLock.RLock()
|
||||||
|
publisherEntry, found := a.publishers[SessionId(m.SessionId)]
|
||||||
|
a.publishersLock.RUnlock()
|
||||||
|
if !found {
|
||||||
|
return fmt.Errorf("publish session id %d not found", m.SessionId)
|
||||||
|
}
|
||||||
|
defer func() {
|
||||||
|
publisherEntry.lastActiveTsNs = time.Now().UnixNano()
|
||||||
|
}()
|
||||||
|
publisherEntry.lastActiveTsNs = 0
|
||||||
|
|
||||||
|
if m.Value != nil {
|
||||||
|
if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
m, err := stream.Recv()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if m.Value == nil {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if err := publisherEntry.entry.PublishRecord(m.Key, m.Value); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
57
weed/mq/agent/agent_grpc_sub_session.go
Normal file
57
weed/mq/agent/agent_grpc_sub_session.go
Normal file
@@ -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
Normal file
75
weed/mq/agent/agent_grpc_subscribe.go
Normal file
@@ -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,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
52
weed/mq/agent/agent_server.go
Normal file
52
weed/mq/agent/agent_server.go
Normal file
@@ -0,0 +1,52 @@
|
|||||||
|
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"
|
||||||
|
"sync"
|
||||||
|
)
|
||||||
|
|
||||||
|
type SessionId int64
|
||||||
|
type SessionEntry[T any] struct {
|
||||||
|
entry T
|
||||||
|
lastActiveTsNs int64
|
||||||
|
}
|
||||||
|
|
||||||
|
type MessageQueueAgentOptions struct {
|
||||||
|
SeedBrokers []pb.ServerAddress
|
||||||
|
}
|
||||||
|
|
||||||
|
type MessageQueueAgent struct {
|
||||||
|
mq_agent_pb.UnimplementedSeaweedMessagingAgentServer
|
||||||
|
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 {
|
||||||
|
|
||||||
|
// check masters to list all brokers
|
||||||
|
|
||||||
|
return &MessageQueueAgent{
|
||||||
|
option: option,
|
||||||
|
brokers: []pb.ServerAddress{},
|
||||||
|
grpcDialOption: grpcDialOption,
|
||||||
|
publishers: make(map[SessionId]*SessionEntry[*pub_client.TopicPublisher]),
|
||||||
|
subscribers: make(map[SessionId]*SessionEntry[*sub_client.TopicSubscriber]),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *MessageQueueAgent) brokersList() []string {
|
||||||
|
var brokers []string
|
||||||
|
for _, broker := range a.brokers {
|
||||||
|
brokers = append(brokers, broker.String())
|
||||||
|
}
|
||||||
|
return brokers
|
||||||
|
}
|
||||||
@@ -9,6 +9,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"sync"
|
"sync"
|
||||||
)
|
)
|
||||||
|
|
||||||
@@ -55,7 +56,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
|
|||||||
|
|
||||||
// called by broker leader to drain existing partitions.
|
// called by broker leader to drain existing partitions.
|
||||||
// new/updated partitions will be detected by broker from the filer
|
// 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
|
// notify the brokers to create the topic partitions in parallel
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
for _, bpa := range assignments {
|
for _, bpa := range assignments {
|
||||||
|
|||||||
@@ -5,7 +5,6 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
@@ -30,9 +29,6 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
|
|||||||
|
|
||||||
// validate the schema
|
// validate the schema
|
||||||
if request.RecordType != nil {
|
if request.RecordType != nil {
|
||||||
if _, err = schema.NewSchema(request.RecordType); err != nil {
|
|
||||||
return nil, status.Errorf(codes.InvalidArgument, "invalid record type %+v: %v", request.RecordType, err)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
t := topic.FromPbTopic(request.Topic)
|
t := topic.FromPbTopic(request.Topic)
|
||||||
|
|||||||
@@ -7,6 +7,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_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
|
// 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
|
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
||||||
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
|
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
|
||||||
topicPartitionStat := topicPartitionStatsItem.Val
|
topicPartitionStat := topicPartitionStatsItem.Val
|
||||||
topic := &mq_pb.Topic{
|
topic := &schema_pb.Topic{
|
||||||
Namespace: topicPartitionStat.TopicPartition.Namespace,
|
Namespace: topicPartitionStat.TopicPartition.Namespace,
|
||||||
Name: topicPartitionStat.TopicPartition.Name,
|
Name: topicPartitionStat.TopicPartition.Name,
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -9,6 +9,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_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"
|
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||||
"io"
|
"io"
|
||||||
"time"
|
"time"
|
||||||
@@ -54,7 +55,7 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
|||||||
}()
|
}()
|
||||||
|
|
||||||
startPosition := b.getRequestPosition(req.GetInit())
|
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
|
// connect to the follower
|
||||||
var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
|
var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
|
||||||
@@ -212,9 +213,9 @@ func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMess
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
|
if offset.StartType == schema_pb.PartitionOffsetStartType_EARLIEST {
|
||||||
startPosition = log_buffer.NewMessagePosition(1, -3)
|
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)
|
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
|
|||||||
63
weed/mq/client/agent_client/SubscribeSession.go
Normal file
63
weed/mq/client/agent_client/SubscribeSession.go
Normal file
@@ -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
|
||||||
|
}
|
||||||
14
weed/mq/client/agent_client/agent_publish.go
Normal file
14
weed/mq/client/agent_client/agent_publish.go
Normal file
@@ -0,0 +1,14 @@
|
|||||||
|
package agent_client
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
|
)
|
||||||
|
|
||||||
|
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
|
||||||
|
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||||
|
SessionId: a.sessionId,
|
||||||
|
Key: key,
|
||||||
|
Value: record,
|
||||||
|
})
|
||||||
|
}
|
||||||
17
weed/mq/client/agent_client/agent_subscribe.go
Normal file
17
weed/mq/client/agent_client/agent_subscribe.go
Normal file
@@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
70
weed/mq/client/agent_client/publish_session.go
Normal file
70
weed/mq/client/agent_client/publish_session.go
Normal file
@@ -0,0 +1,70 @@
|
|||||||
|
package agent_client
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
)
|
||||||
|
|
||||||
|
type PublishSession struct {
|
||||||
|
schema *schema.Schema
|
||||||
|
partitionCount int
|
||||||
|
publisherName string
|
||||||
|
stream grpc.BidiStreamingClient[mq_agent_pb.PublishRecordRequest, mq_agent_pb.PublishRecordResponse]
|
||||||
|
sessionId int64
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitionCount int, publisherName string) (*PublishSession, error) {
|
||||||
|
|
||||||
|
// call local agent grpc server to create a new session
|
||||||
|
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
|
||||||
|
if err != nil {
|
||||||
|
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: &schema_pb.Topic{
|
||||||
|
Namespace: topicSchema.Namespace,
|
||||||
|
Name: topicSchema.Name,
|
||||||
|
},
|
||||||
|
PartitionCount: int32(partitionCount),
|
||||||
|
RecordType: topicSchema.RecordType,
|
||||||
|
PublisherName: publisherName,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if resp.Error != "" {
|
||||||
|
return nil, fmt.Errorf("start publish session: %v", resp.Error)
|
||||||
|
}
|
||||||
|
|
||||||
|
stream, err := agentClient.PublishRecord(context.Background())
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("publish record: %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
return &PublishSession{
|
||||||
|
schema: topicSchema,
|
||||||
|
partitionCount: partitionCount,
|
||||||
|
publisherName: publisherName,
|
||||||
|
stream: stream,
|
||||||
|
sessionId: resp.SessionId,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *PublishSession) CloseSession() error {
|
||||||
|
if a.schema == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
err := a.stream.CloseSend()
|
||||||
|
if err != nil {
|
||||||
|
return fmt.Errorf("close send: %v", err)
|
||||||
|
}
|
||||||
|
a.schema = nil
|
||||||
|
return err
|
||||||
|
}
|
||||||
128
weed/mq/client/cmd/agent_pub_record/publisher_record.go
Normal file
128
weed/mq/client/cmd/agent_pub_record/publisher_record.go
Normal file
@@ -0,0 +1,128 @@
|
|||||||
|
package main
|
||||||
|
|
||||||
|
import (
|
||||||
|
"flag"
|
||||||
|
"fmt"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
|
"log"
|
||||||
|
"sync"
|
||||||
|
"sync/atomic"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
messageCount = flag.Int("n", 1000, "message count")
|
||||||
|
messageDelay = flag.Duration("d", time.Second, "delay between messages")
|
||||||
|
concurrency = flag.Int("c", 4, "concurrent publishers")
|
||||||
|
partitionCount = flag.Int("p", 6, "partition count")
|
||||||
|
|
||||||
|
clientName = flag.String("client", "c1", "client name")
|
||||||
|
|
||||||
|
namespace = flag.String("ns", "test", "namespace")
|
||||||
|
t = flag.String("t", "test", "t")
|
||||||
|
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
|
||||||
|
|
||||||
|
counter int32
|
||||||
|
)
|
||||||
|
|
||||||
|
func doPublish(publisher *agent_client.PublishSession, id int) {
|
||||||
|
startTime := time.Now()
|
||||||
|
for {
|
||||||
|
i := atomic.AddInt32(&counter, 1)
|
||||||
|
if i > int32(*messageCount) {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
// Simulate publishing a message
|
||||||
|
myRecord := genMyRecord(int32(i))
|
||||||
|
if err := publisher.PublishMessageRecord(myRecord.Key, myRecord.ToRecordValue()); err != nil {
|
||||||
|
fmt.Println(err)
|
||||||
|
break
|
||||||
|
}
|
||||||
|
if *messageDelay > 0 {
|
||||||
|
time.Sleep(*messageDelay)
|
||||||
|
fmt.Printf("sent %+v\n", string(myRecord.Key))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
elapsed := time.Since(startTime)
|
||||||
|
log.Printf("Publisher %s-%d finished in %s", *clientName, id, elapsed)
|
||||||
|
}
|
||||||
|
|
||||||
|
type MyRecord struct {
|
||||||
|
Key []byte
|
||||||
|
Field1 []byte
|
||||||
|
Field2 string
|
||||||
|
Field3 int32
|
||||||
|
Field4 int64
|
||||||
|
Field5 float32
|
||||||
|
Field6 float64
|
||||||
|
Field7 bool
|
||||||
|
}
|
||||||
|
|
||||||
|
func genMyRecord(id int32) *MyRecord {
|
||||||
|
return &MyRecord{
|
||||||
|
Key: []byte(fmt.Sprintf("key-%s-%d", *clientName, id)),
|
||||||
|
Field1: []byte(fmt.Sprintf("field1-%s-%d", *clientName, id)),
|
||||||
|
Field2: fmt.Sprintf("field2-%s-%d", *clientName, id),
|
||||||
|
Field3: id,
|
||||||
|
Field4: int64(id),
|
||||||
|
Field5: float32(id),
|
||||||
|
Field6: float64(id),
|
||||||
|
Field7: id%2 == 0,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (r *MyRecord) ToRecordValue() *schema_pb.RecordValue {
|
||||||
|
return schema.RecordBegin().
|
||||||
|
SetBytes("key", r.Key).
|
||||||
|
SetBytes("field1", r.Field1).
|
||||||
|
SetString("field2", r.Field2).
|
||||||
|
SetInt32("field3", r.Field3).
|
||||||
|
SetInt64("field4", r.Field4).
|
||||||
|
SetFloat("field5", r.Field5).
|
||||||
|
SetDouble("field6", r.Field6).
|
||||||
|
SetBool("field7", r.Field7).
|
||||||
|
RecordEnd()
|
||||||
|
}
|
||||||
|
|
||||||
|
func main() {
|
||||||
|
flag.Parse()
|
||||||
|
|
||||||
|
recordType := schema.RecordTypeBegin().
|
||||||
|
WithField("key", schema.TypeBytes).
|
||||||
|
WithField("field1", schema.TypeBytes).
|
||||||
|
WithField("field2", schema.TypeString).
|
||||||
|
WithField("field3", schema.TypeInt32).
|
||||||
|
WithField("field4", schema.TypeInt64).
|
||||||
|
WithField("field5", schema.TypeFloat).
|
||||||
|
WithField("field6", schema.TypeDouble).
|
||||||
|
WithField("field7", schema.TypeBoolean).
|
||||||
|
RecordTypeEnd()
|
||||||
|
|
||||||
|
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
|
||||||
|
}
|
||||||
|
defer session.CloseSession()
|
||||||
|
|
||||||
|
startTime := time.Now()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
// Start multiple publishers
|
||||||
|
for i := 0; i < *concurrency; i++ {
|
||||||
|
wg.Add(1)
|
||||||
|
go func(id int) {
|
||||||
|
defer wg.Done()
|
||||||
|
doPublish(session, id)
|
||||||
|
}(i)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for all publishers to finish
|
||||||
|
wg.Wait()
|
||||||
|
elapsed := time.Since(startTime)
|
||||||
|
|
||||||
|
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
|
||||||
|
|
||||||
|
}
|
||||||
@@ -6,12 +6,12 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/credentials/insecure"
|
"google.golang.org/grpc/credentials/insecure"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@@ -33,23 +33,24 @@ func main() {
|
|||||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||||
MaxPartitionCount: int32(*maxPartitionCount),
|
MaxPartitionCount: int32(*maxPartitionCount),
|
||||||
PerPartitionConcurrency: int32(*perPartitionConcurrency),
|
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||||
}
|
}
|
||||||
|
|
||||||
contentConfig := &sub_client.ContentConfiguration{
|
contentConfig := &sub_client.ContentConfiguration{
|
||||||
Topic: topic.NewTopic(*namespace, *t),
|
Topic: topic.NewTopic(*namespace, *t),
|
||||||
Filter: "",
|
Filter: "",
|
||||||
StartTime: time.Unix(1, 1),
|
|
||||||
}
|
}
|
||||||
|
|
||||||
brokers := strings.Split(*seedBrokers, ",")
|
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
|
counter := 0
|
||||||
subscriber.SetEachMessageFunc(func(key, value []byte) error {
|
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||||
counter++
|
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||||
println(string(key), "=>", string(value), counter)
|
executors.Execute(func() {
|
||||||
return nil
|
counter++
|
||||||
|
println(string(m.Data.Key), "=>", string(m.Data.Value), counter)
|
||||||
|
})
|
||||||
})
|
})
|
||||||
|
|
||||||
subscriber.SetCompletionFunc(func() {
|
subscriber.SetCompletionFunc(func() {
|
||||||
|
|||||||
@@ -6,6 +6,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"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/pb/schema_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||||
@@ -60,30 +61,31 @@ func main() {
|
|||||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||||
MaxPartitionCount: int32(*maxPartitionCount),
|
MaxPartitionCount: int32(*maxPartitionCount),
|
||||||
PerPartitionConcurrency: int32(*perPartitionConcurrency),
|
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||||
}
|
}
|
||||||
|
|
||||||
contentConfig := &sub_client.ContentConfiguration{
|
contentConfig := &sub_client.ContentConfiguration{
|
||||||
Topic: topic.NewTopic(*namespace, *t),
|
Topic: topic.NewTopic(*namespace, *t),
|
||||||
Filter: "",
|
Filter: "",
|
||||||
StartTime: time.Now().Add(-*timeAgo),
|
// StartTime: time.Now().Add(-*timeAgo),
|
||||||
}
|
}
|
||||||
|
|
||||||
brokers := strings.Split(*seedBrokers, ",")
|
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
|
counter := 0
|
||||||
subscriber.SetEachMessageFunc(func(key, value []byte) error {
|
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||||
counter++
|
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||||
record := &schema_pb.RecordValue{}
|
executors.Execute(func() {
|
||||||
err := proto.Unmarshal(value, record)
|
counter++
|
||||||
if err != nil {
|
record := &schema_pb.RecordValue{}
|
||||||
fmt.Printf("unmarshal record value: %v\n", err)
|
err := proto.Unmarshal(m.Data.Value, record)
|
||||||
} else {
|
if err != nil {
|
||||||
fmt.Printf("%s %d: %v\n", string(key), len(value), record)
|
fmt.Printf("unmarshal record value: %v\n", err)
|
||||||
}
|
} else {
|
||||||
//time.Sleep(1300 * time.Millisecond)
|
fmt.Printf("%s %d: %v\n", string(m.Data.Key), len(m.Data.Value), record)
|
||||||
return nil
|
}
|
||||||
|
})
|
||||||
})
|
})
|
||||||
|
|
||||||
subscriber.SetCompletionFunc(func() {
|
subscriber.SetCompletionFunc(func() {
|
||||||
|
|||||||
@@ -6,12 +6,18 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"io"
|
"io"
|
||||||
"reflect"
|
"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
|
// connect to the partition broker
|
||||||
return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
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)
|
return fmt.Errorf("create subscribe client: %v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency
|
slidingWindowSize := sub.SubscriberConfig.SlidingWindowSize
|
||||||
if perPartitionConcurrency <= 0 {
|
if slidingWindowSize <= 0 {
|
||||||
perPartitionConcurrency = 1
|
slidingWindowSize = 1
|
||||||
}
|
}
|
||||||
|
|
||||||
var stopTsNs int64
|
po := findPartitionOffset(sub.ContentConfig.PartitionOffsets, assigned.Partition)
|
||||||
if !sub.ContentConfig.StopTime.IsZero() {
|
if po == nil {
|
||||||
stopTsNs = sub.ContentConfig.StopTime.UnixNano()
|
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{
|
if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{
|
||||||
Message: &mq_pb.SubscribeMessageRequest_Init{
|
Message: &mq_pb.SubscribeMessageRequest_Init{
|
||||||
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
|
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
|
||||||
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
||||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||||
PartitionOffset: &mq_pb.PartitionOffset{
|
PartitionOffset: po,
|
||||||
Partition: assigned.Partition,
|
Filter: sub.ContentConfig.Filter,
|
||||||
StartTsNs: sub.ContentConfig.StartTime.UnixNano(),
|
FollowerBroker: assigned.FollowerBroker,
|
||||||
StopTsNs: stopTsNs,
|
SlidingWindowSize: slidingWindowSize,
|
||||||
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
|
||||||
},
|
|
||||||
Filter: sub.ContentConfig.Filter,
|
|
||||||
FollowerBroker: assigned.FollowerBroker,
|
|
||||||
Concurrency: perPartitionConcurrency,
|
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
}); err != nil {
|
}); err != nil {
|
||||||
@@ -57,24 +62,13 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
|||||||
defer sub.OnCompletionFunc()
|
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() {
|
go func() {
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-stopCh:
|
case <-stopCh:
|
||||||
subscribeClient.CloseSend()
|
subscribeClient.CloseSend()
|
||||||
return
|
return
|
||||||
case ack, ok := <-partitionOffsetChan:
|
case ack, ok := <-sub.PartitionOffsetChan:
|
||||||
if !ok {
|
if !ok {
|
||||||
subscribeClient.CloseSend()
|
subscribeClient.CloseSend()
|
||||||
return
|
return
|
||||||
@@ -91,9 +85,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
|||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
var lastErr error
|
for {
|
||||||
|
|
||||||
for lastErr == nil {
|
|
||||||
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||||
resp, err := subscribeClient.Recv()
|
resp, err := subscribeClient.Recv()
|
||||||
if err != nil {
|
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))
|
fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
executors.Execute(func() {
|
onDataMessageFn(m)
|
||||||
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
|
|
||||||
}
|
|
||||||
})
|
|
||||||
case *mq_pb.SubscribeMessageResponse_Ctrl:
|
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)
|
// 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 {
|
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
|
||||||
|
}
|
||||||
|
|||||||
@@ -4,6 +4,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
@@ -20,6 +21,7 @@ func (sub *TopicSubscriber) Subscribe() error {
|
|||||||
go sub.startProcessors()
|
go sub.startProcessors()
|
||||||
|
|
||||||
// loop forever
|
// loop forever
|
||||||
|
// TODO shutdown the subscriber when not needed anymore
|
||||||
sub.doKeepConnectedToSubCoordinator()
|
sub.doKeepConnectedToSubCoordinator()
|
||||||
|
|
||||||
return nil
|
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 {
|
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)
|
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
|
||||||
} else {
|
} else {
|
||||||
|
|||||||
@@ -3,9 +3,9 @@ package sub_client
|
|||||||
import (
|
import (
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type SubscriberConfiguration struct {
|
type SubscriberConfiguration struct {
|
||||||
@@ -14,16 +14,16 @@ type SubscriberConfiguration struct {
|
|||||||
ConsumerGroupInstanceId string
|
ConsumerGroupInstanceId string
|
||||||
GrpcDialOption grpc.DialOption
|
GrpcDialOption grpc.DialOption
|
||||||
MaxPartitionCount int32 // how many partitions to process concurrently
|
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 {
|
type ContentConfiguration struct {
|
||||||
Topic topic.Topic
|
Topic topic.Topic
|
||||||
Filter string
|
Filter string
|
||||||
StartTime time.Time
|
PartitionOffsets []*schema_pb.PartitionOffset
|
||||||
StopTime time.Time
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data)
|
||||||
type OnEachMessageFunc func(key, value []byte) (err error)
|
type OnEachMessageFunc func(key, value []byte) (err error)
|
||||||
type OnCompletionFunc func()
|
type OnCompletionFunc func()
|
||||||
|
|
||||||
@@ -32,15 +32,17 @@ type TopicSubscriber struct {
|
|||||||
ContentConfig *ContentConfiguration
|
ContentConfig *ContentConfiguration
|
||||||
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||||
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
|
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
|
||||||
|
OnDataMessageFnnc OnDataMessageFn
|
||||||
OnEachMessageFunc OnEachMessageFunc
|
OnEachMessageFunc OnEachMessageFunc
|
||||||
OnCompletionFunc OnCompletionFunc
|
OnCompletionFunc OnCompletionFunc
|
||||||
bootstrapBrokers []string
|
bootstrapBrokers []string
|
||||||
waitForMoreMessage bool
|
waitForMoreMessage bool
|
||||||
activeProcessors map[topic.Partition]*ProcessorState
|
activeProcessors map[topic.Partition]*ProcessorState
|
||||||
activeProcessorsLock sync.Mutex
|
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{
|
return &TopicSubscriber{
|
||||||
SubscriberConfig: subscriber,
|
SubscriberConfig: subscriber,
|
||||||
ContentConfig: content,
|
ContentConfig: content,
|
||||||
@@ -49,6 +51,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
|
|||||||
bootstrapBrokers: bootstrapBrokers,
|
bootstrapBrokers: bootstrapBrokers,
|
||||||
waitForMoreMessage: true,
|
waitForMoreMessage: true,
|
||||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||||
|
PartitionOffsetChan: partitionOffsetChan,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -56,6 +59,10 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc
|
|||||||
sub.OnEachMessageFunc = onEachMessageFn
|
sub.OnEachMessageFunc = onEachMessageFn
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) {
|
||||||
|
sub.OnDataMessageFnnc = fn
|
||||||
|
}
|
||||||
|
|
||||||
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
|
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
|
||||||
sub.OnCompletionFunc = onCompletionFn
|
sub.OnCompletionFunc = onCompletionFn
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -4,6 +4,7 @@ import (
|
|||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
@@ -14,7 +15,7 @@ func AllocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p
|
|||||||
rangeSize := MaxPartitionCount / partitionCount
|
rangeSize := MaxPartitionCount / partitionCount
|
||||||
for i := int32(0); i < partitionCount; i++ {
|
for i := int32(0); i < partitionCount; i++ {
|
||||||
assignment := &mq_pb.BrokerPartitionAssignment{
|
assignment := &mq_pb.BrokerPartitionAssignment{
|
||||||
Partition: &mq_pb.Partition{
|
Partition: &schema_pb.Partition{
|
||||||
RingSize: MaxPartitionCount,
|
RingSize: MaxPartitionCount,
|
||||||
RangeStart: int32(i * rangeSize),
|
RangeStart: int32(i * rangeSize),
|
||||||
RangeStop: int32((i + 1) * rangeSize),
|
RangeStop: int32((i + 1) * rangeSize),
|
||||||
|
|||||||
@@ -4,6 +4,7 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"testing"
|
"testing"
|
||||||
)
|
)
|
||||||
@@ -29,7 +30,7 @@ func Test_allocateOneBroker(t *testing.T) {
|
|||||||
wantAssignments: []*mq_pb.BrokerPartitionAssignment{
|
wantAssignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:17777",
|
LeaderBroker: "localhost:17777",
|
||||||
Partition: &mq_pb.Partition{
|
Partition: &schema_pb.Partition{
|
||||||
RingSize: MaxPartitionCount,
|
RingSize: MaxPartitionCount,
|
||||||
RangeStart: 0,
|
RangeStart: 0,
|
||||||
RangeStop: MaxPartitionCount,
|
RangeStop: MaxPartitionCount,
|
||||||
@@ -96,7 +97,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "",
|
LeaderBroker: "",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "localhost:2",
|
FollowerBroker: "localhost:2",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -111,7 +112,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:1",
|
LeaderBroker: "localhost:1",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "",
|
FollowerBroker: "",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -126,7 +127,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:1",
|
LeaderBroker: "localhost:1",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "localhost:200",
|
FollowerBroker: "localhost:200",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -141,7 +142,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:100",
|
LeaderBroker: "localhost:100",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "localhost:200",
|
FollowerBroker: "localhost:200",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -156,7 +157,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:1",
|
LeaderBroker: "localhost:1",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "localhost:2",
|
FollowerBroker: "localhost:2",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -171,7 +172,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:1",
|
LeaderBroker: "localhost:1",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
@@ -185,7 +186,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
|||||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||||
{
|
{
|
||||||
LeaderBroker: "localhost:1",
|
LeaderBroker: "localhost:1",
|
||||||
Partition: &mq_pb.Partition{},
|
Partition: &schema_pb.Partition{},
|
||||||
FollowerBroker: "localhost:2",
|
FollowerBroker: "localhost:2",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
|
|||||||
@@ -5,6 +5,7 @@ import (
|
|||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
)
|
)
|
||||||
|
|
||||||
type BrokerStats struct {
|
type BrokerStats struct {
|
||||||
@@ -65,7 +66,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
|
|||||||
bs.SubscriberCount = subscriberCount
|
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{
|
tps := &TopicPartitionStats{
|
||||||
TopicPartition: topic.TopicPartition{
|
TopicPartition: topic.TopicPartition{
|
||||||
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},
|
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},
|
||||||
|
|||||||
@@ -3,13 +3,14 @@ package pub_balancer
|
|||||||
import (
|
import (
|
||||||
"errors"
|
"errors"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
ErrNoBroker = errors.New("no broker")
|
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
|
// find existing topic partition assignments
|
||||||
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
|
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
|
||||||
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
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 &&
|
if topicPartitionStat.TopicPartition.Namespace == topic.Namespace &&
|
||||||
topicPartitionStat.TopicPartition.Name == topic.Name {
|
topicPartitionStat.TopicPartition.Name == topic.Name {
|
||||||
assignment := &mq_pb.BrokerPartitionAssignment{
|
assignment := &mq_pb.BrokerPartitionAssignment{
|
||||||
Partition: &mq_pb.Partition{
|
Partition: &schema_pb.Partition{
|
||||||
RingSize: MaxPartitionCount,
|
RingSize: MaxPartitionCount,
|
||||||
RangeStart: topicPartitionStat.RangeStart,
|
RangeStart: topicPartitionStat.RangeStart,
|
||||||
RangeStop: topicPartitionStat.RangeStop,
|
RangeStop: topicPartitionStat.RangeStop,
|
||||||
|
|||||||
@@ -2,7 +2,7 @@ package pub_balancer
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"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 {
|
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 {
|
for _, partitionSlot := range ps.PartitionSlots {
|
||||||
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
|
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
|
||||||
if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {
|
if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {
|
||||||
|
|||||||
@@ -4,6 +4,7 @@ import (
|
|||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
@@ -32,7 +33,7 @@ type PubBalancer struct {
|
|||||||
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
|
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
|
||||||
// Collected from all brokers when they connect to the broker leader
|
// Collected from all brokers when they connect to the broker leader
|
||||||
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
|
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 {
|
func NewPubBalancer() *PubBalancer {
|
||||||
|
|||||||
@@ -5,19 +5,24 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
type Schema struct {
|
type Schema struct {
|
||||||
|
Namespace string
|
||||||
|
Name string
|
||||||
|
RevisionId uint32
|
||||||
RecordType *schema_pb.RecordType
|
RecordType *schema_pb.RecordType
|
||||||
fieldMap map[string]*schema_pb.Field
|
fieldMap map[string]*schema_pb.Field
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewSchema(recordType *schema_pb.RecordType) (*Schema, error) {
|
func NewSchema(namespace string, name string, recordType *schema_pb.RecordType) *Schema {
|
||||||
fieldMap := make(map[string]*schema_pb.Field)
|
fieldMap := make(map[string]*schema_pb.Field)
|
||||||
for _, field := range recordType.Fields {
|
for _, field := range recordType.Fields {
|
||||||
fieldMap[field.Name] = field
|
fieldMap[field.Name] = field
|
||||||
}
|
}
|
||||||
return &Schema{
|
return &Schema{
|
||||||
|
Namespace: namespace,
|
||||||
|
Name: name,
|
||||||
RecordType: recordType,
|
RecordType: recordType,
|
||||||
fieldMap: fieldMap,
|
fieldMap: fieldMap,
|
||||||
}, nil
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Schema) GetField(name string) (*schema_pb.Field, bool) {
|
func (s *Schema) GetField(name string) (*schema_pb.Field, bool) {
|
||||||
|
|||||||
@@ -7,6 +7,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
@@ -20,7 +21,7 @@ type ConsumerGroup struct {
|
|||||||
stopCh chan 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{
|
cg := &ConsumerGroup{
|
||||||
topic: topic.FromPbTopic(t),
|
topic: topic.FromPbTopic(t),
|
||||||
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
|
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
|
||||||
|
|||||||
@@ -5,6 +5,7 @@ import (
|
|||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/filer_client"
|
"github.com/seaweedfs/seaweedfs/weed/filer_client"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
)
|
)
|
||||||
|
|
||||||
type TopicConsumerGroups struct {
|
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)
|
topicName := toTopicName(topic)
|
||||||
tcg, _ := c.TopicSubscribers.Get(topicName)
|
tcg, _ := c.TopicSubscribers.Get(topicName)
|
||||||
if tcg == nil && createIfMissing {
|
if tcg == nil && createIfMissing {
|
||||||
@@ -41,12 +42,12 @@ func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMiss
|
|||||||
}
|
}
|
||||||
return tcg
|
return tcg
|
||||||
}
|
}
|
||||||
func (c *SubCoordinator) RemoveTopic(topic *mq_pb.Topic) {
|
func (c *SubCoordinator) RemoveTopic(topic *schema_pb.Topic) {
|
||||||
topicName := toTopicName(topic)
|
topicName := toTopicName(topic)
|
||||||
c.TopicSubscribers.Remove(topicName)
|
c.TopicSubscribers.Remove(topicName)
|
||||||
}
|
}
|
||||||
|
|
||||||
func toTopicName(topic *mq_pb.Topic) string {
|
func toTopicName(topic *schema_pb.Topic) string {
|
||||||
topicName := topic.Namespace + "." + topic.Name
|
topicName := topic.Namespace + "." + topic.Name
|
||||||
return topicName
|
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))
|
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
|
||||||
if tcg == nil {
|
if tcg == nil {
|
||||||
return
|
return
|
||||||
|
|||||||
@@ -3,6 +3,7 @@ package topic
|
|||||||
import (
|
import (
|
||||||
cmap "github.com/orcaman/concurrent-map/v2"
|
cmap "github.com/orcaman/concurrent-map/v2"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"github.com/shirou/gopsutil/v3/cpu"
|
"github.com/shirou/gopsutil/v3/cpu"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
@@ -89,7 +90,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
|
|||||||
Partition: localPartition.Partition,
|
Partition: localPartition.Partition,
|
||||||
}
|
}
|
||||||
stats.Stats[topicPartition.TopicPartitionId()] = &mq_pb.TopicPartitionStats{
|
stats.Stats[topicPartition.TopicPartitionId()] = &mq_pb.TopicPartitionStats{
|
||||||
Topic: &mq_pb.Topic{
|
Topic: &schema_pb.Topic{
|
||||||
Namespace: string(localTopic.Namespace),
|
Namespace: string(localTopic.Namespace),
|
||||||
Name: localTopic.Name,
|
Name: localTopic.Name,
|
||||||
},
|
},
|
||||||
|
|||||||
@@ -2,7 +2,7 @@ package topic
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
@@ -40,7 +40,7 @@ func (partition Partition) Equals(other Partition) bool {
|
|||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
func FromPbPartition(partition *mq_pb.Partition) Partition {
|
func FromPbPartition(partition *schema_pb.Partition) Partition {
|
||||||
return Partition{
|
return Partition{
|
||||||
RangeStart: partition.RangeStart,
|
RangeStart: partition.RangeStart,
|
||||||
RangeStop: partition.RangeStop,
|
RangeStop: partition.RangeStop,
|
||||||
@@ -67,8 +67,8 @@ func SplitPartitions(targetCount int32, ts int64) []*Partition {
|
|||||||
return partitions
|
return partitions
|
||||||
}
|
}
|
||||||
|
|
||||||
func (partition Partition) ToPbPartition() *mq_pb.Partition {
|
func (partition Partition) ToPbPartition() *schema_pb.Partition {
|
||||||
return &mq_pb.Partition{
|
return &schema_pb.Partition{
|
||||||
RangeStart: partition.RangeStart,
|
RangeStart: partition.RangeStart,
|
||||||
RangeStop: partition.RangeStop,
|
RangeStop: partition.RangeStop,
|
||||||
RingSize: partition.RingSize,
|
RingSize: partition.RingSize,
|
||||||
|
|||||||
@@ -7,6 +7,7 @@ import (
|
|||||||
"github.com/seaweedfs/seaweedfs/weed/filer"
|
"github.com/seaweedfs/seaweedfs/weed/filer"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
jsonpb "google.golang.org/protobuf/encoding/protojson"
|
jsonpb "google.golang.org/protobuf/encoding/protojson"
|
||||||
)
|
)
|
||||||
|
|
||||||
@@ -21,15 +22,15 @@ func NewTopic(namespace string, name string) Topic {
|
|||||||
Name: name,
|
Name: name,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
func FromPbTopic(topic *mq_pb.Topic) Topic {
|
func FromPbTopic(topic *schema_pb.Topic) Topic {
|
||||||
return Topic{
|
return Topic{
|
||||||
Namespace: topic.Namespace,
|
Namespace: topic.Namespace,
|
||||||
Name: topic.Name,
|
Name: topic.Name,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (t Topic) ToPbTopic() *mq_pb.Topic {
|
func (t Topic) ToPbTopic() *schema_pb.Topic {
|
||||||
return &mq_pb.Topic{
|
return &schema_pb.Topic{
|
||||||
Namespace: t.Namespace,
|
Namespace: t.Namespace,
|
||||||
Name: t.Name,
|
Name: t.Name,
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -10,8 +10,9 @@ gen:
|
|||||||
protoc iam.proto --go_out=./iam_pb --go-grpc_out=./iam_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
protoc iam.proto --go_out=./iam_pb --go-grpc_out=./iam_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
protoc mount.proto --go_out=./mount_pb --go-grpc_out=./mount_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
protoc mount.proto --go_out=./mount_pb --go-grpc_out=./mount_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
protoc s3.proto --go_out=./s3_pb --go-grpc_out=./s3_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
protoc s3.proto --go_out=./s3_pb --go-grpc_out=./s3_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
protoc mq.proto --go_out=./mq_pb --go-grpc_out=./mq_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
protoc mq_broker.proto --go_out=./mq_pb --go-grpc_out=./mq_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
protoc schema.proto --go_out=./schema_pb --go-grpc_out=./schema_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
protoc mq_schema.proto --go_out=./schema_pb --go-grpc_out=./schema_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
|
protoc mq_agent.proto --go_out=./mq_agent_pb --go-grpc_out=./mq_agent_pb --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative
|
||||||
# protoc filer.proto --java_out=../../other/java/client/src/main/java
|
# protoc filer.proto --java_out=../../other/java/client/src/main/java
|
||||||
cp filer.proto ../../other/java/client/src/main/proto
|
cp filer.proto ../../other/java/client/src/main/proto
|
||||||
|
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: filer.proto
|
// source: filer.proto
|
||||||
|
|
||||||
package filer_pb
|
package filer_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: filer.proto
|
// source: filer.proto
|
||||||
|
|
||||||
package filer_pb
|
package filer_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: iam.proto
|
// source: iam.proto
|
||||||
|
|
||||||
package iam_pb
|
package iam_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: iam.proto
|
// source: iam.proto
|
||||||
|
|
||||||
package iam_pb
|
package iam_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: master.proto
|
// source: master.proto
|
||||||
|
|
||||||
package master_pb
|
package master_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: master.proto
|
// source: master.proto
|
||||||
|
|
||||||
package master_pb
|
package master_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: mount.proto
|
// source: mount.proto
|
||||||
|
|
||||||
package mount_pb
|
package mount_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: mount.proto
|
// source: mount.proto
|
||||||
|
|
||||||
package mount_pb
|
package mount_pb
|
||||||
|
|||||||
95
weed/pb/mq_agent.proto
Normal file
95
weed/pb/mq_agent.proto
Normal file
@@ -0,0 +1,95 @@
|
|||||||
|
syntax = "proto3";
|
||||||
|
|
||||||
|
package messaging_pb;
|
||||||
|
|
||||||
|
import "mq_schema.proto";
|
||||||
|
|
||||||
|
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb";
|
||||||
|
option java_package = "seaweedfs.mq_agent";
|
||||||
|
option java_outer_classname = "MessageQueueAgentProto";
|
||||||
|
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
|
||||||
|
service SeaweedMessagingAgent {
|
||||||
|
|
||||||
|
// Publishing
|
||||||
|
rpc StartPublishSession (StartPublishSessionRequest) returns (StartPublishSessionResponse) {
|
||||||
|
}
|
||||||
|
rpc ClosePublishSession (ClosePublishSessionRequest) returns (ClosePublishSessionResponse) {
|
||||||
|
}
|
||||||
|
rpc PublishRecord (stream PublishRecordRequest) returns (stream PublishRecordResponse) {
|
||||||
|
}
|
||||||
|
|
||||||
|
// Subscribing
|
||||||
|
rpc StartSubscribeSession (StartSubscribeSessionRequest) returns (StartSubscribeSessionResponse) {
|
||||||
|
}
|
||||||
|
rpc CloseSubscribeSession (CloseSubscribeSessionRequest) returns (CloseSubscribeSessionResponse) {
|
||||||
|
}
|
||||||
|
rpc SubscribeRecord (stream SubscribeRecordRequest) returns (stream SubscribeRecordResponse) {
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
message StartPublishSessionRequest {
|
||||||
|
schema_pb.Topic topic = 1;
|
||||||
|
int32 partition_count = 2;
|
||||||
|
schema_pb.RecordType record_type = 3;
|
||||||
|
string publisher_name = 4;
|
||||||
|
}
|
||||||
|
message StartPublishSessionResponse {
|
||||||
|
string error = 1;
|
||||||
|
int64 session_id = 2;
|
||||||
|
}
|
||||||
|
message ClosePublishSessionRequest {
|
||||||
|
int64 session_id = 1;
|
||||||
|
}
|
||||||
|
message ClosePublishSessionResponse {
|
||||||
|
string error = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
message PublishRecordRequest {
|
||||||
|
int64 session_id = 1; // session_id is required for the first record
|
||||||
|
bytes key = 2;
|
||||||
|
schema_pb.RecordValue value = 3;
|
||||||
|
}
|
||||||
|
message PublishRecordResponse {
|
||||||
|
int64 ack_sequence = 1;
|
||||||
|
string error = 2;
|
||||||
|
}
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
message StartSubscribeSessionRequest {
|
||||||
|
string consumer_group = 1;
|
||||||
|
string consumer_group_instance_id = 2;
|
||||||
|
schema_pb.Topic topic = 4;
|
||||||
|
repeated schema_pb.PartitionOffset partition_offsets = 5;
|
||||||
|
string filter = 6;
|
||||||
|
int32 max_subscribed_partitions = 8;
|
||||||
|
int32 sliding_window_size = 9;
|
||||||
|
}
|
||||||
|
message StartSubscribeSessionResponse {
|
||||||
|
string error = 1;
|
||||||
|
int64 session_id = 2;
|
||||||
|
}
|
||||||
|
message CloseSubscribeSessionRequest {
|
||||||
|
int64 session_id = 1;
|
||||||
|
}
|
||||||
|
message CloseSubscribeSessionResponse {
|
||||||
|
string error = 1;
|
||||||
|
}
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
message SubscribeRecordRequest {
|
||||||
|
int64 session_id = 1; // session_id is required for the first record
|
||||||
|
int64 ack_sequence = 2;
|
||||||
|
bytes ack_key = 3;
|
||||||
|
}
|
||||||
|
message SubscribeRecordResponse {
|
||||||
|
bytes key = 2;
|
||||||
|
schema_pb.RecordValue value = 3;
|
||||||
|
int64 ts_ns = 4;
|
||||||
|
string error = 5;
|
||||||
|
bool is_end_of_stream = 6;
|
||||||
|
bool is_end_of_topic = 7;
|
||||||
|
}
|
||||||
|
//////////////////////////////////////////////////
|
||||||
1148
weed/pb/mq_agent_pb/mq_agent.pb.go
Normal file
1148
weed/pb/mq_agent_pb/mq_agent.pb.go
Normal file
File diff suppressed because it is too large
Load Diff
304
weed/pb/mq_agent_pb/mq_agent_grpc.pb.go
Normal file
304
weed/pb/mq_agent_pb/mq_agent_grpc.pb.go
Normal file
@@ -0,0 +1,304 @@
|
|||||||
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
|
// versions:
|
||||||
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
|
// - protoc v5.28.3
|
||||||
|
// source: mq_agent.proto
|
||||||
|
|
||||||
|
package mq_agent_pb
|
||||||
|
|
||||||
|
import (
|
||||||
|
context "context"
|
||||||
|
grpc "google.golang.org/grpc"
|
||||||
|
codes "google.golang.org/grpc/codes"
|
||||||
|
status "google.golang.org/grpc/status"
|
||||||
|
)
|
||||||
|
|
||||||
|
// This is a compile-time assertion to ensure that this generated file
|
||||||
|
// is compatible with the grpc package it is being compiled against.
|
||||||
|
// Requires gRPC-Go v1.64.0 or later.
|
||||||
|
const _ = grpc.SupportPackageIsVersion9
|
||||||
|
|
||||||
|
const (
|
||||||
|
SeaweedMessagingAgent_StartPublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/StartPublishSession"
|
||||||
|
SeaweedMessagingAgent_ClosePublishSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/ClosePublishSession"
|
||||||
|
SeaweedMessagingAgent_PublishRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/PublishRecord"
|
||||||
|
SeaweedMessagingAgent_StartSubscribeSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/StartSubscribeSession"
|
||||||
|
SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/CloseSubscribeSession"
|
||||||
|
SeaweedMessagingAgent_SubscribeRecord_FullMethodName = "/messaging_pb.SeaweedMessagingAgent/SubscribeRecord"
|
||||||
|
)
|
||||||
|
|
||||||
|
// SeaweedMessagingAgentClient is the client API for SeaweedMessagingAgent service.
|
||||||
|
//
|
||||||
|
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
|
||||||
|
type SeaweedMessagingAgentClient interface {
|
||||||
|
// Publishing
|
||||||
|
StartPublishSession(ctx context.Context, in *StartPublishSessionRequest, opts ...grpc.CallOption) (*StartPublishSessionResponse, error)
|
||||||
|
ClosePublishSession(ctx context.Context, in *ClosePublishSessionRequest, opts ...grpc.CallOption) (*ClosePublishSessionResponse, error)
|
||||||
|
PublishRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse], error)
|
||||||
|
// Subscribing
|
||||||
|
StartSubscribeSession(ctx context.Context, in *StartSubscribeSessionRequest, opts ...grpc.CallOption) (*StartSubscribeSessionResponse, error)
|
||||||
|
CloseSubscribeSession(ctx context.Context, in *CloseSubscribeSessionRequest, opts ...grpc.CallOption) (*CloseSubscribeSessionResponse, error)
|
||||||
|
SubscribeRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse], error)
|
||||||
|
}
|
||||||
|
|
||||||
|
type seaweedMessagingAgentClient struct {
|
||||||
|
cc grpc.ClientConnInterface
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewSeaweedMessagingAgentClient(cc grpc.ClientConnInterface) SeaweedMessagingAgentClient {
|
||||||
|
return &seaweedMessagingAgentClient{cc}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) StartPublishSession(ctx context.Context, in *StartPublishSessionRequest, opts ...grpc.CallOption) (*StartPublishSessionResponse, error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
out := new(StartPublishSessionResponse)
|
||||||
|
err := c.cc.Invoke(ctx, SeaweedMessagingAgent_StartPublishSession_FullMethodName, in, out, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) ClosePublishSession(ctx context.Context, in *ClosePublishSessionRequest, opts ...grpc.CallOption) (*ClosePublishSessionResponse, error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
out := new(ClosePublishSessionResponse)
|
||||||
|
err := c.cc.Invoke(ctx, SeaweedMessagingAgent_ClosePublishSession_FullMethodName, in, out, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) PublishRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse], error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
stream, err := c.cc.NewStream(ctx, &SeaweedMessagingAgent_ServiceDesc.Streams[0], SeaweedMessagingAgent_PublishRecord_FullMethodName, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
x := &grpc.GenericClientStream[PublishRecordRequest, PublishRecordResponse]{ClientStream: stream}
|
||||||
|
return x, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||||
|
type SeaweedMessagingAgent_PublishRecordClient = grpc.BidiStreamingClient[PublishRecordRequest, PublishRecordResponse]
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) StartSubscribeSession(ctx context.Context, in *StartSubscribeSessionRequest, opts ...grpc.CallOption) (*StartSubscribeSessionResponse, error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
out := new(StartSubscribeSessionResponse)
|
||||||
|
err := c.cc.Invoke(ctx, SeaweedMessagingAgent_StartSubscribeSession_FullMethodName, in, out, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) CloseSubscribeSession(ctx context.Context, in *CloseSubscribeSessionRequest, opts ...grpc.CallOption) (*CloseSubscribeSessionResponse, error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
out := new(CloseSubscribeSessionResponse)
|
||||||
|
err := c.cc.Invoke(ctx, SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName, in, out, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
return out, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *seaweedMessagingAgentClient) SubscribeRecord(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse], error) {
|
||||||
|
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
|
||||||
|
stream, err := c.cc.NewStream(ctx, &SeaweedMessagingAgent_ServiceDesc.Streams[1], SeaweedMessagingAgent_SubscribeRecord_FullMethodName, cOpts...)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
x := &grpc.GenericClientStream[SubscribeRecordRequest, SubscribeRecordResponse]{ClientStream: stream}
|
||||||
|
return x, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||||
|
type SeaweedMessagingAgent_SubscribeRecordClient = grpc.BidiStreamingClient[SubscribeRecordRequest, SubscribeRecordResponse]
|
||||||
|
|
||||||
|
// SeaweedMessagingAgentServer is the server API for SeaweedMessagingAgent service.
|
||||||
|
// All implementations must embed UnimplementedSeaweedMessagingAgentServer
|
||||||
|
// for forward compatibility.
|
||||||
|
type SeaweedMessagingAgentServer interface {
|
||||||
|
// Publishing
|
||||||
|
StartPublishSession(context.Context, *StartPublishSessionRequest) (*StartPublishSessionResponse, error)
|
||||||
|
ClosePublishSession(context.Context, *ClosePublishSessionRequest) (*ClosePublishSessionResponse, error)
|
||||||
|
PublishRecord(grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse]) error
|
||||||
|
// Subscribing
|
||||||
|
StartSubscribeSession(context.Context, *StartSubscribeSessionRequest) (*StartSubscribeSessionResponse, error)
|
||||||
|
CloseSubscribeSession(context.Context, *CloseSubscribeSessionRequest) (*CloseSubscribeSessionResponse, error)
|
||||||
|
SubscribeRecord(grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse]) error
|
||||||
|
mustEmbedUnimplementedSeaweedMessagingAgentServer()
|
||||||
|
}
|
||||||
|
|
||||||
|
// UnimplementedSeaweedMessagingAgentServer must be embedded to have
|
||||||
|
// forward compatible implementations.
|
||||||
|
//
|
||||||
|
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
||||||
|
// pointer dereference when methods are called.
|
||||||
|
type UnimplementedSeaweedMessagingAgentServer struct{}
|
||||||
|
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) StartPublishSession(context.Context, *StartPublishSessionRequest) (*StartPublishSessionResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method StartPublishSession not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) ClosePublishSession(context.Context, *ClosePublishSessionRequest) (*ClosePublishSessionResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method ClosePublishSession not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) PublishRecord(grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse]) error {
|
||||||
|
return status.Errorf(codes.Unimplemented, "method PublishRecord not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) StartSubscribeSession(context.Context, *StartSubscribeSessionRequest) (*StartSubscribeSessionResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method StartSubscribeSession not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) CloseSubscribeSession(context.Context, *CloseSubscribeSessionRequest) (*CloseSubscribeSessionResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method CloseSubscribeSession not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) SubscribeRecord(grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse]) error {
|
||||||
|
return status.Errorf(codes.Unimplemented, "method SubscribeRecord not implemented")
|
||||||
|
}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) mustEmbedUnimplementedSeaweedMessagingAgentServer() {}
|
||||||
|
func (UnimplementedSeaweedMessagingAgentServer) testEmbeddedByValue() {}
|
||||||
|
|
||||||
|
// UnsafeSeaweedMessagingAgentServer may be embedded to opt out of forward compatibility for this service.
|
||||||
|
// Use of this interface is not recommended, as added methods to SeaweedMessagingAgentServer will
|
||||||
|
// result in compilation errors.
|
||||||
|
type UnsafeSeaweedMessagingAgentServer interface {
|
||||||
|
mustEmbedUnimplementedSeaweedMessagingAgentServer()
|
||||||
|
}
|
||||||
|
|
||||||
|
func RegisterSeaweedMessagingAgentServer(s grpc.ServiceRegistrar, srv SeaweedMessagingAgentServer) {
|
||||||
|
// If the following call pancis, it indicates UnimplementedSeaweedMessagingAgentServer was
|
||||||
|
// embedded by pointer and is nil. This will cause panics if an
|
||||||
|
// unimplemented method is ever invoked, so we test this at initialization
|
||||||
|
// time to prevent it from happening at runtime later due to I/O.
|
||||||
|
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok {
|
||||||
|
t.testEmbeddedByValue()
|
||||||
|
}
|
||||||
|
s.RegisterService(&SeaweedMessagingAgent_ServiceDesc, srv)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_StartPublishSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(StartPublishSessionRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).StartPublishSession(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: SeaweedMessagingAgent_StartPublishSession_FullMethodName,
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).StartPublishSession(ctx, req.(*StartPublishSessionRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_ClosePublishSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(ClosePublishSessionRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).ClosePublishSession(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: SeaweedMessagingAgent_ClosePublishSession_FullMethodName,
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).ClosePublishSession(ctx, req.(*ClosePublishSessionRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_PublishRecord_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).PublishRecord(&grpc.GenericServerStream[PublishRecordRequest, PublishRecordResponse]{ServerStream: stream})
|
||||||
|
}
|
||||||
|
|
||||||
|
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||||
|
type SeaweedMessagingAgent_PublishRecordServer = grpc.BidiStreamingServer[PublishRecordRequest, PublishRecordResponse]
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_StartSubscribeSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(StartSubscribeSessionRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).StartSubscribeSession(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: SeaweedMessagingAgent_StartSubscribeSession_FullMethodName,
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).StartSubscribeSession(ctx, req.(*StartSubscribeSessionRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_CloseSubscribeSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(CloseSubscribeSessionRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).CloseSubscribeSession(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: SeaweedMessagingAgent_CloseSubscribeSession_FullMethodName,
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).CloseSubscribeSession(ctx, req.(*CloseSubscribeSessionRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _SeaweedMessagingAgent_SubscribeRecord_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||||
|
return srv.(SeaweedMessagingAgentServer).SubscribeRecord(&grpc.GenericServerStream[SubscribeRecordRequest, SubscribeRecordResponse]{ServerStream: stream})
|
||||||
|
}
|
||||||
|
|
||||||
|
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
||||||
|
type SeaweedMessagingAgent_SubscribeRecordServer = grpc.BidiStreamingServer[SubscribeRecordRequest, SubscribeRecordResponse]
|
||||||
|
|
||||||
|
// SeaweedMessagingAgent_ServiceDesc is the grpc.ServiceDesc for SeaweedMessagingAgent service.
|
||||||
|
// It's only intended for direct use with grpc.RegisterService,
|
||||||
|
// and not to be introspected or modified (even as a copy)
|
||||||
|
var SeaweedMessagingAgent_ServiceDesc = grpc.ServiceDesc{
|
||||||
|
ServiceName: "messaging_pb.SeaweedMessagingAgent",
|
||||||
|
HandlerType: (*SeaweedMessagingAgentServer)(nil),
|
||||||
|
Methods: []grpc.MethodDesc{
|
||||||
|
{
|
||||||
|
MethodName: "StartPublishSession",
|
||||||
|
Handler: _SeaweedMessagingAgent_StartPublishSession_Handler,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
MethodName: "ClosePublishSession",
|
||||||
|
Handler: _SeaweedMessagingAgent_ClosePublishSession_Handler,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
MethodName: "StartSubscribeSession",
|
||||||
|
Handler: _SeaweedMessagingAgent_StartSubscribeSession_Handler,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
MethodName: "CloseSubscribeSession",
|
||||||
|
Handler: _SeaweedMessagingAgent_CloseSubscribeSession_Handler,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
Streams: []grpc.StreamDesc{
|
||||||
|
{
|
||||||
|
StreamName: "PublishRecord",
|
||||||
|
Handler: _SeaweedMessagingAgent_PublishRecord_Handler,
|
||||||
|
ServerStreams: true,
|
||||||
|
ClientStreams: true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
StreamName: "SubscribeRecord",
|
||||||
|
Handler: _SeaweedMessagingAgent_SubscribeRecord_Handler,
|
||||||
|
ServerStreams: true,
|
||||||
|
ClientStreams: true,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
Metadata: "mq_agent.proto",
|
||||||
|
}
|
||||||
@@ -2,7 +2,7 @@ syntax = "proto3";
|
|||||||
|
|
||||||
package messaging_pb;
|
package messaging_pb;
|
||||||
|
|
||||||
import "schema.proto";
|
import "mq_schema.proto";
|
||||||
|
|
||||||
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb";
|
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb";
|
||||||
option java_package = "seaweedfs.mq";
|
option java_package = "seaweedfs.mq";
|
||||||
@@ -64,43 +64,14 @@ message FindBrokerLeaderResponse {
|
|||||||
string broker = 1;
|
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 {
|
message BrokerStats {
|
||||||
int32 cpu_usage_percent = 1;
|
int32 cpu_usage_percent = 1;
|
||||||
map<string, TopicPartitionStats> stats = 2;
|
map<string, TopicPartitionStats> stats = 2;
|
||||||
}
|
}
|
||||||
message TopicPartitionStats {
|
message TopicPartitionStats {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
Partition partition = 2;
|
schema_pb.Partition partition = 2;
|
||||||
int32 publisher_count = 3;
|
int32 publisher_count = 3;
|
||||||
int32 subscriber_count = 4;
|
int32 subscriber_count = 4;
|
||||||
string follower = 5;
|
string follower = 5;
|
||||||
@@ -126,7 +97,7 @@ message BalanceTopicsResponse {
|
|||||||
|
|
||||||
//////////////////////////////////////////////////
|
//////////////////////////////////////////////////
|
||||||
message ConfigureTopicRequest {
|
message ConfigureTopicRequest {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
int32 partition_count = 2;
|
int32 partition_count = 2;
|
||||||
schema_pb.RecordType record_type = 3;
|
schema_pb.RecordType record_type = 3;
|
||||||
}
|
}
|
||||||
@@ -137,23 +108,23 @@ message ConfigureTopicResponse {
|
|||||||
message ListTopicsRequest {
|
message ListTopicsRequest {
|
||||||
}
|
}
|
||||||
message ListTopicsResponse {
|
message ListTopicsResponse {
|
||||||
repeated Topic topics = 1;
|
repeated schema_pb.Topic topics = 1;
|
||||||
}
|
}
|
||||||
message LookupTopicBrokersRequest {
|
message LookupTopicBrokersRequest {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
}
|
}
|
||||||
message LookupTopicBrokersResponse {
|
message LookupTopicBrokersResponse {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
||||||
}
|
}
|
||||||
message BrokerPartitionAssignment {
|
message BrokerPartitionAssignment {
|
||||||
Partition partition = 1;
|
schema_pb.Partition partition = 1;
|
||||||
string leader_broker = 2;
|
string leader_broker = 2;
|
||||||
string follower_broker = 3;
|
string follower_broker = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message AssignTopicPartitionsRequest {
|
message AssignTopicPartitionsRequest {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
||||||
bool is_leader = 3;
|
bool is_leader = 3;
|
||||||
bool is_draining = 4;
|
bool is_draining = 4;
|
||||||
@@ -165,7 +136,7 @@ message SubscriberToSubCoordinatorRequest {
|
|||||||
message InitMessage {
|
message InitMessage {
|
||||||
string consumer_group = 1;
|
string consumer_group = 1;
|
||||||
string consumer_group_instance_id = 2;
|
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.
|
// 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,
|
// 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.
|
// the consumer group instance may be assigned partitions less than max_partition_count.
|
||||||
@@ -178,10 +149,10 @@ message SubscriberToSubCoordinatorRequest {
|
|||||||
int32 rebalance_seconds = 5;
|
int32 rebalance_seconds = 5;
|
||||||
}
|
}
|
||||||
message AckUnAssignmentMessage {
|
message AckUnAssignmentMessage {
|
||||||
Partition partition = 1;
|
schema_pb.Partition partition = 1;
|
||||||
}
|
}
|
||||||
message AckAssignmentMessage {
|
message AckAssignmentMessage {
|
||||||
Partition partition = 1;
|
schema_pb.Partition partition = 1;
|
||||||
}
|
}
|
||||||
oneof message {
|
oneof message {
|
||||||
InitMessage init = 1;
|
InitMessage init = 1;
|
||||||
@@ -194,7 +165,7 @@ message SubscriberToSubCoordinatorResponse {
|
|||||||
BrokerPartitionAssignment partition_assignment = 1;
|
BrokerPartitionAssignment partition_assignment = 1;
|
||||||
}
|
}
|
||||||
message UnAssignment {
|
message UnAssignment {
|
||||||
Partition partition = 1;
|
schema_pb.Partition partition = 1;
|
||||||
}
|
}
|
||||||
oneof message {
|
oneof message {
|
||||||
Assignment assignment = 1;
|
Assignment assignment = 1;
|
||||||
@@ -215,8 +186,8 @@ message DataMessage {
|
|||||||
}
|
}
|
||||||
message PublishMessageRequest {
|
message PublishMessageRequest {
|
||||||
message InitMessage {
|
message InitMessage {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
Partition partition = 2;
|
schema_pb.Partition partition = 2;
|
||||||
int32 ack_interval = 3;
|
int32 ack_interval = 3;
|
||||||
string follower_broker = 4;
|
string follower_broker = 4;
|
||||||
string publisher_name = 5; // for debugging
|
string publisher_name = 5; // for debugging
|
||||||
@@ -233,8 +204,8 @@ message PublishMessageResponse {
|
|||||||
}
|
}
|
||||||
message PublishFollowMeRequest {
|
message PublishFollowMeRequest {
|
||||||
message InitMessage {
|
message InitMessage {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
Partition partition = 2;
|
schema_pb.Partition partition = 2;
|
||||||
}
|
}
|
||||||
message FlushMessage {
|
message FlushMessage {
|
||||||
int64 ts_ns = 1;
|
int64 ts_ns = 1;
|
||||||
@@ -256,11 +227,11 @@ message SubscribeMessageRequest {
|
|||||||
string consumer_group = 1;
|
string consumer_group = 1;
|
||||||
string consumer_id = 2;
|
string consumer_id = 2;
|
||||||
string client_id = 3;
|
string client_id = 3;
|
||||||
Topic topic = 4;
|
schema_pb.Topic topic = 4;
|
||||||
PartitionOffset partition_offset = 5;
|
schema_pb.PartitionOffset partition_offset = 5;
|
||||||
string filter = 6;
|
string filter = 6;
|
||||||
string follower_broker = 7;
|
string follower_broker = 7;
|
||||||
int32 concurrency = 8;
|
int32 sliding_window_size = 8;
|
||||||
}
|
}
|
||||||
message AckMessage {
|
message AckMessage {
|
||||||
int64 sequence = 1;
|
int64 sequence = 1;
|
||||||
@@ -284,8 +255,8 @@ message SubscribeMessageResponse {
|
|||||||
}
|
}
|
||||||
message SubscribeFollowMeRequest {
|
message SubscribeFollowMeRequest {
|
||||||
message InitMessage {
|
message InitMessage {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
Partition partition = 2;
|
schema_pb.Partition partition = 2;
|
||||||
string consumer_group = 3;
|
string consumer_group = 3;
|
||||||
}
|
}
|
||||||
message AckMessage {
|
message AckMessage {
|
||||||
@@ -303,13 +274,13 @@ message SubscribeFollowMeResponse {
|
|||||||
int64 ack_ts_ns = 1;
|
int64 ack_ts_ns = 1;
|
||||||
}
|
}
|
||||||
message ClosePublishersRequest {
|
message ClosePublishersRequest {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
int64 unix_time_ns = 2;
|
int64 unix_time_ns = 2;
|
||||||
}
|
}
|
||||||
message ClosePublishersResponse {
|
message ClosePublishersResponse {
|
||||||
}
|
}
|
||||||
message CloseSubscribersRequest {
|
message CloseSubscribersRequest {
|
||||||
Topic topic = 1;
|
schema_pb.Topic topic = 1;
|
||||||
int64 unix_time_ns = 2;
|
int64 unix_time_ns = 2;
|
||||||
}
|
}
|
||||||
message CloseSubscribersResponse {
|
message CloseSubscribersResponse {
|
||||||
File diff suppressed because it is too large
Load Diff
@@ -1,8 +1,8 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: mq.proto
|
// source: mq_broker.proto
|
||||||
|
|
||||||
package mq_pb
|
package mq_pb
|
||||||
|
|
||||||
@@ -589,5 +589,5 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
|
|||||||
ClientStreams: true,
|
ClientStreams: true,
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
Metadata: "mq.proto",
|
Metadata: "mq_broker.proto",
|
||||||
}
|
}
|
||||||
@@ -4,6 +4,39 @@ package schema_pb;
|
|||||||
|
|
||||||
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/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
|
// schema definition
|
||||||
///////////////////////////
|
///////////////////////////
|
||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: remote.proto
|
// source: remote.proto
|
||||||
|
|
||||||
package remote_pb
|
package remote_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: s3.proto
|
// source: s3.proto
|
||||||
|
|
||||||
package s3_pb
|
package s3_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: s3.proto
|
// source: s3.proto
|
||||||
|
|
||||||
package s3_pb
|
package s3_pb
|
||||||
|
|||||||
@@ -1,8 +1,8 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: schema.proto
|
// source: mq_schema.proto
|
||||||
|
|
||||||
package schema_pb
|
package schema_pb
|
||||||
|
|
||||||
@@ -20,6 +20,55 @@ const (
|
|||||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
_ = 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
|
type ScalarType int32
|
||||||
|
|
||||||
const (
|
const (
|
||||||
@@ -65,11 +114,11 @@ func (x ScalarType) String() string {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (ScalarType) Descriptor() protoreflect.EnumDescriptor {
|
func (ScalarType) Descriptor() protoreflect.EnumDescriptor {
|
||||||
return file_schema_proto_enumTypes[0].Descriptor()
|
return file_mq_schema_proto_enumTypes[1].Descriptor()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ScalarType) Type() protoreflect.EnumType {
|
func (ScalarType) Type() protoreflect.EnumType {
|
||||||
return &file_schema_proto_enumTypes[0]
|
return &file_mq_schema_proto_enumTypes[1]
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x ScalarType) Number() protoreflect.EnumNumber {
|
func (x ScalarType) Number() protoreflect.EnumNumber {
|
||||||
@@ -78,7 +127,259 @@ func (x ScalarType) Number() protoreflect.EnumNumber {
|
|||||||
|
|
||||||
// Deprecated: Use ScalarType.Descriptor instead.
|
// Deprecated: Use ScalarType.Descriptor instead.
|
||||||
func (ScalarType) EnumDescriptor() ([]byte, []int) {
|
func (ScalarType) EnumDescriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{0}
|
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 {
|
type RecordType struct {
|
||||||
@@ -92,7 +393,7 @@ type RecordType struct {
|
|||||||
func (x *RecordType) Reset() {
|
func (x *RecordType) Reset() {
|
||||||
*x = RecordType{}
|
*x = RecordType{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[0]
|
mi := &file_mq_schema_proto_msgTypes[4]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -105,7 +406,7 @@ func (x *RecordType) String() string {
|
|||||||
func (*RecordType) ProtoMessage() {}
|
func (*RecordType) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *RecordType) ProtoReflect() protoreflect.Message {
|
func (x *RecordType) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[0]
|
mi := &file_mq_schema_proto_msgTypes[4]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -118,7 +419,7 @@ func (x *RecordType) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use RecordType.ProtoReflect.Descriptor instead.
|
// Deprecated: Use RecordType.ProtoReflect.Descriptor instead.
|
||||||
func (*RecordType) Descriptor() ([]byte, []int) {
|
func (*RecordType) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{0}
|
return file_mq_schema_proto_rawDescGZIP(), []int{4}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *RecordType) GetFields() []*Field {
|
func (x *RecordType) GetFields() []*Field {
|
||||||
@@ -143,7 +444,7 @@ type Field struct {
|
|||||||
func (x *Field) Reset() {
|
func (x *Field) Reset() {
|
||||||
*x = Field{}
|
*x = Field{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[1]
|
mi := &file_mq_schema_proto_msgTypes[5]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -156,7 +457,7 @@ func (x *Field) String() string {
|
|||||||
func (*Field) ProtoMessage() {}
|
func (*Field) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *Field) ProtoReflect() protoreflect.Message {
|
func (x *Field) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[1]
|
mi := &file_mq_schema_proto_msgTypes[5]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -169,7 +470,7 @@ func (x *Field) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use Field.ProtoReflect.Descriptor instead.
|
// Deprecated: Use Field.ProtoReflect.Descriptor instead.
|
||||||
func (*Field) Descriptor() ([]byte, []int) {
|
func (*Field) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{1}
|
return file_mq_schema_proto_rawDescGZIP(), []int{5}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *Field) GetName() string {
|
func (x *Field) GetName() string {
|
||||||
@@ -223,7 +524,7 @@ type Type struct {
|
|||||||
func (x *Type) Reset() {
|
func (x *Type) Reset() {
|
||||||
*x = Type{}
|
*x = Type{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[2]
|
mi := &file_mq_schema_proto_msgTypes[6]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -236,7 +537,7 @@ func (x *Type) String() string {
|
|||||||
func (*Type) ProtoMessage() {}
|
func (*Type) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *Type) ProtoReflect() protoreflect.Message {
|
func (x *Type) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[2]
|
mi := &file_mq_schema_proto_msgTypes[6]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -249,7 +550,7 @@ func (x *Type) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use Type.ProtoReflect.Descriptor instead.
|
// Deprecated: Use Type.ProtoReflect.Descriptor instead.
|
||||||
func (*Type) Descriptor() ([]byte, []int) {
|
func (*Type) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{2}
|
return file_mq_schema_proto_rawDescGZIP(), []int{6}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Type) GetKind() isType_Kind {
|
func (m *Type) GetKind() isType_Kind {
|
||||||
@@ -313,7 +614,7 @@ type ListType struct {
|
|||||||
func (x *ListType) Reset() {
|
func (x *ListType) Reset() {
|
||||||
*x = ListType{}
|
*x = ListType{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[3]
|
mi := &file_mq_schema_proto_msgTypes[7]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -326,7 +627,7 @@ func (x *ListType) String() string {
|
|||||||
func (*ListType) ProtoMessage() {}
|
func (*ListType) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *ListType) ProtoReflect() protoreflect.Message {
|
func (x *ListType) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[3]
|
mi := &file_mq_schema_proto_msgTypes[7]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -339,7 +640,7 @@ func (x *ListType) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use ListType.ProtoReflect.Descriptor instead.
|
// Deprecated: Use ListType.ProtoReflect.Descriptor instead.
|
||||||
func (*ListType) Descriptor() ([]byte, []int) {
|
func (*ListType) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{3}
|
return file_mq_schema_proto_rawDescGZIP(), []int{7}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *ListType) GetElementType() *Type {
|
func (x *ListType) GetElementType() *Type {
|
||||||
@@ -363,7 +664,7 @@ type RecordValue struct {
|
|||||||
func (x *RecordValue) Reset() {
|
func (x *RecordValue) Reset() {
|
||||||
*x = RecordValue{}
|
*x = RecordValue{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[4]
|
mi := &file_mq_schema_proto_msgTypes[8]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -376,7 +677,7 @@ func (x *RecordValue) String() string {
|
|||||||
func (*RecordValue) ProtoMessage() {}
|
func (*RecordValue) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *RecordValue) ProtoReflect() protoreflect.Message {
|
func (x *RecordValue) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[4]
|
mi := &file_mq_schema_proto_msgTypes[8]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -389,7 +690,7 @@ func (x *RecordValue) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use RecordValue.ProtoReflect.Descriptor instead.
|
// Deprecated: Use RecordValue.ProtoReflect.Descriptor instead.
|
||||||
func (*RecordValue) Descriptor() ([]byte, []int) {
|
func (*RecordValue) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{4}
|
return file_mq_schema_proto_rawDescGZIP(), []int{8}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *RecordValue) GetFields() map[string]*Value {
|
func (x *RecordValue) GetFields() map[string]*Value {
|
||||||
@@ -421,7 +722,7 @@ type Value struct {
|
|||||||
func (x *Value) Reset() {
|
func (x *Value) Reset() {
|
||||||
*x = Value{}
|
*x = Value{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[5]
|
mi := &file_mq_schema_proto_msgTypes[9]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -434,7 +735,7 @@ func (x *Value) String() string {
|
|||||||
func (*Value) ProtoMessage() {}
|
func (*Value) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *Value) ProtoReflect() protoreflect.Message {
|
func (x *Value) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[5]
|
mi := &file_mq_schema_proto_msgTypes[9]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -447,7 +748,7 @@ func (x *Value) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use Value.ProtoReflect.Descriptor instead.
|
// Deprecated: Use Value.ProtoReflect.Descriptor instead.
|
||||||
func (*Value) Descriptor() ([]byte, []int) {
|
func (*Value) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{5}
|
return file_mq_schema_proto_rawDescGZIP(), []int{9}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Value) GetKind() isValue_Kind {
|
func (m *Value) GetKind() isValue_Kind {
|
||||||
@@ -589,7 +890,7 @@ type ListValue struct {
|
|||||||
func (x *ListValue) Reset() {
|
func (x *ListValue) Reset() {
|
||||||
*x = ListValue{}
|
*x = ListValue{}
|
||||||
if protoimpl.UnsafeEnabled {
|
if protoimpl.UnsafeEnabled {
|
||||||
mi := &file_schema_proto_msgTypes[6]
|
mi := &file_mq_schema_proto_msgTypes[10]
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
ms.StoreMessageInfo(mi)
|
ms.StoreMessageInfo(mi)
|
||||||
}
|
}
|
||||||
@@ -602,7 +903,7 @@ func (x *ListValue) String() string {
|
|||||||
func (*ListValue) ProtoMessage() {}
|
func (*ListValue) ProtoMessage() {}
|
||||||
|
|
||||||
func (x *ListValue) ProtoReflect() protoreflect.Message {
|
func (x *ListValue) ProtoReflect() protoreflect.Message {
|
||||||
mi := &file_schema_proto_msgTypes[6]
|
mi := &file_mq_schema_proto_msgTypes[10]
|
||||||
if protoimpl.UnsafeEnabled && x != nil {
|
if protoimpl.UnsafeEnabled && x != nil {
|
||||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||||
if ms.LoadMessageInfo() == nil {
|
if ms.LoadMessageInfo() == nil {
|
||||||
@@ -615,7 +916,7 @@ func (x *ListValue) ProtoReflect() protoreflect.Message {
|
|||||||
|
|
||||||
// Deprecated: Use ListValue.ProtoReflect.Descriptor instead.
|
// Deprecated: Use ListValue.ProtoReflect.Descriptor instead.
|
||||||
func (*ListValue) Descriptor() ([]byte, []int) {
|
func (*ListValue) Descriptor() ([]byte, []int) {
|
||||||
return file_schema_proto_rawDescGZIP(), []int{6}
|
return file_mq_schema_proto_rawDescGZIP(), []int{10}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *ListValue) GetValues() []*Value {
|
func (x *ListValue) GetValues() []*Value {
|
||||||
@@ -625,11 +926,44 @@ func (x *ListValue) GetValues() []*Value {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
var File_schema_proto protoreflect.FileDescriptor
|
var File_mq_schema_proto protoreflect.FileDescriptor
|
||||||
|
|
||||||
var file_schema_proto_rawDesc = []byte{
|
var file_mq_schema_proto_rawDesc = []byte{
|
||||||
0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09,
|
0x0a, 0x0f, 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||||
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x36, 0x0a, 0x0a, 0x52, 0x65, 0x63,
|
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,
|
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,
|
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,
|
0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
|
||||||
@@ -695,70 +1029,132 @@ var file_schema_proto_rawDesc = []byte{
|
|||||||
0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x22, 0x35, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61,
|
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,
|
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,
|
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,
|
0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x2a, 0x4c, 0x0a,
|
||||||
0x0a, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42,
|
0x18, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74,
|
||||||
0x4f, 0x4f, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01,
|
0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52,
|
||||||
0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46,
|
0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49,
|
||||||
0x4c, 0x4f, 0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45,
|
0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12,
|
||||||
0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x0a, 0x0a,
|
0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, 0x02, 0x2a, 0x5a, 0x0a, 0x0a, 0x53,
|
||||||
0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74,
|
0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42, 0x4f, 0x4f,
|
||||||
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66,
|
0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01, 0x12, 0x09,
|
||||||
0x73, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64,
|
0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f,
|
||||||
0x2f, 0x70, 0x62, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70,
|
0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x05,
|
||||||
0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
var (
|
||||||
file_schema_proto_rawDescOnce sync.Once
|
file_mq_schema_proto_rawDescOnce sync.Once
|
||||||
file_schema_proto_rawDescData = file_schema_proto_rawDesc
|
file_mq_schema_proto_rawDescData = file_mq_schema_proto_rawDesc
|
||||||
)
|
)
|
||||||
|
|
||||||
func file_schema_proto_rawDescGZIP() []byte {
|
func file_mq_schema_proto_rawDescGZIP() []byte {
|
||||||
file_schema_proto_rawDescOnce.Do(func() {
|
file_mq_schema_proto_rawDescOnce.Do(func() {
|
||||||
file_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_schema_proto_rawDescData)
|
file_mq_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_mq_schema_proto_rawDescData)
|
||||||
})
|
})
|
||||||
return file_schema_proto_rawDescData
|
return file_mq_schema_proto_rawDescData
|
||||||
}
|
}
|
||||||
|
|
||||||
var file_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
|
var file_mq_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 2)
|
||||||
var file_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 8)
|
var file_mq_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 12)
|
||||||
var file_schema_proto_goTypes = []any{
|
var file_mq_schema_proto_goTypes = []any{
|
||||||
(ScalarType)(0), // 0: schema_pb.ScalarType
|
(PartitionOffsetStartType)(0), // 0: schema_pb.PartitionOffsetStartType
|
||||||
(*RecordType)(nil), // 1: schema_pb.RecordType
|
(ScalarType)(0), // 1: schema_pb.ScalarType
|
||||||
(*Field)(nil), // 2: schema_pb.Field
|
(*Topic)(nil), // 2: schema_pb.Topic
|
||||||
(*Type)(nil), // 3: schema_pb.Type
|
(*Partition)(nil), // 3: schema_pb.Partition
|
||||||
(*ListType)(nil), // 4: schema_pb.ListType
|
(*Offset)(nil), // 4: schema_pb.Offset
|
||||||
(*RecordValue)(nil), // 5: schema_pb.RecordValue
|
(*PartitionOffset)(nil), // 5: schema_pb.PartitionOffset
|
||||||
(*Value)(nil), // 6: schema_pb.Value
|
(*RecordType)(nil), // 6: schema_pb.RecordType
|
||||||
(*ListValue)(nil), // 7: schema_pb.ListValue
|
(*Field)(nil), // 7: schema_pb.Field
|
||||||
nil, // 8: schema_pb.RecordValue.FieldsEntry
|
(*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_schema_proto_depIdxs = []int32{
|
var file_mq_schema_proto_depIdxs = []int32{
|
||||||
2, // 0: schema_pb.RecordType.fields:type_name -> schema_pb.Field
|
2, // 0: schema_pb.Offset.topic:type_name -> schema_pb.Topic
|
||||||
3, // 1: schema_pb.Field.type:type_name -> schema_pb.Type
|
5, // 1: schema_pb.Offset.partition_offsets:type_name -> schema_pb.PartitionOffset
|
||||||
0, // 2: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType
|
3, // 2: schema_pb.PartitionOffset.partition:type_name -> schema_pb.Partition
|
||||||
1, // 3: schema_pb.Type.record_type:type_name -> schema_pb.RecordType
|
0, // 3: schema_pb.PartitionOffset.start_type:type_name -> schema_pb.PartitionOffsetStartType
|
||||||
4, // 4: schema_pb.Type.list_type:type_name -> schema_pb.ListType
|
7, // 4: schema_pb.RecordType.fields:type_name -> schema_pb.Field
|
||||||
3, // 5: schema_pb.ListType.element_type:type_name -> schema_pb.Type
|
8, // 5: schema_pb.Field.type:type_name -> schema_pb.Type
|
||||||
8, // 6: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry
|
1, // 6: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType
|
||||||
7, // 7: schema_pb.Value.list_value:type_name -> schema_pb.ListValue
|
6, // 7: schema_pb.Type.record_type:type_name -> schema_pb.RecordType
|
||||||
5, // 8: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue
|
9, // 8: schema_pb.Type.list_type:type_name -> schema_pb.ListType
|
||||||
6, // 9: schema_pb.ListValue.values:type_name -> schema_pb.Value
|
8, // 9: schema_pb.ListType.element_type:type_name -> schema_pb.Type
|
||||||
6, // 10: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value
|
13, // 10: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry
|
||||||
11, // [11:11] is the sub-list for method output_type
|
12, // 11: schema_pb.Value.list_value:type_name -> schema_pb.ListValue
|
||||||
11, // [11:11] is the sub-list for method input_type
|
10, // 12: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue
|
||||||
11, // [11:11] is the sub-list for extension type_name
|
11, // 13: schema_pb.ListValue.values:type_name -> schema_pb.Value
|
||||||
11, // [11:11] is the sub-list for extension extendee
|
11, // 14: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value
|
||||||
0, // [0:11] is the sub-list for field type_name
|
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_schema_proto_init() }
|
func init() { file_mq_schema_proto_init() }
|
||||||
func file_schema_proto_init() {
|
func file_mq_schema_proto_init() {
|
||||||
if File_schema_proto != nil {
|
if File_mq_schema_proto != nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if !protoimpl.UnsafeEnabled {
|
if !protoimpl.UnsafeEnabled {
|
||||||
file_schema_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||||
|
switch v := v.(*Topic); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_mq_schema_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||||
|
switch v := v.(*Partition); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_mq_schema_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||||
|
switch v := v.(*Offset); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_mq_schema_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
||||||
|
switch v := v.(*PartitionOffset); i {
|
||||||
|
case 0:
|
||||||
|
return &v.state
|
||||||
|
case 1:
|
||||||
|
return &v.sizeCache
|
||||||
|
case 2:
|
||||||
|
return &v.unknownFields
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
file_mq_schema_proto_msgTypes[4].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*RecordType); i {
|
switch v := v.(*RecordType); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -770,7 +1166,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[5].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*Field); i {
|
switch v := v.(*Field); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -782,7 +1178,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*Type); i {
|
switch v := v.(*Type); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -794,7 +1190,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[3].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[7].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*ListType); i {
|
switch v := v.(*ListType); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -806,7 +1202,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[4].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[8].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*RecordValue); i {
|
switch v := v.(*RecordValue); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -818,7 +1214,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[5].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[9].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*Value); i {
|
switch v := v.(*Value); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -830,7 +1226,7 @@ func file_schema_proto_init() {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[6].Exporter = func(v any, i int) any {
|
file_mq_schema_proto_msgTypes[10].Exporter = func(v any, i int) any {
|
||||||
switch v := v.(*ListValue); i {
|
switch v := v.(*ListValue); i {
|
||||||
case 0:
|
case 0:
|
||||||
return &v.state
|
return &v.state
|
||||||
@@ -843,12 +1239,12 @@ func file_schema_proto_init() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[2].OneofWrappers = []any{
|
file_mq_schema_proto_msgTypes[6].OneofWrappers = []any{
|
||||||
(*Type_ScalarType)(nil),
|
(*Type_ScalarType)(nil),
|
||||||
(*Type_RecordType)(nil),
|
(*Type_RecordType)(nil),
|
||||||
(*Type_ListType)(nil),
|
(*Type_ListType)(nil),
|
||||||
}
|
}
|
||||||
file_schema_proto_msgTypes[5].OneofWrappers = []any{
|
file_mq_schema_proto_msgTypes[9].OneofWrappers = []any{
|
||||||
(*Value_BoolValue)(nil),
|
(*Value_BoolValue)(nil),
|
||||||
(*Value_Int32Value)(nil),
|
(*Value_Int32Value)(nil),
|
||||||
(*Value_Int64Value)(nil),
|
(*Value_Int64Value)(nil),
|
||||||
@@ -863,19 +1259,19 @@ func file_schema_proto_init() {
|
|||||||
out := protoimpl.TypeBuilder{
|
out := protoimpl.TypeBuilder{
|
||||||
File: protoimpl.DescBuilder{
|
File: protoimpl.DescBuilder{
|
||||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||||
RawDescriptor: file_schema_proto_rawDesc,
|
RawDescriptor: file_mq_schema_proto_rawDesc,
|
||||||
NumEnums: 1,
|
NumEnums: 2,
|
||||||
NumMessages: 8,
|
NumMessages: 12,
|
||||||
NumExtensions: 0,
|
NumExtensions: 0,
|
||||||
NumServices: 0,
|
NumServices: 0,
|
||||||
},
|
},
|
||||||
GoTypes: file_schema_proto_goTypes,
|
GoTypes: file_mq_schema_proto_goTypes,
|
||||||
DependencyIndexes: file_schema_proto_depIdxs,
|
DependencyIndexes: file_mq_schema_proto_depIdxs,
|
||||||
EnumInfos: file_schema_proto_enumTypes,
|
EnumInfos: file_mq_schema_proto_enumTypes,
|
||||||
MessageInfos: file_schema_proto_msgTypes,
|
MessageInfos: file_mq_schema_proto_msgTypes,
|
||||||
}.Build()
|
}.Build()
|
||||||
File_schema_proto = out.File
|
File_mq_schema_proto = out.File
|
||||||
file_schema_proto_rawDesc = nil
|
file_mq_schema_proto_rawDesc = nil
|
||||||
file_schema_proto_goTypes = nil
|
file_mq_schema_proto_goTypes = nil
|
||||||
file_schema_proto_depIdxs = nil
|
file_mq_schema_proto_depIdxs = nil
|
||||||
}
|
}
|
||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// protoc-gen-go v1.34.2
|
// protoc-gen-go v1.34.2
|
||||||
// protoc v5.28.1
|
// protoc v5.28.3
|
||||||
// source: volume_server.proto
|
// source: volume_server.proto
|
||||||
|
|
||||||
package volume_server_pb
|
package volume_server_pb
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||||
// versions:
|
// versions:
|
||||||
// - protoc-gen-go-grpc v1.5.1
|
// - protoc-gen-go-grpc v1.5.1
|
||||||
// - protoc v5.28.1
|
// - protoc v5.28.3
|
||||||
// source: volume_server.proto
|
// source: volume_server.proto
|
||||||
|
|
||||||
package volume_server_pb
|
package volume_server_pb
|
||||||
|
|||||||
@@ -7,6 +7,7 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"io"
|
"io"
|
||||||
)
|
)
|
||||||
|
|
||||||
@@ -54,7 +55,7 @@ func (c *commandMqTopicConfigure) Do(args []string, commandEnv *CommandEnv, writ
|
|||||||
// create topic
|
// create topic
|
||||||
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||||
resp, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
|
resp, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
|
||||||
Topic: &mq_pb.Topic{
|
Topic: &schema_pb.Topic{
|
||||||
Namespace: *namespace,
|
Namespace: *namespace,
|
||||||
Name: *topicName,
|
Name: *topicName,
|
||||||
},
|
},
|
||||||
|
|||||||
@@ -6,6 +6,7 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||||
"io"
|
"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 {
|
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||||
resp, err := client.LookupTopicBrokers(context.Background(), &mq_pb.LookupTopicBrokersRequest{
|
resp, err := client.LookupTopicBrokers(context.Background(), &mq_pb.LookupTopicBrokersRequest{
|
||||||
Topic: &mq_pb.Topic{
|
Topic: &schema_pb.Topic{
|
||||||
Namespace: *namespace,
|
Namespace: *namespace,
|
||||||
Name: *topicName,
|
Name: *topicName,
|
||||||
},
|
},
|
||||||
|
|||||||
Reference in New Issue
Block a user