Re: Consumer group disappears and consumers loops

2016-01-12 Thread Phillip Walker
I've done some work in isolating the problem a bit more, running the code normally instead of through my IDE, etc. KIP-41 should resolve part of the problem, but I found potential related issues. The UNKNOWN_MEMBER_ID error does occur, as expected, when a given thread fails to call poll() again

Re: Consumer group disappears and consumers loops

2016-01-08 Thread Jason Gustafson
We have an active KIP which aims to give better options for avoiding this problem: https://cwiki.apache.org/confluence/display/KAFKA/KIP-41%3A+KafkaConsumer+Max+Records. If you have any feedback, the discussion is on the dev list. The pause/resume trick should have also worked. The consumer

Re: Consumer group disappears and consumers loops

2015-12-31 Thread Phillip Walker
I've run into exactly the same problem, and a similar workaround, along with a config change, seems to be working for now, but I can probably add a little detail. Scenario: The code reads from a very high-volume topic on a remote server (with 120 partitions and no SSL or security). I am currently

Re: Consumer group disappears and consumers loops

2015-12-22 Thread Rune Sørensen
Hi, Sorry for the long delay in replying. As for your questions: No we are not using SSL. The problem went away for Martin when he was running against his kafka instance locally on his development machine, but we are still seeing the issue when we run it in our testing environment, where the

Re: Consumer group disappears and consumers loops

2015-12-01 Thread Jason Gustafson
I've been unable to reproduce this issue running locally. Even with a poll timeout of 1 millisecond, it seems to work as expected. It would be helpful to know a little more about your setup. Are you using SSL? Are the brokers remote? Is the network stable? Thanks, Jason On Tue, Dec 1, 2015 at

Re: Consumer group disappears and consumers loops

2015-12-01 Thread Jason Gustafson
Hi Martin, I'm also not sure why the poll timeout would affect this. Perhaps the handler is still doing work (e.g. sending requests) when the record set is empty? As a general rule, I would recommend longer poll timeouts. I've actually tended to use Long.MAX_VALUE myself. I'll have a look just

Re: Consumer group disappears and consumers loops

2015-12-01 Thread Martin Skøtt
Hi Jason, That actually sounds like a very plausible explanation. My current consumer is using the default settings, but I have previously used these (taken from the sample in the Javadoc for the new KafkaConsumer): "auto.commit.interval.ms", "1000" "session.timeout.ms", "3" My consumer

Re: Consumer group disappears and consumers loops

2015-11-30 Thread Martin Skøtt
Well, I made the problem go away, but I'm not sure why it works :-/ Previously I used a time out value of 100 for Consumer.poll(). Increasing it to 10.000 makes the problem go away completely?! I tried other values as well: - 0 problem remained - 3000, same as heartbeat.interval, problem

Re: Consumer group disappears and consumers loops

2015-11-30 Thread Jason Gustafson
Hey Martin, At a glance, it looks like your consumer's session timeout is expiring. This shouldn't happen unless there is a delay between successive calls to poll which is longer than the session timeout. It might help if you include a snippet of your poll loop and your configuration (i.e. any

Re: Consumer group disappears and consumers loops

2015-11-30 Thread Martin Skøtt
Hi Guozhang, I have done some testing with various values of heartbeat.interval.ms and they don't seem to have any influence on the error messages. Running kafka-consumer-groups also continues to return that the consumer groups does not exists or is rebalancing. Do you have any suggestions to how

Re: Consumer group disappears and consumers loops

2015-11-25 Thread Guozhang Wang
Hello Martin, It seems your consumer's heartbeat.interval.ms config value is too small (default is 3 seconds) for your environment, consider increasing it and see if this issue goes away. At the same time, we have some better error handling fixes in trunk which will be included in the next point

Consumer group disappears and consumers loops

2015-11-25 Thread Martin Skøtt
Hi, I'm experiencing some very strange issues with 0.9. I get these log messages from the new consumer: [main] ERROR org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - Error ILLEGAL_GENERATION occurred while committing offsets for group aaa-bbb-reader [main] WARN