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.

240 lines
6.9 KiB

9 months ago
9 months ago
7 months ago
9 months ago
7 months ago
7 months ago
9 months ago
10 months ago
9 months ago
9 months ago
7 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
9 months ago
9 months ago
7 months ago
7 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
9 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
9 months ago
7 months ago
9 months ago
9 months ago
9 months ago
9 months ago
9 months ago
9 months ago
7 months ago
9 months ago
7 months ago
9 months ago
7 months ago
9 months ago
9 months ago
7 months ago
9 months ago
9 months ago
9 months ago
9 months ago
7 months ago
7 months ago
7 months ago
9 months ago
7 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. "google.golang.org/grpc/codes"
  11. "google.golang.org/grpc/status"
  12. "sync"
  13. "sync/atomic"
  14. "time"
  15. )
  16. type LocalPartition struct {
  17. ListenersWaits int64
  18. AckTsNs int64
  19. // notifying clients
  20. ListenersLock sync.Mutex
  21. ListenersCond *sync.Cond
  22. Partition
  23. LogBuffer *log_buffer.LogBuffer
  24. Publishers *LocalPartitionPublishers
  25. Subscribers *LocalPartitionSubscribers
  26. publishFolloweMeStream mq_pb.SeaweedMessaging_PublishFollowMeClient
  27. followerGrpcConnection *grpc.ClientConn
  28. Follower string
  29. }
  30. var TIME_FORMAT = "2006-01-02-15-04-05"
  31. var PartitionGenerationFormat = "v2006-01-02-15-04-05"
  32. func NewLocalPartition(partition Partition, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  33. lp := &LocalPartition{
  34. Partition: partition,
  35. Publishers: NewLocalPartitionPublishers(),
  36. Subscribers: NewLocalPartitionSubscribers(),
  37. }
  38. lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
  39. lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
  40. 2*time.Minute, logFlushFn, readFromDiskFn, func() {
  41. if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
  42. lp.ListenersCond.Broadcast()
  43. }
  44. })
  45. return lp
  46. }
  47. func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
  48. p.LogBuffer.AddToBuffer(message)
  49. // maybe send to the follower
  50. if p.publishFolloweMeStream != nil {
  51. // println("recv", string(message.Key), message.TsNs)
  52. if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
  53. Message: &mq_pb.PublishFollowMeRequest_Data{
  54. Data: message,
  55. },
  56. }); followErr != nil {
  57. return fmt.Errorf("send to follower %s: %v", p.Follower, followErr)
  58. }
  59. } else {
  60. atomic.StoreInt64(&p.AckTsNs, message.TsNs)
  61. }
  62. return nil
  63. }
  64. func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
  65. onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
  66. var processedPosition log_buffer.MessagePosition
  67. var readPersistedLogErr error
  68. var readInMemoryLogErr error
  69. var isDone bool
  70. for {
  71. processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
  72. if readPersistedLogErr != nil {
  73. glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
  74. return readPersistedLogErr
  75. }
  76. if isDone {
  77. return nil
  78. }
  79. startPosition = processedPosition
  80. processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
  81. if isDone {
  82. return nil
  83. }
  84. startPosition = processedPosition
  85. if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
  86. continue
  87. }
  88. if readInMemoryLogErr != nil {
  89. glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
  90. return readInMemoryLogErr
  91. }
  92. }
  93. }
  94. func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
  95. return p.LogBuffer.GetEarliestTime()
  96. }
  97. func (p *LocalPartition) HasData() bool {
  98. return !p.LogBuffer.GetEarliestTime().IsZero()
  99. }
  100. func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
  101. return p.LogBuffer.GetEarliestPosition()
  102. }
  103. func (p *LocalPartition) closePublishers() {
  104. p.Publishers.SignalShutdown()
  105. }
  106. func (p *LocalPartition) closeSubscribers() {
  107. p.Subscribers.SignalShutdown()
  108. }
  109. func (p *LocalPartition) WaitUntilNoPublishers() {
  110. for {
  111. if p.Publishers.Size() == 0 {
  112. return
  113. }
  114. time.Sleep(113 * time.Millisecond)
  115. }
  116. }
  117. func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
  118. if p.publishFolloweMeStream != nil {
  119. return nil
  120. }
  121. if initMessage.FollowerBroker == "" {
  122. return nil
  123. }
  124. p.Follower = initMessage.FollowerBroker
  125. ctx := context.Background()
  126. p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.Follower, true, grpcDialOption)
  127. if err != nil {
  128. return fmt.Errorf("fail to dial %s: %v", p.Follower, err)
  129. }
  130. followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
  131. p.publishFolloweMeStream, err = followerClient.PublishFollowMe(ctx)
  132. if err != nil {
  133. return fmt.Errorf("fail to create publish client: %v", err)
  134. }
  135. if err = p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
  136. Message: &mq_pb.PublishFollowMeRequest_Init{
  137. Init: &mq_pb.PublishFollowMeRequest_InitMessage{
  138. Topic: initMessage.Topic,
  139. Partition: initMessage.Partition,
  140. },
  141. },
  142. }); err != nil {
  143. return err
  144. }
  145. // start receiving ack from follower
  146. go func() {
  147. defer func() {
  148. // println("stop receiving ack from follower")
  149. }()
  150. for {
  151. ack, err := p.publishFolloweMeStream.Recv()
  152. if err != nil {
  153. e, _ := status.FromError(err)
  154. if e.Code() == codes.Canceled {
  155. glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.Follower)
  156. return
  157. }
  158. glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.Follower, err)
  159. return
  160. }
  161. atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
  162. // println("recv ack", ack.AckTsNs)
  163. }
  164. }()
  165. return nil
  166. }
  167. func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
  168. if p.Publishers.Size() == 0 && p.Subscribers.Size() == 0 {
  169. p.LogBuffer.ShutdownLogBuffer()
  170. for !p.LogBuffer.IsAllFlushed() {
  171. time.Sleep(113 * time.Millisecond)
  172. }
  173. if p.publishFolloweMeStream != nil {
  174. // send close to the follower
  175. if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
  176. Message: &mq_pb.PublishFollowMeRequest_Close{
  177. Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
  178. },
  179. }); followErr != nil {
  180. glog.Errorf("Error closing follower stream: %v", followErr)
  181. }
  182. glog.V(4).Infof("closing grpcConnection to follower")
  183. p.followerGrpcConnection.Close()
  184. p.publishFolloweMeStream = nil
  185. p.Follower = ""
  186. }
  187. hasShutdown = true
  188. }
  189. glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.Follower, hasShutdown)
  190. return
  191. }
  192. func (p *LocalPartition) Shutdown() {
  193. p.closePublishers()
  194. p.closeSubscribers()
  195. p.LogBuffer.ShutdownLogBuffer()
  196. glog.V(0).Infof("local partition %v shutting down", p.Partition)
  197. }
  198. func (p *LocalPartition) NotifyLogFlushed(flushTsNs int64) {
  199. if p.publishFolloweMeStream != nil {
  200. if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
  201. Message: &mq_pb.PublishFollowMeRequest_Flush{
  202. Flush: &mq_pb.PublishFollowMeRequest_FlushMessage{
  203. TsNs: flushTsNs,
  204. },
  205. },
  206. }); followErr != nil {
  207. glog.Errorf("send follower %s flush message: %v", p.Follower, followErr)
  208. }
  209. // println("notifying", p.Follower, "flushed at", flushTsNs)
  210. }
  211. }