Skip to content
This repository has been archived by the owner on May 13, 2019. It is now read-only.

Trigger rebalance after claim partition failed #103

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
18 changes: 14 additions & 4 deletions consumergroup/consumer_group.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,10 +264,11 @@ func (cg *ConsumerGroup) topicListConsumer(topics []string) {
cg.Logf("Currently registered consumers: %d\n", len(cg.consumers))

stopper := make(chan struct{})
claimPartitionFailed := make(chan struct{})

for _, topic := range topics {
cg.wg.Add(1)
go cg.topicConsumer(topic, cg.messages, cg.errors, stopper)
go cg.topicConsumer(topic, cg.messages, cg.errors, stopper, claimPartitionFailed)
}

select {
Expand All @@ -291,11 +292,16 @@ func (cg *ConsumerGroup) topicListConsumer(topics []string) {
cg.Logf("Triggering rebalance due to consumer list change\n")
close(stopper)
cg.wg.Wait()

case <-claimPartitionFailed:
cg.Logf("Triggering rebalance due to claim partition failed\n")
close(stopper)
cg.wg.Wait()
}
}
}

func (cg *ConsumerGroup) topicConsumer(topic string, messages chan<- *sarama.ConsumerMessage, errors chan<- *sarama.ConsumerError, stopper <-chan struct{}) {
func (cg *ConsumerGroup) topicConsumer(topic string, messages chan<- *sarama.ConsumerMessage, errors chan<- *sarama.ConsumerError, stopper <-chan struct{}, claimPartitionFailed chan<- struct{}) {
defer cg.wg.Done()

select {
Expand Down Expand Up @@ -338,7 +344,7 @@ func (cg *ConsumerGroup) topicConsumer(topic string, messages chan<- *sarama.Con
for _, pid := range myPartitions {

wg.Add(1)
go cg.partitionConsumer(topic, pid.ID, messages, errors, &wg, stopper)
go cg.partitionConsumer(topic, pid.ID, messages, errors, &wg, stopper, claimPartitionFailed)
}

wg.Wait()
Expand Down Expand Up @@ -370,7 +376,7 @@ func (cg *ConsumerGroup) consumePartition(topic string, partition int32, nextOff
}

// Consumes a partition
func (cg *ConsumerGroup) partitionConsumer(topic string, partition int32, messages chan<- *sarama.ConsumerMessage, errors chan<- *sarama.ConsumerError, wg *sync.WaitGroup, stopper <-chan struct{}) {
func (cg *ConsumerGroup) partitionConsumer(topic string, partition int32, messages chan<- *sarama.ConsumerMessage, errors chan<- *sarama.ConsumerError, wg *sync.WaitGroup, stopper <-chan struct{}, claimPartitionFailed chan<- struct{}) {
defer wg.Done()

select {
Expand All @@ -386,6 +392,10 @@ func (cg *ConsumerGroup) partitionConsumer(topic string, partition int32, messag
time.Sleep(1 * time.Second)
} else {
cg.Logf("%s/%d :: FAILED to claim the partition: %s\n", topic, partition, err)
select {
case claimPartitionFailed <- struct{}{}:
default:
}
return
}
}
Expand Down