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.

216 lines
5.9 KiB

10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
11 months ago
10 months ago
10 months ago
Merge accumulated changes related to message queue (#5098) * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * tracking topic=>broker * merge * comment
1 year ago
10 months ago
10 months ago
10 months ago
1 year ago
Merge accumulated changes related to message queue (#5098) * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * balance partitions on brokers * prepare topic partition first and then publish, move partition * purge unused APIs * clean up * adjust logs * add BalanceTopics() grpc API * configure topic * configure topic command * refactor * repair missing partitions * sequence of operations to ensure ordering * proto to close publishers and consumers * rename file * topic partition versioned by unixTimeNs * create local topic partition * close publishers * randomize the client name * wait until no publishers * logs * close stop publisher channel * send last ack * comments * comment * comments * support list of brokers * add cli options * Update .gitignore * logs * return io.eof directly * refactor * optionally create topic * refactoring * detect consumer disconnection * sub client wait for more messages * subscribe by time stamp * rename * rename to sub_balancer * rename * adjust comments * rename * fix compilation * rename * rename * SubscriberToSubCoordinator * sticky rebalance * go fmt * add tests * tracking topic=>broker * merge * comment
1 year ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
10 months ago
  1. package topic
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/seaweedfs/seaweedfs/weed/glog"
  6. "github.com/seaweedfs/seaweedfs/weed/pb"
  7. "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
  8. "github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
  9. "google.golang.org/grpc"
  10. "sync"
  11. "sync/atomic"
  12. "time"
  13. )
  14. type LocalPartition struct {
  15. ListenersWaits int64
  16. AckTsNs int64
  17. // notifying clients
  18. ListenersLock sync.Mutex
  19. ListenersCond *sync.Cond
  20. Partition
  21. LogBuffer *log_buffer.LogBuffer
  22. Publishers *LocalPartitionPublishers
  23. Subscribers *LocalPartitionSubscribers
  24. FollowerId int32
  25. followerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
  26. followerGrpcConnection *grpc.ClientConn
  27. follower string
  28. }
  29. var TIME_FORMAT = "2006-01-02-15-04-05"
  30. func NewLocalPartition(partition Partition, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  31. lp := &LocalPartition{
  32. Partition: partition,
  33. Publishers: NewLocalPartitionPublishers(),
  34. Subscribers: NewLocalPartitionSubscribers(),
  35. }
  36. lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
  37. lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
  38. 2*time.Minute, logFlushFn, readFromDiskFn, func() {
  39. if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
  40. lp.ListenersCond.Broadcast()
  41. }
  42. })
  43. return lp
  44. }
  45. func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
  46. p.LogBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
  47. // maybe send to the follower
  48. if p.followerStream != nil {
  49. println("recv", string(message.Key), message.TsNs)
  50. if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
  51. Message: &mq_pb.PublishFollowMeRequest_Data{
  52. Data: message,
  53. },
  54. }); followErr != nil {
  55. return fmt.Errorf("send to follower %s: %v", p.follower, followErr)
  56. }
  57. } else {
  58. atomic.StoreInt64(&p.AckTsNs, message.TsNs)
  59. }
  60. return nil
  61. }
  62. func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
  63. onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
  64. var processedPosition log_buffer.MessagePosition
  65. var readPersistedLogErr error
  66. var readInMemoryLogErr error
  67. var isDone bool
  68. for {
  69. processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
  70. if readPersistedLogErr != nil {
  71. glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
  72. return readPersistedLogErr
  73. }
  74. if isDone {
  75. return nil
  76. }
  77. startPosition = processedPosition
  78. processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
  79. if isDone {
  80. return nil
  81. }
  82. startPosition = processedPosition
  83. if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
  84. continue
  85. }
  86. if readInMemoryLogErr != nil {
  87. glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
  88. return readInMemoryLogErr
  89. }
  90. }
  91. }
  92. func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
  93. return p.LogBuffer.GetEarliestTime()
  94. }
  95. func (p *LocalPartition) HasData() bool {
  96. return !p.LogBuffer.GetEarliestTime().IsZero()
  97. }
  98. func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
  99. return p.LogBuffer.GetEarliestPosition()
  100. }
  101. func (p *LocalPartition) closePublishers() {
  102. p.Publishers.SignalShutdown()
  103. }
  104. func (p *LocalPartition) closeSubscribers() {
  105. p.Subscribers.SignalShutdown()
  106. }
  107. func (p *LocalPartition) WaitUntilNoPublishers() {
  108. for {
  109. if p.Publishers.IsEmpty() {
  110. return
  111. }
  112. time.Sleep(113 * time.Millisecond)
  113. }
  114. }
  115. func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
  116. if p.followerStream != nil {
  117. return nil
  118. }
  119. if len(initMessage.FollowerBrokers) == 0 {
  120. return nil
  121. }
  122. p.follower = initMessage.FollowerBrokers[0]
  123. ctx := context.Background()
  124. p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.follower, true, grpcDialOption)
  125. if err != nil {
  126. return fmt.Errorf("fail to dial %s: %v", p.follower, err)
  127. }
  128. followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
  129. p.followerStream, err = followerClient.PublishFollowMe(ctx)
  130. if err != nil {
  131. return fmt.Errorf("fail to create publish client: %v", err)
  132. }
  133. if err = p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
  134. Message: &mq_pb.PublishFollowMeRequest_Init{
  135. Init: &mq_pb.PublishFollowMeRequest_InitMessage{
  136. Topic: initMessage.Topic,
  137. Partition: initMessage.Partition,
  138. },
  139. },
  140. }); err != nil {
  141. return err
  142. }
  143. // start receiving ack from follower
  144. go func() {
  145. defer func() {
  146. println("stop receiving ack from follower")
  147. }()
  148. for {
  149. ack, err := p.followerStream.Recv()
  150. if err != nil {
  151. glog.Errorf("Error receiving follower ack: %v", err)
  152. return
  153. }
  154. atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
  155. println("recv ack", ack.AckTsNs)
  156. }
  157. }()
  158. return nil
  159. }
  160. func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
  161. if !p.Publishers.IsEmpty() {
  162. return
  163. }
  164. if !p.Subscribers.IsEmpty() {
  165. return
  166. }
  167. p.LogBuffer.ShutdownLogBuffer()
  168. if p.followerStream != nil {
  169. // send close to the follower
  170. if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
  171. Message: &mq_pb.PublishFollowMeRequest_Close{
  172. Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
  173. },
  174. }); followErr != nil {
  175. glog.Errorf("Error closing follower stream: %v", followErr)
  176. }
  177. glog.V(4).Infof("closing grpcConnection to follower")
  178. p.followerGrpcConnection.Close()
  179. p.followerStream = nil
  180. }
  181. return
  182. }
  183. func (p *LocalPartition) Shutdown() {
  184. p.closePublishers()
  185. p.closeSubscribers()
  186. p.LogBuffer.ShutdownLogBuffer()
  187. atomic.StoreInt32(&p.FollowerId, 0)
  188. glog.V(0).Infof("local partition %v shutting down", p.Partition)
  189. }