kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jason Gustafson <ja...@confluent.io>
Subject Re: Consumer group disappears and consumers loops
Date Tue, 01 Dec 2015 18:06:59 GMT
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 to make sure
everything still works with smaller values though.

-Jason



On Tue, Dec 1, 2015 at 2:35 AM, Martin Skøtt <martin.skoett@falconsocial.com
> wrote:

> 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", "30000"
>
> My consumer loop is quite simple as it just calls a domain specific
> service:
>
> while (true) {
>     ConsumerRecords<String, Object> records = consumer.poll(10000);
>     for (ConsumerRecord<String, Object> record : records) {
>         serve.handle(record.topic(), record.value());
>     }
> }
>
> The domain service does a number of things (including lookups in a RDBMS
> and saving to ElasticSearch). In my local test setup a poll will often
> result between 5.000 and 10.000 records and I can easily see the processing
> of those taking more than 30 seconds.
>
> I'll probably take a look at adding some threading to my consumer and add
> more partitions to my topics.
>
> That is all fine, but it doesn't really explain why increasing poll timeout
> made the problem go away :-/
>
> Martin
>
> On 30 November 2015 at 19:30, Jason Gustafson <jason@confluent.io> wrote:
>
> > 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 overridden
> > settings).
> >
> > -Jason
> >
> > On Mon, Nov 30, 2015 at 8:12 AM, Martin Skøtt <
> > martin.skoett@falconsocial.com> wrote:
> >
> > > 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 remained, but less
> > frequent
> > >
> > > Not really sure what is going on, but happy that the problem went away
> > :-)
> > >
> > > Martin
> > >
> > > On 30 November 2015 at 15:33, Martin Skøtt <
> > martin.skoett@falconsocial.com
> > > >
> > > wrote:
> > >
> > > > 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 I
> > > > could debug this further?
> > > >
> > > > Regards,
> > > > Martin
> > > >
> > > >
> > > > On 25 November 2015 at 18:37, Guozhang Wang <wangguoz@gmail.com>
> > wrote:
> > > >
> > > >> 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 release.
> > > >>
> > > >> https://issues.apache.org/jira/browse/KAFKA-2860
> > > >>
> > > >> Guozhang
> > > >>
> > > >>
> > > >>
> > > >> On Wed, Nov 25, 2015 at 6:54 AM, Martin Skøtt <
> > > >> martin.skoett@falconsocial.com> wrote:
> > > >>
> > > >> > 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
> > > >> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator
> > > >> > - Auto offset commit failed: Commit cannot be completed due to
> group
> > > >> > rebalance
> > > >> > [main] ERROR
> > > >> > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator
-
> > > Error
> > > >> > ILLEGAL_GENERATION occurred while committing offsets for group
> > > >> > aaa-bbb-reader
> > > >> > [main] WARN
> > > >> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator
> > > >> > - Auto offset commit failed:
> > > >> > [main] INFO
> > > >> org.apache.kafka.clients.consumer.internals.AbstractCoordinator
> > > >> > - Attempt to join group aaa-bbb-reader failed due to unknown
> member
> > > id,
> > > >> > resetting and retrying.
> > > >> >
> > > >> > And this in the broker log:
> > > >> > [2015-11-25 15:41:01,542] INFO [GroupCoordinator 0]: Preparing
to
> > > >> > restabilize group aaa-bbb-reader with old generation 1
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:01,544] INFO [GroupCoordinator 0]:
> > > >> > Group aaa-bbb-reader generation 1 is dead and removed
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:13,474] INFO [GroupCoordinator 0]: Preparing
to
> > > >> > restabilize group aaa-bbb-reader with old generation 0
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:13,475] INFO [GroupCoordinator 0]: Stabilized
> > > >> > group aaa-bbb-reader generation 1
> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:13,477] INFO [GroupCoordinator 0]: Assignment
> > > received
> > > >> > from leader for group aaa-bbb-reader for generation 1
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:43,478] INFO [GroupCoordinator 0]: Preparing
to
> > > >> > restabilize group aaa-bbb-reader with old generation 1
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> > [2015-11-25 15:41:43,478] INFO [GroupCoordinator 0]:
> > > >> > Group aaa-bbb-reader generation 1 is dead and removed
> > > >> > (kafka.coordinator.GroupCoordinator)
> > > >> >
> > > >> > When this happens the kafka-consumer-groups describe command
keeps
> > > >> saying
> > > >> > that the group no longer exists or is rebalancing. What is
> probably
> > > even
> > > >> > worse is that my consumers appears to be looping constantly
> through
> > > >> > everything written to the topics!?
> > > >> >
> > > >> > Does anyone have any input on what might be happening?
> > > >> >
> > > >> > I'm running 0.9 locally on my laptop using one Zookeeper and
one
> > > broker,
> > > >> > both using the configuration provided in the distribution. I
have
> 13
> > > >> topics
> > > >> > with two partitions each and a replication factor of 1. I run
one
> > > >> producer
> > > >> > and once consumer also on the same machine.
> > > >> >
> > > >> > --
> > > >> > Martin Skøtt
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> -- Guozhang
> > > >>
> > > >
> > > >
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message