broker_grpc_server_subscribe.go 5.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177
  1. package broker
  2. import (
  3. "fmt"
  4. "github.com/chrislusf/seaweedfs/weed/util/log_buffer"
  5. "io"
  6. "strings"
  7. "time"
  8. "github.com/golang/protobuf/proto"
  9. "github.com/chrislusf/seaweedfs/weed/filer"
  10. "github.com/chrislusf/seaweedfs/weed/glog"
  11. "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
  12. "github.com/chrislusf/seaweedfs/weed/pb/messaging_pb"
  13. )
  14. func (broker *MessageBroker) Subscribe(stream messaging_pb.SeaweedMessaging_SubscribeServer) error {
  15. // process initial request
  16. in, err := stream.Recv()
  17. if err == io.EOF {
  18. return nil
  19. }
  20. if err != nil {
  21. return err
  22. }
  23. var processedTsNs int64
  24. var messageCount int64
  25. subscriberId := in.Init.SubscriberId
  26. // TODO look it up
  27. topicConfig := &messaging_pb.TopicConfiguration{
  28. // IsTransient: true,
  29. }
  30. // get lock
  31. tp := TopicPartition{
  32. Namespace: in.Init.Namespace,
  33. Topic: in.Init.Topic,
  34. Partition: in.Init.Partition,
  35. }
  36. fmt.Printf("+ subscriber %s for %s\n", subscriberId, tp.String())
  37. defer func() {
  38. fmt.Printf("- subscriber %s for %s %d messages last %v\n", subscriberId, tp.String(), messageCount, time.Unix(0, processedTsNs))
  39. }()
  40. lock := broker.topicManager.RequestLock(tp, topicConfig, false)
  41. defer broker.topicManager.ReleaseLock(tp, false)
  42. isConnected := true
  43. go func() {
  44. for isConnected {
  45. if _, err := stream.Recv(); err != nil {
  46. // println("disconnecting connection to", subscriberId, tp.String())
  47. isConnected = false
  48. lock.cond.Signal()
  49. }
  50. }
  51. }()
  52. lastReadTime := time.Now()
  53. switch in.Init.StartPosition {
  54. case messaging_pb.SubscriberMessage_InitMessage_TIMESTAMP:
  55. lastReadTime = time.Unix(0, in.Init.TimestampNs)
  56. case messaging_pb.SubscriberMessage_InitMessage_LATEST:
  57. case messaging_pb.SubscriberMessage_InitMessage_EARLIEST:
  58. lastReadTime = time.Unix(0, 0)
  59. }
  60. // how to process each message
  61. // an error returned will end the subscription
  62. eachMessageFn := func(m *messaging_pb.Message) error {
  63. err := stream.Send(&messaging_pb.BrokerMessage{
  64. Data: m,
  65. })
  66. if err != nil {
  67. glog.V(0).Infof("=> subscriber %v: %+v", subscriberId, err)
  68. }
  69. return err
  70. }
  71. eachLogEntryFn := func(logEntry *filer_pb.LogEntry) error {
  72. m := &messaging_pb.Message{}
  73. if err = proto.Unmarshal(logEntry.Data, m); err != nil {
  74. glog.Errorf("unexpected unmarshal messaging_pb.Message: %v", err)
  75. return err
  76. }
  77. // fmt.Printf("sending : %d bytes ts %d\n", len(m.Value), logEntry.TsNs)
  78. if err = eachMessageFn(m); err != nil {
  79. glog.Errorf("sending %d bytes to %s: %s", len(m.Value), subscriberId, err)
  80. return err
  81. }
  82. if m.IsClose {
  83. // println("processed EOF")
  84. return io.EOF
  85. }
  86. processedTsNs = logEntry.TsNs
  87. messageCount++
  88. return nil
  89. }
  90. // fmt.Printf("subscriber %s read %d on disk log %v\n", subscriberId, messageCount, lastReadTime)
  91. for {
  92. if err = broker.readPersistedLogBuffer(&tp, lastReadTime, eachLogEntryFn); err != nil {
  93. if err != io.EOF {
  94. // println("stopping from persisted logs", err.Error())
  95. return err
  96. }
  97. }
  98. if processedTsNs != 0 {
  99. lastReadTime = time.Unix(0, processedTsNs)
  100. }
  101. lastReadTime, err = lock.logBuffer.LoopProcessLogData("broker", lastReadTime, func() bool {
  102. lock.Mutex.Lock()
  103. lock.cond.Wait()
  104. lock.Mutex.Unlock()
  105. return isConnected
  106. }, eachLogEntryFn)
  107. if err != nil {
  108. if err == log_buffer.ResumeFromDiskError {
  109. continue
  110. }
  111. glog.Errorf("processed to %v: %v", lastReadTime, err)
  112. time.Sleep(3127 * time.Millisecond)
  113. if err != log_buffer.ResumeError {
  114. break
  115. }
  116. }
  117. }
  118. return err
  119. }
  120. func (broker *MessageBroker) readPersistedLogBuffer(tp *TopicPartition, startTime time.Time, eachLogEntryFn func(logEntry *filer_pb.LogEntry) error) (err error) {
  121. startTime = startTime.UTC()
  122. startDate := fmt.Sprintf("%04d-%02d-%02d", startTime.Year(), startTime.Month(), startTime.Day())
  123. startHourMinute := fmt.Sprintf("%02d-%02d.segment", startTime.Hour(), startTime.Minute())
  124. sizeBuf := make([]byte, 4)
  125. startTsNs := startTime.UnixNano()
  126. topicDir := genTopicDir(tp.Namespace, tp.Topic)
  127. partitionSuffix := fmt.Sprintf(".part%02d", tp.Partition)
  128. return filer_pb.List(broker, topicDir, "", func(dayEntry *filer_pb.Entry, isLast bool) error {
  129. dayDir := fmt.Sprintf("%s/%s", topicDir, dayEntry.Name)
  130. return filer_pb.List(broker, dayDir, "", func(hourMinuteEntry *filer_pb.Entry, isLast bool) error {
  131. if dayEntry.Name == startDate {
  132. if strings.Compare(hourMinuteEntry.Name, startHourMinute) < 0 {
  133. return nil
  134. }
  135. }
  136. if !strings.HasSuffix(hourMinuteEntry.Name, partitionSuffix) {
  137. return nil
  138. }
  139. // println("partition", tp.Partition, "processing", dayDir, "/", hourMinuteEntry.Name)
  140. chunkedFileReader := filer.NewChunkStreamReader(broker, hourMinuteEntry.Chunks)
  141. defer chunkedFileReader.Close()
  142. if _, err := filer.ReadEachLogEntry(chunkedFileReader, sizeBuf, startTsNs, eachLogEntryFn); err != nil {
  143. chunkedFileReader.Close()
  144. if err == io.EOF {
  145. return err
  146. }
  147. return fmt.Errorf("reading %s/%s: %v", dayDir, hourMinuteEntry.Name, err)
  148. }
  149. return nil
  150. }, "", false, 24*60)
  151. }, startDate, true, 366)
  152. }