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

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

offsets.retention.minutes is designed to handle the case that a consumer group 
goes away forever. In that case, we don't want to store the offsets for that 
group forever. That retention time potentially can be less than the log 
retention time. In your case, if the consumer is still alive, you would want to 
keep committing the offsets even though they are not changing. Once the 
consumer group coordinator sees the new offset commit, it will extend the time 
to preserve the offsets. This is what auto commit will do. 

> 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