[ 
https://issues.apache.org/jira/browse/KAFKA-1740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14620788#comment-14620788
 ] 

Onur Karaman commented on KAFKA-1740:
-------------------------------------

Here are two scenarios from 3f8480ccfb011eb43da774737597c597f703e11b which 
cause issues:

Let the following be a sample consumer:
{code}
KafkaConsumer<String, String> consumer = new KafkaConsumer<String, 
String>(props);
consumer.subscribe("topic1", "topic2");
while (true) {
  ConsumerRecords<String, String> records = consumer.poll(100);
  for (ConsumerRecord<String, String> record : records) {
    System.out.printf("topic = %s, partition = %d, offset = %d, key = %s, value 
= %s",
        record.topic(), record.partition(), record.offset(), record.key(), 
record.value());
  }
}
{code}
1. Start the consumer. Then expand the partitions for topic1. The coordinator 
logs will show the state going from Stable -> PreparingRebalance -> Dead -> 
Stable (because the consmer is running in a loop and will rejoin). This state 
will now be steady.

2. Start consumer c1. Wait for the group to be Stable. Start consumer c2. The 
coordinator logs will show the state forever loop from Stable -> 
PreparingRebalance -> Rebalancing -> Stable -> PreparingRebalance -> 
Rebalancing -> Stable. Basically c1 joining causes c2 to timeout and c2 joining 
causes c1 to timeout.

> Merge Offset manager into Coordinator
> -------------------------------------
>
>                 Key: KAFKA-1740
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1740
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: consumer
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>            Priority: Critical
>             Fix For: 0.8.3
>
>         Attachments: KAFKA-1740.patch, KAFKA-1740_2015-06-29_18:21:42.patch, 
> KAFKA-1740_2015-06-29_18:44:54.patch
>
>
> This JIRA involves refactoring offset manager and merge it into coordinator, 
> including adding the logic for consumer-id / generation-id checking.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to