aboutsummaryrefslogtreecommitdiff
path: root/weed/mq/client/sub_client
diff options
context:
space:
mode:
Diffstat (limited to 'weed/mq/client/sub_client')
-rw-r--r--weed/mq/client/sub_client/connect_to_sub_coordinator.go171
-rw-r--r--weed/mq/client/sub_client/lookup.go34
-rw-r--r--weed/mq/client/sub_client/process.go85
-rw-r--r--weed/mq/client/sub_client/subscribe.go41
-rw-r--r--weed/mq/client/sub_client/subscriber.go18
5 files changed, 185 insertions, 164 deletions
diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
new file mode 100644
index 000000000..6d0eb0d43
--- /dev/null
+++ b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
@@ -0,0 +1,171 @@
+package sub_client
+
+import (
+ "context"
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "io"
+ "sync"
+ "time"
+)
+
+func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
+ waitTime := 1 * time.Second
+ for {
+ for _, broker := range sub.bootstrapBrokers {
+ // TODO find the balancer
+ // connect to the balancer
+ pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ stream, err := client.SubscriberToSubCoordinator(ctx)
+ if err != nil {
+ glog.V(0).Infof("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ return err
+ }
+ waitTime = 1 * time.Second
+
+ // Maybe later: subscribe to multiple topics instead of just one
+
+ if err := stream.Send(&mq_pb.SubscriberToSubCoordinatorRequest{
+ Message: &mq_pb.SubscriberToSubCoordinatorRequest_Init{
+ Init: &mq_pb.SubscriberToSubCoordinatorRequest_InitMessage{
+ ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
+ ConsumerGroupInstanceId: sub.SubscriberConfig.ConsumerGroupInstanceId,
+ Topic: &mq_pb.Topic{
+ Namespace: sub.ContentConfig.Namespace,
+ Name: sub.ContentConfig.Topic,
+ },
+ },
+ },
+ }); err != nil {
+ glog.V(0).Infof("subscriber %s/%s send init: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ return err
+ }
+
+ // keep receiving messages from the sub coordinator
+ for {
+ resp, err := stream.Recv()
+ if err != nil {
+ glog.V(0).Infof("subscriber %s/%s receive: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ return err
+ }
+ assignment := resp.GetAssignment()
+ if assignment != nil {
+ glog.V(0).Infof("subscriber %s/%s receive assignment: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, assignment)
+ }
+ sub.onEachAssignment(assignment)
+ }
+
+ return nil
+ })
+ }
+ glog.V(0).Infof("subscriber %s/%s/%s waiting for more assignments", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ if waitTime < 10*time.Second {
+ waitTime += 1 * time.Second
+ }
+ time.Sleep(waitTime)
+ }
+}
+
+func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCoordinatorResponse_Assignment) {
+ if assignment == nil {
+ return
+ }
+ // process each partition, with a concurrency limit
+ var wg sync.WaitGroup
+ semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit)
+
+ for _, assigned := range assignment.AssignedPartitions {
+ wg.Add(1)
+ semaphore <- struct{}{}
+ go func(partition *mq_pb.Partition, broker string) {
+ defer wg.Done()
+ defer func() { <-semaphore }()
+ glog.V(0).Infof("subscriber %s/%s/%s assigned partition %+v at %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
+ err := sub.onEachPartition(partition, broker)
+ if err != nil {
+ glog.V(0).Infof("subscriber %s/%s/%s partition %+v at %v: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err)
+ }
+ }(assigned.Partition, assigned.Broker)
+ }
+
+ wg.Wait()
+}
+
+func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker string) error {
+ // connect to the partition broker
+ return pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
+ subscribeClient, err := client.SubscribeMessage(context.Background(), &mq_pb.SubscribeMessageRequest{
+ Message: &mq_pb.SubscribeMessageRequest_Init{
+ Init: &mq_pb.SubscribeMessageRequest_InitMessage{
+ ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
+ ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
+ Topic: &mq_pb.Topic{
+ Namespace: sub.ContentConfig.Namespace,
+ Name: sub.ContentConfig.Topic,
+ },
+ PartitionOffset: &mq_pb.PartitionOffset{
+ Partition: partition,
+ StartTsNs: sub.alreadyProcessedTsNs,
+ StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
+ },
+ Filter: sub.ContentConfig.Filter,
+ },
+ },
+ })
+
+ if err != nil {
+ return fmt.Errorf("create subscribe client: %v", err)
+ }
+
+ glog.V(0).Infof("subscriber %s/%s/%s connected to partition %+v at %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
+
+ if sub.OnCompletionFunc != nil {
+ defer sub.OnCompletionFunc()
+ }
+ defer func() {
+ subscribeClient.SendMsg(&mq_pb.SubscribeMessageRequest{
+ Message: &mq_pb.SubscribeMessageRequest_Ack{
+ Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
+ Sequence: 0,
+ },
+ },
+ })
+ subscribeClient.CloseSend()
+ }()
+
+ 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 {
+ return fmt.Errorf("subscribe recv: %v", err)
+ }
+ if resp.Message == nil {
+ glog.V(0).Infof("subscriber %s/%s/%s received nil message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ continue
+ }
+ switch m := resp.Message.(type) {
+ case *mq_pb.SubscribeMessageResponse_Data:
+ shouldContinue, processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
+ if processErr != nil {
+ return fmt.Errorf("process error: %v", processErr)
+ }
+ sub.alreadyProcessedTsNs = m.Data.TsNs
+ if !shouldContinue {
+ return nil
+ }
+ 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 {
+ return io.EOF
+ }
+ }
+ }
+
+ return nil
+ })
+}
diff --git a/weed/mq/client/sub_client/lookup.go b/weed/mq/client/sub_client/lookup.go
deleted file mode 100644
index b6d2a8c53..000000000
--- a/weed/mq/client/sub_client/lookup.go
+++ /dev/null
@@ -1,34 +0,0 @@
-package sub_client
-
-import (
- "context"
- "fmt"
- "github.com/seaweedfs/seaweedfs/weed/pb"
- "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
-)
-
-func (sub *TopicSubscriber) doLookup(brokerAddress string) error {
- err := pb.WithBrokerGrpcClient(true,
- brokerAddress,
- sub.SubscriberConfig.GrpcDialOption,
- func(client mq_pb.SeaweedMessagingClient) error {
- lookupResp, err := client.LookupTopicBrokers(context.Background(),
- &mq_pb.LookupTopicBrokersRequest{
- Topic: &mq_pb.Topic{
- Namespace: sub.ContentConfig.Namespace,
- Name: sub.ContentConfig.Topic,
- },
- IsForPublish: false,
- })
- if err != nil {
- return err
- }
- sub.brokerPartitionAssignments = lookupResp.BrokerPartitionAssignments
- return nil
- })
-
- if err != nil {
- return fmt.Errorf("lookup topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
- }
- return nil
-}
diff --git a/weed/mq/client/sub_client/process.go b/weed/mq/client/sub_client/process.go
deleted file mode 100644
index b6bdb14ee..000000000
--- a/weed/mq/client/sub_client/process.go
+++ /dev/null
@@ -1,85 +0,0 @@
-package sub_client
-
-import (
- "context"
- "fmt"
- "github.com/seaweedfs/seaweedfs/weed/pb"
- "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
- "sync"
-)
-
-func (sub *TopicSubscriber) doProcess() error {
- var wg sync.WaitGroup
- for _, brokerPartitionAssignment := range sub.brokerPartitionAssignments {
- brokerAddress := brokerPartitionAssignment.LeaderBroker
- grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, sub.SubscriberConfig.GrpcDialOption)
- if err != nil {
- return fmt.Errorf("dial broker %s: %v", brokerAddress, err)
- }
- brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
- subscribeClient, err := brokerClient.Subscribe(context.Background(), &mq_pb.SubscribeRequest{
- Message: &mq_pb.SubscribeRequest_Init{
- Init: &mq_pb.SubscribeRequest_InitMessage{
- ConsumerGroup: sub.SubscriberConfig.GroupId,
- ConsumerId: sub.SubscriberConfig.GroupInstanceId,
- Topic: &mq_pb.Topic{
- Namespace: sub.ContentConfig.Namespace,
- Name: sub.ContentConfig.Topic,
- },
- Partition: &mq_pb.Partition{
- RingSize: brokerPartitionAssignment.Partition.RingSize,
- RangeStart: brokerPartitionAssignment.Partition.RangeStart,
- RangeStop: brokerPartitionAssignment.Partition.RangeStop,
- },
- Filter: sub.ContentConfig.Filter,
- Offset: &mq_pb.SubscribeRequest_InitMessage_StartTimestampNs{
- StartTimestampNs: sub.alreadyProcessedTsNs,
- },
- },
- },
- })
- if err != nil {
- return fmt.Errorf("create subscribe client: %v", err)
- }
- wg.Add(1)
- go func() {
- defer wg.Done()
- if sub.OnCompletionFunc != nil {
- defer sub.OnCompletionFunc()
- }
- defer func() {
- subscribeClient.SendMsg(&mq_pb.SubscribeRequest{
- Message: &mq_pb.SubscribeRequest_Ack{
- Ack: &mq_pb.SubscribeRequest_AckMessage{
- Sequence: 0,
- },
- },
- })
- subscribeClient.CloseSend()
- }()
- for {
- resp, err := subscribeClient.Recv()
- if err != nil {
- fmt.Printf("subscribe error: %v\n", err)
- return
- }
- if resp.Message == nil {
- continue
- }
- switch m := resp.Message.(type) {
- case *mq_pb.SubscribeResponse_Data:
- if !sub.OnEachMessageFunc(m.Data.Key, m.Data.Value) {
- return
- }
- sub.alreadyProcessedTsNs = m.Data.TsNs
- case *mq_pb.SubscribeResponse_Ctrl:
- if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
- return
- }
- }
- }
- }()
- }
- wg.Wait()
- return nil
-}
diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go
index 370f5aa3c..df62ea674 100644
--- a/weed/mq/client/sub_client/subscribe.go
+++ b/weed/mq/client/sub_client/subscribe.go
@@ -1,48 +1,11 @@
package sub_client
-import (
- "fmt"
- "github.com/seaweedfs/seaweedfs/weed/util"
- "io"
- "log"
- "time"
-)
-
// Subscribe subscribes to a topic's specified partitions.
// If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
func (sub *TopicSubscriber) Subscribe() error {
- index := -1
- util.RetryUntil("subscribe", func() error {
- index++
- index = index % len(sub.bootstrapBrokers)
- // ask balancer for brokers of the topic
- if err := sub.doLookup(sub.bootstrapBrokers[index]); err != nil {
- return fmt.Errorf("lookup topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
- }
- if len(sub.brokerPartitionAssignments) == 0 {
- if sub.waitForMoreMessage {
- time.Sleep(1 * time.Second)
- return fmt.Errorf("no broker partition assignments")
- } else {
- return nil
- }
- }
- // treat the first broker as the topic leader
- // connect to the leader broker
+ // loop forever
+ sub.doKeepConnectedToSubCoordinator()
- // subscribe to the topic
- if err := sub.doProcess(); err != nil {
- return fmt.Errorf("subscribe topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
- }
- return nil
- }, func(err error) bool {
- if err == io.EOF {
- log.Printf("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
- sub.waitForMoreMessage = false
- return false
- }
- return true
- })
return nil
}
diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go
index 9b96b14cb..9adc5197e 100644
--- a/weed/mq/client/sub_client/subscriber.go
+++ b/weed/mq/client/sub_client/subscriber.go
@@ -7,10 +7,10 @@ import (
)
type SubscriberConfiguration struct {
- ClientId string
- GroupId string
- GroupInstanceId string
- GroupMinimumPeers int32
+ ClientId string
+ ConsumerGroup string
+ ConsumerGroupInstanceId string
+ GroupMinimumPeers int32
GroupMaximumPeers int32
BootstrapServers []string
GrpcDialOption grpc.DialOption
@@ -23,12 +23,17 @@ type ContentConfiguration struct {
StartTime time.Time
}
-type OnEachMessageFunc func(key, value []byte) (shouldContinue bool)
+type ProcessorConfiguration struct {
+ ConcurrentPartitionLimit int // how many partitions to process concurrently
+}
+
+type OnEachMessageFunc func(key, value []byte) (shouldContinue bool, err error)
type OnCompletionFunc func()
type TopicSubscriber struct {
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
+ ProcessorConfig *ProcessorConfiguration
brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
@@ -37,10 +42,11 @@ type TopicSubscriber struct {
alreadyProcessedTsNs int64
}
-func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
+func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, processor ProcessorConfiguration) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
+ ProcessorConfig: &processor,
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
alreadyProcessedTsNs: content.StartTime.UnixNano(),