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

Vahid Hashemian edited comment on KAFKA-4682 at 8/11/17 5:56 PM:
-----------------------------------------------------------------

[~hachikuji] Thank you for your comments. You seem to be looking at this with 
an inclination to get rid of the retention time from the OffsetCommit protocol. 
I think with my comments below I'm considering the alternative:

# Ewen's KIP proposes to increase the default retention from 1 day to 7 days. 
So, allowing consumers to set a lower timeout (for the console consumer) seems 
to be helpful after his KIP; the same way allowing them to set a higher timeout 
(for actual consumer applications) is helpful before his KIP.
# Even if we have offset-level expiration, all offsets in the group should 
expire together, because the expiration timer starts ticking for all partitions 
at the same time (when the group becomes empty). The only exception is when a 
consumer has set a non-default retention time for particular partitions (e.g. 
using the OffsetCommit API).
# Agreed. The expiration timestamp won't make sense. Perhaps the retention time 
should be stored and whether to expire or not could be calculated on the fly 
from the time group becomes empty + retention time (we would need to somehow 
keep the timestamp of the group becoming empty). This expiration check needs to 
be performed only if the group is empty; otherwise there is no need to expire 
at all.
# I don't have a strong feeling about this. It's for sure simpler to let all 
offsets expire at the same time. And if we keep the individual offset retention 
it would be easier to change this in case the cache size becomes an issue.

I think there is a risk involved in removing the individual retention from the 
protocol: could some requirement arise in the future that makes us bring it 
back to the protocol? One option is to let that field stay for now, and remove 
it later once we are more certain that it won't be needed back.


was (Author: vahid):
[~hachikuji] Thank you for your comments. You seem to be looking at this with 
an inclination to get rid of the retention time from the OffsetCommit protocol. 
I think with my comments below I'm considering the alternative:

# Ewen's KIP proposes to increase the default retention from 1 day to 7 days. 
So, allowing consumers to set a lower timeout (for the console consumer) seems 
to be helpful after his KIP; the same way allowing them to set a higher timeout 
(for actual consumer applications) is helpful before his KIP.

# Even if we have offset-level expiration, all offsets in the group should 
expire together, because the expiration timer starts ticking for all partitions 
at the same time (when the group becomes empty). The only exception is when a 
consumer has set a non-default retention time for particular partitions (e.g. 
using the OffsetCommit API).

# Agreed. The expiration timestamp won't make sense. Perhaps the retention time 
should be stored and whether to expire or not could be calculated on the fly 
from the time group becomes empty + retention time (we would need to somehow 
keep the timestamp of the group becoming empty). This expiration check needs to 
be performed only if the group is empty; otherwise there is no need to expire 
at all.

# I don't have a strong feeling about this. It's for sure simpler to let all 
offsets expire at the same time. And if we keep the individual offset retention 
it would be easier to change this in case the cache size becomes an issue.

I think there is a risk involved in removing the individual retention from the 
protocol: could some requirement arise in the future that makes us bring it 
back to the protocol? One option is to let that field stay for now, and remove 
it later once we are more certain that it won't be needed back.

> Committed offsets should not be deleted if a consumer is still active
> ---------------------------------------------------------------------
>
>                 Key: KAFKA-4682
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4682
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: James Cheng
>
> Kafka will delete committed offsets that are older than 
> offsets.retention.minutes
> If there is an active consumer on a low traffic partition, it is possible 
> that Kafka will delete the committed offset for that consumer. Once the 
> offset is deleted, a restart or a rebalance of that consumer will cause the 
> consumer to not find any committed offset and start consuming from 
> earliest/latest (depending on auto.offset.reset). I'm not sure, but a broker 
> failover might also cause you to start reading from auto.offset.reset (due to 
> broker restart, or coordinator failover).
> I think that Kafka should only delete offsets for inactive consumers. The 
> timer should only start after a consumer group goes inactive. For example, if 
> a consumer group goes inactive, then after 1 week, delete the offsets for 
> that consumer group. This is a solution that [~junrao] mentioned in 
> https://issues.apache.org/jira/browse/KAFKA-3806?focusedCommentId=15323521&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15323521
> The current workarounds are to:
> # Commit an offset on every partition you own on a regular basis, making sure 
> that it is more frequent than offsets.retention.minutes (a broker-side 
> setting that a consumer might not be aware of)
> or
> # Turn the value of offsets.retention.minutes up really really high. You have 
> to make sure it is higher than any valid low-traffic rate that you want to 
> support. For example, if you want to support a topic where someone produces 
> once a month, you would have to set offsetes.retention.mintues to 1 month. 
> or
> # Turn on enable.auto.commit (this is essentially #1, but easier to 
> implement).
> None of these are ideal. 
> #1 can be spammy. It requires your consumers know something about how the 
> brokers are configured. Sometimes it is out of your control. Mirrormaker, for 
> example, only commits offsets on partitions where it receives data. And it is 
> duplication that you need to put into all of your consumers.
> #2 has disk-space impact on the broker (in __consumer_offsets) as well as 
> memory-size on the broker (to answer OffsetFetch).
> #3 I think has the potential for message loss (the consumer might commit on 
> messages that are not yet fully processed)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to