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.

213 lines
6.3 KiB

11 months ago
11 months ago
11 months ago
11 months ago
11 months ago
11 months ago
1 year ago
11 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
11 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
2 years 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
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
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
11 months ago
11 months ago
11 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. isLeader bool
  22. FollowerBrokers []pb.ServerAddress
  23. LogBuffer *log_buffer.LogBuffer
  24. ConsumerCount int32
  25. Publishers *LocalPartitionPublishers
  26. Subscribers *LocalPartitionSubscribers
  27. FollowerId int32
  28. FollowerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
  29. FollowerGrpcConnection *grpc.ClientConn
  30. }
  31. var TIME_FORMAT = "2006-01-02-15-04-05"
  32. func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  33. lp := &LocalPartition{
  34. Partition: partition,
  35. isLeader: isLeader,
  36. FollowerBrokers: followerBrokers,
  37. Publishers: NewLocalPartitionPublishers(),
  38. Subscribers: NewLocalPartitionSubscribers(),
  39. }
  40. lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
  41. lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
  42. 2*time.Minute, logFlushFn, readFromDiskFn, func() {
  43. if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
  44. lp.ListenersCond.Broadcast()
  45. }
  46. })
  47. return lp
  48. }
  49. func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
  50. p.LogBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
  51. }
  52. func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
  53. onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
  54. var processedPosition log_buffer.MessagePosition
  55. var readPersistedLogErr error
  56. var readInMemoryLogErr error
  57. var isDone bool
  58. for {
  59. processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
  60. if readPersistedLogErr != nil {
  61. glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
  62. return readPersistedLogErr
  63. }
  64. if isDone {
  65. return nil
  66. }
  67. startPosition = processedPosition
  68. processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
  69. if isDone {
  70. return nil
  71. }
  72. startPosition = processedPosition
  73. if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
  74. continue
  75. }
  76. if readInMemoryLogErr != nil {
  77. glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
  78. return readInMemoryLogErr
  79. }
  80. }
  81. }
  82. func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
  83. return p.LogBuffer.GetEarliestTime()
  84. }
  85. func (p *LocalPartition) HasData() bool {
  86. return !p.LogBuffer.GetEarliestTime().IsZero()
  87. }
  88. func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
  89. return p.LogBuffer.GetEarliestPosition()
  90. }
  91. func FromPbBrokerPartitionAssignment(self pb.ServerAddress, partition Partition, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  92. isLeader := assignment.LeaderBroker == string(self)
  93. followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
  94. for i, followerBroker := range assignment.FollowerBrokers {
  95. followers[i] = pb.ServerAddress(followerBroker)
  96. }
  97. return NewLocalPartition(partition, isLeader, followers, logFlushFn, readFromDiskFn)
  98. }
  99. func (p *LocalPartition) closePublishers() {
  100. p.Publishers.SignalShutdown()
  101. }
  102. func (p *LocalPartition) closeSubscribers() {
  103. p.Subscribers.SignalShutdown()
  104. }
  105. func (p *LocalPartition) WaitUntilNoPublishers() {
  106. for {
  107. if p.Publishers.IsEmpty() {
  108. return
  109. }
  110. time.Sleep(113 * time.Millisecond)
  111. }
  112. }
  113. func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
  114. if p.FollowerStream != nil {
  115. return nil
  116. }
  117. if len(initMessage.FollowerBrokers) == 0 {
  118. return nil
  119. }
  120. follower := initMessage.FollowerBrokers[0]
  121. ctx := context.Background()
  122. p.FollowerGrpcConnection, err = pb.GrpcDial(ctx, follower, true, grpcDialOption)
  123. if err != nil {
  124. return fmt.Errorf("fail to dial %s: %v", follower, err)
  125. }
  126. followerClient := mq_pb.NewSeaweedMessagingClient(p.FollowerGrpcConnection)
  127. p.FollowerStream, err = followerClient.PublishFollowMe(ctx)
  128. if err != nil {
  129. return fmt.Errorf("fail to create publish client: %v", err)
  130. }
  131. if err = p.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{
  132. Message: &mq_pb.PublishFollowMeRequest_Init{
  133. Init: &mq_pb.PublishFollowMeRequest_InitMessage{
  134. Topic: initMessage.Topic,
  135. Partition: initMessage.Partition,
  136. },
  137. },
  138. }); err != nil {
  139. return err
  140. }
  141. // start receiving ack from follower
  142. go func() {
  143. defer func() {
  144. println("stop receiving ack from follower")
  145. }()
  146. for {
  147. ack, err := p.FollowerStream.Recv()
  148. if err != nil {
  149. glog.Errorf("Error receiving follower ack: %v", err)
  150. return
  151. }
  152. atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
  153. println("recv ack", ack.AckTsNs)
  154. }
  155. }()
  156. return nil
  157. }
  158. func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
  159. if p.MaybeShutdownLocalPartition() {
  160. if p.FollowerStream != nil {
  161. // send close to the follower
  162. if followErr := p.FollowerStream.Send(&mq_pb.PublishFollowMeRequest{
  163. Message: &mq_pb.PublishFollowMeRequest_Close{
  164. Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
  165. },
  166. }); followErr != nil {
  167. glog.Errorf("Error closing follower stream: %v", followErr)
  168. }
  169. println("closing grpcConnection to follower")
  170. p.FollowerGrpcConnection.Close()
  171. }
  172. }
  173. return
  174. }
  175. func (p *LocalPartition) canShutdownLocalPartition() (hasShutdown bool) {
  176. if p.Publishers.IsEmpty() && p.Subscribers.IsEmpty() {
  177. p.LogBuffer.ShutdownLogBuffer()
  178. hasShutdown = true
  179. }
  180. return
  181. }
  182. func (p *LocalPartition) Shutdown() {
  183. p.closePublishers()
  184. p.closeSubscribers()
  185. p.LogBuffer.ShutdownLogBuffer()
  186. atomic.StoreInt32(&p.FollowerId, 0)
  187. glog.V(0).Infof("local partition %v shutting down", p.Partition)
  188. }