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:
Chris Lu
2025-01-20 22:19:27 -08:00
committed by GitHub
parent b2f56d9add
commit cc05874d06
57 changed files with 3802 additions and 1562 deletions

View 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
}

View 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
}
}
}

View 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
}

View 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,
}
}
}
}

View 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
}

View File

@@ -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 {

View File

@@ -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)

View File

@@ -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,
}

View File

@@ -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

View 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
}

View 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,
})
}

View 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)
}
}

View 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
}

View 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())
}

View File

@@ -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() {

View File

@@ -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() {

View File

@@ -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
}

View File

@@ -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 {

View File

@@ -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
}

View File

@@ -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),

View File

@@ -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",
},
},

View File

@@ -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},

View File

@@ -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,

View File

@@ -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 {

View File

@@ -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 {

View File

@@ -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) {

View File

@@ -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](),

View File

@@ -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

View File

@@ -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,
},

View File

@@ -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,

View File

@@ -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,
}