broker_grpc_sub.go 7.3 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221
  1. package broker
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/seaweedfs/seaweedfs/weed/glog"
  6. "github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
  7. "github.com/seaweedfs/seaweedfs/weed/mq/topic"
  8. "github.com/seaweedfs/seaweedfs/weed/pb"
  9. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  10. "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
  11. "github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
  12. "io"
  13. "time"
  14. )
  15. func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_SubscribeMessageServer) error {
  16. req, err := stream.Recv()
  17. if err != nil {
  18. return err
  19. }
  20. if req.GetInit() == nil {
  21. glog.Errorf("missing init message")
  22. return fmt.Errorf("missing init message")
  23. }
  24. ctx := stream.Context()
  25. clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
  26. t := topic.FromPbTopic(req.GetInit().Topic)
  27. partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
  28. glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
  29. localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, partition)
  30. if getOrGenErr != nil {
  31. return getOrGenErr
  32. }
  33. localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())
  34. glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
  35. isConnected := true
  36. sleepIntervalCount := 0
  37. var counter int64
  38. defer func() {
  39. isConnected = false
  40. localTopicPartition.Subscribers.RemoveSubscriber(clientName)
  41. glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
  42. if localTopicPartition.MaybeShutdownLocalPartition() {
  43. b.localTopicManager.RemoveLocalPartition(t, partition)
  44. }
  45. }()
  46. startPosition := b.getRequestPosition(req.GetInit())
  47. imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().Concurrency))
  48. // connect to the follower
  49. var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
  50. glog.V(0).Infof("follower broker: %v", req.GetInit().FollowerBroker)
  51. if req.GetInit().FollowerBroker != "" {
  52. follower := req.GetInit().FollowerBroker
  53. if followerGrpcConnection, err := pb.GrpcDial(ctx, follower, true, b.grpcDialOption); err != nil {
  54. return fmt.Errorf("fail to dial %s: %v", follower, err)
  55. } else {
  56. defer func() {
  57. println("closing SubscribeFollowMe connection", follower)
  58. subscribeFollowMeStream.CloseSend()
  59. // followerGrpcConnection.Close()
  60. }()
  61. followerClient := mq_pb.NewSeaweedMessagingClient(followerGrpcConnection)
  62. if subscribeFollowMeStream, err = followerClient.SubscribeFollowMe(ctx); err != nil {
  63. return fmt.Errorf("fail to subscribe to %s: %v", follower, err)
  64. } else {
  65. if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
  66. Message: &mq_pb.SubscribeFollowMeRequest_Init{
  67. Init: &mq_pb.SubscribeFollowMeRequest_InitMessage{
  68. Topic: req.GetInit().Topic,
  69. Partition: req.GetInit().GetPartitionOffset().Partition,
  70. ConsumerGroup: req.GetInit().ConsumerGroup,
  71. },
  72. },
  73. }); err != nil {
  74. return fmt.Errorf("fail to send init to %s: %v", follower, err)
  75. }
  76. }
  77. }
  78. glog.V(0).Infof("follower %s connected", follower)
  79. }
  80. go func() {
  81. var lastOffset int64
  82. for {
  83. ack, err := stream.Recv()
  84. if err != nil {
  85. if err == io.EOF {
  86. // the client has called CloseSend(). This is to ack the close.
  87. stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Ctrl{
  88. Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
  89. IsEndOfStream: true,
  90. },
  91. }})
  92. break
  93. }
  94. glog.V(0).Infof("topic %v partition %v subscriber %s lastOffset %d error: %v", t, partition, clientName, lastOffset, err)
  95. break
  96. }
  97. if ack.GetAck().Key == nil {
  98. // skip ack for control messages
  99. continue
  100. }
  101. imt.AcknowledgeMessage(ack.GetAck().Key, ack.GetAck().Sequence)
  102. currentLastOffset := imt.GetOldestAckedTimestamp()
  103. // fmt.Printf("%+v recv (%s,%d), oldest %d\n", partition, string(ack.GetAck().Key), ack.GetAck().Sequence, currentLastOffset)
  104. if subscribeFollowMeStream != nil && currentLastOffset > lastOffset {
  105. if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
  106. Message: &mq_pb.SubscribeFollowMeRequest_Ack{
  107. Ack: &mq_pb.SubscribeFollowMeRequest_AckMessage{
  108. TsNs: currentLastOffset,
  109. },
  110. },
  111. }); err != nil {
  112. glog.Errorf("Error sending ack to follower: %v", err)
  113. break
  114. }
  115. lastOffset = currentLastOffset
  116. // fmt.Printf("%+v forwarding ack %d\n", partition, lastOffset)
  117. }
  118. }
  119. if lastOffset > 0 {
  120. glog.V(0).Infof("saveConsumerGroupOffset %v %v %v %v", t, partition, req.GetInit().ConsumerGroup, lastOffset)
  121. if err := b.saveConsumerGroupOffset(t, partition, req.GetInit().ConsumerGroup, lastOffset); err != nil {
  122. glog.Errorf("saveConsumerGroupOffset partition %v lastOffset %d: %v", partition, lastOffset, err)
  123. }
  124. }
  125. if subscribeFollowMeStream != nil {
  126. if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
  127. Message: &mq_pb.SubscribeFollowMeRequest_Close{
  128. Close: &mq_pb.SubscribeFollowMeRequest_CloseMessage{},
  129. },
  130. }); err != nil {
  131. glog.Errorf("Error sending close to follower: %v", err)
  132. }
  133. }
  134. }()
  135. return localTopicPartition.Subscribe(clientName, startPosition, func() bool {
  136. if !isConnected {
  137. return false
  138. }
  139. sleepIntervalCount++
  140. if sleepIntervalCount > 32 {
  141. sleepIntervalCount = 32
  142. }
  143. time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond)
  144. // Check if the client has disconnected by monitoring the context
  145. select {
  146. case <-ctx.Done():
  147. err := ctx.Err()
  148. if err == context.Canceled {
  149. // Client disconnected
  150. return false
  151. }
  152. glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
  153. return false
  154. default:
  155. // Continue processing the request
  156. }
  157. return true
  158. }, func(logEntry *filer_pb.LogEntry) (bool, error) {
  159. // reset the sleep interval count
  160. sleepIntervalCount = 0
  161. for imt.IsInflight(logEntry.Key) {
  162. time.Sleep(137 * time.Millisecond)
  163. }
  164. if logEntry.Key != nil {
  165. imt.EnflightMessage(logEntry.Key, logEntry.TsNs)
  166. }
  167. if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
  168. Data: &mq_pb.DataMessage{
  169. Key: logEntry.Key,
  170. Value: logEntry.Data,
  171. TsNs: logEntry.TsNs,
  172. },
  173. }}); err != nil {
  174. glog.Errorf("Error sending data: %v", err)
  175. return false, err
  176. }
  177. counter++
  178. return false, nil
  179. })
  180. }
  181. func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (startPosition log_buffer.MessagePosition) {
  182. if initMessage == nil {
  183. return
  184. }
  185. offset := initMessage.GetPartitionOffset()
  186. if offset.StartTsNs != 0 {
  187. startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
  188. return
  189. }
  190. if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil {
  191. glog.V(0).Infof("resume from saved offset %v %v %v: %v", initMessage.Topic, initMessage.PartitionOffset.Partition, initMessage.ConsumerGroup, storedOffset)
  192. startPosition = log_buffer.NewMessagePosition(storedOffset, -2)
  193. return
  194. }
  195. if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
  196. startPosition = log_buffer.NewMessagePosition(1, -3)
  197. } else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
  198. startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
  199. }
  200. return
  201. }