coordinator receives unassignment ack

This commit is contained in:
chrislu
2024-05-23 08:23:35 -07:00
parent 73d008fafa
commit d40b350f54
7 changed files with 620 additions and 496 deletions

View File

@@ -56,6 +56,15 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
return err
}
go func() {
for reply := range sub.brokerPartitionAssignmentAckChan {
if err := stream.Send(reply); err != nil {
glog.V(0).Infof("subscriber %s reply: %v", sub.ContentConfig.Topic, err)
return
}
}
}()
// keep receiving messages from the sub coordinator
for {
resp, err := stream.Recv()

View File

@@ -65,6 +65,13 @@ func (sub *TopicSubscriber) startProcessors() {
} else {
glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
}
sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignment{
AckUnAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{
Partition: assigned.Partition,
},
},
}
}(assigned.PartitionAssignment, topicPartition)
}
if unAssignment := message.GetUnAssignment(); unAssignment != nil {

View File

@@ -27,25 +27,27 @@ type OnEachMessageFunc func(key, value []byte) (err error)
type OnCompletionFunc func()
type TopicSubscriber struct {
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
bootstrapBrokers []string
waitForMoreMessage bool
activeProcessors map[topic.Partition]*ProcessorState
activeProcessorsLock sync.Mutex
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
bootstrapBrokers []string
waitForMoreMessage bool
activeProcessors map[topic.Partition]*ProcessorState
activeProcessorsLock sync.Mutex
}
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024),
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
activeProcessors: make(map[topic.Partition]*ProcessorState),
SubscriberConfig: subscriber,
ContentConfig: content,
brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024),
brokerPartitionAssignmentAckChan: make(chan *mq_pb.SubscriberToSubCoordinatorRequest, 1024),
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
activeProcessors: make(map[topic.Partition]*ProcessorState),
}
}