You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Jordon Tolotti <jt...@rehearsal.com> on 2018/03/16 15:47:48 UTC

Questions regarding undesired offset reset in Kafka Streams Application

Hello,

I am seeing an issue where I have a single streams app running (so a
consumer group of one) that is subscribed to about 10 topics. If the
streams app gets killed and restarted, many of the offsets for the consumer
group are reset to 0 and a lot of data is unintentionally reprocessed. The
offsets that get reset seem to be random but they usually only affect a few
partitions of the affected topics.

I don't seem to notice this problem if I maintain at least one running
instance of the streams app. For example, if I have a consumer group of
two, and take them down one at a time and update them, the issue is not
present.

Is there any obvious reason that I am missing that might be causing this to
happen? It appears that the app is cleanly shutting down, but if it is not,
could that explain what I am seeing?

Context:

- The streams application is running in Docker
- When a new version is deployed (application-id stays the same though) the
current running container is shut down and a new container is started, so
there is a time when no consumer instance is active.
- The container logs make it seem that the app is cleanly shut down.

Steps I go through to reproduce this issue:

1. Disallow writes to Kafka to ensure that no writes occur during the test
(dev environment)
2. Use kafka-consumer-group.sh script to verify there is a zero lag on all
partitions of all topics
3. Deploy a new version of the application (again the code is updated but
the application-id stays the same) which causes the streams app to die and
then be restarted.
4. Use kafka-consumer-group.sh script to check the lag, which shows high
lags on many topics and partitions


Any help is greatly appreciated. Thanks!
Jordon

Re: Questions regarding undesired offset reset in Kafka Streams Application

Posted by Jordon Tolotti <jt...@rehearsal.com>.
Matthias,

Thank you for the quick response. I was able to verify that was the
problem. I had seen similar solutions but thought that the expiration would
not happen with an active consumer group (but it makes sense considering it
is just another topic). I appreciate the help and timely response very much.

Jordon

On Fri, Mar 16, 2018 at 10:16 AM, Matthias J. Sax <ma...@confluent.io>
wrote:

> Jordon,
>
> not sure, if this applies to your situation, but brokers only maintain
> committed offsets for 24h by default. This offset-retention time is
> applied for each partitions individually and starts when the commit was
> done (ie, offset can expired even if the consumer group is active).
>
> Thus, if you have some partitions for which the application did process
> all data, but no new data arrives for 24h for those partitions, the
> committed offsets expired and on restart, auto.offset.reset triggers.
>
> It's a know issue and there are two KIPs for it already
> (https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 186%3A+Increase+offsets+retention+default+to+7+days
> and
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 211%3A+Revise+Expiration+Semantics+of+Consumer+Group+Offsets).
>
> You can increase the offset retention time by changing the corresponding
> config (cf. KIP-186) in your own setup.
>
> If this does not solve the issue, it would be good to get DEBUG logs of
> your Kafka Streams application to dig into it.
>
>
>
> -Matthias
>
>
>
>
>
> On 3/16/18 8:47 AM, Jordon Tolotti wrote:
> > Hello,
> >
> > I am seeing an issue where I have a single streams app running (so a
> > consumer group of one) that is subscribed to about 10 topics. If the
> > streams app gets killed and restarted, many of the offsets for the
> consumer
> > group are reset to 0 and a lot of data is unintentionally reprocessed.
> The
> > offsets that get reset seem to be random but they usually only affect a
> few
> > partitions of the affected topics.
> >
> > I don't seem to notice this problem if I maintain at least one running
> > instance of the streams app. For example, if I have a consumer group of
> > two, and take them down one at a time and update them, the issue is not
> > present.
> >
> > Is there any obvious reason that I am missing that might be causing this
> to
> > happen? It appears that the app is cleanly shutting down, but if it is
> not,
> > could that explain what I am seeing?
> >
> > Context:
> >
> > - The streams application is running in Docker
> > - When a new version is deployed (application-id stays the same though)
> the
> > current running container is shut down and a new container is started, so
> > there is a time when no consumer instance is active.
> > - The container logs make it seem that the app is cleanly shut down.
> >
> > Steps I go through to reproduce this issue:
> >
> > 1. Disallow writes to Kafka to ensure that no writes occur during the
> test
> > (dev environment)
> > 2. Use kafka-consumer-group.sh script to verify there is a zero lag on
> all
> > partitions of all topics
> > 3. Deploy a new version of the application (again the code is updated but
> > the application-id stays the same) which causes the streams app to die
> and
> > then be restarted.
> > 4. Use kafka-consumer-group.sh script to check the lag, which shows high
> > lags on many topics and partitions
> >
> >
> > Any help is greatly appreciated. Thanks!
> > Jordon
> >
>
>

Re: Questions regarding undesired offset reset in Kafka Streams Application

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Jordon,

not sure, if this applies to your situation, but brokers only maintain
committed offsets for 24h by default. This offset-retention time is
applied for each partitions individually and starts when the commit was
done (ie, offset can expired even if the consumer group is active).

Thus, if you have some partitions for which the application did process
all data, but no new data arrives for 24h for those partitions, the
committed offsets expired and on restart, auto.offset.reset triggers.

It's a know issue and there are two KIPs for it already
(https://cwiki.apache.org/confluence/display/KAFKA/KIP-186%3A+Increase+offsets+retention+default+to+7+days
and
https://cwiki.apache.org/confluence/display/KAFKA/KIP-211%3A+Revise+Expiration+Semantics+of+Consumer+Group+Offsets).

You can increase the offset retention time by changing the corresponding
config (cf. KIP-186) in your own setup.

If this does not solve the issue, it would be good to get DEBUG logs of
your Kafka Streams application to dig into it.



-Matthias





On 3/16/18 8:47 AM, Jordon Tolotti wrote:
> Hello,
> 
> I am seeing an issue where I have a single streams app running (so a
> consumer group of one) that is subscribed to about 10 topics. If the
> streams app gets killed and restarted, many of the offsets for the consumer
> group are reset to 0 and a lot of data is unintentionally reprocessed. The
> offsets that get reset seem to be random but they usually only affect a few
> partitions of the affected topics.
> 
> I don't seem to notice this problem if I maintain at least one running
> instance of the streams app. For example, if I have a consumer group of
> two, and take them down one at a time and update them, the issue is not
> present.
> 
> Is there any obvious reason that I am missing that might be causing this to
> happen? It appears that the app is cleanly shutting down, but if it is not,
> could that explain what I am seeing?
> 
> Context:
> 
> - The streams application is running in Docker
> - When a new version is deployed (application-id stays the same though) the
> current running container is shut down and a new container is started, so
> there is a time when no consumer instance is active.
> - The container logs make it seem that the app is cleanly shut down.
> 
> Steps I go through to reproduce this issue:
> 
> 1. Disallow writes to Kafka to ensure that no writes occur during the test
> (dev environment)
> 2. Use kafka-consumer-group.sh script to verify there is a zero lag on all
> partitions of all topics
> 3. Deploy a new version of the application (again the code is updated but
> the application-id stays the same) which causes the streams app to die and
> then be restarted.
> 4. Use kafka-consumer-group.sh script to check the lag, which shows high
> lags on many topics and partitions
> 
> 
> Any help is greatly appreciated. Thanks!
> Jordon
>