Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Hi Mayuresh, low_watermark will be updated when log retention fires on the broker. It may also be updated on the follower when follower receives FetchResponse from leader; and it may be updated on the leader when leader receives PurgeRequest from admin client. Thanks, Dong On Wed, Jan 11, 2017 a

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Mayuresh Gharat
Hi Dong, As per "If the message's offset is below low_watermark, then it should have been deleted by log retention policy." ---> I am not sure if I understand this correctly. Do you mean to say that the low_watermark will be updated only when the log retention fires on the broker? Thanks, Mayu

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-10 Thread Dong Lin
Bump up. I am going to initiate the vote If there is no further concern with the KIP. On Fri, Jan 6, 2017 at 11:23 PM, Dong Lin wrote: > Hey Mayuresh, > > Thanks for the comment. If the message's offset is below low_watermark, > then it should have been deleted by log retention policy. Thus it i

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Dong Lin
Hey Mayuresh, Thanks for the comment. If the message's offset is below low_watermark, then it should have been deleted by log retention policy. Thus it is OK not to expose this message to consumer. Does this answer your question? Thanks, Dong On Fri, Jan 6, 2017 at 4:21 PM, Mayuresh Gharat wrot

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Mayuresh Gharat
Hi Dong, Thanks for the KIP. I had a question (which might have been answered before). 1) The KIP says that the low_water_mark will be updated periodically by the broker like high_water_mark. Essentially we want to use low_water_mark for cases where an entire segment cannot be deleted because ma

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Dong Lin
Hey Jun, Thanks for reviewing the KIP! 1. The low_watermark will be checkpointed in a new file named "replication-low-watermark-checkpoint". It will have the same format as the existing replication-offset-checkpoint file. This allows us the keep the existing format of checkpoint files which maps

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Jun Rao
Hi, Dong, Thanks for the proposal. Looks good overall. A couple of comments. 1. Where is the low_watermark checkpointed? Is that in replication-offset-checkpoint? If so, do we need to bump up the version? Could you also describe the format change? 2. For topics with "delete" retention, currently

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-04 Thread radai
one more example of complicated config - mirror maker. we definitely cant trust each and every topic owner to configure their topics not to purge before they've been mirrored. which would mean there's a per-topic config (set by the owner) and a "global" config (where mirror makers are specified) a

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-04 Thread radai
in summary - i'm not opposed to the idea of a per-topic clean up config that tracks some set of consumer groups' offsets (which would probably work for 80% of use cases), but i definitely see a need to expose a simple API for the more advanced/obscure/custom use cases (the other 20%). On Wed, Jan

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-04 Thread radai
a major motivation for this KIP is cost savings. lots of internal systems at LI use kafka as an intermediate pipe, and set the topic retention period to a "safe enough" amount of time to be able to recover from crashes/downtime and catch up to "now". this results in a few days' worth of retention

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Dong Lin
On Tue, Jan 3, 2017 at 11:01 PM, Ewen Cheslack-Postava wrote: > On Tue, Jan 3, 2017 at 6:14 PM, Dong Lin wrote: > > > Hey Ewen, > > > > Thanks for the review. As Radai explained, it would be complex in terms > of > > user configuration if we were to use committed offset to decide data > > deleti

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Ewen Cheslack-Postava
On Tue, Jan 3, 2017 at 6:14 PM, Dong Lin wrote: > Hey Ewen, > > Thanks for the review. As Radai explained, it would be complex in terms of > user configuration if we were to use committed offset to decide data > deletion. We need a way to specify which groups need to consume data of > this partit

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Ewen Cheslack-Postava
On Tue, Jan 3, 2017 at 5:30 PM, radai wrote: > also 4. some apps may do their own offset bookkeeping > This is definitely a fair point, but if you want aggressive cleanup of data in Kafka, you can dual commit with the Kafka commit happening second. I don't see how this would be a problem -- inco

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Dong Lin
Hey Ewen, Thanks for the review. As Radai explained, it would be complex in terms of user configuration if we were to use committed offset to decide data deletion. We need a way to specify which groups need to consume data of this partition. The broker will also need to consume the entire offsets

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread radai
also 4. some apps may do their own offset bookkeeping On Tue, Jan 3, 2017 at 5:29 PM, radai wrote: > the issue with tracking committed offsets is whos offsets do you track? > > 1. some topics have multiple groups > 2. some "groups" are really one-offs like developers spinning up console > consum

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread radai
the issue with tracking committed offsets is whos offsets do you track? 1. some topics have multiple groups 2. some "groups" are really one-offs like developers spinning up console consumer "just to see if there's data" 3. there are use cases where you want to deliberately "wipe" data EVEN IF its

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Ewen Cheslack-Postava
Dong, Looks like that's an internal link, https://cwiki.apache.org/confluence/display/KAFKA/KIP-107%3A+Add+purgeDataBefore%28%29+API+in+AdminClient is the right one. I have a question about one of the rejected alternatives: > Using committed offset instead of an extra API to trigger data purge o

[DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-03 Thread Dong Lin
Hi all, We created KIP-107 to propose addition of purgeDataBefore() API in AdminClient. Please find the KIP wiki in the link https://iwww.corp.linkedin. com/wiki/cf/display/ENGS/Kafka+purgeDataBefore%28%29+API+design+proposal. We would love to hear your comments and suggestions. Thanks, Dong