You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

114 lines
3.7 KiB

1 year ago
1 year ago
  1. package sub_client
  2. import (
  3. "github.com/seaweedfs/seaweedfs/weed/glog"
  4. "github.com/seaweedfs/seaweedfs/weed/mq/topic"
  5. "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
  6. "sync"
  7. "time"
  8. )
  9. type ProcessorState struct {
  10. stopCh chan struct{}
  11. }
  12. // Subscribe subscribes to a topic's specified partitions.
  13. // If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
  14. func (sub *TopicSubscriber) Subscribe() error {
  15. go sub.startProcessors()
  16. // loop forever
  17. sub.doKeepConnectedToSubCoordinator()
  18. return nil
  19. }
  20. func (sub *TopicSubscriber) startProcessors() {
  21. // listen to the messages from the sub coordinator
  22. // start one processor per partition
  23. var wg sync.WaitGroup
  24. semaphore := make(chan struct{}, sub.SubscriberConfig.MaxPartitionCount)
  25. for message := range sub.brokerPartitionAssignmentChan {
  26. if assigned := message.GetAssignment(); assigned != nil {
  27. wg.Add(1)
  28. semaphore <- struct{}{}
  29. topicPartition := topic.FromPbPartition(assigned.PartitionAssignment.Partition)
  30. // wait until no covering partition is still in progress
  31. sub.waitUntilNoOverlappingPartitionInFlight(topicPartition)
  32. // start a processors
  33. stopChan := make(chan struct{})
  34. sub.activeProcessorsLock.Lock()
  35. sub.activeProcessors[topicPartition] = &ProcessorState{
  36. stopCh: stopChan,
  37. }
  38. sub.activeProcessorsLock.Unlock()
  39. go func(assigned *mq_pb.BrokerPartitionAssignment, topicPartition topic.Partition) {
  40. defer func() {
  41. sub.activeProcessorsLock.Lock()
  42. delete(sub.activeProcessors, topicPartition)
  43. sub.activeProcessorsLock.Unlock()
  44. <-semaphore
  45. wg.Done()
  46. }()
  47. glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
  48. err := sub.onEachPartition(assigned, stopChan)
  49. if err != nil {
  50. glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
  51. } else {
  52. glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
  53. }
  54. sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
  55. Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignment{
  56. AckUnAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{
  57. Partition: assigned.Partition,
  58. },
  59. },
  60. }
  61. }(assigned.PartitionAssignment, topicPartition)
  62. }
  63. if unAssignment := message.GetUnAssignment(); unAssignment != nil {
  64. topicPartition := topic.FromPbPartition(unAssignment.Partition)
  65. sub.activeProcessorsLock.Lock()
  66. if processor, found := sub.activeProcessors[topicPartition]; found {
  67. close(processor.stopCh)
  68. delete(sub.activeProcessors, topicPartition)
  69. }
  70. sub.activeProcessorsLock.Unlock()
  71. }
  72. }
  73. wg.Wait()
  74. }
  75. func (sub *TopicSubscriber) waitUntilNoOverlappingPartitionInFlight(topicPartition topic.Partition) {
  76. foundOverlapping := true
  77. for foundOverlapping {
  78. sub.activeProcessorsLock.Lock()
  79. foundOverlapping = false
  80. var overlappedPartition topic.Partition
  81. for partition, _ := range sub.activeProcessors {
  82. if partition.Overlaps(topicPartition) {
  83. if partition.Equals(topicPartition) {
  84. continue
  85. }
  86. foundOverlapping = true
  87. overlappedPartition = partition
  88. break
  89. }
  90. }
  91. sub.activeProcessorsLock.Unlock()
  92. if foundOverlapping {
  93. glog.V(0).Infof("subscriber %s new partition %v waiting for partition %+v to complete", sub.ContentConfig.Topic, topicPartition, overlappedPartition)
  94. time.Sleep(1 * time.Second)
  95. }
  96. }
  97. }