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.

145 lines
4.4 KiB

11 months ago
11 months ago
11 months ago
11 months ago
11 months ago
12 months 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
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
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
  1. package topic
  2. import (
  3. "fmt"
  4. "github.com/seaweedfs/seaweedfs/weed/glog"
  5. "github.com/seaweedfs/seaweedfs/weed/pb"
  6. "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
  7. "github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
  8. "google.golang.org/grpc"
  9. "sync"
  10. "sync/atomic"
  11. "time"
  12. )
  13. type LocalPartition struct {
  14. ListenersWaits int64
  15. AckTsNs int64
  16. // notifying clients
  17. ListenersLock sync.Mutex
  18. ListenersCond *sync.Cond
  19. Partition
  20. isLeader bool
  21. FollowerBrokers []pb.ServerAddress
  22. LogBuffer *log_buffer.LogBuffer
  23. ConsumerCount int32
  24. Publishers *LocalPartitionPublishers
  25. Subscribers *LocalPartitionSubscribers
  26. FollowerId int32
  27. FollowerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
  28. FollowerGrpcConnection *grpc.ClientConn
  29. }
  30. var TIME_FORMAT = "2006-01-02-15-04-05"
  31. func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  32. lp := &LocalPartition{
  33. Partition: partition,
  34. isLeader: isLeader,
  35. FollowerBrokers: followerBrokers,
  36. Publishers: NewLocalPartitionPublishers(),
  37. Subscribers: NewLocalPartitionSubscribers(),
  38. }
  39. lp.ListenersCond = sync.NewCond(&lp.ListenersLock)
  40. lp.LogBuffer = log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
  41. 2*time.Minute, logFlushFn, readFromDiskFn, func() {
  42. if atomic.LoadInt64(&lp.ListenersWaits) > 0 {
  43. lp.ListenersCond.Broadcast()
  44. }
  45. })
  46. return lp
  47. }
  48. func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
  49. p.LogBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
  50. }
  51. func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
  52. onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
  53. var processedPosition log_buffer.MessagePosition
  54. var readPersistedLogErr error
  55. var readInMemoryLogErr error
  56. var isDone bool
  57. for {
  58. processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
  59. if readPersistedLogErr != nil {
  60. glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
  61. return readPersistedLogErr
  62. }
  63. if isDone {
  64. return nil
  65. }
  66. startPosition = processedPosition
  67. processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
  68. if isDone {
  69. return nil
  70. }
  71. startPosition = processedPosition
  72. if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
  73. continue
  74. }
  75. if readInMemoryLogErr != nil {
  76. glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
  77. return readInMemoryLogErr
  78. }
  79. }
  80. }
  81. func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
  82. return p.LogBuffer.GetEarliestTime()
  83. }
  84. func (p *LocalPartition) HasData() bool {
  85. return !p.LogBuffer.GetEarliestTime().IsZero()
  86. }
  87. func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
  88. return p.LogBuffer.GetEarliestPosition()
  89. }
  90. func FromPbBrokerPartitionAssignment(self pb.ServerAddress, partition Partition, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
  91. isLeader := assignment.LeaderBroker == string(self)
  92. followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
  93. for i, followerBroker := range assignment.FollowerBrokers {
  94. followers[i] = pb.ServerAddress(followerBroker)
  95. }
  96. return NewLocalPartition(partition, isLeader, followers, logFlushFn, readFromDiskFn)
  97. }
  98. func (p *LocalPartition) closePublishers() {
  99. p.Publishers.SignalShutdown()
  100. }
  101. func (p *LocalPartition) closeSubscribers() {
  102. p.Subscribers.SignalShutdown()
  103. }
  104. func (p *LocalPartition) WaitUntilNoPublishers() {
  105. for {
  106. if p.Publishers.IsEmpty() {
  107. return
  108. }
  109. time.Sleep(113 * time.Millisecond)
  110. }
  111. }
  112. func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
  113. if p.Publishers.IsEmpty() && p.Subscribers.IsEmpty() {
  114. p.LogBuffer.ShutdownLogBuffer()
  115. hasShutdown = true
  116. }
  117. return
  118. }
  119. func (p *LocalPartition) Shutdown() {
  120. p.closePublishers()
  121. p.closeSubscribers()
  122. p.LogBuffer.ShutdownLogBuffer()
  123. atomic.StoreInt32(&p.FollowerId, 0)
  124. glog.V(0).Infof("local partition %v shutting down", p.Partition)
  125. }