mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-11-24 16:53:14 +08:00
refactor
ToDo 1. handle unassignment on client side 2. get notified of completed unassignment 3. in consumer_group.go, balance and send unassignment
This commit is contained in:
@@ -30,37 +30,42 @@ func (sub *TopicSubscriber) startProcessors() {
|
||||
var wg sync.WaitGroup
|
||||
semaphore := make(chan struct{}, sub.SubscriberConfig.MaxPartitionCount)
|
||||
|
||||
for assigned := range sub.brokerPartitionAssignmentChan {
|
||||
wg.Add(1)
|
||||
semaphore <- struct{}{}
|
||||
for message := range sub.brokerPartitionAssignmentChan {
|
||||
if assigned := message.GetAssignment(); assigned != nil {
|
||||
wg.Add(1)
|
||||
semaphore <- struct{}{}
|
||||
|
||||
topicPartition := topic.FromPbPartition(assigned.Partition)
|
||||
topicPartition := topic.FromPbPartition(assigned.PartitionAssignment.Partition)
|
||||
|
||||
// wait until no covering partition is still in progress
|
||||
sub.waitUntilNoOverlappingPartitionInFlight(topicPartition)
|
||||
// wait until no covering partition is still in progress
|
||||
sub.waitUntilNoOverlappingPartitionInFlight(topicPartition)
|
||||
|
||||
// start a processors
|
||||
sub.activeProcessorsLock.Lock()
|
||||
sub.activeProcessors[topicPartition] = &ProcessorState{}
|
||||
sub.activeProcessorsLock.Unlock()
|
||||
// start a processors
|
||||
sub.activeProcessorsLock.Lock()
|
||||
sub.activeProcessors[topicPartition] = &ProcessorState{}
|
||||
sub.activeProcessorsLock.Unlock()
|
||||
|
||||
go func(assigned *mq_pb.BrokerPartitionAssignment, topicPartition topic.Partition) {
|
||||
defer func() {
|
||||
sub.activeProcessorsLock.Lock()
|
||||
delete(sub.activeProcessors, topicPartition)
|
||||
sub.activeProcessorsLock.Unlock()
|
||||
go func(assigned *mq_pb.BrokerPartitionAssignment, topicPartition topic.Partition) {
|
||||
defer func() {
|
||||
sub.activeProcessorsLock.Lock()
|
||||
delete(sub.activeProcessors, topicPartition)
|
||||
sub.activeProcessorsLock.Unlock()
|
||||
|
||||
<-semaphore
|
||||
wg.Done()
|
||||
}()
|
||||
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
||||
err := sub.onEachPartition(assigned)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
|
||||
} else {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
||||
}
|
||||
}(assigned, topicPartition)
|
||||
<-semaphore
|
||||
wg.Done()
|
||||
}()
|
||||
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
||||
err := sub.onEachPartition(assigned)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
|
||||
} else {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
||||
}
|
||||
}(assigned.PartitionAssignment, topicPartition)
|
||||
}
|
||||
if unAssignment := message.GetUnAssignment(); unAssignment != nil {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
|
||||
@@ -29,7 +29,7 @@ type OnCompletionFunc func()
|
||||
type TopicSubscriber struct {
|
||||
SubscriberConfig *SubscriberConfiguration
|
||||
ContentConfig *ContentConfiguration
|
||||
brokerPartitionAssignmentChan chan *mq_pb.BrokerPartitionAssignment
|
||||
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||
OnEachMessageFunc OnEachMessageFunc
|
||||
OnCompletionFunc OnCompletionFunc
|
||||
bootstrapBrokers []string
|
||||
@@ -42,7 +42,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
|
||||
return &TopicSubscriber{
|
||||
SubscriberConfig: subscriber,
|
||||
ContentConfig: content,
|
||||
brokerPartitionAssignmentChan: make(chan *mq_pb.BrokerPartitionAssignment, 1024),
|
||||
brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024),
|
||||
bootstrapBrokers: bootstrapBrokers,
|
||||
waitForMoreMessage: true,
|
||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||
|
||||
@@ -131,27 +131,24 @@ func (cg *ConsumerGroup) BalanceConsumerGroupInstances(knownPartitionSlotToBroke
|
||||
partitionSlots = make([]*PartitionSlotToConsumerInstance, 0)
|
||||
}
|
||||
consumerGroupInstance.Partitions = ToPartitions(partitionSlotToBrokerList.RingSize, partitionSlots)
|
||||
assignedPartitions := make([]*mq_pb.BrokerPartitionAssignment, len(partitionSlots))
|
||||
for i, partitionSlot := range partitionSlots {
|
||||
assignedPartitions[i] = &mq_pb.BrokerPartitionAssignment{
|
||||
Partition: &mq_pb.Partition{
|
||||
RangeStop: partitionSlot.RangeStop,
|
||||
RangeStart: partitionSlot.RangeStart,
|
||||
RingSize: partitionSlotToBrokerList.RingSize,
|
||||
UnixTimeNs: partitionSlot.UnixTimeNs,
|
||||
for _, partitionSlot := range partitionSlots {
|
||||
consumerGroupInstance.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
|
||||
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
|
||||
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
|
||||
PartitionAssignment: &mq_pb.BrokerPartitionAssignment{
|
||||
Partition: &mq_pb.Partition{
|
||||
RangeStop: partitionSlot.RangeStop,
|
||||
RangeStart: partitionSlot.RangeStart,
|
||||
RingSize: partitionSlotToBrokerList.RingSize,
|
||||
UnixTimeNs: partitionSlot.UnixTimeNs,
|
||||
},
|
||||
LeaderBroker: partitionSlot.Broker,
|
||||
FollowerBroker: partitionSlot.FollowerBroker,
|
||||
},
|
||||
},
|
||||
},
|
||||
LeaderBroker: partitionSlot.Broker,
|
||||
FollowerBroker: partitionSlot.FollowerBroker,
|
||||
}
|
||||
}
|
||||
response := &mq_pb.SubscriberToSubCoordinatorResponse{
|
||||
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
|
||||
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
|
||||
PartitionAssignments: assignedPartitions,
|
||||
},
|
||||
},
|
||||
}
|
||||
consumerGroupInstance.ResponseChan <- response
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user