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

Jun Rao commented on KAFKA-3806:
--------------------------------

[~dwatzke], I think the original intention is that consumers in Kafka are 
typically real time. If a consumer is down for more than a day, chances are 
that it will never come back again and therefore it's offsets won't be needed.

[~tu...@avast.com], it's true that for a given consumer-group,topic,partition, 
after compaction, there will be only one message stored in the offset topic 
eventually. What we wanted to protect against is lots of short lived consumer 
groups. For example, currently, each time when one runs tools like 
console-consumer, a new consumer group is created. In an environment where many 
people are running those tools for a short period of time, it would be useful 
to garbage collect the unused offsets. If you know this is not the case, you 
can increase offsets.retention.minutes.

Yes, ideally, perhaps we should only start the expiration clock once the 
consumer becomes inactive, instead when an offset is last committed. For 
example, one simple improvement is if there is no new update to the offset but 
the consumer is still alive, we can automatically write the last offset with a 
new expiration time. This will address [~wushujames]'s issue better since the 
consumer no longer has to recommit the same offset.

> Adjust default values of log.retention.hours and offsets.retention.minutes
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-3806
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3806
>             Project: Kafka
>          Issue Type: Improvement
>          Components: config
>    Affects Versions: 0.9.0.1, 0.10.0.0
>            Reporter: Michal Turek
>            Priority: Minor
>
> Combination of default values of log.retention.hours (168 hours = 7 days) and 
> offsets.retention.minutes (1440 minutes = 1 day) may be dangerous in special 
> cases. Offset retention should be always greater than log retention.
> We have observed the following scenario and issue:
> - Producing of data to a topic was disabled two days ago by producer update, 
> topic wasn't deleted.
> - Consumer consumed all data and properly committed offsets to Kafka.
> - Consumer made no more offset commits for that topic because there was no 
> more incoming data and there was nothing to confirm. (We have auto-commit 
> disabled, I'm not sure how behaves enabled auto-commit.)
> - After one day: Kafka cleared too old offsets according to 
> offsets.retention.minutes.
> - After two days: Long-term running consumer was restarted after update, it 
> didn't find any committed offsets for that topic since they were deleted by 
> offsets.retention.minutes so it started consuming from the beginning.
> - The messages were still in Kafka due to larger log.retention.hours, about 5 
> days of messages were read again.
> Known workaround to solve this issue:
> - Explicitly configure log.retention.hours and offsets.retention.minutes, 
> don't use defaults.
> Proposals:
> - Prolong default value of offsets.retention.minutes to be at least twice 
> larger than log.retention.hours.
> - Check these values during Kafka startup and log a warning if 
> offsets.retention.minutes is smaller than log.retention.hours.
> - Add a note to migration guide about differences between storing of offsets 
> in ZooKeeper and Kafka (http://kafka.apache.org/documentation.html#upgrade).



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

Reply via email to