seaweedfs/weed/mq/topic/local_partition.go
Chris Lu 580940bf82
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
2023-12-11 12:05:54 -08:00

79 lines
2.3 KiB
Go

package topic
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"time"
)
type LocalPartition struct {
Partition
isLeader bool
FollowerBrokers []pb.ServerAddress
logBuffer *log_buffer.LogBuffer
ConsumerCount int32
StopPublishersCh chan struct{}
Publishers *LocalPartitionPublishers
StopSubscribersCh chan struct{}
Subscribers *LocalPartitionSubscribers
}
func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition {
return &LocalPartition{
Partition: partition,
isLeader: isLeader,
FollowerBrokers: followerBrokers,
logBuffer: log_buffer.NewLogBuffer(
fmt.Sprintf("%d/%4d-%4d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
2*time.Minute,
func(startTime, stopTime time.Time, buf []byte) {
},
func() {
},
),
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
}
}
type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error
func (p *LocalPartition) Publish(message *mq_pb.DataMessage) {
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
}
func (p *LocalPartition) Subscribe(clientName string, startReadTime time.Time, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) {
p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, onNoMessageFn, eachMessageFn)
}
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition {
isLeader := assignment.LeaderBroker == string(self)
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
for i, followerBroker := range assignment.FollowerBrokers {
followers[i] = pb.ServerAddress(followerBroker)
}
return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers)
}
func (p *LocalPartition) closePublishers() {
p.Publishers.SignalShutdown()
close(p.StopPublishersCh)
}
func (p *LocalPartition) closeSubscribers() {
p.Subscribers.SignalShutdown()
}
func (p *LocalPartition) WaitUntilNoPublishers() {
for {
if p.Publishers.IsEmpty() {
return
}
time.Sleep(113 * time.Millisecond)
}
}