Re: Offset for consumer group set to Null in __consumer_offsets, leads to data loss

2018-08-27 Thread Satish Duggana
In Kafka 2.0, default offsets retention period is changed to 7 days[a] and
better expiration semantics of consumer groups was added as part of
KIP-211[b].


a - https://kafka.apache.org/documentation/#upgrade_200_notable
b -
https://cwiki.apache.org/confluence/display/KAFKA/KIP-211%3A+Revise+Expiration+Semantics+of+Consumer+Group+Offsets


On Mon, Aug 27, 2018 at 7:26 PM, Biplob Biswas 
wrote:

> Hi Manikumar,
>
> Thanks for the reply, it seems you caught the correct nerve as we never set
> this property, and if kafka is taking the default value then it would mean
> that the consumer group would expire in a day, which completely explains
> our behaviour.
>
> Thanks a lot.  We will test this and verify and report back.
>
> Thanks & Regards
> Biplob Biswas
>
>
> On Mon, Aug 27, 2018 at 10:12 AM Manikumar 
> wrote:
>
> > What is the value set for offsets.retention.minutes?  Default value is
> > `1day` in Kafka 0.11 release.
> > Based on your requirement, you may need to increase this value.
> >
> > On Mon, Aug 27, 2018 at 1:33 PM Biplob Biswas 
> > wrote:
> >
> > > Hi,
> > >
> > > Can anyone provide any insights to the issue we faced?
> > >
> > > Thanks & Regards
> > > Biplob Biswas
> > >
> > >
> > > On Fri, Aug 24, 2018 at 4:50 PM Biplob Biswas <
> revolutioni...@gmail.com>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > *Short Version:*
> > > > We had an unusual situation where after restart of our spark job,
> > rather
> > > > than reading from the last known offset of the consumer group, it
> > started
> > > > consuming from the latest offset, thus leading to data loss.
> > > >
> > > > *Long Version:*
> > > > We have a spark job which crashed last Friday due to an NPE
> introduced
> > by
> > > > developer code. Over the weekend the job was not restarted, although
> on
> > > > monday the bug was fixed and the job was restarted. The
> consumer-group
> > > was
> > > > *NOT* changed, but when we restarted the job, the spark job started
> > > > reading form the latest offset.
> > > >
> > > > We started investigating the reasons for the same and we verified our
> > > > spark job configuration and it looks good, we tried to reproduce the
> > > issue
> > > > by killing the job and restarting while new events are coming in and
> it
> > > > reads the buffered data properly. We also verified that the consumer
> > > group
> > > > had a valid lag which was reducing over time.
> > > >
> > > > Next we checked __consumer_offsets topic and we found the something
> > which
> > > > could explain why spark job started reading fro latest. We found the
> > > > following lines which were not compressed by the topic itself
> > > >
> > > > [spark-job-consumer-group,spark-job-reading-topic,1]::NULL
> > > >> [spark-job-consumer-group,spark-job-reading-topic,3]::NULL
> > > >> [spark-job-consumer-group,spark-job-reading-topic,0]::NULL
> > > >> [spark-job-consumer-group,spark-job-reading-topic,2]::NULL
> > > >
> > > > --
> > > > These following offset commits came after we started ingesting again
> on
> > > > monday after bug fix:
> > > >
> > > >
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,3]::[
> OffsetMetadata[606979,NO_METADATA],CommitTime
> > > >> 1534778505013,ExpirationTime 1534864905013]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,1]::[
> OffsetMetadata[607195,NO_METADATA],CommitTime
> > > >> 1534778505013,ExpirationTime 1534864905013]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,2]::[
> OffsetMetadata[607100,NO_METADATA],CommitTime
> > > >> 1534778505013,ExpirationTime 1534864905013]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,0]::[
> OffsetMetadata[607330,NO_METADATA],CommitTime
> > > >> 1534778505013,ExpirationTime 1534864905013]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,3]::[
> OffsetMetadata[606979,NO_METADATA],CommitTime
> > > >> 1534778508010,ExpirationTime 1534864908010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,1]::[
> OffsetMetadata[607195,NO_METADATA],CommitTime
> > > >> 1534778508010,ExpirationTime 1534864908010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,2]::[
> OffsetMetadata[607100,NO_METADATA],CommitTime
> > > >> 1534778508010,ExpirationTime 1534864908010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,0]::[
> OffsetMetadata[607330,NO_METADATA],CommitTime
> > > >> 1534778508010,ExpirationTime 1534864908010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,3]::[
> OffsetMetadata[606979,NO_METADATA],CommitTime
> > > >> 1534778511010,ExpirationTime 1534864911010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,1]::[
> OffsetMetadata[607195,NO_METADATA],CommitTime
> > > >> 1534778511010,ExpirationTime 1534864911010]
> > > >>
> > >
> > [spark-job-consumer-group,spark-job-reading-topic,2]::[
> OffsetMetadata[607100,NO_METADATA],CommitTime
> > > >> 1534778511010,ExpirationTime 

Re: Offset for consumer group set to Null in __consumer_offsets, leads to data loss

2018-08-27 Thread Biplob Biswas
Hi Manikumar,

Thanks for the reply, it seems you caught the correct nerve as we never set
this property, and if kafka is taking the default value then it would mean
that the consumer group would expire in a day, which completely explains
our behaviour.

Thanks a lot.  We will test this and verify and report back.

Thanks & Regards
Biplob Biswas


On Mon, Aug 27, 2018 at 10:12 AM Manikumar 
wrote:

> What is the value set for offsets.retention.minutes?  Default value is
> `1day` in Kafka 0.11 release.
> Based on your requirement, you may need to increase this value.
>
> On Mon, Aug 27, 2018 at 1:33 PM Biplob Biswas 
> wrote:
>
> > Hi,
> >
> > Can anyone provide any insights to the issue we faced?
> >
> > Thanks & Regards
> > Biplob Biswas
> >
> >
> > On Fri, Aug 24, 2018 at 4:50 PM Biplob Biswas 
> > wrote:
> >
> > > Hi everyone,
> > >
> > > *Short Version:*
> > > We had an unusual situation where after restart of our spark job,
> rather
> > > than reading from the last known offset of the consumer group, it
> started
> > > consuming from the latest offset, thus leading to data loss.
> > >
> > > *Long Version:*
> > > We have a spark job which crashed last Friday due to an NPE introduced
> by
> > > developer code. Over the weekend the job was not restarted, although on
> > > monday the bug was fixed and the job was restarted. The consumer-group
> > was
> > > *NOT* changed, but when we restarted the job, the spark job started
> > > reading form the latest offset.
> > >
> > > We started investigating the reasons for the same and we verified our
> > > spark job configuration and it looks good, we tried to reproduce the
> > issue
> > > by killing the job and restarting while new events are coming in and it
> > > reads the buffered data properly. We also verified that the consumer
> > group
> > > had a valid lag which was reducing over time.
> > >
> > > Next we checked __consumer_offsets topic and we found the something
> which
> > > could explain why spark job started reading fro latest. We found the
> > > following lines which were not compressed by the topic itself
> > >
> > > [spark-job-consumer-group,spark-job-reading-topic,1]::NULL
> > >> [spark-job-consumer-group,spark-job-reading-topic,3]::NULL
> > >> [spark-job-consumer-group,spark-job-reading-topic,0]::NULL
> > >> [spark-job-consumer-group,spark-job-reading-topic,2]::NULL
> > >
> > > --
> > > These following offset commits came after we started ingesting again on
> > > monday after bug fix:
> > >
> > >
> >
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> > >> 1534778505013,ExpirationTime 1534864905013]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> > >> 1534778505013,ExpirationTime 1534864905013]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> > >> 1534778505013,ExpirationTime 1534864905013]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> > >> 1534778505013,ExpirationTime 1534864905013]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> > >> 1534778508010,ExpirationTime 1534864908010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> > >> 1534778508010,ExpirationTime 1534864908010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> > >> 1534778508010,ExpirationTime 1534864908010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> > >> 1534778508010,ExpirationTime 1534864908010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> > >> 1534778511010,ExpirationTime 1534864911010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> > >> 1534778511010,ExpirationTime 1534864911010]
> > >>
> >
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> > >> 1534778511010,ExpirationTime 1534864911010]
> > >
> > >
> > >
> > > But this only showed the reason of the behaviour not the cause, so we
> > > looked into kafka logs -  during the time from when our kafka job died
> > and
> > > when we restarted, we found a few exception although these doesn't look
> > > like that it could be the root cause but I still wanted to make sure:
> > >
> > >>
> > >> 2018-08-20 09:01:20,016 INFO kafka.controller.KafkaController:
> > >> [Controller 180]: Starting preferred replica leader election for
> > partitions
> > >> [__consumer_offsets,2]
> > >> 2018-08-20 09:01:20,016 INFO kafka.controller.PartitionStateMachine:
> > >> [Partition state machine on Controller 180]: Invoking state change to
> > 

Re: Offset for consumer group set to Null in __consumer_offsets, leads to data loss

2018-08-27 Thread Manikumar
What is the value set for offsets.retention.minutes?  Default value is
`1day` in Kafka 0.11 release.
Based on your requirement, you may need to increase this value.

On Mon, Aug 27, 2018 at 1:33 PM Biplob Biswas 
wrote:

> Hi,
>
> Can anyone provide any insights to the issue we faced?
>
> Thanks & Regards
> Biplob Biswas
>
>
> On Fri, Aug 24, 2018 at 4:50 PM Biplob Biswas 
> wrote:
>
> > Hi everyone,
> >
> > *Short Version:*
> > We had an unusual situation where after restart of our spark job, rather
> > than reading from the last known offset of the consumer group, it started
> > consuming from the latest offset, thus leading to data loss.
> >
> > *Long Version:*
> > We have a spark job which crashed last Friday due to an NPE introduced by
> > developer code. Over the weekend the job was not restarted, although on
> > monday the bug was fixed and the job was restarted. The consumer-group
> was
> > *NOT* changed, but when we restarted the job, the spark job started
> > reading form the latest offset.
> >
> > We started investigating the reasons for the same and we verified our
> > spark job configuration and it looks good, we tried to reproduce the
> issue
> > by killing the job and restarting while new events are coming in and it
> > reads the buffered data properly. We also verified that the consumer
> group
> > had a valid lag which was reducing over time.
> >
> > Next we checked __consumer_offsets topic and we found the something which
> > could explain why spark job started reading fro latest. We found the
> > following lines which were not compressed by the topic itself
> >
> > [spark-job-consumer-group,spark-job-reading-topic,1]::NULL
> >> [spark-job-consumer-group,spark-job-reading-topic,3]::NULL
> >> [spark-job-consumer-group,spark-job-reading-topic,0]::NULL
> >> [spark-job-consumer-group,spark-job-reading-topic,2]::NULL
> >
> > --
> > These following offset commits came after we started ingesting again on
> > monday after bug fix:
> >
> >
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> >> 1534778505013,ExpirationTime 1534864905013]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> >> 1534778505013,ExpirationTime 1534864905013]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> >> 1534778505013,ExpirationTime 1534864905013]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> >> 1534778505013,ExpirationTime 1534864905013]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> >> 1534778508010,ExpirationTime 1534864908010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> >> 1534778508010,ExpirationTime 1534864908010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> >> 1534778508010,ExpirationTime 1534864908010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> >> 1534778508010,ExpirationTime 1534864908010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> >> 1534778511010,ExpirationTime 1534864911010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> >> 1534778511010,ExpirationTime 1534864911010]
> >>
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> >> 1534778511010,ExpirationTime 1534864911010]
> >
> >
> >
> > But this only showed the reason of the behaviour not the cause, so we
> > looked into kafka logs -  during the time from when our kafka job died
> and
> > when we restarted, we found a few exception although these doesn't look
> > like that it could be the root cause but I still wanted to make sure:
> >
> >>
> >> 2018-08-20 09:01:20,016 INFO kafka.controller.KafkaController:
> >> [Controller 180]: Starting preferred replica leader election for
> partitions
> >> [__consumer_offsets,2]
> >> 2018-08-20 09:01:20,016 INFO kafka.controller.PartitionStateMachine:
> >> [Partition state machine on Controller 180]: Invoking state change to
> >> OnlinePartition for partitions [__consumer_offsets,2]
> >> 2018-08-20 09:01:20,019 INFO
> >> kafka.controller.PreferredReplicaPartitionLeaderSelector:
> >> [PreferredReplicaPartitionLeaderSelector]: Current leader 180 for
> partition
> >> [__consumer_offsets,2] is not the preferred replica. Triggering
> preferred
> >> replica leader election
> >> 2018-08-20 09:01:20,019 ERROR state.change.logger: Controller 180 epoch
> >> 37 encountered error while electing leader for partition
> >> [__consumer_offsets,2] due to: Preferred replica 182 for partition
> >> [__consumer_offsets,2] is either not alive or not in the isr. Current
> >> 

Re: Offset for consumer group set to Null in __consumer_offsets, leads to data loss

2018-08-27 Thread Biplob Biswas
Hi,

Can anyone provide any insights to the issue we faced?

Thanks & Regards
Biplob Biswas


On Fri, Aug 24, 2018 at 4:50 PM Biplob Biswas 
wrote:

> Hi everyone,
>
> *Short Version:*
> We had an unusual situation where after restart of our spark job, rather
> than reading from the last known offset of the consumer group, it started
> consuming from the latest offset, thus leading to data loss.
>
> *Long Version:*
> We have a spark job which crashed last Friday due to an NPE introduced by
> developer code. Over the weekend the job was not restarted, although on
> monday the bug was fixed and the job was restarted. The consumer-group was
> *NOT* changed, but when we restarted the job, the spark job started
> reading form the latest offset.
>
> We started investigating the reasons for the same and we verified our
> spark job configuration and it looks good, we tried to reproduce the issue
> by killing the job and restarting while new events are coming in and it
> reads the buffered data properly. We also verified that the consumer group
> had a valid lag which was reducing over time.
>
> Next we checked __consumer_offsets topic and we found the something which
> could explain why spark job started reading fro latest. We found the
> following lines which were not compressed by the topic itself
>
> [spark-job-consumer-group,spark-job-reading-topic,1]::NULL
>> [spark-job-consumer-group,spark-job-reading-topic,3]::NULL
>> [spark-job-consumer-group,spark-job-reading-topic,0]::NULL
>> [spark-job-consumer-group,spark-job-reading-topic,2]::NULL
>
> --
> These following offset commits came after we started ingesting again on
> monday after bug fix:
>
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
>> 1534778505013,ExpirationTime 1534864905013]
>> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
>> 1534778505013,ExpirationTime 1534864905013]
>> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
>> 1534778505013,ExpirationTime 1534864905013]
>> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
>> 1534778505013,ExpirationTime 1534864905013]
>> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
>> 1534778508010,ExpirationTime 1534864908010]
>> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
>> 1534778508010,ExpirationTime 1534864908010]
>> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
>> 1534778508010,ExpirationTime 1534864908010]
>> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
>> 1534778508010,ExpirationTime 1534864908010]
>> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
>> 1534778511010,ExpirationTime 1534864911010]
>> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
>> 1534778511010,ExpirationTime 1534864911010]
>> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
>> 1534778511010,ExpirationTime 1534864911010]
>
>
>
> But this only showed the reason of the behaviour not the cause, so we
> looked into kafka logs -  during the time from when our kafka job died and
> when we restarted, we found a few exception although these doesn't look
> like that it could be the root cause but I still wanted to make sure:
>
>>
>> 2018-08-20 09:01:20,016 INFO kafka.controller.KafkaController:
>> [Controller 180]: Starting preferred replica leader election for partitions
>> [__consumer_offsets,2]
>> 2018-08-20 09:01:20,016 INFO kafka.controller.PartitionStateMachine:
>> [Partition state machine on Controller 180]: Invoking state change to
>> OnlinePartition for partitions [__consumer_offsets,2]
>> 2018-08-20 09:01:20,019 INFO
>> kafka.controller.PreferredReplicaPartitionLeaderSelector:
>> [PreferredReplicaPartitionLeaderSelector]: Current leader 180 for partition
>> [__consumer_offsets,2] is not the preferred replica. Triggering preferred
>> replica leader election
>> 2018-08-20 09:01:20,019 ERROR state.change.logger: Controller 180 epoch
>> 37 encountered error while electing leader for partition
>> [__consumer_offsets,2] due to: Preferred replica 182 for partition
>> [__consumer_offsets,2] is either not alive or not in the isr. Current
>> leader and ISR: [{"leader":180,"leader_epoch":2,"isr":[181,180]}].
>> 2018-08-20 09:01:20,019 ERROR state.change.logger: Controller 180 epoch
>> 37 initiated state change for partition [__consumer_offsets,2] from
>> OnlinePartition to OnlinePartition failed
>> kafka.common.StateChangeFailedException: encountered error while electing
>> leader for partition [__consumer_offsets,2] due to: Preferred replica 182
>> for partition 

Offset for consumer group set to Null in __consumer_offsets, leads to data loss

2018-08-24 Thread Biplob Biswas
Hi everyone,

*Short Version:*
We had an unusual situation where after restart of our spark job, rather
than reading from the last known offset of the consumer group, it started
consuming from the latest offset, thus leading to data loss.

*Long Version:*
We have a spark job which crashed last Friday due to an NPE introduced by
developer code. Over the weekend the job was not restarted, although on
monday the bug was fixed and the job was restarted. The consumer-group was
*NOT* changed, but when we restarted the job, the spark job started reading
form the latest offset.

We started investigating the reasons for the same and we verified our spark
job configuration and it looks good, we tried to reproduce the issue by
killing the job and restarting while new events are coming in and it reads
the buffered data properly. We also verified that the consumer group had a
valid lag which was reducing over time.

Next we checked __consumer_offsets topic and we found the something which
could explain why spark job started reading fro latest. We found the
following lines which were not compressed by the topic itself

[spark-job-consumer-group,spark-job-reading-topic,1]::NULL
> [spark-job-consumer-group,spark-job-reading-topic,3]::NULL
> [spark-job-consumer-group,spark-job-reading-topic,0]::NULL
> [spark-job-consumer-group,spark-job-reading-topic,2]::NULL

--
These following offset commits came after we started ingesting again on
monday after bug fix:

[spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> 1534778505013,ExpirationTime 1534864905013]
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> 1534778505013,ExpirationTime 1534864905013]
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> 1534778505013,ExpirationTime 1534864905013]
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> 1534778505013,ExpirationTime 1534864905013]
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> 1534778508010,ExpirationTime 1534864908010]
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> 1534778508010,ExpirationTime 1534864908010]
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> 1534778508010,ExpirationTime 1534864908010]
> [spark-job-consumer-group,spark-job-reading-topic,0]::[OffsetMetadata[607330,NO_METADATA],CommitTime
> 1534778508010,ExpirationTime 1534864908010]
> [spark-job-consumer-group,spark-job-reading-topic,3]::[OffsetMetadata[606979,NO_METADATA],CommitTime
> 1534778511010,ExpirationTime 1534864911010]
> [spark-job-consumer-group,spark-job-reading-topic,1]::[OffsetMetadata[607195,NO_METADATA],CommitTime
> 1534778511010,ExpirationTime 1534864911010]
> [spark-job-consumer-group,spark-job-reading-topic,2]::[OffsetMetadata[607100,NO_METADATA],CommitTime
> 1534778511010,ExpirationTime 1534864911010]



But this only showed the reason of the behaviour not the cause, so we
looked into kafka logs -  during the time from when our kafka job died and
when we restarted, we found a few exception although these doesn't look
like that it could be the root cause but I still wanted to make sure:

>
> 2018-08-20 09:01:20,016 INFO kafka.controller.KafkaController: [Controller
> 180]: Starting preferred replica leader election for partitions
> [__consumer_offsets,2]
> 2018-08-20 09:01:20,016 INFO kafka.controller.PartitionStateMachine:
> [Partition state machine on Controller 180]: Invoking state change to
> OnlinePartition for partitions [__consumer_offsets,2]
> 2018-08-20 09:01:20,019 INFO
> kafka.controller.PreferredReplicaPartitionLeaderSelector:
> [PreferredReplicaPartitionLeaderSelector]: Current leader 180 for partition
> [__consumer_offsets,2] is not the preferred replica. Triggering preferred
> replica leader election
> 2018-08-20 09:01:20,019 ERROR state.change.logger: Controller 180 epoch 37
> encountered error while electing leader for partition
> [__consumer_offsets,2] due to: Preferred replica 182 for partition
> [__consumer_offsets,2] is either not alive or not in the isr. Current
> leader and ISR: [{"leader":180,"leader_epoch":2,"isr":[181,180]}].
> 2018-08-20 09:01:20,019 ERROR state.change.logger: Controller 180 epoch 37
> initiated state change for partition [__consumer_offsets,2] from
> OnlinePartition to OnlinePartition failed
> kafka.common.StateChangeFailedException: encountered error while electing
> leader for partition [__consumer_offsets,2] due to: Preferred replica 182
> for partition [__consumer_offsets,2] is either not alive or not in the isr.
> Current leader and ISR: [{"leader":180,"leader_epoch":2,"isr":[181,180]}].
> at
> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:330)
> at
>