mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-08-20 09:53:01 +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:
parent
b2f56d9add
commit
cc05874d06
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 = ""
|
||||
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
|
||||
# i.e., "weed backup|benchmark|filer.copy|filer.replicate|mount|s3|upload"
|
||||
[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/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"sync"
|
||||
)
|
||||
|
||||
@ -55,7 +56,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
|
||||
|
||||
// called by broker leader to drain existing partitions.
|
||||
// new/updated partitions will be detected by broker from the filer
|
||||
func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error {
|
||||
func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error {
|
||||
// notify the brokers to create the topic partitions in parallel
|
||||
var wg sync.WaitGroup
|
||||
for _, bpa := range assignments {
|
||||
|
||||
@ -5,7 +5,6 @@ import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"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/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
|
||||
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)
|
||||
|
||||
@ -7,6 +7,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
// LookupTopicBrokers returns the brokers that are serving the topic
|
||||
@ -54,7 +55,7 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List
|
||||
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
||||
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
|
||||
topicPartitionStat := topicPartitionStatsItem.Val
|
||||
topic := &mq_pb.Topic{
|
||||
topic := &schema_pb.Topic{
|
||||
Namespace: topicPartitionStat.TopicPartition.Namespace,
|
||||
Name: topicPartitionStat.TopicPartition.Name,
|
||||
}
|
||||
|
||||
@ -9,6 +9,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||
"io"
|
||||
"time"
|
||||
@ -54,7 +55,7 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
||||
}()
|
||||
|
||||
startPosition := b.getRequestPosition(req.GetInit())
|
||||
imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().Concurrency))
|
||||
imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().SlidingWindowSize))
|
||||
|
||||
// connect to the follower
|
||||
var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
|
||||
@ -212,9 +213,9 @@ func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMess
|
||||
return
|
||||
}
|
||||
|
||||
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
|
||||
if offset.StartType == schema_pb.PartitionOffsetStartType_EARLIEST {
|
||||
startPosition = log_buffer.NewMessagePosition(1, -3)
|
||||
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
|
||||
} else if offset.StartType == schema_pb.PartitionOffsetStartType_LATEST {
|
||||
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
|
||||
}
|
||||
return
|
||||
|
||||
63
weed/mq/client/agent_client/SubscribeSession.go
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/mq/client/sub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -33,23 +33,24 @@ func main() {
|
||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
MaxPartitionCount: int32(*maxPartitionCount),
|
||||
PerPartitionConcurrency: int32(*perPartitionConcurrency),
|
||||
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||
}
|
||||
|
||||
contentConfig := &sub_client.ContentConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
StartTime: time.Unix(1, 1),
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
}
|
||||
|
||||
brokers := strings.Split(*seedBrokers, ",")
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
|
||||
|
||||
counter := 0
|
||||
subscriber.SetEachMessageFunc(func(key, value []byte) error {
|
||||
counter++
|
||||
println(string(key), "=>", string(value), counter)
|
||||
return nil
|
||||
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
counter++
|
||||
println(string(m.Data.Key), "=>", string(m.Data.Value), counter)
|
||||
})
|
||||
})
|
||||
|
||||
subscriber.SetCompletionFunc(func() {
|
||||
|
||||
@ -6,6 +6,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
|
||||
@ -60,30 +61,31 @@ func main() {
|
||||
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
|
||||
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
MaxPartitionCount: int32(*maxPartitionCount),
|
||||
PerPartitionConcurrency: int32(*perPartitionConcurrency),
|
||||
SlidingWindowSize: int32(*perPartitionConcurrency),
|
||||
}
|
||||
|
||||
contentConfig := &sub_client.ContentConfiguration{
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
StartTime: time.Now().Add(-*timeAgo),
|
||||
Topic: topic.NewTopic(*namespace, *t),
|
||||
Filter: "",
|
||||
// StartTime: time.Now().Add(-*timeAgo),
|
||||
}
|
||||
|
||||
brokers := strings.Split(*seedBrokers, ",")
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
|
||||
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, make(chan sub_client.KeyedOffset, 1024))
|
||||
|
||||
counter := 0
|
||||
subscriber.SetEachMessageFunc(func(key, value []byte) error {
|
||||
counter++
|
||||
record := &schema_pb.RecordValue{}
|
||||
err := proto.Unmarshal(value, record)
|
||||
if err != nil {
|
||||
fmt.Printf("unmarshal record value: %v\n", err)
|
||||
} else {
|
||||
fmt.Printf("%s %d: %v\n", string(key), len(value), record)
|
||||
}
|
||||
//time.Sleep(1300 * time.Millisecond)
|
||||
return nil
|
||||
executors := util.NewLimitedConcurrentExecutor(int(subscriberConfig.SlidingWindowSize))
|
||||
subscriber.SetOnDataMessageFn(func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
counter++
|
||||
record := &schema_pb.RecordValue{}
|
||||
err := proto.Unmarshal(m.Data.Value, record)
|
||||
if err != nil {
|
||||
fmt.Printf("unmarshal record value: %v\n", err)
|
||||
} else {
|
||||
fmt.Printf("%s %d: %v\n", string(m.Data.Key), len(m.Data.Value), record)
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
subscriber.SetCompletionFunc(func() {
|
||||
|
||||
@ -6,12 +6,18 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"io"
|
||||
"reflect"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}) error {
|
||||
type KeyedOffset struct {
|
||||
Key []byte
|
||||
Offset int64
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}, onDataMessageFn OnDataMessageFn) error {
|
||||
// connect to the partition broker
|
||||
return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
|
||||
@ -20,31 +26,30 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
return fmt.Errorf("create subscribe client: %v", err)
|
||||
}
|
||||
|
||||
perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency
|
||||
if perPartitionConcurrency <= 0 {
|
||||
perPartitionConcurrency = 1
|
||||
slidingWindowSize := sub.SubscriberConfig.SlidingWindowSize
|
||||
if slidingWindowSize <= 0 {
|
||||
slidingWindowSize = 1
|
||||
}
|
||||
|
||||
var stopTsNs int64
|
||||
if !sub.ContentConfig.StopTime.IsZero() {
|
||||
stopTsNs = sub.ContentConfig.StopTime.UnixNano()
|
||||
po := findPartitionOffset(sub.ContentConfig.PartitionOffsets, assigned.Partition)
|
||||
if po == nil {
|
||||
po = &schema_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: time.Now().UnixNano(),
|
||||
StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
}
|
||||
}
|
||||
|
||||
if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{
|
||||
Message: &mq_pb.SubscribeMessageRequest_Init{
|
||||
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
|
||||
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: &mq_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: sub.ContentConfig.StartTime.UnixNano(),
|
||||
StopTsNs: stopTsNs,
|
||||
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
},
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
Concurrency: perPartitionConcurrency,
|
||||
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: po,
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
SlidingWindowSize: slidingWindowSize,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
@ -57,24 +62,13 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
defer sub.OnCompletionFunc()
|
||||
}
|
||||
|
||||
type KeyedOffset struct {
|
||||
Key []byte
|
||||
Offset int64
|
||||
}
|
||||
|
||||
partitionOffsetChan := make(chan KeyedOffset, 1024)
|
||||
defer func() {
|
||||
close(partitionOffsetChan)
|
||||
}()
|
||||
executors := util.NewLimitedConcurrentExecutor(int(perPartitionConcurrency))
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stopCh:
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
case ack, ok := <-partitionOffsetChan:
|
||||
case ack, ok := <-sub.PartitionOffsetChan:
|
||||
if !ok {
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
@ -91,9 +85,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
}
|
||||
}()
|
||||
|
||||
var lastErr error
|
||||
|
||||
for lastErr == nil {
|
||||
for {
|
||||
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||
resp, err := subscribeClient.Recv()
|
||||
if err != nil {
|
||||
@ -113,17 +105,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
|
||||
continue
|
||||
}
|
||||
executors.Execute(func() {
|
||||
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
|
||||
if processErr == nil {
|
||||
partitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
} else {
|
||||
lastErr = processErr
|
||||
}
|
||||
})
|
||||
onDataMessageFn(m)
|
||||
case *mq_pb.SubscribeMessageResponse_Ctrl:
|
||||
// glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
|
||||
if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
|
||||
@ -132,6 +114,14 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
}
|
||||
}
|
||||
|
||||
return lastErr
|
||||
})
|
||||
}
|
||||
|
||||
func findPartitionOffset(partitionOffsets []*schema_pb.PartitionOffset, partition *schema_pb.Partition) *schema_pb.PartitionOffset {
|
||||
for _, po := range partitionOffsets {
|
||||
if po.Partition == partition {
|
||||
return po
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -4,6 +4,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
@ -20,6 +21,7 @@ func (sub *TopicSubscriber) Subscribe() error {
|
||||
go sub.startProcessors()
|
||||
|
||||
// loop forever
|
||||
// TODO shutdown the subscriber when not needed anymore
|
||||
sub.doKeepConnectedToSubCoordinator()
|
||||
|
||||
return nil
|
||||
@ -66,7 +68,21 @@ func (sub *TopicSubscriber) startProcessors() {
|
||||
},
|
||||
},
|
||||
}
|
||||
err := sub.onEachPartition(assigned, stopChan)
|
||||
|
||||
executors := util.NewLimitedConcurrentExecutor(int(sub.SubscriberConfig.SlidingWindowSize))
|
||||
onDataMessageFn := func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
|
||||
if processErr == nil {
|
||||
sub.PartitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
err := sub.onEachPartition(assigned, stopChan, onDataMessageFn)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
|
||||
} else {
|
||||
|
||||
@ -3,9 +3,9 @@ package sub_client
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
type SubscriberConfiguration struct {
|
||||
@ -14,16 +14,16 @@ type SubscriberConfiguration struct {
|
||||
ConsumerGroupInstanceId string
|
||||
GrpcDialOption grpc.DialOption
|
||||
MaxPartitionCount int32 // how many partitions to process concurrently
|
||||
PerPartitionConcurrency int32 // how many messages to process concurrently per partition
|
||||
SlidingWindowSize int32 // how many messages to process concurrently per partition
|
||||
}
|
||||
|
||||
type ContentConfiguration struct {
|
||||
Topic topic.Topic
|
||||
Filter string
|
||||
StartTime time.Time
|
||||
StopTime time.Time
|
||||
Topic topic.Topic
|
||||
Filter string
|
||||
PartitionOffsets []*schema_pb.PartitionOffset
|
||||
}
|
||||
|
||||
type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data)
|
||||
type OnEachMessageFunc func(key, value []byte) (err error)
|
||||
type OnCompletionFunc func()
|
||||
|
||||
@ -32,15 +32,17 @@ type TopicSubscriber struct {
|
||||
ContentConfig *ContentConfiguration
|
||||
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
|
||||
OnDataMessageFnnc OnDataMessageFn
|
||||
OnEachMessageFunc OnEachMessageFunc
|
||||
OnCompletionFunc OnCompletionFunc
|
||||
bootstrapBrokers []string
|
||||
waitForMoreMessage bool
|
||||
activeProcessors map[topic.Partition]*ProcessorState
|
||||
activeProcessorsLock sync.Mutex
|
||||
PartitionOffsetChan chan KeyedOffset
|
||||
}
|
||||
|
||||
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
|
||||
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber {
|
||||
return &TopicSubscriber{
|
||||
SubscriberConfig: subscriber,
|
||||
ContentConfig: content,
|
||||
@ -49,6 +51,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
|
||||
bootstrapBrokers: bootstrapBrokers,
|
||||
waitForMoreMessage: true,
|
||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||
PartitionOffsetChan: partitionOffsetChan,
|
||||
}
|
||||
}
|
||||
|
||||
@ -56,6 +59,10 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc
|
||||
sub.OnEachMessageFunc = onEachMessageFn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) {
|
||||
sub.OnDataMessageFnnc = fn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
|
||||
sub.OnCompletionFunc = onCompletionFn
|
||||
}
|
||||
|
||||
@ -4,6 +4,7 @@ import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"math/rand"
|
||||
"time"
|
||||
)
|
||||
@ -14,7 +15,7 @@ func AllocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p
|
||||
rangeSize := MaxPartitionCount / partitionCount
|
||||
for i := int32(0); i < partitionCount; i++ {
|
||||
assignment := &mq_pb.BrokerPartitionAssignment{
|
||||
Partition: &mq_pb.Partition{
|
||||
Partition: &schema_pb.Partition{
|
||||
RingSize: MaxPartitionCount,
|
||||
RangeStart: int32(i * rangeSize),
|
||||
RangeStop: int32((i + 1) * rangeSize),
|
||||
|
||||
@ -4,6 +4,7 @@ import (
|
||||
"fmt"
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"testing"
|
||||
)
|
||||
@ -29,7 +30,7 @@ func Test_allocateOneBroker(t *testing.T) {
|
||||
wantAssignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:17777",
|
||||
Partition: &mq_pb.Partition{
|
||||
Partition: &schema_pb.Partition{
|
||||
RingSize: MaxPartitionCount,
|
||||
RangeStart: 0,
|
||||
RangeStop: MaxPartitionCount,
|
||||
@ -96,7 +97,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "localhost:2",
|
||||
},
|
||||
},
|
||||
@ -111,7 +112,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:1",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "",
|
||||
},
|
||||
},
|
||||
@ -126,7 +127,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:1",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "localhost:200",
|
||||
},
|
||||
},
|
||||
@ -141,7 +142,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:100",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "localhost:200",
|
||||
},
|
||||
},
|
||||
@ -156,7 +157,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:1",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "localhost:2",
|
||||
},
|
||||
},
|
||||
@ -171,7 +172,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:1",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
},
|
||||
},
|
||||
},
|
||||
@ -185,7 +186,7 @@ func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
|
||||
assignments: []*mq_pb.BrokerPartitionAssignment{
|
||||
{
|
||||
LeaderBroker: "localhost:1",
|
||||
Partition: &mq_pb.Partition{},
|
||||
Partition: &schema_pb.Partition{},
|
||||
FollowerBroker: "localhost:2",
|
||||
},
|
||||
},
|
||||
|
||||
@ -5,6 +5,7 @@ import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
type BrokerStats struct {
|
||||
@ -65,7 +66,7 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
|
||||
bs.SubscriberCount = subscriberCount
|
||||
}
|
||||
|
||||
func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition, isAdd bool) {
|
||||
func (bs *BrokerStats) RegisterAssignment(t *schema_pb.Topic, partition *schema_pb.Partition, isAdd bool) {
|
||||
tps := &TopicPartitionStats{
|
||||
TopicPartition: topic.TopicPartition{
|
||||
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},
|
||||
|
||||
@ -3,13 +3,14 @@ package pub_balancer
|
||||
import (
|
||||
"errors"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
var (
|
||||
ErrNoBroker = errors.New("no broker")
|
||||
)
|
||||
|
||||
func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
|
||||
func (balancer *PubBalancer) LookupTopicPartitions(topic *schema_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
|
||||
// find existing topic partition assignments
|
||||
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
|
||||
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
|
||||
@ -18,7 +19,7 @@ func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignme
|
||||
if topicPartitionStat.TopicPartition.Namespace == topic.Namespace &&
|
||||
topicPartitionStat.TopicPartition.Name == topic.Name {
|
||||
assignment := &mq_pb.BrokerPartitionAssignment{
|
||||
Partition: &mq_pb.Partition{
|
||||
Partition: &schema_pb.Partition{
|
||||
RingSize: MaxPartitionCount,
|
||||
RangeStart: topicPartitionStat.RangeStart,
|
||||
RangeStop: topicPartitionStat.RangeStop,
|
||||
|
||||
@ -2,7 +2,7 @@ package pub_balancer
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
type PartitionSlotToBroker struct {
|
||||
@ -24,7 +24,7 @@ func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList {
|
||||
}
|
||||
}
|
||||
|
||||
func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string, follower string) {
|
||||
func (ps *PartitionSlotToBrokerList) AddBroker(partition *schema_pb.Partition, broker string, follower string) {
|
||||
for _, partitionSlot := range ps.PartitionSlots {
|
||||
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
|
||||
if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {
|
||||
|
||||
@ -4,6 +4,7 @@ import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -32,7 +33,7 @@ type PubBalancer struct {
|
||||
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
|
||||
// Collected from all brokers when they connect to the broker leader
|
||||
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
|
||||
OnPartitionChange func(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
|
||||
OnPartitionChange func(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
|
||||
}
|
||||
|
||||
func NewPubBalancer() *PubBalancer {
|
||||
|
||||
@ -5,19 +5,24 @@ import (
|
||||
)
|
||||
|
||||
type Schema struct {
|
||||
Namespace string
|
||||
Name string
|
||||
RevisionId uint32
|
||||
RecordType *schema_pb.RecordType
|
||||
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)
|
||||
for _, field := range recordType.Fields {
|
||||
fieldMap[field.Name] = field
|
||||
}
|
||||
return &Schema{
|
||||
Namespace: namespace,
|
||||
Name: name,
|
||||
RecordType: recordType,
|
||||
fieldMap: fieldMap,
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
|
||||
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/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"time"
|
||||
)
|
||||
|
||||
@ -20,7 +21,7 @@ type ConsumerGroup struct {
|
||||
stopCh chan struct{}
|
||||
}
|
||||
|
||||
func NewConsumerGroup(t *mq_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup {
|
||||
func NewConsumerGroup(t *schema_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup {
|
||||
cg := &ConsumerGroup{
|
||||
topic: topic.FromPbTopic(t),
|
||||
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
|
||||
|
||||
@ -5,6 +5,7 @@ import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/filer_client"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
type TopicConsumerGroups struct {
|
||||
@ -28,7 +29,7 @@ func NewSubCoordinator() *SubCoordinator {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
|
||||
func (c *SubCoordinator) GetTopicConsumerGroups(topic *schema_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
|
||||
topicName := toTopicName(topic)
|
||||
tcg, _ := c.TopicSubscribers.Get(topicName)
|
||||
if tcg == nil && createIfMissing {
|
||||
@ -41,12 +42,12 @@ func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMiss
|
||||
}
|
||||
return tcg
|
||||
}
|
||||
func (c *SubCoordinator) RemoveTopic(topic *mq_pb.Topic) {
|
||||
func (c *SubCoordinator) RemoveTopic(topic *schema_pb.Topic) {
|
||||
topicName := toTopicName(topic)
|
||||
c.TopicSubscribers.Remove(topicName)
|
||||
}
|
||||
|
||||
func toTopicName(topic *mq_pb.Topic) string {
|
||||
func toTopicName(topic *schema_pb.Topic) string {
|
||||
topicName := topic.Namespace + "." + topic.Name
|
||||
return topicName
|
||||
}
|
||||
@ -96,7 +97,7 @@ func (c *SubCoordinator) RemoveSubscriber(initMessage *mq_pb.SubscriberToSubCoor
|
||||
}
|
||||
}
|
||||
|
||||
func (c *SubCoordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
|
||||
func (c *SubCoordinator) OnPartitionChange(topic *schema_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
|
||||
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
|
||||
if tcg == nil {
|
||||
return
|
||||
|
||||
@ -3,6 +3,7 @@ package topic
|
||||
import (
|
||||
cmap "github.com/orcaman/concurrent-map/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"github.com/shirou/gopsutil/v3/cpu"
|
||||
"time"
|
||||
)
|
||||
@ -89,7 +90,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
|
||||
Partition: localPartition.Partition,
|
||||
}
|
||||
stats.Stats[topicPartition.TopicPartitionId()] = &mq_pb.TopicPartitionStats{
|
||||
Topic: &mq_pb.Topic{
|
||||
Topic: &schema_pb.Topic{
|
||||
Namespace: string(localTopic.Namespace),
|
||||
Name: localTopic.Name,
|
||||
},
|
||||
|
||||
@ -2,7 +2,7 @@ package topic
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"time"
|
||||
)
|
||||
|
||||
@ -40,7 +40,7 @@ func (partition Partition) Equals(other Partition) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func FromPbPartition(partition *mq_pb.Partition) Partition {
|
||||
func FromPbPartition(partition *schema_pb.Partition) Partition {
|
||||
return Partition{
|
||||
RangeStart: partition.RangeStart,
|
||||
RangeStop: partition.RangeStop,
|
||||
@ -67,8 +67,8 @@ func SplitPartitions(targetCount int32, ts int64) []*Partition {
|
||||
return partitions
|
||||
}
|
||||
|
||||
func (partition Partition) ToPbPartition() *mq_pb.Partition {
|
||||
return &mq_pb.Partition{
|
||||
func (partition Partition) ToPbPartition() *schema_pb.Partition {
|
||||
return &schema_pb.Partition{
|
||||
RangeStart: partition.RangeStart,
|
||||
RangeStop: partition.RangeStop,
|
||||
RingSize: partition.RingSize,
|
||||
|
||||
@ -7,6 +7,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/filer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
jsonpb "google.golang.org/protobuf/encoding/protojson"
|
||||
)
|
||||
|
||||
@ -21,15 +22,15 @@ func NewTopic(namespace string, name string) Topic {
|
||||
Name: name,
|
||||
}
|
||||
}
|
||||
func FromPbTopic(topic *mq_pb.Topic) Topic {
|
||||
func FromPbTopic(topic *schema_pb.Topic) Topic {
|
||||
return Topic{
|
||||
Namespace: topic.Namespace,
|
||||
Name: topic.Name,
|
||||
}
|
||||
}
|
||||
|
||||
func (t Topic) ToPbTopic() *mq_pb.Topic {
|
||||
return &mq_pb.Topic{
|
||||
func (t Topic) ToPbTopic() *schema_pb.Topic {
|
||||
return &schema_pb.Topic{
|
||||
Namespace: t.Namespace,
|
||||
Name: t.Name,
|
||||
}
|
||||
|
||||
@ -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 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 mq.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_broker.proto --go_out=./mq_pb --go-grpc_out=./mq_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
|
||||
cp filer.proto ../../other/java/client/src/main/proto
|
||||
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: filer.proto
|
||||
|
||||
package filer_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: filer.proto
|
||||
|
||||
package filer_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: iam.proto
|
||||
|
||||
package iam_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: iam.proto
|
||||
|
||||
package iam_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: master.proto
|
||||
|
||||
package master_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: master.proto
|
||||
|
||||
package master_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: mount.proto
|
||||
|
||||
package mount_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: mount.proto
|
||||
|
||||
package mount_pb
|
||||
|
||||
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;
|
||||
|
||||
import "schema.proto";
|
||||
import "mq_schema.proto";
|
||||
|
||||
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb";
|
||||
option java_package = "seaweedfs.mq";
|
||||
@ -64,43 +64,14 @@ message FindBrokerLeaderResponse {
|
||||
string broker = 1;
|
||||
}
|
||||
|
||||
message Topic {
|
||||
string namespace = 1;
|
||||
string name = 2;
|
||||
}
|
||||
message Partition {
|
||||
int32 ring_size = 1;
|
||||
int32 range_start = 2;
|
||||
int32 range_stop = 3;
|
||||
int64 unix_time_ns = 4;
|
||||
}
|
||||
|
||||
message Offset {
|
||||
Topic topic = 1;
|
||||
repeated PartitionOffset partition_offsets = 2;
|
||||
}
|
||||
|
||||
enum PartitionOffsetStartType {
|
||||
EARLIEST = 0;
|
||||
EARLIEST_IN_MEMORY = 1;
|
||||
LATEST = 2;
|
||||
}
|
||||
|
||||
message PartitionOffset {
|
||||
Partition partition = 1;
|
||||
int64 start_ts_ns = 2;
|
||||
int64 stop_ts_ns = 3;
|
||||
PartitionOffsetStartType start_type = 4;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////
|
||||
message BrokerStats {
|
||||
int32 cpu_usage_percent = 1;
|
||||
map<string, TopicPartitionStats> stats = 2;
|
||||
}
|
||||
message TopicPartitionStats {
|
||||
Topic topic = 1;
|
||||
Partition partition = 2;
|
||||
schema_pb.Topic topic = 1;
|
||||
schema_pb.Partition partition = 2;
|
||||
int32 publisher_count = 3;
|
||||
int32 subscriber_count = 4;
|
||||
string follower = 5;
|
||||
@ -126,7 +97,7 @@ message BalanceTopicsResponse {
|
||||
|
||||
//////////////////////////////////////////////////
|
||||
message ConfigureTopicRequest {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
int32 partition_count = 2;
|
||||
schema_pb.RecordType record_type = 3;
|
||||
}
|
||||
@ -137,23 +108,23 @@ message ConfigureTopicResponse {
|
||||
message ListTopicsRequest {
|
||||
}
|
||||
message ListTopicsResponse {
|
||||
repeated Topic topics = 1;
|
||||
repeated schema_pb.Topic topics = 1;
|
||||
}
|
||||
message LookupTopicBrokersRequest {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
}
|
||||
message LookupTopicBrokersResponse {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
||||
}
|
||||
message BrokerPartitionAssignment {
|
||||
Partition partition = 1;
|
||||
schema_pb.Partition partition = 1;
|
||||
string leader_broker = 2;
|
||||
string follower_broker = 3;
|
||||
}
|
||||
|
||||
message AssignTopicPartitionsRequest {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
|
||||
bool is_leader = 3;
|
||||
bool is_draining = 4;
|
||||
@ -165,7 +136,7 @@ message SubscriberToSubCoordinatorRequest {
|
||||
message InitMessage {
|
||||
string consumer_group = 1;
|
||||
string consumer_group_instance_id = 2;
|
||||
Topic topic = 3;
|
||||
schema_pb.Topic topic = 3;
|
||||
// The consumer group instance will be assigned at most max_partition_count partitions.
|
||||
// If the number of partitions is less than the sum of max_partition_count,
|
||||
// the consumer group instance may be assigned partitions less than max_partition_count.
|
||||
@ -178,10 +149,10 @@ message SubscriberToSubCoordinatorRequest {
|
||||
int32 rebalance_seconds = 5;
|
||||
}
|
||||
message AckUnAssignmentMessage {
|
||||
Partition partition = 1;
|
||||
schema_pb.Partition partition = 1;
|
||||
}
|
||||
message AckAssignmentMessage {
|
||||
Partition partition = 1;
|
||||
schema_pb.Partition partition = 1;
|
||||
}
|
||||
oneof message {
|
||||
InitMessage init = 1;
|
||||
@ -194,7 +165,7 @@ message SubscriberToSubCoordinatorResponse {
|
||||
BrokerPartitionAssignment partition_assignment = 1;
|
||||
}
|
||||
message UnAssignment {
|
||||
Partition partition = 1;
|
||||
schema_pb.Partition partition = 1;
|
||||
}
|
||||
oneof message {
|
||||
Assignment assignment = 1;
|
||||
@ -215,8 +186,8 @@ message DataMessage {
|
||||
}
|
||||
message PublishMessageRequest {
|
||||
message InitMessage {
|
||||
Topic topic = 1;
|
||||
Partition partition = 2;
|
||||
schema_pb.Topic topic = 1;
|
||||
schema_pb.Partition partition = 2;
|
||||
int32 ack_interval = 3;
|
||||
string follower_broker = 4;
|
||||
string publisher_name = 5; // for debugging
|
||||
@ -233,8 +204,8 @@ message PublishMessageResponse {
|
||||
}
|
||||
message PublishFollowMeRequest {
|
||||
message InitMessage {
|
||||
Topic topic = 1;
|
||||
Partition partition = 2;
|
||||
schema_pb.Topic topic = 1;
|
||||
schema_pb.Partition partition = 2;
|
||||
}
|
||||
message FlushMessage {
|
||||
int64 ts_ns = 1;
|
||||
@ -256,11 +227,11 @@ message SubscribeMessageRequest {
|
||||
string consumer_group = 1;
|
||||
string consumer_id = 2;
|
||||
string client_id = 3;
|
||||
Topic topic = 4;
|
||||
PartitionOffset partition_offset = 5;
|
||||
schema_pb.Topic topic = 4;
|
||||
schema_pb.PartitionOffset partition_offset = 5;
|
||||
string filter = 6;
|
||||
string follower_broker = 7;
|
||||
int32 concurrency = 8;
|
||||
int32 sliding_window_size = 8;
|
||||
}
|
||||
message AckMessage {
|
||||
int64 sequence = 1;
|
||||
@ -284,8 +255,8 @@ message SubscribeMessageResponse {
|
||||
}
|
||||
message SubscribeFollowMeRequest {
|
||||
message InitMessage {
|
||||
Topic topic = 1;
|
||||
Partition partition = 2;
|
||||
schema_pb.Topic topic = 1;
|
||||
schema_pb.Partition partition = 2;
|
||||
string consumer_group = 3;
|
||||
}
|
||||
message AckMessage {
|
||||
@ -303,13 +274,13 @@ message SubscribeFollowMeResponse {
|
||||
int64 ack_ts_ns = 1;
|
||||
}
|
||||
message ClosePublishersRequest {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
int64 unix_time_ns = 2;
|
||||
}
|
||||
message ClosePublishersResponse {
|
||||
}
|
||||
message CloseSubscribersRequest {
|
||||
Topic topic = 1;
|
||||
schema_pb.Topic topic = 1;
|
||||
int64 unix_time_ns = 2;
|
||||
}
|
||||
message CloseSubscribersResponse {
|
||||
File diff suppressed because it is too large
Load Diff
@ -1,8 +1,8 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// source: mq.proto
|
||||
// - protoc v5.28.3
|
||||
// source: mq_broker.proto
|
||||
|
||||
package mq_pb
|
||||
|
||||
@ -589,5 +589,5 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
|
||||
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";
|
||||
|
||||
///////////////////////////
|
||||
// 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
|
||||
///////////////////////////
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: remote.proto
|
||||
|
||||
package remote_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: s3.proto
|
||||
|
||||
package s3_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: s3.proto
|
||||
|
||||
package s3_pb
|
||||
|
||||
@ -1,8 +1,8 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// source: schema.proto
|
||||
// protoc v5.28.3
|
||||
// source: mq_schema.proto
|
||||
|
||||
package schema_pb
|
||||
|
||||
@ -20,6 +20,55 @@ const (
|
||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
||||
)
|
||||
|
||||
type PartitionOffsetStartType int32
|
||||
|
||||
const (
|
||||
PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0
|
||||
PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1
|
||||
PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2
|
||||
)
|
||||
|
||||
// Enum value maps for PartitionOffsetStartType.
|
||||
var (
|
||||
PartitionOffsetStartType_name = map[int32]string{
|
||||
0: "EARLIEST",
|
||||
1: "EARLIEST_IN_MEMORY",
|
||||
2: "LATEST",
|
||||
}
|
||||
PartitionOffsetStartType_value = map[string]int32{
|
||||
"EARLIEST": 0,
|
||||
"EARLIEST_IN_MEMORY": 1,
|
||||
"LATEST": 2,
|
||||
}
|
||||
)
|
||||
|
||||
func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType {
|
||||
p := new(PartitionOffsetStartType)
|
||||
*p = x
|
||||
return p
|
||||
}
|
||||
|
||||
func (x PartitionOffsetStartType) String() string {
|
||||
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
|
||||
}
|
||||
|
||||
func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor {
|
||||
return file_mq_schema_proto_enumTypes[0].Descriptor()
|
||||
}
|
||||
|
||||
func (PartitionOffsetStartType) Type() protoreflect.EnumType {
|
||||
return &file_mq_schema_proto_enumTypes[0]
|
||||
}
|
||||
|
||||
func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber {
|
||||
return protoreflect.EnumNumber(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use PartitionOffsetStartType.Descriptor instead.
|
||||
func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) {
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
type ScalarType int32
|
||||
|
||||
const (
|
||||
@ -65,11 +114,11 @@ func (x ScalarType) String() string {
|
||||
}
|
||||
|
||||
func (ScalarType) Descriptor() protoreflect.EnumDescriptor {
|
||||
return file_schema_proto_enumTypes[0].Descriptor()
|
||||
return file_mq_schema_proto_enumTypes[1].Descriptor()
|
||||
}
|
||||
|
||||
func (ScalarType) Type() protoreflect.EnumType {
|
||||
return &file_schema_proto_enumTypes[0]
|
||||
return &file_mq_schema_proto_enumTypes[1]
|
||||
}
|
||||
|
||||
func (x ScalarType) Number() protoreflect.EnumNumber {
|
||||
@ -78,7 +127,259 @@ func (x ScalarType) Number() protoreflect.EnumNumber {
|
||||
|
||||
// Deprecated: Use ScalarType.Descriptor instead.
|
||||
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 {
|
||||
@ -92,7 +393,7 @@ type RecordType struct {
|
||||
func (x *RecordType) Reset() {
|
||||
*x = RecordType{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[0]
|
||||
mi := &file_mq_schema_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -105,7 +406,7 @@ func (x *RecordType) String() string {
|
||||
func (*RecordType) ProtoMessage() {}
|
||||
|
||||
func (x *RecordType) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[0]
|
||||
mi := &file_mq_schema_proto_msgTypes[4]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -118,7 +419,7 @@ func (x *RecordType) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use RecordType.ProtoReflect.Descriptor instead.
|
||||
func (*RecordType) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{0}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{4}
|
||||
}
|
||||
|
||||
func (x *RecordType) GetFields() []*Field {
|
||||
@ -143,7 +444,7 @@ type Field struct {
|
||||
func (x *Field) Reset() {
|
||||
*x = Field{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[1]
|
||||
mi := &file_mq_schema_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -156,7 +457,7 @@ func (x *Field) String() string {
|
||||
func (*Field) ProtoMessage() {}
|
||||
|
||||
func (x *Field) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[1]
|
||||
mi := &file_mq_schema_proto_msgTypes[5]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -169,7 +470,7 @@ func (x *Field) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use Field.ProtoReflect.Descriptor instead.
|
||||
func (*Field) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{1}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{5}
|
||||
}
|
||||
|
||||
func (x *Field) GetName() string {
|
||||
@ -223,7 +524,7 @@ type Type struct {
|
||||
func (x *Type) Reset() {
|
||||
*x = Type{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[2]
|
||||
mi := &file_mq_schema_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -236,7 +537,7 @@ func (x *Type) String() string {
|
||||
func (*Type) ProtoMessage() {}
|
||||
|
||||
func (x *Type) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[2]
|
||||
mi := &file_mq_schema_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -249,7 +550,7 @@ func (x *Type) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use Type.ProtoReflect.Descriptor instead.
|
||||
func (*Type) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{2}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{6}
|
||||
}
|
||||
|
||||
func (m *Type) GetKind() isType_Kind {
|
||||
@ -313,7 +614,7 @@ type ListType struct {
|
||||
func (x *ListType) Reset() {
|
||||
*x = ListType{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[3]
|
||||
mi := &file_mq_schema_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -326,7 +627,7 @@ func (x *ListType) String() string {
|
||||
func (*ListType) ProtoMessage() {}
|
||||
|
||||
func (x *ListType) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[3]
|
||||
mi := &file_mq_schema_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -339,7 +640,7 @@ func (x *ListType) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use ListType.ProtoReflect.Descriptor instead.
|
||||
func (*ListType) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{3}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{7}
|
||||
}
|
||||
|
||||
func (x *ListType) GetElementType() *Type {
|
||||
@ -363,7 +664,7 @@ type RecordValue struct {
|
||||
func (x *RecordValue) Reset() {
|
||||
*x = RecordValue{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[4]
|
||||
mi := &file_mq_schema_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -376,7 +677,7 @@ func (x *RecordValue) String() string {
|
||||
func (*RecordValue) ProtoMessage() {}
|
||||
|
||||
func (x *RecordValue) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[4]
|
||||
mi := &file_mq_schema_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -389,7 +690,7 @@ func (x *RecordValue) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use RecordValue.ProtoReflect.Descriptor instead.
|
||||
func (*RecordValue) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{4}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{8}
|
||||
}
|
||||
|
||||
func (x *RecordValue) GetFields() map[string]*Value {
|
||||
@ -421,7 +722,7 @@ type Value struct {
|
||||
func (x *Value) Reset() {
|
||||
*x = Value{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[5]
|
||||
mi := &file_mq_schema_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -434,7 +735,7 @@ func (x *Value) String() string {
|
||||
func (*Value) ProtoMessage() {}
|
||||
|
||||
func (x *Value) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[5]
|
||||
mi := &file_mq_schema_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -447,7 +748,7 @@ func (x *Value) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use Value.ProtoReflect.Descriptor instead.
|
||||
func (*Value) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{5}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{9}
|
||||
}
|
||||
|
||||
func (m *Value) GetKind() isValue_Kind {
|
||||
@ -589,7 +890,7 @@ type ListValue struct {
|
||||
func (x *ListValue) Reset() {
|
||||
*x = ListValue{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_schema_proto_msgTypes[6]
|
||||
mi := &file_mq_schema_proto_msgTypes[10]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -602,7 +903,7 @@ func (x *ListValue) String() string {
|
||||
func (*ListValue) ProtoMessage() {}
|
||||
|
||||
func (x *ListValue) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_schema_proto_msgTypes[6]
|
||||
mi := &file_mq_schema_proto_msgTypes[10]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -615,7 +916,7 @@ func (x *ListValue) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use ListValue.ProtoReflect.Descriptor instead.
|
||||
func (*ListValue) Descriptor() ([]byte, []int) {
|
||||
return file_schema_proto_rawDescGZIP(), []int{6}
|
||||
return file_mq_schema_proto_rawDescGZIP(), []int{10}
|
||||
}
|
||||
|
||||
func (x *ListValue) GetValues() []*Value {
|
||||
@ -625,11 +926,44 @@ func (x *ListValue) GetValues() []*Value {
|
||||
return nil
|
||||
}
|
||||
|
||||
var File_schema_proto protoreflect.FileDescriptor
|
||||
var File_mq_schema_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_schema_proto_rawDesc = []byte{
|
||||
0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09,
|
||||
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x36, 0x0a, 0x0a, 0x52, 0x65, 0x63,
|
||||
var file_mq_schema_proto_rawDesc = []byte{
|
||||
0x0a, 0x0f, 0x6d, 0x71, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x12, 0x09, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x22, 0x39, 0x0a, 0x05,
|
||||
0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61,
|
||||
0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70,
|
||||
0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74,
|
||||
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x69,
|
||||
0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x69,
|
||||
0x7a, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72,
|
||||
0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74,
|
||||
0x61, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x6f,
|
||||
0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74,
|
||||
0x6f, 0x70, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f,
|
||||
0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69,
|
||||
0x6d, 0x65, 0x4e, 0x73, 0x22, 0x79, 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x26,
|
||||
0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e,
|
||||
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52,
|
||||
0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x47, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,
|
||||
0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61,
|
||||
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x10, 0x70,
|
||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x22,
|
||||
0xc7, 0x01, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66,
|
||||
0x73, 0x65, 0x74, 0x12, 0x32, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f,
|
||||
0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61,
|
||||
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74,
|
||||
0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x74,
|
||||
0x61, 0x72, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x1c, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x70, 0x5f,
|
||||
0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x6f,
|
||||
0x70, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x42, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74,
|
||||
0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x23, 0x2e, 0x73, 0x63, 0x68, 0x65,
|
||||
0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f,
|
||||
0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09,
|
||||
0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x36, 0x0a, 0x0a, 0x52, 0x65, 0x63,
|
||||
0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
|
||||
0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,
|
||||
0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
|
||||
@ -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,
|
||||
0x6c, 0x75, 0x65, 0x12, 0x28, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20,
|
||||
0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e,
|
||||
0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x2a, 0x5a, 0x0a,
|
||||
0x0a, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42,
|
||||
0x4f, 0x4f, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01,
|
||||
0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46,
|
||||
0x4c, 0x4f, 0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45,
|
||||
0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x0a, 0x0a,
|
||||
0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74,
|
||||
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66,
|
||||
0x73, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64,
|
||||
0x2f, 0x70, 0x62, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x2a, 0x4c, 0x0a,
|
||||
0x18, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74,
|
||||
0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52,
|
||||
0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49,
|
||||
0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12,
|
||||
0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, 0x02, 0x2a, 0x5a, 0x0a, 0x0a, 0x53,
|
||||
0x63, 0x61, 0x6c, 0x61, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x42, 0x4f, 0x4f,
|
||||
0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01, 0x12, 0x09,
|
||||
0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f,
|
||||
0x41, 0x54, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x05,
|
||||
0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x0a, 0x0a, 0x06, 0x53,
|
||||
0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75,
|
||||
0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f,
|
||||
0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70,
|
||||
0x62, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_schema_proto_rawDescOnce sync.Once
|
||||
file_schema_proto_rawDescData = file_schema_proto_rawDesc
|
||||
file_mq_schema_proto_rawDescOnce sync.Once
|
||||
file_mq_schema_proto_rawDescData = file_mq_schema_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_schema_proto_rawDescGZIP() []byte {
|
||||
file_schema_proto_rawDescOnce.Do(func() {
|
||||
file_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_schema_proto_rawDescData)
|
||||
func file_mq_schema_proto_rawDescGZIP() []byte {
|
||||
file_mq_schema_proto_rawDescOnce.Do(func() {
|
||||
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_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 8)
|
||||
var file_schema_proto_goTypes = []any{
|
||||
(ScalarType)(0), // 0: schema_pb.ScalarType
|
||||
(*RecordType)(nil), // 1: schema_pb.RecordType
|
||||
(*Field)(nil), // 2: schema_pb.Field
|
||||
(*Type)(nil), // 3: schema_pb.Type
|
||||
(*ListType)(nil), // 4: schema_pb.ListType
|
||||
(*RecordValue)(nil), // 5: schema_pb.RecordValue
|
||||
(*Value)(nil), // 6: schema_pb.Value
|
||||
(*ListValue)(nil), // 7: schema_pb.ListValue
|
||||
nil, // 8: schema_pb.RecordValue.FieldsEntry
|
||||
var file_mq_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 2)
|
||||
var file_mq_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 12)
|
||||
var file_mq_schema_proto_goTypes = []any{
|
||||
(PartitionOffsetStartType)(0), // 0: schema_pb.PartitionOffsetStartType
|
||||
(ScalarType)(0), // 1: schema_pb.ScalarType
|
||||
(*Topic)(nil), // 2: schema_pb.Topic
|
||||
(*Partition)(nil), // 3: schema_pb.Partition
|
||||
(*Offset)(nil), // 4: schema_pb.Offset
|
||||
(*PartitionOffset)(nil), // 5: schema_pb.PartitionOffset
|
||||
(*RecordType)(nil), // 6: schema_pb.RecordType
|
||||
(*Field)(nil), // 7: schema_pb.Field
|
||||
(*Type)(nil), // 8: schema_pb.Type
|
||||
(*ListType)(nil), // 9: schema_pb.ListType
|
||||
(*RecordValue)(nil), // 10: schema_pb.RecordValue
|
||||
(*Value)(nil), // 11: schema_pb.Value
|
||||
(*ListValue)(nil), // 12: schema_pb.ListValue
|
||||
nil, // 13: schema_pb.RecordValue.FieldsEntry
|
||||
}
|
||||
var file_schema_proto_depIdxs = []int32{
|
||||
2, // 0: schema_pb.RecordType.fields:type_name -> schema_pb.Field
|
||||
3, // 1: schema_pb.Field.type:type_name -> schema_pb.Type
|
||||
0, // 2: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType
|
||||
1, // 3: schema_pb.Type.record_type:type_name -> schema_pb.RecordType
|
||||
4, // 4: schema_pb.Type.list_type:type_name -> schema_pb.ListType
|
||||
3, // 5: schema_pb.ListType.element_type:type_name -> schema_pb.Type
|
||||
8, // 6: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry
|
||||
7, // 7: schema_pb.Value.list_value:type_name -> schema_pb.ListValue
|
||||
5, // 8: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue
|
||||
6, // 9: schema_pb.ListValue.values:type_name -> schema_pb.Value
|
||||
6, // 10: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value
|
||||
11, // [11:11] is the sub-list for method output_type
|
||||
11, // [11:11] is the sub-list for method input_type
|
||||
11, // [11:11] is the sub-list for extension type_name
|
||||
11, // [11:11] is the sub-list for extension extendee
|
||||
0, // [0:11] is the sub-list for field type_name
|
||||
var file_mq_schema_proto_depIdxs = []int32{
|
||||
2, // 0: schema_pb.Offset.topic:type_name -> schema_pb.Topic
|
||||
5, // 1: schema_pb.Offset.partition_offsets:type_name -> schema_pb.PartitionOffset
|
||||
3, // 2: schema_pb.PartitionOffset.partition:type_name -> schema_pb.Partition
|
||||
0, // 3: schema_pb.PartitionOffset.start_type:type_name -> schema_pb.PartitionOffsetStartType
|
||||
7, // 4: schema_pb.RecordType.fields:type_name -> schema_pb.Field
|
||||
8, // 5: schema_pb.Field.type:type_name -> schema_pb.Type
|
||||
1, // 6: schema_pb.Type.scalar_type:type_name -> schema_pb.ScalarType
|
||||
6, // 7: schema_pb.Type.record_type:type_name -> schema_pb.RecordType
|
||||
9, // 8: schema_pb.Type.list_type:type_name -> schema_pb.ListType
|
||||
8, // 9: schema_pb.ListType.element_type:type_name -> schema_pb.Type
|
||||
13, // 10: schema_pb.RecordValue.fields:type_name -> schema_pb.RecordValue.FieldsEntry
|
||||
12, // 11: schema_pb.Value.list_value:type_name -> schema_pb.ListValue
|
||||
10, // 12: schema_pb.Value.record_value:type_name -> schema_pb.RecordValue
|
||||
11, // 13: schema_pb.ListValue.values:type_name -> schema_pb.Value
|
||||
11, // 14: schema_pb.RecordValue.FieldsEntry.value:type_name -> schema_pb.Value
|
||||
15, // [15:15] is the sub-list for method output_type
|
||||
15, // [15:15] is the sub-list for method input_type
|
||||
15, // [15:15] is the sub-list for extension type_name
|
||||
15, // [15:15] is the sub-list for extension extendee
|
||||
0, // [0:15] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_schema_proto_init() }
|
||||
func file_schema_proto_init() {
|
||||
if File_schema_proto != nil {
|
||||
func init() { file_mq_schema_proto_init() }
|
||||
func file_mq_schema_proto_init() {
|
||||
if File_mq_schema_proto != nil {
|
||||
return
|
||||
}
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -770,7 +1166,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -782,7 +1178,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -794,7 +1190,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -806,7 +1202,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -818,7 +1214,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -830,7 +1226,7 @@ func file_schema_proto_init() {
|
||||
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 {
|
||||
case 0:
|
||||
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_RecordType)(nil),
|
||||
(*Type_ListType)(nil),
|
||||
}
|
||||
file_schema_proto_msgTypes[5].OneofWrappers = []any{
|
||||
file_mq_schema_proto_msgTypes[9].OneofWrappers = []any{
|
||||
(*Value_BoolValue)(nil),
|
||||
(*Value_Int32Value)(nil),
|
||||
(*Value_Int64Value)(nil),
|
||||
@ -863,19 +1259,19 @@ func file_schema_proto_init() {
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_schema_proto_rawDesc,
|
||||
NumEnums: 1,
|
||||
NumMessages: 8,
|
||||
RawDescriptor: file_mq_schema_proto_rawDesc,
|
||||
NumEnums: 2,
|
||||
NumMessages: 12,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
GoTypes: file_schema_proto_goTypes,
|
||||
DependencyIndexes: file_schema_proto_depIdxs,
|
||||
EnumInfos: file_schema_proto_enumTypes,
|
||||
MessageInfos: file_schema_proto_msgTypes,
|
||||
GoTypes: file_mq_schema_proto_goTypes,
|
||||
DependencyIndexes: file_mq_schema_proto_depIdxs,
|
||||
EnumInfos: file_mq_schema_proto_enumTypes,
|
||||
MessageInfos: file_mq_schema_proto_msgTypes,
|
||||
}.Build()
|
||||
File_schema_proto = out.File
|
||||
file_schema_proto_rawDesc = nil
|
||||
file_schema_proto_goTypes = nil
|
||||
file_schema_proto_depIdxs = nil
|
||||
File_mq_schema_proto = out.File
|
||||
file_mq_schema_proto_rawDesc = nil
|
||||
file_mq_schema_proto_goTypes = nil
|
||||
file_mq_schema_proto_depIdxs = nil
|
||||
}
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v5.28.1
|
||||
// protoc v5.28.3
|
||||
// source: volume_server.proto
|
||||
|
||||
package volume_server_pb
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.5.1
|
||||
// - protoc v5.28.1
|
||||
// - protoc v5.28.3
|
||||
// source: volume_server.proto
|
||||
|
||||
package volume_server_pb
|
||||
|
||||
@ -7,6 +7,7 @@ import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"io"
|
||||
)
|
||||
|
||||
@ -54,7 +55,7 @@ func (c *commandMqTopicConfigure) Do(args []string, commandEnv *CommandEnv, writ
|
||||
// create topic
|
||||
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
resp, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
|
||||
Topic: &mq_pb.Topic{
|
||||
Topic: &schema_pb.Topic{
|
||||
Namespace: *namespace,
|
||||
Name: *topicName,
|
||||
},
|
||||
|
||||
@ -6,6 +6,7 @@ import (
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"io"
|
||||
)
|
||||
|
||||
@ -46,7 +47,7 @@ func (c *commandMqTopicDescribe) Do(args []string, commandEnv *CommandEnv, write
|
||||
|
||||
return pb.WithBrokerGrpcClient(false, brokerBalancer, commandEnv.option.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
resp, err := client.LookupTopicBrokers(context.Background(), &mq_pb.LookupTopicBrokersRequest{
|
||||
Topic: &mq_pb.Topic{
|
||||
Topic: &schema_pb.Topic{
|
||||
Namespace: *namespace,
|
||||
Name: *topicName,
|
||||
},
|
||||
|
||||
Loading…
Reference in New Issue
Block a user