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

2017-03-15 Thread Dong Lin
When changing the code, I realized that it feels weird to have
DeleteRequest and DeleteTopicsRequest. Thus I would follow the suggestion
and change it to DeleteRecordsRequest in this KIP, unless we decide to use
PurgeRequest.

On Wed, Mar 15, 2017 at 12:04 PM, Dong Lin  wrote:

> Hey Jason, Ismael, Jeff,
>
> Regarding Purge vs PurgeRecords, would it be OK for me to make a followup
> patch to rename PurgeRequest to PurgeRecordsRequest (similarly for
> ProduceRequest and FetchRequest)? This is because I favor PurgeRequest over
> PurgeRecordsRequest before we rename ProduceRequest and FetchRequest.
> Also, since the patch is ready for merge other than the naming issue we
> are discussing here, I would like to make less cosmetic code change and
> have it merged soon. I can submit the patch to rename the requests right
> after the pull request is committed.
>
> Hey Jun,
>
> You mentioned that the purpose of having purge is to distinguish between
> removing while log vs removing portion of the log. The PurgeRequest
> proposed in this KIP will remove portion of the Log since it works on the
> granularity of records. This will be more explicit after we rename it to
> PurgeRecordsRequest. If we want to have request in the future to remove the
> entire log, we can name it PurgeLogRequest. Thus I think it is OK to use
> "delete" instead of "purge" in the name and still be able to distinguish
> between removing while log vs removing portion of the log.
>
> I have updated the KIP to replace "purge" with "delete" in the names of
> the Java API and requests. Are you OK with the change?
>
> Thanks,
> Dong
>
>
> On Wed, Mar 15, 2017 at 9:59 AM, Jason Gustafson 
> wrote:
>
>> Hey Dong,
>>
>> Sorry for the late reply. Yes, I prefer PurgeRecordsRequest instead of
>> PurgeRequest. DeleteRecords seems even better. As mentioned, I also think
>> it would be a good idea to rename FetchRequest and ProduceRequest
>> accordingly, but we need not consider that here. We could potentially
>> rename Purge to PurgeRecords if and when we rename Fetch and Produce, but
>> if that's the plan, we may as well do it from the start. Anyway, it's just
>> my preference, so don't block on my opinion if the consensus is unclear.
>>
>> -Jason
>>
>>
>>
>> On Wed, Mar 15, 2017 at 8:45 AM, Ismael Juma  wrote:
>>
>> > Hi Dong,
>> >
>> > I think your suggestion of including `Records` in the name of the new
>> > request and renaming `Fetch` and `Produce` to be `FetchRecords` and
>> > `ProduceRecords` is a good one. We can do the the renames separately.
>> It's
>> > a compatible change since the name of the API is never exchanged with
>> > clients and the request/response classes are internal (we have done such
>> > renames before as Jason pointed out offline). The number of APIs will
>> > continue to grow and it will be much clearer if we avoid implicit
>> > assumptions about the target of an API request/response.
>> >
>> > Given that, I also think that DeleteRecords makes sense since we also
>> have
>> > DeleteTopics. Both are batch APIs that delete multiple items (the space
>> is
>> > only freed later). If we use Oracle's definition of "purge", it seems
>> to be
>> > what happens to cause the space to be freed (and happens in the
>> background
>> > in Kafka):
>> >
>> > "Purging is the process of freeing up space in the database or of
>> deleting
>> > obsolete data that is not required by the system. The purge process can
>> be
>> > based on the age of the data or the type of data"
>> > https://docs.oracle.com/cd/E12057_01/doc.1014/e12050/archpur
>> g.htm#BABHDECI
>> >
>> > Regarding the AdminClient, I thought the KIP was proposing adding this
>> > method to the Java AdminClient (KIP-117). If it's the Scala one, then it
>> > doesn't even need to be in the KIP as the Scala AdminClient is internal
>> and
>> > no compatibility guarantees are offered (the methods that exist there
>> never
>> > went through a KIP for example). So, I'm OK with keeping the method
>> > signature as it is.
>> >
>> > Ismael
>> >
>> > On Fri, Mar 10, 2017 at 6:06 PM, Dong Lin  wrote:
>> >
>> > > Hey Jason,
>> > >
>> > > Just to clarify, are you, together with Ismael and Jeff, suggesting
>> that
>> > > the new request should be named PurgeRecordsRequest instead of
>> > > PurgeRequest? The advantage of PurgeRecordsRequest is the name itself
>> is
>> > > more explicit about what it does. The disadvantage of
>> PurgeRecordsRequest
>> > > is that it is a bit consistent with ProduceRequest and FetchRequest,
>> > which
>> > > already assumes that if the target is not explicitly specified then
>> the
>> > > target is "Records".
>> > >
>> > > I would be in favor of PurgeRecordsRequest if we plan to change
>> > > FetchRequest
>> > > to FetchRecordsRequest and ProduceRequest to ProduceRequestsRequest.
>> > > Otherwise, I would prefer PurgeRequest since it is more consistent
>> with
>> > > existing style. Would 

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

2017-03-15 Thread Dong Lin
Hey Jason, Ismael, Jeff,

Regarding Purge vs PurgeRecords, would it be OK for me to make a followup
patch to rename PurgeRequest to PurgeRecordsRequest (similarly for
ProduceRequest and FetchRequest)? This is because I favor PurgeRequest over
PurgeRecordsRequest before we rename ProduceRequest and FetchRequest. Also,
since the patch is ready for merge other than the naming issue we are
discussing here, I would like to make less cosmetic code change and have it
merged soon. I can submit the patch to rename the requests right after the
pull request is committed.

Hey Jun,

You mentioned that the purpose of having purge is to distinguish between
removing while log vs removing portion of the log. The PurgeRequest
proposed in this KIP will remove portion of the Log since it works on the
granularity of records. This will be more explicit after we rename it to
PurgeRecordsRequest. If we want to have request in the future to remove the
entire log, we can name it PurgeLogRequest. Thus I think it is OK to use
"delete" instead of "purge" in the name and still be able to distinguish
between removing while log vs removing portion of the log.

I have updated the KIP to replace "purge" with "delete" in the names of the
Java API and requests. Are you OK with the change?

Thanks,
Dong


On Wed, Mar 15, 2017 at 9:59 AM, Jason Gustafson  wrote:

> Hey Dong,
>
> Sorry for the late reply. Yes, I prefer PurgeRecordsRequest instead of
> PurgeRequest. DeleteRecords seems even better. As mentioned, I also think
> it would be a good idea to rename FetchRequest and ProduceRequest
> accordingly, but we need not consider that here. We could potentially
> rename Purge to PurgeRecords if and when we rename Fetch and Produce, but
> if that's the plan, we may as well do it from the start. Anyway, it's just
> my preference, so don't block on my opinion if the consensus is unclear.
>
> -Jason
>
>
>
> On Wed, Mar 15, 2017 at 8:45 AM, Ismael Juma  wrote:
>
> > Hi Dong,
> >
> > I think your suggestion of including `Records` in the name of the new
> > request and renaming `Fetch` and `Produce` to be `FetchRecords` and
> > `ProduceRecords` is a good one. We can do the the renames separately.
> It's
> > a compatible change since the name of the API is never exchanged with
> > clients and the request/response classes are internal (we have done such
> > renames before as Jason pointed out offline). The number of APIs will
> > continue to grow and it will be much clearer if we avoid implicit
> > assumptions about the target of an API request/response.
> >
> > Given that, I also think that DeleteRecords makes sense since we also
> have
> > DeleteTopics. Both are batch APIs that delete multiple items (the space
> is
> > only freed later). If we use Oracle's definition of "purge", it seems to
> be
> > what happens to cause the space to be freed (and happens in the
> background
> > in Kafka):
> >
> > "Purging is the process of freeing up space in the database or of
> deleting
> > obsolete data that is not required by the system. The purge process can
> be
> > based on the age of the data or the type of data"
> > https://docs.oracle.com/cd/E12057_01/doc.1014/e12050/archpur
> g.htm#BABHDECI
> >
> > Regarding the AdminClient, I thought the KIP was proposing adding this
> > method to the Java AdminClient (KIP-117). If it's the Scala one, then it
> > doesn't even need to be in the KIP as the Scala AdminClient is internal
> and
> > no compatibility guarantees are offered (the methods that exist there
> never
> > went through a KIP for example). So, I'm OK with keeping the method
> > signature as it is.
> >
> > Ismael
> >
> > On Fri, Mar 10, 2017 at 6:06 PM, Dong Lin  wrote:
> >
> > > Hey Jason,
> > >
> > > Just to clarify, are you, together with Ismael and Jeff, suggesting
> that
> > > the new request should be named PurgeRecordsRequest instead of
> > > PurgeRequest? The advantage of PurgeRecordsRequest is the name itself
> is
> > > more explicit about what it does. The disadvantage of
> PurgeRecordsRequest
> > > is that it is a bit consistent with ProduceRequest and FetchRequest,
> > which
> > > already assumes that if the target is not explicitly specified then the
> > > target is "Records".
> > >
> > > I would be in favor of PurgeRecordsRequest if we plan to change
> > > FetchRequest
> > > to FetchRecordsRequest and ProduceRequest to ProduceRequestsRequest.
> > > Otherwise, I would prefer PurgeRequest since it is more consistent with
> > > existing style. Would PurgeRequest look more reasonable if we simply
> > assume
> > > that the operation target is "Records" if it is not explicitly
> specified
> > in
> > > the name?
> > >
> > > Becket is also in favor of PurgeRequest for the same reason. Maybe we
> can
> > > start a vote for this if people can not reach consensus on this name? I
> > > won't fight for PurgeRequest if most people like PurgeRecordsRequest.
> > >
> > > Thanks,
> > > Dong
> > >
> 

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

2017-03-15 Thread Jason Gustafson
Hey Dong,

Sorry for the late reply. Yes, I prefer PurgeRecordsRequest instead of
PurgeRequest. DeleteRecords seems even better. As mentioned, I also think
it would be a good idea to rename FetchRequest and ProduceRequest
accordingly, but we need not consider that here. We could potentially
rename Purge to PurgeRecords if and when we rename Fetch and Produce, but
if that's the plan, we may as well do it from the start. Anyway, it's just
my preference, so don't block on my opinion if the consensus is unclear.

-Jason



On Wed, Mar 15, 2017 at 8:45 AM, Ismael Juma  wrote:

> Hi Dong,
>
> I think your suggestion of including `Records` in the name of the new
> request and renaming `Fetch` and `Produce` to be `FetchRecords` and
> `ProduceRecords` is a good one. We can do the the renames separately. It's
> a compatible change since the name of the API is never exchanged with
> clients and the request/response classes are internal (we have done such
> renames before as Jason pointed out offline). The number of APIs will
> continue to grow and it will be much clearer if we avoid implicit
> assumptions about the target of an API request/response.
>
> Given that, I also think that DeleteRecords makes sense since we also have
> DeleteTopics. Both are batch APIs that delete multiple items (the space is
> only freed later). If we use Oracle's definition of "purge", it seems to be
> what happens to cause the space to be freed (and happens in the background
> in Kafka):
>
> "Purging is the process of freeing up space in the database or of deleting
> obsolete data that is not required by the system. The purge process can be
> based on the age of the data or the type of data"
> https://docs.oracle.com/cd/E12057_01/doc.1014/e12050/archpurg.htm#BABHDECI
>
> Regarding the AdminClient, I thought the KIP was proposing adding this
> method to the Java AdminClient (KIP-117). If it's the Scala one, then it
> doesn't even need to be in the KIP as the Scala AdminClient is internal and
> no compatibility guarantees are offered (the methods that exist there never
> went through a KIP for example). So, I'm OK with keeping the method
> signature as it is.
>
> Ismael
>
> On Fri, Mar 10, 2017 at 6:06 PM, Dong Lin  wrote:
>
> > Hey Jason,
> >
> > Just to clarify, are you, together with Ismael and Jeff, suggesting that
> > the new request should be named PurgeRecordsRequest instead of
> > PurgeRequest? The advantage of PurgeRecordsRequest is the name itself is
> > more explicit about what it does. The disadvantage of PurgeRecordsRequest
> > is that it is a bit consistent with ProduceRequest and FetchRequest,
> which
> > already assumes that if the target is not explicitly specified then the
> > target is "Records".
> >
> > I would be in favor of PurgeRecordsRequest if we plan to change
> > FetchRequest
> > to FetchRecordsRequest and ProduceRequest to ProduceRequestsRequest.
> > Otherwise, I would prefer PurgeRequest since it is more consistent with
> > existing style. Would PurgeRequest look more reasonable if we simply
> assume
> > that the operation target is "Records" if it is not explicitly specified
> in
> > the name?
> >
> > Becket is also in favor of PurgeRequest for the same reason. Maybe we can
> > start a vote for this if people can not reach consensus on this name? I
> > won't fight for PurgeRequest if most people like PurgeRecordsRequest.
> >
> > Thanks,
> > Dong
> >
> >
> >
> >
> > On Thu, Mar 9, 2017 at 5:39 PM, Jason Gustafson 
> > wrote:
> >
> > > Re; Purge vs PurgeRecords: I think I'm with Ismael and Jeff that the
> > > increasing surface area of the request APIs calls for more explicit
> > naming.
> > > PurgeRecords sounds reasonable to me. Using simple verbs like "fetch"
> and
> > > "produce" made sense when there were 6 or 7 APIs, but we'll soon be up
> to
> > > 30. I could also imagine having other Purge* APIs in the future (e.g.
> > > PurgeCommittedOffsets?), so it would be nice to avoid the need to
> rename
> > in
> > > the future, though it's probably not too big of a problem if we have
> to.
> > > (FWIW, I'd also be in favor of change FetchRequest to
> FetchRecordsRequest
> > > and ProduceRequest to ProduceRequestsRequest.)
> > >
> > > -Jason
> > >
> > > On Tue, Mar 7, 2017 at 10:11 AM, Dong Lin  wrote:
> > >
> > > > Hi Jun, Ismael,
> > > >
> > > > I think making the API similar to a future KIP is desirable but not
> > > > required. Implementation is easy but discussion of the API may take a
> > lot
> > > > of time given that we haven't yet reached agreement on KIP-117. Thus
> I
> > > > prefer to just mark the API in Scala as unstable.
> > > >
> > > > I am OK with either delete or purge in the name.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > > On Tue, Mar 7, 2017 at 9:59 AM, Jun Rao  wrote:
> > > >
> > > > > Hi, Dong, Ismael,
> > > > >
> > > > > 1. I just meant that it would be useful to distinguish 

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

2017-03-15 Thread Ismael Juma
Hi Dong,

I think your suggestion of including `Records` in the name of the new
request and renaming `Fetch` and `Produce` to be `FetchRecords` and
`ProduceRecords` is a good one. We can do the the renames separately. It's
a compatible change since the name of the API is never exchanged with
clients and the request/response classes are internal (we have done such
renames before as Jason pointed out offline). The number of APIs will
continue to grow and it will be much clearer if we avoid implicit
assumptions about the target of an API request/response.

Given that, I also think that DeleteRecords makes sense since we also have
DeleteTopics. Both are batch APIs that delete multiple items (the space is
only freed later). If we use Oracle's definition of "purge", it seems to be
what happens to cause the space to be freed (and happens in the background
in Kafka):

"Purging is the process of freeing up space in the database or of deleting
obsolete data that is not required by the system. The purge process can be
based on the age of the data or the type of data"
https://docs.oracle.com/cd/E12057_01/doc.1014/e12050/archpurg.htm#BABHDECI

Regarding the AdminClient, I thought the KIP was proposing adding this
method to the Java AdminClient (KIP-117). If it's the Scala one, then it
doesn't even need to be in the KIP as the Scala AdminClient is internal and
no compatibility guarantees are offered (the methods that exist there never
went through a KIP for example). So, I'm OK with keeping the method
signature as it is.

Ismael

On Fri, Mar 10, 2017 at 6:06 PM, Dong Lin  wrote:

> Hey Jason,
>
> Just to clarify, are you, together with Ismael and Jeff, suggesting that
> the new request should be named PurgeRecordsRequest instead of
> PurgeRequest? The advantage of PurgeRecordsRequest is the name itself is
> more explicit about what it does. The disadvantage of PurgeRecordsRequest
> is that it is a bit consistent with ProduceRequest and FetchRequest, which
> already assumes that if the target is not explicitly specified then the
> target is "Records".
>
> I would be in favor of PurgeRecordsRequest if we plan to change
> FetchRequest
> to FetchRecordsRequest and ProduceRequest to ProduceRequestsRequest.
> Otherwise, I would prefer PurgeRequest since it is more consistent with
> existing style. Would PurgeRequest look more reasonable if we simply assume
> that the operation target is "Records" if it is not explicitly specified in
> the name?
>
> Becket is also in favor of PurgeRequest for the same reason. Maybe we can
> start a vote for this if people can not reach consensus on this name? I
> won't fight for PurgeRequest if most people like PurgeRecordsRequest.
>
> Thanks,
> Dong
>
>
>
>
> On Thu, Mar 9, 2017 at 5:39 PM, Jason Gustafson 
> wrote:
>
> > Re; Purge vs PurgeRecords: I think I'm with Ismael and Jeff that the
> > increasing surface area of the request APIs calls for more explicit
> naming.
> > PurgeRecords sounds reasonable to me. Using simple verbs like "fetch" and
> > "produce" made sense when there were 6 or 7 APIs, but we'll soon be up to
> > 30. I could also imagine having other Purge* APIs in the future (e.g.
> > PurgeCommittedOffsets?), so it would be nice to avoid the need to rename
> in
> > the future, though it's probably not too big of a problem if we have to.
> > (FWIW, I'd also be in favor of change FetchRequest to FetchRecordsRequest
> > and ProduceRequest to ProduceRequestsRequest.)
> >
> > -Jason
> >
> > On Tue, Mar 7, 2017 at 10:11 AM, Dong Lin  wrote:
> >
> > > Hi Jun, Ismael,
> > >
> > > I think making the API similar to a future KIP is desirable but not
> > > required. Implementation is easy but discussion of the API may take a
> lot
> > > of time given that we haven't yet reached agreement on KIP-117. Thus I
> > > prefer to just mark the API in Scala as unstable.
> > >
> > > I am OK with either delete or purge in the name.
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Tue, Mar 7, 2017 at 9:59 AM, Jun Rao  wrote:
> > >
> > > > Hi, Dong, Ismael,
> > > >
> > > > 1. I just meant that it would be useful to distinguish between
> removing
> > > the
> > > > whole log vs removing a portion of the log. The exact naming is less
> > > > important.
> > > >
> > > > 4. When we move the purgeBefore() api to the Java AdminClient, it
> would
> > > be
> > > > great if the api looks comparable to what's in KIP-117. For now,
> > perhaps
> > > we
> > > > can mark the api in Scala as unstable so that people are aware that
> > it's
> > > > subject to change?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 3, 2017 at 11:25 AM, Dong Lin 
> wrote:
> > > >
> > > > > Hey Ismael,
> > > > >
> > > > > Thank for the detailed explanation. Here is my thought:
> > > > >
> > > > > 1. purge vs. delete
> > > > >
> > > > > We have originally considered purge, delete, truncate and remove. I
> > > 

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

2017-03-10 Thread Dong Lin
Hey Jason,

Just to clarify, are you, together with Ismael and Jeff, suggesting that
the new request should be named PurgeRecordsRequest instead of
PurgeRequest? The advantage of PurgeRecordsRequest is the name itself is
more explicit about what it does. The disadvantage of PurgeRecordsRequest
is that it is a bit consistent with ProduceRequest and FetchRequest, which
already assumes that if the target is not explicitly specified then the
target is "Records".

I would be in favor of PurgeRecordsRequest if we plan to change FetchRequest
to FetchRecordsRequest and ProduceRequest to ProduceRequestsRequest.
Otherwise, I would prefer PurgeRequest since it is more consistent with
existing style. Would PurgeRequest look more reasonable if we simply assume
that the operation target is "Records" if it is not explicitly specified in
the name?

Becket is also in favor of PurgeRequest for the same reason. Maybe we can
start a vote for this if people can not reach consensus on this name? I
won't fight for PurgeRequest if most people like PurgeRecordsRequest.

Thanks,
Dong




On Thu, Mar 9, 2017 at 5:39 PM, Jason Gustafson  wrote:

> Re; Purge vs PurgeRecords: I think I'm with Ismael and Jeff that the
> increasing surface area of the request APIs calls for more explicit naming.
> PurgeRecords sounds reasonable to me. Using simple verbs like "fetch" and
> "produce" made sense when there were 6 or 7 APIs, but we'll soon be up to
> 30. I could also imagine having other Purge* APIs in the future (e.g.
> PurgeCommittedOffsets?), so it would be nice to avoid the need to rename in
> the future, though it's probably not too big of a problem if we have to.
> (FWIW, I'd also be in favor of change FetchRequest to FetchRecordsRequest
> and ProduceRequest to ProduceRequestsRequest.)
>
> -Jason
>
> On Tue, Mar 7, 2017 at 10:11 AM, Dong Lin  wrote:
>
> > Hi Jun, Ismael,
> >
> > I think making the API similar to a future KIP is desirable but not
> > required. Implementation is easy but discussion of the API may take a lot
> > of time given that we haven't yet reached agreement on KIP-117. Thus I
> > prefer to just mark the API in Scala as unstable.
> >
> > I am OK with either delete or purge in the name.
> >
> > Thanks,
> > Dong
> >
> >
> > On Tue, Mar 7, 2017 at 9:59 AM, Jun Rao  wrote:
> >
> > > Hi, Dong, Ismael,
> > >
> > > 1. I just meant that it would be useful to distinguish between removing
> > the
> > > whole log vs removing a portion of the log. The exact naming is less
> > > important.
> > >
> > > 4. When we move the purgeBefore() api to the Java AdminClient, it would
> > be
> > > great if the api looks comparable to what's in KIP-117. For now,
> perhaps
> > we
> > > can mark the api in Scala as unstable so that people are aware that
> it's
> > > subject to change?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Fri, Mar 3, 2017 at 11:25 AM, Dong Lin  wrote:
> > >
> > > > Hey Ismael,
> > > >
> > > > Thank for the detailed explanation. Here is my thought:
> > > >
> > > > 1. purge vs. delete
> > > >
> > > > We have originally considered purge, delete, truncate and remove. I
> > don't
> > > > have a strong preference among them and would be OK with any choice
> > here.
> > > > That is why I didn't provide specific reasoning for selecting purge
> and
> > > > instead asked you and Jun for reason to choose between purge/delete.
> > > >
> > > > Can you be more specific where do we use "delete" in
> > AdminClient.scala? I
> > > > couldn't find any usage of "delete" there.
> > > >
> > > > "delete" seems to be the only one that is exposed in the wire
> protocol
> > > and
> > > > script to the user. For example, "delete" as an option for
> > > kafka-topics.sh.
> > > > And it is used in the name of "DeleteTopicRequest" and a field name
> in
> > > the
> > > > StopReplicaRequest. That is why I slightly prefer "delete" over
> > "purge".
> > > >
> > > > But all these names have been used in the Java API that is not
> exposed
> > > > directly to the user. For example, We have Log.truncateTo(),
> > > > DelayedOperation.purgeCompleted(), and MemoryNavigableLRUCache.
> > remove().
> > > > Also, we haven't yet exposed any Java API to user that uses any of
> > these
> > > > choices. Thus there is no unanimous choice here and it should be OK
> to
> > > > choose any of the "delete", "purge", "truncate" or "remove" and at
> this
> > > > stage. I personally don't have any obvious difference among them and
> am
> > > OK
> > > > with any of them.
> > > >
> > > > 2. Message vs. Record vs. data in the Java API name.
> > > >
> > > > Both "message" and "record"  are used in the Kafka, e.g.
> MemoryRecords,
> > > > ProducerRecord, ConsumerRecords, ReplicaManager.appendRecords(),
> > > > ReplicaManager.fetchMessages(). I remember there was a patch that
> > > changed
> > > > method name from using "message" to "record". Since Record is used
> more
> > > > widely, I think we 

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

2017-03-09 Thread Jason Gustafson
Re; Purge vs PurgeRecords: I think I'm with Ismael and Jeff that the
increasing surface area of the request APIs calls for more explicit naming.
PurgeRecords sounds reasonable to me. Using simple verbs like "fetch" and
"produce" made sense when there were 6 or 7 APIs, but we'll soon be up to
30. I could also imagine having other Purge* APIs in the future (e.g.
PurgeCommittedOffsets?), so it would be nice to avoid the need to rename in
the future, though it's probably not too big of a problem if we have to.
(FWIW, I'd also be in favor of change FetchRequest to FetchRecordsRequest
and ProduceRequest to ProduceRequestsRequest.)

-Jason

On Tue, Mar 7, 2017 at 10:11 AM, Dong Lin  wrote:

> Hi Jun, Ismael,
>
> I think making the API similar to a future KIP is desirable but not
> required. Implementation is easy but discussion of the API may take a lot
> of time given that we haven't yet reached agreement on KIP-117. Thus I
> prefer to just mark the API in Scala as unstable.
>
> I am OK with either delete or purge in the name.
>
> Thanks,
> Dong
>
>
> On Tue, Mar 7, 2017 at 9:59 AM, Jun Rao  wrote:
>
> > Hi, Dong, Ismael,
> >
> > 1. I just meant that it would be useful to distinguish between removing
> the
> > whole log vs removing a portion of the log. The exact naming is less
> > important.
> >
> > 4. When we move the purgeBefore() api to the Java AdminClient, it would
> be
> > great if the api looks comparable to what's in KIP-117. For now, perhaps
> we
> > can mark the api in Scala as unstable so that people are aware that it's
> > subject to change?
> >
> > Thanks,
> >
> > Jun
> >
> > On Fri, Mar 3, 2017 at 11:25 AM, Dong Lin  wrote:
> >
> > > Hey Ismael,
> > >
> > > Thank for the detailed explanation. Here is my thought:
> > >
> > > 1. purge vs. delete
> > >
> > > We have originally considered purge, delete, truncate and remove. I
> don't
> > > have a strong preference among them and would be OK with any choice
> here.
> > > That is why I didn't provide specific reasoning for selecting purge and
> > > instead asked you and Jun for reason to choose between purge/delete.
> > >
> > > Can you be more specific where do we use "delete" in
> AdminClient.scala? I
> > > couldn't find any usage of "delete" there.
> > >
> > > "delete" seems to be the only one that is exposed in the wire protocol
> > and
> > > script to the user. For example, "delete" as an option for
> > kafka-topics.sh.
> > > And it is used in the name of "DeleteTopicRequest" and a field name in
> > the
> > > StopReplicaRequest. That is why I slightly prefer "delete" over
> "purge".
> > >
> > > But all these names have been used in the Java API that is not exposed
> > > directly to the user. For example, We have Log.truncateTo(),
> > > DelayedOperation.purgeCompleted(), and MemoryNavigableLRUCache.
> remove().
> > > Also, we haven't yet exposed any Java API to user that uses any of
> these
> > > choices. Thus there is no unanimous choice here and it should be OK to
> > > choose any of the "delete", "purge", "truncate" or "remove" and at this
> > > stage. I personally don't have any obvious difference among them and am
> > OK
> > > with any of them.
> > >
> > > 2. Message vs. Record vs. data in the Java API name.
> > >
> > > Both "message" and "record"  are used in the Kafka, e.g. MemoryRecords,
> > > ProducerRecord, ConsumerRecords, ReplicaManager.appendRecords(),
> > > ReplicaManager.fetchMessages(). I remember there was a patch that
> > changed
> > > method name from using "message" to "record". Since Record is used more
> > > widely, I think we should use Record instead of Message going forward.
> > >
> > > I agree that data is not used anyway and I prefer to change it to
> record,
> > > e.g. purgeRecordBefore(). Does anyone have comment on this?
> > >
> > >
> > > 3. PurgeRecordRequest vs. PurgeRequest
> > >
> > > As you said, PurgeRequest is consistent with FetchRequest and
> > > ProduceRequest and it makes sense if we reserve the word
> > > "Purge" for dealing with records/messages. I am not aware of anything
> > other
> > > than "record/message" that we may want to purge in the future. Even if
> > > there is, I am not sure this would be an issue. For example, we can
> just
> > > create PurgeXXXRequest similar to DeleteTopicsRequest. If we name the
> new
> > > request ad PurgeRecordsRequest, it will be different from FetchRequest
> > and
> > > ProduceRequest which is probably more confusing to user. Thus I prefer
> to
> > > keep the request name as PurgeRequest.
> > >
> > >
> > > 4. Change method signature to encapsulate the parameters and result as
> > does
> > > in KIP-117.
> > >
> > > I don't think we should do it in KIP-107. First, KIP-117 is still under
> > > discussion while KIP-107 has been reviewed for a few rounds and is
> almost
> > > ready for commit. Changing the API at this moment will require more
> > > discussion and delay progress. We should try to avoid that. Second, 

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

2017-03-07 Thread Dong Lin
Hi Jun, Ismael,

I think making the API similar to a future KIP is desirable but not
required. Implementation is easy but discussion of the API may take a lot
of time given that we haven't yet reached agreement on KIP-117. Thus I
prefer to just mark the API in Scala as unstable.

I am OK with either delete or purge in the name.

Thanks,
Dong


On Tue, Mar 7, 2017 at 9:59 AM, Jun Rao  wrote:

> Hi, Dong, Ismael,
>
> 1. I just meant that it would be useful to distinguish between removing the
> whole log vs removing a portion of the log. The exact naming is less
> important.
>
> 4. When we move the purgeBefore() api to the Java AdminClient, it would be
> great if the api looks comparable to what's in KIP-117. For now, perhaps we
> can mark the api in Scala as unstable so that people are aware that it's
> subject to change?
>
> Thanks,
>
> Jun
>
> On Fri, Mar 3, 2017 at 11:25 AM, Dong Lin  wrote:
>
> > Hey Ismael,
> >
> > Thank for the detailed explanation. Here is my thought:
> >
> > 1. purge vs. delete
> >
> > We have originally considered purge, delete, truncate and remove. I don't
> > have a strong preference among them and would be OK with any choice here.
> > That is why I didn't provide specific reasoning for selecting purge and
> > instead asked you and Jun for reason to choose between purge/delete.
> >
> > Can you be more specific where do we use "delete" in AdminClient.scala? I
> > couldn't find any usage of "delete" there.
> >
> > "delete" seems to be the only one that is exposed in the wire protocol
> and
> > script to the user. For example, "delete" as an option for
> kafka-topics.sh.
> > And it is used in the name of "DeleteTopicRequest" and a field name in
> the
> > StopReplicaRequest. That is why I slightly prefer "delete" over "purge".
> >
> > But all these names have been used in the Java API that is not exposed
> > directly to the user. For example, We have Log.truncateTo(),
> > DelayedOperation.purgeCompleted(), and MemoryNavigableLRUCache.remove().
> > Also, we haven't yet exposed any Java API to user that uses any of these
> > choices. Thus there is no unanimous choice here and it should be OK to
> > choose any of the "delete", "purge", "truncate" or "remove" and at this
> > stage. I personally don't have any obvious difference among them and am
> OK
> > with any of them.
> >
> > 2. Message vs. Record vs. data in the Java API name.
> >
> > Both "message" and "record"  are used in the Kafka, e.g. MemoryRecords,
> > ProducerRecord, ConsumerRecords, ReplicaManager.appendRecords(),
> > ReplicaManager.fetchMessages(). I remember there was a patch that
> changed
> > method name from using "message" to "record". Since Record is used more
> > widely, I think we should use Record instead of Message going forward.
> >
> > I agree that data is not used anyway and I prefer to change it to record,
> > e.g. purgeRecordBefore(). Does anyone have comment on this?
> >
> >
> > 3. PurgeRecordRequest vs. PurgeRequest
> >
> > As you said, PurgeRequest is consistent with FetchRequest and
> > ProduceRequest and it makes sense if we reserve the word
> > "Purge" for dealing with records/messages. I am not aware of anything
> other
> > than "record/message" that we may want to purge in the future. Even if
> > there is, I am not sure this would be an issue. For example, we can just
> > create PurgeXXXRequest similar to DeleteTopicsRequest. If we name the new
> > request ad PurgeRecordsRequest, it will be different from FetchRequest
> and
> > ProduceRequest which is probably more confusing to user. Thus I prefer to
> > keep the request name as PurgeRequest.
> >
> >
> > 4. Change method signature to encapsulate the parameters and result as
> does
> > in KIP-117.
> >
> > I don't think we should do it in KIP-107. First, KIP-117 is still under
> > discussion while KIP-107 has been reviewed for a few rounds and is almost
> > ready for commit. Changing the API at this moment will require more
> > discussion and delay progress. We should try to avoid that. Second, I
> think
> > it is OK for KIP-107 to have different API from KIP-117. The later KIP is
> > free to do what it wants and the earlier KIP should not depend on the
> later
> > KIP. User will need to change API anyway when they switch from Scala
> > AdminClient to Java AdminClient.
> >
> > Dong
> >
> >
> > On Fri, Mar 3, 2017 at 6:34 AM, Ismael Juma  wrote:
> >
> > > First of all, sorry to arrive late on this.
> > >
> > > Jun, do you have a reference that states that "purge" means to remove a
> > > portion? If I do "define: purge" on Google, one of the definitions is
> > > "physically remove (something) completely."
> > >
> > > In the PR, I was asking about the reasoning more than suggesting a
> > change.
> > > But let me clarify my thoughts. There are 2 separate things to think
> > about:
> > >
> > > 1. The protocol change.
> > >
> > > It's currently called Purge with no mention of what it's purging. This

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

2017-03-07 Thread Jun Rao
Hi, Dong, Ismael,

1. I just meant that it would be useful to distinguish between removing the
whole log vs removing a portion of the log. The exact naming is less
important.

4. When we move the purgeBefore() api to the Java AdminClient, it would be
great if the api looks comparable to what's in KIP-117. For now, perhaps we
can mark the api in Scala as unstable so that people are aware that it's
subject to change?

Thanks,

Jun

On Fri, Mar 3, 2017 at 11:25 AM, Dong Lin  wrote:

> Hey Ismael,
>
> Thank for the detailed explanation. Here is my thought:
>
> 1. purge vs. delete
>
> We have originally considered purge, delete, truncate and remove. I don't
> have a strong preference among them and would be OK with any choice here.
> That is why I didn't provide specific reasoning for selecting purge and
> instead asked you and Jun for reason to choose between purge/delete.
>
> Can you be more specific where do we use "delete" in AdminClient.scala? I
> couldn't find any usage of "delete" there.
>
> "delete" seems to be the only one that is exposed in the wire protocol and
> script to the user. For example, "delete" as an option for kafka-topics.sh.
> And it is used in the name of "DeleteTopicRequest" and a field name in the
> StopReplicaRequest. That is why I slightly prefer "delete" over "purge".
>
> But all these names have been used in the Java API that is not exposed
> directly to the user. For example, We have Log.truncateTo(),
> DelayedOperation.purgeCompleted(), and MemoryNavigableLRUCache.remove().
> Also, we haven't yet exposed any Java API to user that uses any of these
> choices. Thus there is no unanimous choice here and it should be OK to
> choose any of the "delete", "purge", "truncate" or "remove" and at this
> stage. I personally don't have any obvious difference among them and am OK
> with any of them.
>
> 2. Message vs. Record vs. data in the Java API name.
>
> Both "message" and "record"  are used in the Kafka, e.g. MemoryRecords,
> ProducerRecord, ConsumerRecords, ReplicaManager.appendRecords(),
> ReplicaManager.fetchMessages(). I remember there was a patch that changed
> method name from using "message" to "record". Since Record is used more
> widely, I think we should use Record instead of Message going forward.
>
> I agree that data is not used anyway and I prefer to change it to record,
> e.g. purgeRecordBefore(). Does anyone have comment on this?
>
>
> 3. PurgeRecordRequest vs. PurgeRequest
>
> As you said, PurgeRequest is consistent with FetchRequest and
> ProduceRequest and it makes sense if we reserve the word
> "Purge" for dealing with records/messages. I am not aware of anything other
> than "record/message" that we may want to purge in the future. Even if
> there is, I am not sure this would be an issue. For example, we can just
> create PurgeXXXRequest similar to DeleteTopicsRequest. If we name the new
> request ad PurgeRecordsRequest, it will be different from FetchRequest and
> ProduceRequest which is probably more confusing to user. Thus I prefer to
> keep the request name as PurgeRequest.
>
>
> 4. Change method signature to encapsulate the parameters and result as does
> in KIP-117.
>
> I don't think we should do it in KIP-107. First, KIP-117 is still under
> discussion while KIP-107 has been reviewed for a few rounds and is almost
> ready for commit. Changing the API at this moment will require more
> discussion and delay progress. We should try to avoid that. Second, I think
> it is OK for KIP-107 to have different API from KIP-117. The later KIP is
> free to do what it wants and the earlier KIP should not depend on the later
> KIP. User will need to change API anyway when they switch from Scala
> AdminClient to Java AdminClient.
>
> Dong
>
>
> On Fri, Mar 3, 2017 at 6:34 AM, Ismael Juma  wrote:
>
> > First of all, sorry to arrive late on this.
> >
> > Jun, do you have a reference that states that "purge" means to remove a
> > portion? If I do "define: purge" on Google, one of the definitions is
> > "physically remove (something) completely."
> >
> > In the PR, I was asking about the reasoning more than suggesting a
> change.
> > But let me clarify my thoughts. There are 2 separate things to think
> about:
> >
> > 1. The protocol change.
> >
> > It's currently called Purge with no mention of what it's purging. This is
> > consistent with Fetch and Produce and it makes sense if we reserve the
> word
> > "Purge" for dealing with records/messages. Having said that, I don't
> think
> > this is particularly intuitive for people who are not familiar with Kafka
> > and its history. The number of APIs in the protocol keeps growing and it
> > would be better to be explicit about what is being purged/deleted, in my
> > opinion. If we are explicit, then we need to decide what to call it,
> since
> > there is no precedent. A few options: PurgeRecords, PurgeMessages,
> > PurgeData, DeleteRecords, DeleteMessages, DeleteData (I personally don't
> > 

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

2017-03-03 Thread Dong Lin
re: Purge vs PurgeMessages/Records - I think we agree that PurgeRequest is
more consistent with current requests while PurgeRecords is more explicit
by itself. The question is whether it is more important to be consistent or
explicit. My opinion is that consistency is more important here. In general
user knows ProduceRequset and FetchRequest before any other request, which
means they will know that such request applies to "Records" if not
explicitly specified. Thus PurgeRequest seems a reasonable choice. Also, it
is not a problem if we have something else to purge in the future - we can
simply name the new request as PurgeXXXRequest similar to
DeleteTopicsRequest. Does this make sense?



On Fri, Mar 3, 2017 at 11:35 AM, Jeff Widman  wrote:

> re: Purge vs PurgeMessages/Records - I also prefer that it be more explicit
> about what is being purged. Despite the inconsistency with Fetch/Produce,
> because it's explicit about what's being purged there shouldn't be
> additional confusion. Who knows what in the future might need purging?
> Adding the extra word provides optionality in the future with very little
> cost.
>
> re: Records vs Messages... it would be nice if it was consistent across
> Kafka. Messages has a sense of expiring once received which seems like a
> better fit for a system like Kafka that has things flowing through it and
> then deleted, whereas records has a connotation of being kept for a
> unspecified period of time, such as a a more typical database scenario. But
> that's a minor point, really I just prefer it's consistent.
>
> On Fri, Mar 3, 2017 at 6:34 AM, Ismael Juma  wrote:
>
> > First of all, sorry to arrive late on this.
> >
> > Jun, do you have a reference that states that "purge" means to remove a
> > portion? If I do "define: purge" on Google, one of the definitions is
> > "physically remove (something) completely."
> >
> > In the PR, I was asking about the reasoning more than suggesting a
> change.
> > But let me clarify my thoughts. There are 2 separate things to think
> about:
> >
> > 1. The protocol change.
> >
> > It's currently called Purge with no mention of what it's purging. This is
> > consistent with Fetch and Produce and it makes sense if we reserve the
> word
> > "Purge" for dealing with records/messages. Having said that, I don't
> think
> > this is particularly intuitive for people who are not familiar with Kafka
> > and its history. The number of APIs in the protocol keeps growing and it
> > would be better to be explicit about what is being purged/deleted, in my
> > opinion. If we are explicit, then we need to decide what to call it,
> since
> > there is no precedent. A few options: PurgeRecords, PurgeMessages,
> > PurgeData, DeleteRecords, DeleteMessages, DeleteData (I personally don't
> > like the Data suffix as it's not used anywhere else).
> >
> > 2. The AdminClient change.
> >
> > Regarding the name of the method, I'd prefer to avoid the `Data` suffix
> > because I don't think we use that anywhere else (please correct me if I'm
> > wrong). In the Producer, we have `send(ProduceRecord)` and in the
> consumer
> > we have `ConsumerRecords poll(...)`. So maybe, the suffix should be
> > `Records`? Like in the protocol, we still need to decide if we want to
> use
> > `purge` or `delete`. We seem to use `delete` for all the other methods in
> > the AdminClient, so unless we have a reason to use a different name, it
> > seems like we should be consistent.
> >
> > The proposed method signature is `Future > PurgeDataResult>> purgeDataBefore(Map
> > offsetForPartition)`. In the AdminClient KIP (KIP-117), we are using
> > classes to encapsulate the parameters and result. We should probably do
> the
> > same in this KIP for consistency. Once we do that, we should also
> consider
> > if `Before` should be in the method name or should be in the parameter
> > class. Just an example to describe what I mean, one could say
> > `deleteRecords(DeleteRecordsParams.before(offsetsForPartition)`. That
> way,
> > we could provide a different way of deleting by simply updating the
> > parameters class.
> >
> > Some food for thought. :)
> >
> > Ismael
> >
> >
> >
> > On Thu, Mar 2, 2017 at 5:46 PM, Dong Lin  wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks for this information. I am not aware of this difference between
> > the
> > > purge and delete. Given this difference, I will prefer to the existing
> > name
> > > of the purge.
> > >
> > > Ismael, please let me if you are strong about using delete.
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Thu, Mar 2, 2017 at 9:40 AM, Jun Rao  wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > It seems that delete means removing everything while purge means
> > > removing a
> > > > portion. So, it seems that it's better to be able to distinguish the
> > two?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Wed, 

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

2017-03-03 Thread Jeff Widman
re: Purge vs PurgeMessages/Records - I also prefer that it be more explicit
about what is being purged. Despite the inconsistency with Fetch/Produce,
because it's explicit about what's being purged there shouldn't be
additional confusion. Who knows what in the future might need purging?
Adding the extra word provides optionality in the future with very little
cost.

re: Records vs Messages... it would be nice if it was consistent across
Kafka. Messages has a sense of expiring once received which seems like a
better fit for a system like Kafka that has things flowing through it and
then deleted, whereas records has a connotation of being kept for a
unspecified period of time, such as a a more typical database scenario. But
that's a minor point, really I just prefer it's consistent.

On Fri, Mar 3, 2017 at 6:34 AM, Ismael Juma  wrote:

> First of all, sorry to arrive late on this.
>
> Jun, do you have a reference that states that "purge" means to remove a
> portion? If I do "define: purge" on Google, one of the definitions is
> "physically remove (something) completely."
>
> In the PR, I was asking about the reasoning more than suggesting a change.
> But let me clarify my thoughts. There are 2 separate things to think about:
>
> 1. The protocol change.
>
> It's currently called Purge with no mention of what it's purging. This is
> consistent with Fetch and Produce and it makes sense if we reserve the word
> "Purge" for dealing with records/messages. Having said that, I don't think
> this is particularly intuitive for people who are not familiar with Kafka
> and its history. The number of APIs in the protocol keeps growing and it
> would be better to be explicit about what is being purged/deleted, in my
> opinion. If we are explicit, then we need to decide what to call it, since
> there is no precedent. A few options: PurgeRecords, PurgeMessages,
> PurgeData, DeleteRecords, DeleteMessages, DeleteData (I personally don't
> like the Data suffix as it's not used anywhere else).
>
> 2. The AdminClient change.
>
> Regarding the name of the method, I'd prefer to avoid the `Data` suffix
> because I don't think we use that anywhere else (please correct me if I'm
> wrong). In the Producer, we have `send(ProduceRecord)` and in the consumer
> we have `ConsumerRecords poll(...)`. So maybe, the suffix should be
> `Records`? Like in the protocol, we still need to decide if we want to use
> `purge` or `delete`. We seem to use `delete` for all the other methods in
> the AdminClient, so unless we have a reason to use a different name, it
> seems like we should be consistent.
>
> The proposed method signature is `Future PurgeDataResult>> purgeDataBefore(Map
> offsetForPartition)`. In the AdminClient KIP (KIP-117), we are using
> classes to encapsulate the parameters and result. We should probably do the
> same in this KIP for consistency. Once we do that, we should also consider
> if `Before` should be in the method name or should be in the parameter
> class. Just an example to describe what I mean, one could say
> `deleteRecords(DeleteRecordsParams.before(offsetsForPartition)`. That way,
> we could provide a different way of deleting by simply updating the
> parameters class.
>
> Some food for thought. :)
>
> Ismael
>
>
>
> On Thu, Mar 2, 2017 at 5:46 PM, Dong Lin  wrote:
>
> > Hey Jun,
> >
> > Thanks for this information. I am not aware of this difference between
> the
> > purge and delete. Given this difference, I will prefer to the existing
> name
> > of the purge.
> >
> > Ismael, please let me if you are strong about using delete.
> >
> > Thanks,
> > Dong
> >
> >
> > On Thu, Mar 2, 2017 at 9:40 AM, Jun Rao  wrote:
> >
> > > Hi, Dong,
> > >
> > > It seems that delete means removing everything while purge means
> > removing a
> > > portion. So, it seems that it's better to be able to distinguish the
> two?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin  wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have updated the KIP to include a script that allows user to purge
> > data
> > > > by providing a map from partition to offset. I think this script may
> be
> > > > convenience and useful, e.g., if user simply wants to purge all data
> of
> > > > given partitions from command line. I am wondering if anyone object
> > this
> > > > script or has suggestions on the interface.
> > > >
> > > > Besides, Ismael commented in the pull request that it may be better
> to
> > > > rename PurgeDataBefore() to DeleteDataBefore() and rename
> PurgeRequest
> > to
> > > > DeleteRequest. I think it may be a good idea because kafka-topics.sh
> > > > already use "delete" as an option. Personally I don't have strong
> > > > preference between "purge" and "delete". I am wondering if anyone
> > object
> > > to
> > > > this change.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > 

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

2017-03-03 Thread Dong Lin
Hey Ismael,

Thank for the detailed explanation. Here is my thought:

1. purge vs. delete

We have originally considered purge, delete, truncate and remove. I don't
have a strong preference among them and would be OK with any choice here.
That is why I didn't provide specific reasoning for selecting purge and
instead asked you and Jun for reason to choose between purge/delete.

Can you be more specific where do we use "delete" in AdminClient.scala? I
couldn't find any usage of "delete" there.

"delete" seems to be the only one that is exposed in the wire protocol and
script to the user. For example, "delete" as an option for kafka-topics.sh.
And it is used in the name of "DeleteTopicRequest" and a field name in the
StopReplicaRequest. That is why I slightly prefer "delete" over "purge".

But all these names have been used in the Java API that is not exposed
directly to the user. For example, We have Log.truncateTo(),
DelayedOperation.purgeCompleted(), and MemoryNavigableLRUCache.remove().
Also, we haven't yet exposed any Java API to user that uses any of these
choices. Thus there is no unanimous choice here and it should be OK to
choose any of the "delete", "purge", "truncate" or "remove" and at this
stage. I personally don't have any obvious difference among them and am OK
with any of them.

2. Message vs. Record vs. data in the Java API name.

Both "message" and "record"  are used in the Kafka, e.g. MemoryRecords,
ProducerRecord, ConsumerRecords, ReplicaManager.appendRecords(),
ReplicaManager.fetchMessages(). I remember there was a patch that changed
method name from using "message" to "record". Since Record is used more
widely, I think we should use Record instead of Message going forward.

I agree that data is not used anyway and I prefer to change it to record,
e.g. purgeRecordBefore(). Does anyone have comment on this?


3. PurgeRecordRequest vs. PurgeRequest

As you said, PurgeRequest is consistent with FetchRequest and
ProduceRequest and it makes sense if we reserve the word
"Purge" for dealing with records/messages. I am not aware of anything other
than "record/message" that we may want to purge in the future. Even if
there is, I am not sure this would be an issue. For example, we can just
create PurgeXXXRequest similar to DeleteTopicsRequest. If we name the new
request ad PurgeRecordsRequest, it will be different from FetchRequest and
ProduceRequest which is probably more confusing to user. Thus I prefer to
keep the request name as PurgeRequest.


4. Change method signature to encapsulate the parameters and result as does
in KIP-117.

I don't think we should do it in KIP-107. First, KIP-117 is still under
discussion while KIP-107 has been reviewed for a few rounds and is almost
ready for commit. Changing the API at this moment will require more
discussion and delay progress. We should try to avoid that. Second, I think
it is OK for KIP-107 to have different API from KIP-117. The later KIP is
free to do what it wants and the earlier KIP should not depend on the later
KIP. User will need to change API anyway when they switch from Scala
AdminClient to Java AdminClient.

Dong


On Fri, Mar 3, 2017 at 6:34 AM, Ismael Juma  wrote:

> First of all, sorry to arrive late on this.
>
> Jun, do you have a reference that states that "purge" means to remove a
> portion? If I do "define: purge" on Google, one of the definitions is
> "physically remove (something) completely."
>
> In the PR, I was asking about the reasoning more than suggesting a change.
> But let me clarify my thoughts. There are 2 separate things to think about:
>
> 1. The protocol change.
>
> It's currently called Purge with no mention of what it's purging. This is
> consistent with Fetch and Produce and it makes sense if we reserve the word
> "Purge" for dealing with records/messages. Having said that, I don't think
> this is particularly intuitive for people who are not familiar with Kafka
> and its history. The number of APIs in the protocol keeps growing and it
> would be better to be explicit about what is being purged/deleted, in my
> opinion. If we are explicit, then we need to decide what to call it, since
> there is no precedent. A few options: PurgeRecords, PurgeMessages,
> PurgeData, DeleteRecords, DeleteMessages, DeleteData (I personally don't
> like the Data suffix as it's not used anywhere else).
>
> 2. The AdminClient change.
>
> Regarding the name of the method, I'd prefer to avoid the `Data` suffix
> because I don't think we use that anywhere else (please correct me if I'm
> wrong). In the Producer, we have `send(ProduceRecord)` and in the consumer
> we have `ConsumerRecords poll(...)`. So maybe, the suffix should be
> `Records`? Like in the protocol, we still need to decide if we want to use
> `purge` or `delete`. We seem to use `delete` for all the other methods in
> the AdminClient, so unless we have a reason to use a different name, it
> seems like we should be consistent.
>
> The proposed method 

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

2017-03-03 Thread Ismael Juma
First of all, sorry to arrive late on this.

Jun, do you have a reference that states that "purge" means to remove a
portion? If I do "define: purge" on Google, one of the definitions is
"physically remove (something) completely."

In the PR, I was asking about the reasoning more than suggesting a change.
But let me clarify my thoughts. There are 2 separate things to think about:

1. The protocol change.

It's currently called Purge with no mention of what it's purging. This is
consistent with Fetch and Produce and it makes sense if we reserve the word
"Purge" for dealing with records/messages. Having said that, I don't think
this is particularly intuitive for people who are not familiar with Kafka
and its history. The number of APIs in the protocol keeps growing and it
would be better to be explicit about what is being purged/deleted, in my
opinion. If we are explicit, then we need to decide what to call it, since
there is no precedent. A few options: PurgeRecords, PurgeMessages,
PurgeData, DeleteRecords, DeleteMessages, DeleteData (I personally don't
like the Data suffix as it's not used anywhere else).

2. The AdminClient change.

Regarding the name of the method, I'd prefer to avoid the `Data` suffix
because I don't think we use that anywhere else (please correct me if I'm
wrong). In the Producer, we have `send(ProduceRecord)` and in the consumer
we have `ConsumerRecords poll(...)`. So maybe, the suffix should be
`Records`? Like in the protocol, we still need to decide if we want to use
`purge` or `delete`. We seem to use `delete` for all the other methods in
the AdminClient, so unless we have a reason to use a different name, it
seems like we should be consistent.

The proposed method signature is `Future> purgeDataBefore(Map
offsetForPartition)`. In the AdminClient KIP (KIP-117), we are using
classes to encapsulate the parameters and result. We should probably do the
same in this KIP for consistency. Once we do that, we should also consider
if `Before` should be in the method name or should be in the parameter
class. Just an example to describe what I mean, one could say
`deleteRecords(DeleteRecordsParams.before(offsetsForPartition)`. That way,
we could provide a different way of deleting by simply updating the
parameters class.

Some food for thought. :)

Ismael



On Thu, Mar 2, 2017 at 5:46 PM, Dong Lin  wrote:

> Hey Jun,
>
> Thanks for this information. I am not aware of this difference between the
> purge and delete. Given this difference, I will prefer to the existing name
> of the purge.
>
> Ismael, please let me if you are strong about using delete.
>
> Thanks,
> Dong
>
>
> On Thu, Mar 2, 2017 at 9:40 AM, Jun Rao  wrote:
>
> > Hi, Dong,
> >
> > It seems that delete means removing everything while purge means
> removing a
> > portion. So, it seems that it's better to be able to distinguish the two?
> >
> > Thanks,
> >
> > Jun
> >
> > On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin  wrote:
> >
> > > Hi all,
> > >
> > > I have updated the KIP to include a script that allows user to purge
> data
> > > by providing a map from partition to offset. I think this script may be
> > > convenience and useful, e.g., if user simply wants to purge all data of
> > > given partitions from command line. I am wondering if anyone object
> this
> > > script or has suggestions on the interface.
> > >
> > > Besides, Ismael commented in the pull request that it may be better to
> > > rename PurgeDataBefore() to DeleteDataBefore() and rename PurgeRequest
> to
> > > DeleteRequest. I think it may be a good idea because kafka-topics.sh
> > > already use "delete" as an option. Personally I don't have strong
> > > preference between "purge" and "delete". I am wondering if anyone
> object
> > to
> > > this change.
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > >
> > > On Wed, Mar 1, 2017 at 9:46 AM, Dong Lin  wrote:
> > >
> > > > Hi Ismael,
> > > >
> > > > I actually mean log_start_offset. I realized that it is a better name
> > > > after I start implementation because "logStartOffset" is already used
> > in
> > > > Log.scala and LogCleanerManager.scala. So I changed it from
> > > > log_begin_offset to log_start_offset in the patch. But I forgot to
> > update
> > > > the KIP and specify it in the mailing thread.
> > > >
> > > > Thanks for catching this. Let me update the KIP to reflect this
> change.
> > > >
> > > > Dong
> > > >
> > > >
> > > > On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma 
> wrote:
> > > >
> > > >> Hi Dong,
> > > >>
> > > >> When you say "logStartOffset", do you mean "log_begin_offset "? I
> > could
> > > >> only find the latter in the KIP. If so, would log_start_offset be a
> > > better
> > > >> name?
> > > >>
> > > >> Ismael
> > > >>
> > > >> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin 
> > wrote:
> > > >>
> > > >> > Hi Jun and 

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

2017-03-02 Thread Dong Lin
Hey Jun,

Thanks for this information. I am not aware of this difference between the
purge and delete. Given this difference, I will prefer to the existing name
of the purge.

Ismael, please let me if you are strong about using delete.

Thanks,
Dong


On Thu, Mar 2, 2017 at 9:40 AM, Jun Rao  wrote:

> Hi, Dong,
>
> It seems that delete means removing everything while purge means removing a
> portion. So, it seems that it's better to be able to distinguish the two?
>
> Thanks,
>
> Jun
>
> On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > I have updated the KIP to include a script that allows user to purge data
> > by providing a map from partition to offset. I think this script may be
> > convenience and useful, e.g., if user simply wants to purge all data of
> > given partitions from command line. I am wondering if anyone object this
> > script or has suggestions on the interface.
> >
> > Besides, Ismael commented in the pull request that it may be better to
> > rename PurgeDataBefore() to DeleteDataBefore() and rename PurgeRequest to
> > DeleteRequest. I think it may be a good idea because kafka-topics.sh
> > already use "delete" as an option. Personally I don't have strong
> > preference between "purge" and "delete". I am wondering if anyone object
> to
> > this change.
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Wed, Mar 1, 2017 at 9:46 AM, Dong Lin  wrote:
> >
> > > Hi Ismael,
> > >
> > > I actually mean log_start_offset. I realized that it is a better name
> > > after I start implementation because "logStartOffset" is already used
> in
> > > Log.scala and LogCleanerManager.scala. So I changed it from
> > > log_begin_offset to log_start_offset in the patch. But I forgot to
> update
> > > the KIP and specify it in the mailing thread.
> > >
> > > Thanks for catching this. Let me update the KIP to reflect this change.
> > >
> > > Dong
> > >
> > >
> > > On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma  wrote:
> > >
> > >> Hi Dong,
> > >>
> > >> When you say "logStartOffset", do you mean "log_begin_offset "? I
> could
> > >> only find the latter in the KIP. If so, would log_start_offset be a
> > better
> > >> name?
> > >>
> > >> Ismael
> > >>
> > >> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin 
> wrote:
> > >>
> > >> > Hi Jun and everyone,
> > >> >
> > >> > I would like to change the KIP in the following way. Currently, if
> any
> > >> > replica if offline, the purge result for a partition will
> > >> > be NotEnoughReplicasException and its low_watermark will be 0. The
> > >> > motivation for this approach is that we want to guarantee that the
> > data
> > >> > before purgedOffset has been deleted on all replicas of this
> partition
> > >> if
> > >> > purge result indicates success.
> > >> >
> > >> > But this approach seems too conservative. It should be sufficient in
> > >> most
> > >> > cases to just tell user success and set low_watermark to minimum
> > >> > logStartOffset of all live replicas in the PurgeResponse if
> > >> logStartOffset
> > >> > of all live replicas have reached purgedOffset. This is because for
> an
> > >> > offline replicas to become online and be elected leader, it should
> > have
> > >> > received one FetchReponse from the current leader which should tell
> it
> > >> to
> > >> > purge beyond purgedOffset. The benefit of doing this change is that
> we
> > >> can
> > >> > allow purge operation to succeed when some replica is offline.
> > >> >
> > >> > Are you OK with this change? If so, I will go ahead to update the
> KIP
> > >> and
> > >> > implement this behavior.
> > >> >
> > >> > Thanks,
> > >> > Dong
> > >> >
> > >> >
> > >> >
> > >> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin 
> > wrote:
> > >> >
> > >> > > Hey Jun,
> > >> > >
> > >> > > Do you have time to review the KIP again or vote for it?
> > >> > >
> > >> > > Hey Ewen,
> > >> > >
> > >> > > Can you also review the KIP again or vote for it? I have discussed
> > >> with
> > >> > > Radai and Becket regarding your concern. We still think putting it
> > in
> > >> > Admin
> > >> > > Client seems more intuitive because there is use-case where
> > >> application
> > >> > > which manages topic or produces data may also want to purge data.
> It
> > >> > seems
> > >> > > weird if they need to create a consumer to do this.
> > >> > >
> > >> > > Thanks,
> > >> > > Dong
> > >> > >
> > >> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> > >> > > gharatmayures...@gmail.com> wrote:
> > >> > >
> > >> > >> +1 (non-binding)
> > >> > >>
> > >> > >> Thanks,
> > >> > >>
> > >> > >> Mayuresh
> > >> > >>
> > >> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
> > >> wrote:
> > >> > >>
> > >> > >> > Sorry for the duplicated email. It seems that gmail will put
> the
> > >> > voting
> > >> > >> > email in this thread if I simply replace DISCUSS with VOTE in
> the
> > >> > >> subject.
> > 

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

2017-03-02 Thread Jun Rao
Hi, Dong,

It seems that delete means removing everything while purge means removing a
portion. So, it seems that it's better to be able to distinguish the two?

Thanks,

Jun

On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin  wrote:

> Hi all,
>
> I have updated the KIP to include a script that allows user to purge data
> by providing a map from partition to offset. I think this script may be
> convenience and useful, e.g., if user simply wants to purge all data of
> given partitions from command line. I am wondering if anyone object this
> script or has suggestions on the interface.
>
> Besides, Ismael commented in the pull request that it may be better to
> rename PurgeDataBefore() to DeleteDataBefore() and rename PurgeRequest to
> DeleteRequest. I think it may be a good idea because kafka-topics.sh
> already use "delete" as an option. Personally I don't have strong
> preference between "purge" and "delete". I am wondering if anyone object to
> this change.
>
> Thanks,
> Dong
>
>
>
> On Wed, Mar 1, 2017 at 9:46 AM, Dong Lin  wrote:
>
> > Hi Ismael,
> >
> > I actually mean log_start_offset. I realized that it is a better name
> > after I start implementation because "logStartOffset" is already used in
> > Log.scala and LogCleanerManager.scala. So I changed it from
> > log_begin_offset to log_start_offset in the patch. But I forgot to update
> > the KIP and specify it in the mailing thread.
> >
> > Thanks for catching this. Let me update the KIP to reflect this change.
> >
> > Dong
> >
> >
> > On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma  wrote:
> >
> >> Hi Dong,
> >>
> >> When you say "logStartOffset", do you mean "log_begin_offset "? I could
> >> only find the latter in the KIP. If so, would log_start_offset be a
> better
> >> name?
> >>
> >> Ismael
> >>
> >> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin  wrote:
> >>
> >> > Hi Jun and everyone,
> >> >
> >> > I would like to change the KIP in the following way. Currently, if any
> >> > replica if offline, the purge result for a partition will
> >> > be NotEnoughReplicasException and its low_watermark will be 0. The
> >> > motivation for this approach is that we want to guarantee that the
> data
> >> > before purgedOffset has been deleted on all replicas of this partition
> >> if
> >> > purge result indicates success.
> >> >
> >> > But this approach seems too conservative. It should be sufficient in
> >> most
> >> > cases to just tell user success and set low_watermark to minimum
> >> > logStartOffset of all live replicas in the PurgeResponse if
> >> logStartOffset
> >> > of all live replicas have reached purgedOffset. This is because for an
> >> > offline replicas to become online and be elected leader, it should
> have
> >> > received one FetchReponse from the current leader which should tell it
> >> to
> >> > purge beyond purgedOffset. The benefit of doing this change is that we
> >> can
> >> > allow purge operation to succeed when some replica is offline.
> >> >
> >> > Are you OK with this change? If so, I will go ahead to update the KIP
> >> and
> >> > implement this behavior.
> >> >
> >> > Thanks,
> >> > Dong
> >> >
> >> >
> >> >
> >> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin 
> wrote:
> >> >
> >> > > Hey Jun,
> >> > >
> >> > > Do you have time to review the KIP again or vote for it?
> >> > >
> >> > > Hey Ewen,
> >> > >
> >> > > Can you also review the KIP again or vote for it? I have discussed
> >> with
> >> > > Radai and Becket regarding your concern. We still think putting it
> in
> >> > Admin
> >> > > Client seems more intuitive because there is use-case where
> >> application
> >> > > which manages topic or produces data may also want to purge data. It
> >> > seems
> >> > > weird if they need to create a consumer to do this.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> >> > > gharatmayures...@gmail.com> wrote:
> >> > >
> >> > >> +1 (non-binding)
> >> > >>
> >> > >> Thanks,
> >> > >>
> >> > >> Mayuresh
> >> > >>
> >> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
> >> wrote:
> >> > >>
> >> > >> > Sorry for the duplicated email. It seems that gmail will put the
> >> > voting
> >> > >> > email in this thread if I simply replace DISCUSS with VOTE in the
> >> > >> subject.
> >> > >> >
> >> > >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> >> > wrote:
> >> > >> >
> >> > >> > > Hi all,
> >> > >> > >
> >> > >> > > It seems that there is no further concern with the KIP-107. At
> >> this
> >> > >> point
> >> > >> > > we would like to start the voting process. The KIP can be found
> >> at
> >> > >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> >> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >> > >> > >
> >> > >> > > Thanks,
> >> > >> > > Dong
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >>
> >> > >>
> >> > >> --
> >> > >> 

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

2017-03-01 Thread Becket Qin
Thanks for the update. The changes sound reasonable.

On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin  wrote:

> Hi all,
>
> I have updated the KIP to include a script that allows user to purge data
> by providing a map from partition to offset. I think this script may be
> convenience and useful, e.g., if user simply wants to purge all data of
> given partitions from command line. I am wondering if anyone object this
> script or has suggestions on the interface.
>
> Besides, Ismael commented in the pull request that it may be better to
> rename PurgeDataBefore() to DeleteDataBefore() and rename PurgeRequest to
> DeleteRequest. I think it may be a good idea because kafka-topics.sh
> already use "delete" as an option. Personally I don't have strong
> preference between "purge" and "delete". I am wondering if anyone object to
> this change.
>
> Thanks,
> Dong
>
>
>
> On Wed, Mar 1, 2017 at 9:46 AM, Dong Lin  wrote:
>
> > Hi Ismael,
> >
> > I actually mean log_start_offset. I realized that it is a better name
> > after I start implementation because "logStartOffset" is already used in
> > Log.scala and LogCleanerManager.scala. So I changed it from
> > log_begin_offset to log_start_offset in the patch. But I forgot to update
> > the KIP and specify it in the mailing thread.
> >
> > Thanks for catching this. Let me update the KIP to reflect this change.
> >
> > Dong
> >
> >
> > On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma  wrote:
> >
> >> Hi Dong,
> >>
> >> When you say "logStartOffset", do you mean "log_begin_offset "? I could
> >> only find the latter in the KIP. If so, would log_start_offset be a
> better
> >> name?
> >>
> >> Ismael
> >>
> >> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin  wrote:
> >>
> >> > Hi Jun and everyone,
> >> >
> >> > I would like to change the KIP in the following way. Currently, if any
> >> > replica if offline, the purge result for a partition will
> >> > be NotEnoughReplicasException and its low_watermark will be 0. The
> >> > motivation for this approach is that we want to guarantee that the
> data
> >> > before purgedOffset has been deleted on all replicas of this partition
> >> if
> >> > purge result indicates success.
> >> >
> >> > But this approach seems too conservative. It should be sufficient in
> >> most
> >> > cases to just tell user success and set low_watermark to minimum
> >> > logStartOffset of all live replicas in the PurgeResponse if
> >> logStartOffset
> >> > of all live replicas have reached purgedOffset. This is because for an
> >> > offline replicas to become online and be elected leader, it should
> have
> >> > received one FetchReponse from the current leader which should tell it
> >> to
> >> > purge beyond purgedOffset. The benefit of doing this change is that we
> >> can
> >> > allow purge operation to succeed when some replica is offline.
> >> >
> >> > Are you OK with this change? If so, I will go ahead to update the KIP
> >> and
> >> > implement this behavior.
> >> >
> >> > Thanks,
> >> > Dong
> >> >
> >> >
> >> >
> >> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin 
> wrote:
> >> >
> >> > > Hey Jun,
> >> > >
> >> > > Do you have time to review the KIP again or vote for it?
> >> > >
> >> > > Hey Ewen,
> >> > >
> >> > > Can you also review the KIP again or vote for it? I have discussed
> >> with
> >> > > Radai and Becket regarding your concern. We still think putting it
> in
> >> > Admin
> >> > > Client seems more intuitive because there is use-case where
> >> application
> >> > > which manages topic or produces data may also want to purge data. It
> >> > seems
> >> > > weird if they need to create a consumer to do this.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> >> > > gharatmayures...@gmail.com> wrote:
> >> > >
> >> > >> +1 (non-binding)
> >> > >>
> >> > >> Thanks,
> >> > >>
> >> > >> Mayuresh
> >> > >>
> >> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
> >> wrote:
> >> > >>
> >> > >> > Sorry for the duplicated email. It seems that gmail will put the
> >> > voting
> >> > >> > email in this thread if I simply replace DISCUSS with VOTE in the
> >> > >> subject.
> >> > >> >
> >> > >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> >> > wrote:
> >> > >> >
> >> > >> > > Hi all,
> >> > >> > >
> >> > >> > > It seems that there is no further concern with the KIP-107. At
> >> this
> >> > >> point
> >> > >> > > we would like to start the voting process. The KIP can be found
> >> at
> >> > >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> >> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >> > >> > >
> >> > >> > > Thanks,
> >> > >> > > Dong
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >>
> >> > >>
> >> > >> --
> >> > >> -Regards,
> >> > >> Mayuresh R. Gharat
> >> > >> (862) 250-7125
> >> > >>
> >> > >
> >> > >
> >> >
> >>
> >
> >
>


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

2017-03-01 Thread Dong Lin
Hi all,

I have updated the KIP to include a script that allows user to purge data
by providing a map from partition to offset. I think this script may be
convenience and useful, e.g., if user simply wants to purge all data of
given partitions from command line. I am wondering if anyone object this
script or has suggestions on the interface.

Besides, Ismael commented in the pull request that it may be better to
rename PurgeDataBefore() to DeleteDataBefore() and rename PurgeRequest to
DeleteRequest. I think it may be a good idea because kafka-topics.sh
already use "delete" as an option. Personally I don't have strong
preference between "purge" and "delete". I am wondering if anyone object to
this change.

Thanks,
Dong



On Wed, Mar 1, 2017 at 9:46 AM, Dong Lin  wrote:

> Hi Ismael,
>
> I actually mean log_start_offset. I realized that it is a better name
> after I start implementation because "logStartOffset" is already used in
> Log.scala and LogCleanerManager.scala. So I changed it from
> log_begin_offset to log_start_offset in the patch. But I forgot to update
> the KIP and specify it in the mailing thread.
>
> Thanks for catching this. Let me update the KIP to reflect this change.
>
> Dong
>
>
> On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma  wrote:
>
>> Hi Dong,
>>
>> When you say "logStartOffset", do you mean "log_begin_offset "? I could
>> only find the latter in the KIP. If so, would log_start_offset be a better
>> name?
>>
>> Ismael
>>
>> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin  wrote:
>>
>> > Hi Jun and everyone,
>> >
>> > I would like to change the KIP in the following way. Currently, if any
>> > replica if offline, the purge result for a partition will
>> > be NotEnoughReplicasException and its low_watermark will be 0. The
>> > motivation for this approach is that we want to guarantee that the data
>> > before purgedOffset has been deleted on all replicas of this partition
>> if
>> > purge result indicates success.
>> >
>> > But this approach seems too conservative. It should be sufficient in
>> most
>> > cases to just tell user success and set low_watermark to minimum
>> > logStartOffset of all live replicas in the PurgeResponse if
>> logStartOffset
>> > of all live replicas have reached purgedOffset. This is because for an
>> > offline replicas to become online and be elected leader, it should have
>> > received one FetchReponse from the current leader which should tell it
>> to
>> > purge beyond purgedOffset. The benefit of doing this change is that we
>> can
>> > allow purge operation to succeed when some replica is offline.
>> >
>> > Are you OK with this change? If so, I will go ahead to update the KIP
>> and
>> > implement this behavior.
>> >
>> > Thanks,
>> > Dong
>> >
>> >
>> >
>> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:
>> >
>> > > Hey Jun,
>> > >
>> > > Do you have time to review the KIP again or vote for it?
>> > >
>> > > Hey Ewen,
>> > >
>> > > Can you also review the KIP again or vote for it? I have discussed
>> with
>> > > Radai and Becket regarding your concern. We still think putting it in
>> > Admin
>> > > Client seems more intuitive because there is use-case where
>> application
>> > > which manages topic or produces data may also want to purge data. It
>> > seems
>> > > weird if they need to create a consumer to do this.
>> > >
>> > > Thanks,
>> > > Dong
>> > >
>> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
>> > > gharatmayures...@gmail.com> wrote:
>> > >
>> > >> +1 (non-binding)
>> > >>
>> > >> Thanks,
>> > >>
>> > >> Mayuresh
>> > >>
>> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
>> wrote:
>> > >>
>> > >> > Sorry for the duplicated email. It seems that gmail will put the
>> > voting
>> > >> > email in this thread if I simply replace DISCUSS with VOTE in the
>> > >> subject.
>> > >> >
>> > >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
>> > wrote:
>> > >> >
>> > >> > > Hi all,
>> > >> > >
>> > >> > > It seems that there is no further concern with the KIP-107. At
>> this
>> > >> point
>> > >> > > we would like to start the voting process. The KIP can be found
>> at
>> > >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
>> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>> > >> > >
>> > >> > > Thanks,
>> > >> > > Dong
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >>
>> > >> --
>> > >> -Regards,
>> > >> Mayuresh R. Gharat
>> > >> (862) 250-7125
>> > >>
>> > >
>> > >
>> >
>>
>
>


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

2017-03-01 Thread Dong Lin
Hi Ismael,

I actually mean log_start_offset. I realized that it is a better name after
I start implementation because "logStartOffset" is already used in
Log.scala and LogCleanerManager.scala. So I changed it from
log_begin_offset to log_start_offset in the patch. But I forgot to update
the KIP and specify it in the mailing thread.

Thanks for catching this. Let me update the KIP to reflect this change.

Dong


On Wed, Mar 1, 2017 at 6:15 AM, Ismael Juma  wrote:

> Hi Dong,
>
> When you say "logStartOffset", do you mean "log_begin_offset "? I could
> only find the latter in the KIP. If so, would log_start_offset be a better
> name?
>
> Ismael
>
> On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin  wrote:
>
> > Hi Jun and everyone,
> >
> > I would like to change the KIP in the following way. Currently, if any
> > replica if offline, the purge result for a partition will
> > be NotEnoughReplicasException and its low_watermark will be 0. The
> > motivation for this approach is that we want to guarantee that the data
> > before purgedOffset has been deleted on all replicas of this partition if
> > purge result indicates success.
> >
> > But this approach seems too conservative. It should be sufficient in most
> > cases to just tell user success and set low_watermark to minimum
> > logStartOffset of all live replicas in the PurgeResponse if
> logStartOffset
> > of all live replicas have reached purgedOffset. This is because for an
> > offline replicas to become online and be elected leader, it should have
> > received one FetchReponse from the current leader which should tell it to
> > purge beyond purgedOffset. The benefit of doing this change is that we
> can
> > allow purge operation to succeed when some replica is offline.
> >
> > Are you OK with this change? If so, I will go ahead to update the KIP and
> > implement this behavior.
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:
> >
> > > Hey Jun,
> > >
> > > Do you have time to review the KIP again or vote for it?
> > >
> > > Hey Ewen,
> > >
> > > Can you also review the KIP again or vote for it? I have discussed with
> > > Radai and Becket regarding your concern. We still think putting it in
> > Admin
> > > Client seems more intuitive because there is use-case where application
> > > which manages topic or produces data may also want to purge data. It
> > seems
> > > weird if they need to create a consumer to do this.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> > > gharatmayures...@gmail.com> wrote:
> > >
> > >> +1 (non-binding)
> > >>
> > >> Thanks,
> > >>
> > >> Mayuresh
> > >>
> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
> wrote:
> > >>
> > >> > Sorry for the duplicated email. It seems that gmail will put the
> > voting
> > >> > email in this thread if I simply replace DISCUSS with VOTE in the
> > >> subject.
> > >> >
> > >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> > wrote:
> > >> >
> > >> > > Hi all,
> > >> > >
> > >> > > It seems that there is no further concern with the KIP-107. At
> this
> > >> point
> > >> > > we would like to start the voting process. The KIP can be found at
> > >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> > >> > >
> > >> > > Thanks,
> > >> > > Dong
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -Regards,
> > >> Mayuresh R. Gharat
> > >> (862) 250-7125
> > >>
> > >
> > >
> >
>


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

2017-03-01 Thread Ismael Juma
Hi Dong,

When you say "logStartOffset", do you mean "log_begin_offset "? I could
only find the latter in the KIP. If so, would log_start_offset be a better
name?

Ismael

On Tue, Feb 28, 2017 at 4:26 AM, Dong Lin  wrote:

> Hi Jun and everyone,
>
> I would like to change the KIP in the following way. Currently, if any
> replica if offline, the purge result for a partition will
> be NotEnoughReplicasException and its low_watermark will be 0. The
> motivation for this approach is that we want to guarantee that the data
> before purgedOffset has been deleted on all replicas of this partition if
> purge result indicates success.
>
> But this approach seems too conservative. It should be sufficient in most
> cases to just tell user success and set low_watermark to minimum
> logStartOffset of all live replicas in the PurgeResponse if logStartOffset
> of all live replicas have reached purgedOffset. This is because for an
> offline replicas to become online and be elected leader, it should have
> received one FetchReponse from the current leader which should tell it to
> purge beyond purgedOffset. The benefit of doing this change is that we can
> allow purge operation to succeed when some replica is offline.
>
> Are you OK with this change? If so, I will go ahead to update the KIP and
> implement this behavior.
>
> Thanks,
> Dong
>
>
>
> On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:
>
> > Hey Jun,
> >
> > Do you have time to review the KIP again or vote for it?
> >
> > Hey Ewen,
> >
> > Can you also review the KIP again or vote for it? I have discussed with
> > Radai and Becket regarding your concern. We still think putting it in
> Admin
> > Client seems more intuitive because there is use-case where application
> > which manages topic or produces data may also want to purge data. It
> seems
> > weird if they need to create a consumer to do this.
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> > gharatmayures...@gmail.com> wrote:
> >
> >> +1 (non-binding)
> >>
> >> Thanks,
> >>
> >> Mayuresh
> >>
> >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin  wrote:
> >>
> >> > Sorry for the duplicated email. It seems that gmail will put the
> voting
> >> > email in this thread if I simply replace DISCUSS with VOTE in the
> >> subject.
> >> >
> >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> wrote:
> >> >
> >> > > Hi all,
> >> > >
> >> > > It seems that there is no further concern with the KIP-107. At this
> >> point
> >> > > we would like to start the voting process. The KIP can be found at
> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> -Regards,
> >> Mayuresh R. Gharat
> >> (862) 250-7125
> >>
> >
> >
>


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

2017-02-28 Thread Dong Lin
Thanks Jun. I have updated the KIP to reflect this change.

On Tue, Feb 28, 2017 at 9:44 AM, Jun Rao  wrote:

> Hi, Dong,
>
> Yes, this change makes sense to me.
>
> Thanks,
>
> Jun
>
> On Mon, Feb 27, 2017 at 8:26 PM, Dong Lin  wrote:
>
> > Hi Jun and everyone,
> >
> > I would like to change the KIP in the following way. Currently, if any
> > replica if offline, the purge result for a partition will
> > be NotEnoughReplicasException and its low_watermark will be 0. The
> > motivation for this approach is that we want to guarantee that the data
> > before purgedOffset has been deleted on all replicas of this partition if
> > purge result indicates success.
> >
> > But this approach seems too conservative. It should be sufficient in most
> > cases to just tell user success and set low_watermark to minimum
> > logStartOffset of all live replicas in the PurgeResponse if
> logStartOffset
> > of all live replicas have reached purgedOffset. This is because for an
> > offline replicas to become online and be elected leader, it should have
> > received one FetchReponse from the current leader which should tell it to
> > purge beyond purgedOffset. The benefit of doing this change is that we
> can
> > allow purge operation to succeed when some replica is offline.
> >
> > Are you OK with this change? If so, I will go ahead to update the KIP and
> > implement this behavior.
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:
> >
> > > Hey Jun,
> > >
> > > Do you have time to review the KIP again or vote for it?
> > >
> > > Hey Ewen,
> > >
> > > Can you also review the KIP again or vote for it? I have discussed with
> > > Radai and Becket regarding your concern. We still think putting it in
> > Admin
> > > Client seems more intuitive because there is use-case where application
> > > which manages topic or produces data may also want to purge data. It
> > seems
> > > weird if they need to create a consumer to do this.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> > > gharatmayures...@gmail.com> wrote:
> > >
> > >> +1 (non-binding)
> > >>
> > >> Thanks,
> > >>
> > >> Mayuresh
> > >>
> > >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin 
> wrote:
> > >>
> > >> > Sorry for the duplicated email. It seems that gmail will put the
> > voting
> > >> > email in this thread if I simply replace DISCUSS with VOTE in the
> > >> subject.
> > >> >
> > >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> > wrote:
> > >> >
> > >> > > Hi all,
> > >> > >
> > >> > > It seems that there is no further concern with the KIP-107. At
> this
> > >> point
> > >> > > we would like to start the voting process. The KIP can be found at
> > >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> > >> > >
> > >> > > Thanks,
> > >> > > Dong
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -Regards,
> > >> Mayuresh R. Gharat
> > >> (862) 250-7125
> > >>
> > >
> > >
> >
>


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

2017-02-28 Thread Jun Rao
Hi, Dong,

Yes, this change makes sense to me.

Thanks,

Jun

On Mon, Feb 27, 2017 at 8:26 PM, Dong Lin  wrote:

> Hi Jun and everyone,
>
> I would like to change the KIP in the following way. Currently, if any
> replica if offline, the purge result for a partition will
> be NotEnoughReplicasException and its low_watermark will be 0. The
> motivation for this approach is that we want to guarantee that the data
> before purgedOffset has been deleted on all replicas of this partition if
> purge result indicates success.
>
> But this approach seems too conservative. It should be sufficient in most
> cases to just tell user success and set low_watermark to minimum
> logStartOffset of all live replicas in the PurgeResponse if logStartOffset
> of all live replicas have reached purgedOffset. This is because for an
> offline replicas to become online and be elected leader, it should have
> received one FetchReponse from the current leader which should tell it to
> purge beyond purgedOffset. The benefit of doing this change is that we can
> allow purge operation to succeed when some replica is offline.
>
> Are you OK with this change? If so, I will go ahead to update the KIP and
> implement this behavior.
>
> Thanks,
> Dong
>
>
>
> On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:
>
> > Hey Jun,
> >
> > Do you have time to review the KIP again or vote for it?
> >
> > Hey Ewen,
> >
> > Can you also review the KIP again or vote for it? I have discussed with
> > Radai and Becket regarding your concern. We still think putting it in
> Admin
> > Client seems more intuitive because there is use-case where application
> > which manages topic or produces data may also want to purge data. It
> seems
> > weird if they need to create a consumer to do this.
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> > gharatmayures...@gmail.com> wrote:
> >
> >> +1 (non-binding)
> >>
> >> Thanks,
> >>
> >> Mayuresh
> >>
> >> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin  wrote:
> >>
> >> > Sorry for the duplicated email. It seems that gmail will put the
> voting
> >> > email in this thread if I simply replace DISCUSS with VOTE in the
> >> subject.
> >> >
> >> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin 
> wrote:
> >> >
> >> > > Hi all,
> >> > >
> >> > > It seems that there is no further concern with the KIP-107. At this
> >> point
> >> > > we would like to start the voting process. The KIP can be found at
> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> -Regards,
> >> Mayuresh R. Gharat
> >> (862) 250-7125
> >>
> >
> >
>


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

2017-02-27 Thread Dong Lin
Hi Jun and everyone,

I would like to change the KIP in the following way. Currently, if any
replica if offline, the purge result for a partition will
be NotEnoughReplicasException and its low_watermark will be 0. The
motivation for this approach is that we want to guarantee that the data
before purgedOffset has been deleted on all replicas of this partition if
purge result indicates success.

But this approach seems too conservative. It should be sufficient in most
cases to just tell user success and set low_watermark to minimum
logStartOffset of all live replicas in the PurgeResponse if logStartOffset
of all live replicas have reached purgedOffset. This is because for an
offline replicas to become online and be elected leader, it should have
received one FetchReponse from the current leader which should tell it to
purge beyond purgedOffset. The benefit of doing this change is that we can
allow purge operation to succeed when some replica is offline.

Are you OK with this change? If so, I will go ahead to update the KIP and
implement this behavior.

Thanks,
Dong



On Tue, Jan 17, 2017 at 10:18 AM, Dong Lin  wrote:

> Hey Jun,
>
> Do you have time to review the KIP again or vote for it?
>
> Hey Ewen,
>
> Can you also review the KIP again or vote for it? I have discussed with
> Radai and Becket regarding your concern. We still think putting it in Admin
> Client seems more intuitive because there is use-case where application
> which manages topic or produces data may also want to purge data. It seems
> weird if they need to create a consumer to do this.
>
> Thanks,
> Dong
>
> On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat <
> gharatmayures...@gmail.com> wrote:
>
>> +1 (non-binding)
>>
>> Thanks,
>>
>> Mayuresh
>>
>> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin  wrote:
>>
>> > Sorry for the duplicated email. It seems that gmail will put the voting
>> > email in this thread if I simply replace DISCUSS with VOTE in the
>> subject.
>> >
>> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin  wrote:
>> >
>> > > Hi all,
>> > >
>> > > It seems that there is no further concern with the KIP-107. At this
>> point
>> > > we would like to start the voting process. The KIP can be found at
>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
>> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>> > >
>> > > Thanks,
>> > > Dong
>> > >
>> >
>>
>>
>>
>> --
>> -Regards,
>> Mayuresh R. Gharat
>> (862) 250-7125
>>
>
>


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

2017-01-31 Thread Apurva Mehta
got it. I missed the message in the middle declaring that the vote passed.

On Tue, Jan 31, 2017 at 5:51 PM, Dong Lin  wrote:

> This thread was been closed on Jan 18. We had more discussion after
> Guozhang's feedback on Jan 21. But no major change was made to the KIP
> after the discussion.
>
>
> On Tue, Jan 31, 2017 at 5:47 PM, Dong Lin  wrote:
>
> > Hey Apurva,
> >
> > I think the KIP table in https://cwiki.apache.org/co
> > nfluence/display/KAFKA/Kafka+Improvement+Proposals has already been
> > updated. Is there anything I missed?
> >
> > Thanks,
> > Dong
> >
> > On Tue, Jan 31, 2017 at 5:45 PM, Apurva Mehta 
> wrote:
> >
> >> Hi Dong,
> >>
> >> It looks like this vote passed. Can you close this thread and update the
> >> KIP table?
> >>
> >> Thanks,
> >> Apurva
> >>
> >> On Tue, Jan 24, 2017 at 1:30 PM, Jun Rao  wrote:
> >>
> >> > Hi, Dong,
> >> >
> >> > The changes sound good to me. Also, thanks for the explanation of
> >> returning
> >> > a future from purgeDataFrom(). We can keep it that way.
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Mon, Jan 23, 2017 at 4:24 PM, Dong Lin 
> wrote:
> >> >
> >> > > Hi all,
> >> > >
> >> > > When I am implementing the patch, I realized that the current usage
> of
> >> > > "low_watermark" is a bit confusing. So I made the following
> interface
> >> > > changes in the KIP:
> >> > >
> >> > > - The newly added checkpoint file will be named
> >> > log-begin-offset-checkpoint
> >> > > - Replace low_watermark with log_begin_offset in
> FetchRequestPartition
> >> > and
> >> > > FetchResponsePartitionHeader
> >> > >
> >> > > The problem with the previous naming conversion is that,
> low_watermark
> >> > > implies minimum log begin offset of all replicas (similar to high
> >> > > watermark) and we return this value in the PurgeResponse. In other
> >> words,
> >> > > low_watermark can not be incremented if a follower is not live.
> >> Therefore
> >> > > we can not use low_watermark in the checkpoint file or in the
> >> > FetchResponse
> >> > > from leader to followers if we want to persists the offset-to-purge
> >> > > received from user across broker rebounce.
> >> > >
> >> > > You can find the changes in KIP here
> >> > >  diffpagesbyversion.action?
> >> > > pageId=67636826=13=14>.
> >> > > Please let me know if you have any concern with this change.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> > > On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin 
> >> wrote:
> >> > >
> >> > > > Thanks for the comment Jun.
> >> > > >
> >> > > > Yeah, I think there is use-case where this can be useful. Allowing
> >> for
> >> > > > asynchronous delete will be useful if an application doesn't need
> >> > strong
> >> > > > guarantee of purgeDataFrom(), e.g. if it is done to help reduce
> disk
> >> > > usage
> >> > > > of kafka. The application may want to purge data for every time it
> >> does
> >> > > > auto-commit without wait for future object to complete. On the
> other
> >> > > hand,
> >> > > > synchronous delete will be useful if an application wants to make
> >> sure
> >> > > that
> >> > > > the sensitive or bad data is definitely deleted. I think
> returning a
> >> > > future
> >> > > > makes both choice available to user and it doesn't complicate
> >> > > > implementation much.
> >> > > >
> >> > > >
> >> > > > On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao 
> wrote:
> >> > > >
> >> > > >> I feel that it's simpler to just keep the format of the
> checkpoint
> >> > file
> >> > > as
> >> > > >> it is and just add a separate checkpoint for low watermark. Low
> >> > > watermark
> >> > > >> and high watermark are maintained independently. So, not sure if
> >> there
> >> > > is
> >> > > >> significant benefit of storing them together.
> >> > > >>
> >> > > >> Looking at the KIP again. I actually have another question on the
> >> api.
> >> > > Is
> >> > > >> there any benefit of returning a Future in the purgeDataBefore()
> >> api?
> >> > > >> Since
> >> > > >> admin apis are used infrequently, it seems that it's simpler to
> >> just
> >> > > have
> >> > > >> a
> >> > > >> blocking api and returns Map?
> >> > > >>
> >> > > >> Thanks,
> >> > > >>
> >> > > >> Jun
> >> > > >>
> >> > > >> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin 
> >> > wrote:
> >> > > >>
> >> > > >> > Thanks for the comment Guozhang. Please don't worry about being
> >> > late.
> >> > > I
> >> > > >> > would like to update the KIP if there is clear benefit of the
> new
> >> > > >> approach.
> >> > > >> > I am wondering if there is any use-case or operation aspects
> that
> >> > > would
> >> > > >> > benefit from the new approach.
> >> > > >> >
> >> > > >> > I am not saying that these checkpoint files have the same
> >> priority.
> >> > I
> >> > > >> > mentioned other checkpoint 

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

2017-01-31 Thread Dong Lin
This thread was been closed on Jan 18. We had more discussion after
Guozhang's feedback on Jan 21. But no major change was made to the KIP
after the discussion.


On Tue, Jan 31, 2017 at 5:47 PM, Dong Lin  wrote:

> Hey Apurva,
>
> I think the KIP table in https://cwiki.apache.org/co
> nfluence/display/KAFKA/Kafka+Improvement+Proposals has already been
> updated. Is there anything I missed?
>
> Thanks,
> Dong
>
> On Tue, Jan 31, 2017 at 5:45 PM, Apurva Mehta  wrote:
>
>> Hi Dong,
>>
>> It looks like this vote passed. Can you close this thread and update the
>> KIP table?
>>
>> Thanks,
>> Apurva
>>
>> On Tue, Jan 24, 2017 at 1:30 PM, Jun Rao  wrote:
>>
>> > Hi, Dong,
>> >
>> > The changes sound good to me. Also, thanks for the explanation of
>> returning
>> > a future from purgeDataFrom(). We can keep it that way.
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> > On Mon, Jan 23, 2017 at 4:24 PM, Dong Lin  wrote:
>> >
>> > > Hi all,
>> > >
>> > > When I am implementing the patch, I realized that the current usage of
>> > > "low_watermark" is a bit confusing. So I made the following interface
>> > > changes in the KIP:
>> > >
>> > > - The newly added checkpoint file will be named
>> > log-begin-offset-checkpoint
>> > > - Replace low_watermark with log_begin_offset in FetchRequestPartition
>> > and
>> > > FetchResponsePartitionHeader
>> > >
>> > > The problem with the previous naming conversion is that, low_watermark
>> > > implies minimum log begin offset of all replicas (similar to high
>> > > watermark) and we return this value in the PurgeResponse. In other
>> words,
>> > > low_watermark can not be incremented if a follower is not live.
>> Therefore
>> > > we can not use low_watermark in the checkpoint file or in the
>> > FetchResponse
>> > > from leader to followers if we want to persists the offset-to-purge
>> > > received from user across broker rebounce.
>> > >
>> > > You can find the changes in KIP here
>> > > > > > pageId=67636826=13=14>.
>> > > Please let me know if you have any concern with this change.
>> > >
>> > > Thanks,
>> > > Dong
>> > >
>> > > On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin 
>> wrote:
>> > >
>> > > > Thanks for the comment Jun.
>> > > >
>> > > > Yeah, I think there is use-case where this can be useful. Allowing
>> for
>> > > > asynchronous delete will be useful if an application doesn't need
>> > strong
>> > > > guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk
>> > > usage
>> > > > of kafka. The application may want to purge data for every time it
>> does
>> > > > auto-commit without wait for future object to complete. On the other
>> > > hand,
>> > > > synchronous delete will be useful if an application wants to make
>> sure
>> > > that
>> > > > the sensitive or bad data is definitely deleted. I think returning a
>> > > future
>> > > > makes both choice available to user and it doesn't complicate
>> > > > implementation much.
>> > > >
>> > > >
>> > > > On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:
>> > > >
>> > > >> I feel that it's simpler to just keep the format of the checkpoint
>> > file
>> > > as
>> > > >> it is and just add a separate checkpoint for low watermark. Low
>> > > watermark
>> > > >> and high watermark are maintained independently. So, not sure if
>> there
>> > > is
>> > > >> significant benefit of storing them together.
>> > > >>
>> > > >> Looking at the KIP again. I actually have another question on the
>> api.
>> > > Is
>> > > >> there any benefit of returning a Future in the purgeDataBefore()
>> api?
>> > > >> Since
>> > > >> admin apis are used infrequently, it seems that it's simpler to
>> just
>> > > have
>> > > >> a
>> > > >> blocking api and returns Map?
>> > > >>
>> > > >> Thanks,
>> > > >>
>> > > >> Jun
>> > > >>
>> > > >> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin 
>> > wrote:
>> > > >>
>> > > >> > Thanks for the comment Guozhang. Please don't worry about being
>> > late.
>> > > I
>> > > >> > would like to update the KIP if there is clear benefit of the new
>> > > >> approach.
>> > > >> > I am wondering if there is any use-case or operation aspects that
>> > > would
>> > > >> > benefit from the new approach.
>> > > >> >
>> > > >> > I am not saying that these checkpoint files have the same
>> priority.
>> > I
>> > > >> > mentioned other checkpoint files to suggest that it is OK to add
>> one
>> > > >> more
>> > > >> > checkpoint file. To me three checkpoint files is not much
>> different
>> > > from
>> > > >> > four checkpoint files. I am just inclined to not update the KIP
>> if
>> > the
>> > > >> only
>> > > >> > benefit is to avoid addition of a new checkpoint file.
>> > > >> >
>> > > >> >
>> > > >> >
>> > > >> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang <
>> wangg...@gmail.com>
>> > > 

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

2017-01-31 Thread Dong Lin
Hey Apurva,

I think the KIP table in
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
has already been updated. Is there anything I missed?

Thanks,
Dong

On Tue, Jan 31, 2017 at 5:45 PM, Apurva Mehta  wrote:

> Hi Dong,
>
> It looks like this vote passed. Can you close this thread and update the
> KIP table?
>
> Thanks,
> Apurva
>
> On Tue, Jan 24, 2017 at 1:30 PM, Jun Rao  wrote:
>
> > Hi, Dong,
> >
> > The changes sound good to me. Also, thanks for the explanation of
> returning
> > a future from purgeDataFrom(). We can keep it that way.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 23, 2017 at 4:24 PM, Dong Lin  wrote:
> >
> > > Hi all,
> > >
> > > When I am implementing the patch, I realized that the current usage of
> > > "low_watermark" is a bit confusing. So I made the following interface
> > > changes in the KIP:
> > >
> > > - The newly added checkpoint file will be named
> > log-begin-offset-checkpoint
> > > - Replace low_watermark with log_begin_offset in FetchRequestPartition
> > and
> > > FetchResponsePartitionHeader
> > >
> > > The problem with the previous naming conversion is that, low_watermark
> > > implies minimum log begin offset of all replicas (similar to high
> > > watermark) and we return this value in the PurgeResponse. In other
> words,
> > > low_watermark can not be incremented if a follower is not live.
> Therefore
> > > we can not use low_watermark in the checkpoint file or in the
> > FetchResponse
> > > from leader to followers if we want to persists the offset-to-purge
> > > received from user across broker rebounce.
> > >
> > > You can find the changes in KIP here
> > >  > > pageId=67636826=13=14>.
> > > Please let me know if you have any concern with this change.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin 
> wrote:
> > >
> > > > Thanks for the comment Jun.
> > > >
> > > > Yeah, I think there is use-case where this can be useful. Allowing
> for
> > > > asynchronous delete will be useful if an application doesn't need
> > strong
> > > > guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk
> > > usage
> > > > of kafka. The application may want to purge data for every time it
> does
> > > > auto-commit without wait for future object to complete. On the other
> > > hand,
> > > > synchronous delete will be useful if an application wants to make
> sure
> > > that
> > > > the sensitive or bad data is definitely deleted. I think returning a
> > > future
> > > > makes both choice available to user and it doesn't complicate
> > > > implementation much.
> > > >
> > > >
> > > > On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:
> > > >
> > > >> I feel that it's simpler to just keep the format of the checkpoint
> > file
> > > as
> > > >> it is and just add a separate checkpoint for low watermark. Low
> > > watermark
> > > >> and high watermark are maintained independently. So, not sure if
> there
> > > is
> > > >> significant benefit of storing them together.
> > > >>
> > > >> Looking at the KIP again. I actually have another question on the
> api.
> > > Is
> > > >> there any benefit of returning a Future in the purgeDataBefore()
> api?
> > > >> Since
> > > >> admin apis are used infrequently, it seems that it's simpler to just
> > > have
> > > >> a
> > > >> blocking api and returns Map?
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jun
> > > >>
> > > >> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin 
> > wrote:
> > > >>
> > > >> > Thanks for the comment Guozhang. Please don't worry about being
> > late.
> > > I
> > > >> > would like to update the KIP if there is clear benefit of the new
> > > >> approach.
> > > >> > I am wondering if there is any use-case or operation aspects that
> > > would
> > > >> > benefit from the new approach.
> > > >> >
> > > >> > I am not saying that these checkpoint files have the same
> priority.
> > I
> > > >> > mentioned other checkpoint files to suggest that it is OK to add
> one
> > > >> more
> > > >> > checkpoint file. To me three checkpoint files is not much
> different
> > > from
> > > >> > four checkpoint files. I am just inclined to not update the KIP if
> > the
> > > >> only
> > > >> > benefit is to avoid addition of a new checkpoint file.
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang <
> wangg...@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> > > To me the distinction between recovery-checkpoint and
> > > >> > > replication-checkpoint are different from the distinction
> between
> > > >> these
> > > >> > two
> > > >> > > hw checkpoint values: when broker starts up and act as the
> leader
> > > for
> > > >> a
> > > >> > > partition, it can live without seeing the recovery checkpoint,
> but
> > > >> 

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

2017-01-31 Thread Apurva Mehta
Hi Dong,

It looks like this vote passed. Can you close this thread and update the
KIP table?

Thanks,
Apurva

On Tue, Jan 24, 2017 at 1:30 PM, Jun Rao  wrote:

> Hi, Dong,
>
> The changes sound good to me. Also, thanks for the explanation of returning
> a future from purgeDataFrom(). We can keep it that way.
>
> Thanks,
>
> Jun
>
> On Mon, Jan 23, 2017 at 4:24 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > When I am implementing the patch, I realized that the current usage of
> > "low_watermark" is a bit confusing. So I made the following interface
> > changes in the KIP:
> >
> > - The newly added checkpoint file will be named
> log-begin-offset-checkpoint
> > - Replace low_watermark with log_begin_offset in FetchRequestPartition
> and
> > FetchResponsePartitionHeader
> >
> > The problem with the previous naming conversion is that, low_watermark
> > implies minimum log begin offset of all replicas (similar to high
> > watermark) and we return this value in the PurgeResponse. In other words,
> > low_watermark can not be incremented if a follower is not live. Therefore
> > we can not use low_watermark in the checkpoint file or in the
> FetchResponse
> > from leader to followers if we want to persists the offset-to-purge
> > received from user across broker rebounce.
> >
> > You can find the changes in KIP here
> >  > pageId=67636826=13=14>.
> > Please let me know if you have any concern with this change.
> >
> > Thanks,
> > Dong
> >
> > On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin  wrote:
> >
> > > Thanks for the comment Jun.
> > >
> > > Yeah, I think there is use-case where this can be useful. Allowing for
> > > asynchronous delete will be useful if an application doesn't need
> strong
> > > guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk
> > usage
> > > of kafka. The application may want to purge data for every time it does
> > > auto-commit without wait for future object to complete. On the other
> > hand,
> > > synchronous delete will be useful if an application wants to make sure
> > that
> > > the sensitive or bad data is definitely deleted. I think returning a
> > future
> > > makes both choice available to user and it doesn't complicate
> > > implementation much.
> > >
> > >
> > > On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:
> > >
> > >> I feel that it's simpler to just keep the format of the checkpoint
> file
> > as
> > >> it is and just add a separate checkpoint for low watermark. Low
> > watermark
> > >> and high watermark are maintained independently. So, not sure if there
> > is
> > >> significant benefit of storing them together.
> > >>
> > >> Looking at the KIP again. I actually have another question on the api.
> > Is
> > >> there any benefit of returning a Future in the purgeDataBefore() api?
> > >> Since
> > >> admin apis are used infrequently, it seems that it's simpler to just
> > have
> > >> a
> > >> blocking api and returns Map?
> > >>
> > >> Thanks,
> > >>
> > >> Jun
> > >>
> > >> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin 
> wrote:
> > >>
> > >> > Thanks for the comment Guozhang. Please don't worry about being
> late.
> > I
> > >> > would like to update the KIP if there is clear benefit of the new
> > >> approach.
> > >> > I am wondering if there is any use-case or operation aspects that
> > would
> > >> > benefit from the new approach.
> > >> >
> > >> > I am not saying that these checkpoint files have the same priority.
> I
> > >> > mentioned other checkpoint files to suggest that it is OK to add one
> > >> more
> > >> > checkpoint file. To me three checkpoint files is not much different
> > from
> > >> > four checkpoint files. I am just inclined to not update the KIP if
> the
> > >> only
> > >> > benefit is to avoid addition of a new checkpoint file.
> > >> >
> > >> >
> > >> >
> > >> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang 
> > >> wrote:
> > >> >
> > >> > > To me the distinction between recovery-checkpoint and
> > >> > > replication-checkpoint are different from the distinction between
> > >> these
> > >> > two
> > >> > > hw checkpoint values: when broker starts up and act as the leader
> > for
> > >> a
> > >> > > partition, it can live without seeing the recovery checkpoint, but
> > >> just
> > >> > > cannot rely on the existing last log segment and need to fetch
> from
> > >> other
> > >> > > replicas; but if the replication-checkpoint file is missing, it
> is a
> > >> > > correctness issue, as it does not know from where to truncate its
> > >> data,
> > >> > and
> > >> > > also how to respond to a fetch request. That is why I think we can
> > >> > separate
> > >> > > these two types of files, since the latter one is more important
> > than
> > >> the
> > >> > > previous one.
> > >> > >
> > >> > > That being said, I do not want to 

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

2017-01-24 Thread Jun Rao
Hi, Dong,

The changes sound good to me. Also, thanks for the explanation of returning
a future from purgeDataFrom(). We can keep it that way.

Thanks,

Jun

On Mon, Jan 23, 2017 at 4:24 PM, Dong Lin  wrote:

> Hi all,
>
> When I am implementing the patch, I realized that the current usage of
> "low_watermark" is a bit confusing. So I made the following interface
> changes in the KIP:
>
> - The newly added checkpoint file will be named log-begin-offset-checkpoint
> - Replace low_watermark with log_begin_offset in FetchRequestPartition and
> FetchResponsePartitionHeader
>
> The problem with the previous naming conversion is that, low_watermark
> implies minimum log begin offset of all replicas (similar to high
> watermark) and we return this value in the PurgeResponse. In other words,
> low_watermark can not be incremented if a follower is not live. Therefore
> we can not use low_watermark in the checkpoint file or in the FetchResponse
> from leader to followers if we want to persists the offset-to-purge
> received from user across broker rebounce.
>
> You can find the changes in KIP here
>  pageId=67636826=13=14>.
> Please let me know if you have any concern with this change.
>
> Thanks,
> Dong
>
> On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin  wrote:
>
> > Thanks for the comment Jun.
> >
> > Yeah, I think there is use-case where this can be useful. Allowing for
> > asynchronous delete will be useful if an application doesn't need strong
> > guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk
> usage
> > of kafka. The application may want to purge data for every time it does
> > auto-commit without wait for future object to complete. On the other
> hand,
> > synchronous delete will be useful if an application wants to make sure
> that
> > the sensitive or bad data is definitely deleted. I think returning a
> future
> > makes both choice available to user and it doesn't complicate
> > implementation much.
> >
> >
> > On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:
> >
> >> I feel that it's simpler to just keep the format of the checkpoint file
> as
> >> it is and just add a separate checkpoint for low watermark. Low
> watermark
> >> and high watermark are maintained independently. So, not sure if there
> is
> >> significant benefit of storing them together.
> >>
> >> Looking at the KIP again. I actually have another question on the api.
> Is
> >> there any benefit of returning a Future in the purgeDataBefore() api?
> >> Since
> >> admin apis are used infrequently, it seems that it's simpler to just
> have
> >> a
> >> blocking api and returns Map?
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin  wrote:
> >>
> >> > Thanks for the comment Guozhang. Please don't worry about being late.
> I
> >> > would like to update the KIP if there is clear benefit of the new
> >> approach.
> >> > I am wondering if there is any use-case or operation aspects that
> would
> >> > benefit from the new approach.
> >> >
> >> > I am not saying that these checkpoint files have the same priority. I
> >> > mentioned other checkpoint files to suggest that it is OK to add one
> >> more
> >> > checkpoint file. To me three checkpoint files is not much different
> from
> >> > four checkpoint files. I am just inclined to not update the KIP if the
> >> only
> >> > benefit is to avoid addition of a new checkpoint file.
> >> >
> >> >
> >> >
> >> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang 
> >> wrote:
> >> >
> >> > > To me the distinction between recovery-checkpoint and
> >> > > replication-checkpoint are different from the distinction between
> >> these
> >> > two
> >> > > hw checkpoint values: when broker starts up and act as the leader
> for
> >> a
> >> > > partition, it can live without seeing the recovery checkpoint, but
> >> just
> >> > > cannot rely on the existing last log segment and need to fetch from
> >> other
> >> > > replicas; but if the replication-checkpoint file is missing, it is a
> >> > > correctness issue, as it does not know from where to truncate its
> >> data,
> >> > and
> >> > > also how to respond to a fetch request. That is why I think we can
> >> > separate
> >> > > these two types of files, since the latter one is more important
> than
> >> the
> >> > > previous one.
> >> > >
> >> > > That being said, I do not want to recall another vote on this since
> >> it is
> >> > > my bad not responding before the vote is called. Just wanted to
> point
> >> out
> >> > > for the record that this approach may have some operational
> scenarios
> >> > where
> >> > > one of the replication files is missing and we need to treat them
> >> > > specifically.
> >> > >
> >> > >
> >> > > Guozhang
> >> > >
> >> > >
> >> > > On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin 

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

2017-01-23 Thread Dong Lin
Hi all,

When I am implementing the patch, I realized that the current usage of
"low_watermark" is a bit confusing. So I made the following interface
changes in the KIP:

- The newly added checkpoint file will be named log-begin-offset-checkpoint
- Replace low_watermark with log_begin_offset in FetchRequestPartition and
FetchResponsePartitionHeader

The problem with the previous naming conversion is that, low_watermark
implies minimum log begin offset of all replicas (similar to high
watermark) and we return this value in the PurgeResponse. In other words,
low_watermark can not be incremented if a follower is not live. Therefore
we can not use low_watermark in the checkpoint file or in the FetchResponse
from leader to followers if we want to persists the offset-to-purge
received from user across broker rebounce.

You can find the changes in KIP here
.
Please let me know if you have any concern with this change.

Thanks,
Dong

On Mon, Jan 23, 2017 at 11:20 AM, Dong Lin  wrote:

> Thanks for the comment Jun.
>
> Yeah, I think there is use-case where this can be useful. Allowing for
> asynchronous delete will be useful if an application doesn't need strong
> guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk usage
> of kafka. The application may want to purge data for every time it does
> auto-commit without wait for future object to complete. On the other hand,
> synchronous delete will be useful if an application wants to make sure that
> the sensitive or bad data is definitely deleted. I think returning a future
> makes both choice available to user and it doesn't complicate
> implementation much.
>
>
> On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:
>
>> I feel that it's simpler to just keep the format of the checkpoint file as
>> it is and just add a separate checkpoint for low watermark. Low watermark
>> and high watermark are maintained independently. So, not sure if there is
>> significant benefit of storing them together.
>>
>> Looking at the KIP again. I actually have another question on the api. Is
>> there any benefit of returning a Future in the purgeDataBefore() api?
>> Since
>> admin apis are used infrequently, it seems that it's simpler to just have
>> a
>> blocking api and returns Map?
>>
>> Thanks,
>>
>> Jun
>>
>> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin  wrote:
>>
>> > Thanks for the comment Guozhang. Please don't worry about being late. I
>> > would like to update the KIP if there is clear benefit of the new
>> approach.
>> > I am wondering if there is any use-case or operation aspects that would
>> > benefit from the new approach.
>> >
>> > I am not saying that these checkpoint files have the same priority. I
>> > mentioned other checkpoint files to suggest that it is OK to add one
>> more
>> > checkpoint file. To me three checkpoint files is not much different from
>> > four checkpoint files. I am just inclined to not update the KIP if the
>> only
>> > benefit is to avoid addition of a new checkpoint file.
>> >
>> >
>> >
>> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang 
>> wrote:
>> >
>> > > To me the distinction between recovery-checkpoint and
>> > > replication-checkpoint are different from the distinction between
>> these
>> > two
>> > > hw checkpoint values: when broker starts up and act as the leader for
>> a
>> > > partition, it can live without seeing the recovery checkpoint, but
>> just
>> > > cannot rely on the existing last log segment and need to fetch from
>> other
>> > > replicas; but if the replication-checkpoint file is missing, it is a
>> > > correctness issue, as it does not know from where to truncate its
>> data,
>> > and
>> > > also how to respond to a fetch request. That is why I think we can
>> > separate
>> > > these two types of files, since the latter one is more important than
>> the
>> > > previous one.
>> > >
>> > > That being said, I do not want to recall another vote on this since
>> it is
>> > > my bad not responding before the vote is called. Just wanted to point
>> out
>> > > for the record that this approach may have some operational scenarios
>> > where
>> > > one of the replication files is missing and we need to treat them
>> > > specifically.
>> > >
>> > >
>> > > Guozhang
>> > >
>> > >
>> > > On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin 
>> wrote:
>> > >
>> > > > Yeah, your solution of adding new APIs certainly works and I don't
>> > think
>> > > > that is an issue. On the other hand I don't think it is an issue to
>> > add a
>> > > > new checkpoint file as well since we already have multiple
>> checkpoint
>> > > > files. The benefit of the new approach you mentioned is probably
>> not an
>> > > > issue in the current approach since high watermark and low watermark
>> > > works
>> > > > completely 

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

2017-01-23 Thread Dong Lin
Thanks for the comment Jun.

Yeah, I think there is use-case where this can be useful. Allowing for
asynchronous delete will be useful if an application doesn't need strong
guarantee of purgeDataFrom(), e.g. if it is done to help reduce disk usage
of kafka. The application may want to purge data for every time it does
auto-commit without wait for future object to complete. On the other hand,
synchronous delete will be useful if an application wants to make sure that
the sensitive or bad data is definitely deleted. I think returning a future
makes both choice available to user and it doesn't complicate
implementation much.


On Mon, Jan 23, 2017 at 10:45 AM, Jun Rao  wrote:

> I feel that it's simpler to just keep the format of the checkpoint file as
> it is and just add a separate checkpoint for low watermark. Low watermark
> and high watermark are maintained independently. So, not sure if there is
> significant benefit of storing them together.
>
> Looking at the KIP again. I actually have another question on the api. Is
> there any benefit of returning a Future in the purgeDataBefore() api? Since
> admin apis are used infrequently, it seems that it's simpler to just have a
> blocking api and returns Map?
>
> Thanks,
>
> Jun
>
> On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin  wrote:
>
> > Thanks for the comment Guozhang. Please don't worry about being late. I
> > would like to update the KIP if there is clear benefit of the new
> approach.
> > I am wondering if there is any use-case or operation aspects that would
> > benefit from the new approach.
> >
> > I am not saying that these checkpoint files have the same priority. I
> > mentioned other checkpoint files to suggest that it is OK to add one more
> > checkpoint file. To me three checkpoint files is not much different from
> > four checkpoint files. I am just inclined to not update the KIP if the
> only
> > benefit is to avoid addition of a new checkpoint file.
> >
> >
> >
> > On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang 
> wrote:
> >
> > > To me the distinction between recovery-checkpoint and
> > > replication-checkpoint are different from the distinction between these
> > two
> > > hw checkpoint values: when broker starts up and act as the leader for a
> > > partition, it can live without seeing the recovery checkpoint, but just
> > > cannot rely on the existing last log segment and need to fetch from
> other
> > > replicas; but if the replication-checkpoint file is missing, it is a
> > > correctness issue, as it does not know from where to truncate its data,
> > and
> > > also how to respond to a fetch request. That is why I think we can
> > separate
> > > these two types of files, since the latter one is more important than
> the
> > > previous one.
> > >
> > > That being said, I do not want to recall another vote on this since it
> is
> > > my bad not responding before the vote is called. Just wanted to point
> out
> > > for the record that this approach may have some operational scenarios
> > where
> > > one of the replication files is missing and we need to treat them
> > > specifically.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin  wrote:
> > >
> > > > Yeah, your solution of adding new APIs certainly works and I don't
> > think
> > > > that is an issue. On the other hand I don't think it is an issue to
> > add a
> > > > new checkpoint file as well since we already have multiple checkpoint
> > > > files. The benefit of the new approach you mentioned is probably not
> an
> > > > issue in the current approach since high watermark and low watermark
> > > works
> > > > completely independently. Since there is no strong reason to choose
> > > either
> > > > of them, I am inclined to choose the one that makes less format
> change
> > > and
> > > > simpler in the Java API. The current approach seems better w.r.t this
> > > minor
> > > > reason.
> > > >
> > > > If you are strong that we should use the new approach, I can do that
> as
> > > > well. Please let me know if you think so, and I will need to ask
> > > > Jun/Joel/Becket to vote on this again since this changes the
> interface
> > of
> > > > the KIP.
> > > >
> > > > On Sun, Jan 22, 2017 at 9:35 AM, Guozhang Wang 
> > > wrote:
> > > >
> > > > > I think this is less of an issue: we can use the same patterns as
> in
> > > the
> > > > > request protocol, i.e.:
> > > > >
> > > > > write(Map[TP, Long]) // write the checkout point in v0 format
> > > > > write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1
> > > format
> > > > >
> > > > > CheckpointedOffsets read() // read the file relying on its version
> id
> > > > >
> > > > > class CheckpointedOffsets {
> > > > >
> > > > > Integer getVersion();
> > > > > Long getFirstOffset();
> > > > > Long getSecondOffset();   // would return NO_AVAILABLE with v0
> > 

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

2017-01-23 Thread Jun Rao
I feel that it's simpler to just keep the format of the checkpoint file as
it is and just add a separate checkpoint for low watermark. Low watermark
and high watermark are maintained independently. So, not sure if there is
significant benefit of storing them together.

Looking at the KIP again. I actually have another question on the api. Is
there any benefit of returning a Future in the purgeDataBefore() api? Since
admin apis are used infrequently, it seems that it's simpler to just have a
blocking api and returns Map?

Thanks,

Jun

On Sun, Jan 22, 2017 at 3:56 PM, Dong Lin  wrote:

> Thanks for the comment Guozhang. Please don't worry about being late. I
> would like to update the KIP if there is clear benefit of the new approach.
> I am wondering if there is any use-case or operation aspects that would
> benefit from the new approach.
>
> I am not saying that these checkpoint files have the same priority. I
> mentioned other checkpoint files to suggest that it is OK to add one more
> checkpoint file. To me three checkpoint files is not much different from
> four checkpoint files. I am just inclined to not update the KIP if the only
> benefit is to avoid addition of a new checkpoint file.
>
>
>
> On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang  wrote:
>
> > To me the distinction between recovery-checkpoint and
> > replication-checkpoint are different from the distinction between these
> two
> > hw checkpoint values: when broker starts up and act as the leader for a
> > partition, it can live without seeing the recovery checkpoint, but just
> > cannot rely on the existing last log segment and need to fetch from other
> > replicas; but if the replication-checkpoint file is missing, it is a
> > correctness issue, as it does not know from where to truncate its data,
> and
> > also how to respond to a fetch request. That is why I think we can
> separate
> > these two types of files, since the latter one is more important than the
> > previous one.
> >
> > That being said, I do not want to recall another vote on this since it is
> > my bad not responding before the vote is called. Just wanted to point out
> > for the record that this approach may have some operational scenarios
> where
> > one of the replication files is missing and we need to treat them
> > specifically.
> >
> >
> > Guozhang
> >
> >
> > On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin  wrote:
> >
> > > Yeah, your solution of adding new APIs certainly works and I don't
> think
> > > that is an issue. On the other hand I don't think it is an issue to
> add a
> > > new checkpoint file as well since we already have multiple checkpoint
> > > files. The benefit of the new approach you mentioned is probably not an
> > > issue in the current approach since high watermark and low watermark
> > works
> > > completely independently. Since there is no strong reason to choose
> > either
> > > of them, I am inclined to choose the one that makes less format change
> > and
> > > simpler in the Java API. The current approach seems better w.r.t this
> > minor
> > > reason.
> > >
> > > If you are strong that we should use the new approach, I can do that as
> > > well. Please let me know if you think so, and I will need to ask
> > > Jun/Joel/Becket to vote on this again since this changes the interface
> of
> > > the KIP.
> > >
> > > On Sun, Jan 22, 2017 at 9:35 AM, Guozhang Wang 
> > wrote:
> > >
> > > > I think this is less of an issue: we can use the same patterns as in
> > the
> > > > request protocol, i.e.:
> > > >
> > > > write(Map[TP, Long]) // write the checkout point in v0 format
> > > > write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1
> > format
> > > >
> > > > CheckpointedOffsets read() // read the file relying on its version id
> > > >
> > > > class CheckpointedOffsets {
> > > >
> > > > Integer getVersion();
> > > > Long getFirstOffset();
> > > > Long getSecondOffset();   // would return NO_AVAILABLE with v0
> > format
> > > > }
> > > >
> > > >
> > > > As I think of it, another benefit is that we wont have a partition
> that
> > > > only have one of the watermarks in case of a failure in between
> writing
> > > two
> > > > files.
> > > >
> > > > Guozhang
> > > >
> > > > On Sun, Jan 22, 2017 at 12:03 AM, Dong Lin 
> > wrote:
> > > >
> > > > > Hey Guozhang,
> > > > >
> > > > > Thanks for the review:) Yes it is possible to combine them. Both
> > > solution
> > > > > will have the same performance. But I think the current solution
> will
> > > > give
> > > > > us simpler Java class design. Note that we will have to change Java
> > API
> > > > > (e.g. read() and write()) of OffsetCheckpoint class in order to
> > > provide a
> > > > > map from TopicPartition to a pair of integers when we write to
> > > checkpoint
> > > > > file. This makes this class less generic since this API is not used
> > by
> 

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

2017-01-22 Thread Dong Lin
Thanks for the comment Guozhang. Please don't worry about being late. I
would like to update the KIP if there is clear benefit of the new approach.
I am wondering if there is any use-case or operation aspects that would
benefit from the new approach.

I am not saying that these checkpoint files have the same priority. I
mentioned other checkpoint files to suggest that it is OK to add one more
checkpoint file. To me three checkpoint files is not much different from
four checkpoint files. I am just inclined to not update the KIP if the only
benefit is to avoid addition of a new checkpoint file.



On Sun, Jan 22, 2017 at 3:48 PM, Guozhang Wang  wrote:

> To me the distinction between recovery-checkpoint and
> replication-checkpoint are different from the distinction between these two
> hw checkpoint values: when broker starts up and act as the leader for a
> partition, it can live without seeing the recovery checkpoint, but just
> cannot rely on the existing last log segment and need to fetch from other
> replicas; but if the replication-checkpoint file is missing, it is a
> correctness issue, as it does not know from where to truncate its data, and
> also how to respond to a fetch request. That is why I think we can separate
> these two types of files, since the latter one is more important than the
> previous one.
>
> That being said, I do not want to recall another vote on this since it is
> my bad not responding before the vote is called. Just wanted to point out
> for the record that this approach may have some operational scenarios where
> one of the replication files is missing and we need to treat them
> specifically.
>
>
> Guozhang
>
>
> On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin  wrote:
>
> > Yeah, your solution of adding new APIs certainly works and I don't think
> > that is an issue. On the other hand I don't think it is an issue to add a
> > new checkpoint file as well since we already have multiple checkpoint
> > files. The benefit of the new approach you mentioned is probably not an
> > issue in the current approach since high watermark and low watermark
> works
> > completely independently. Since there is no strong reason to choose
> either
> > of them, I am inclined to choose the one that makes less format change
> and
> > simpler in the Java API. The current approach seems better w.r.t this
> minor
> > reason.
> >
> > If you are strong that we should use the new approach, I can do that as
> > well. Please let me know if you think so, and I will need to ask
> > Jun/Joel/Becket to vote on this again since this changes the interface of
> > the KIP.
> >
> > On Sun, Jan 22, 2017 at 9:35 AM, Guozhang Wang 
> wrote:
> >
> > > I think this is less of an issue: we can use the same patterns as in
> the
> > > request protocol, i.e.:
> > >
> > > write(Map[TP, Long]) // write the checkout point in v0 format
> > > write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1
> format
> > >
> > > CheckpointedOffsets read() // read the file relying on its version id
> > >
> > > class CheckpointedOffsets {
> > >
> > > Integer getVersion();
> > > Long getFirstOffset();
> > > Long getSecondOffset();   // would return NO_AVAILABLE with v0
> format
> > > }
> > >
> > >
> > > As I think of it, another benefit is that we wont have a partition that
> > > only have one of the watermarks in case of a failure in between writing
> > two
> > > files.
> > >
> > > Guozhang
> > >
> > > On Sun, Jan 22, 2017 at 12:03 AM, Dong Lin 
> wrote:
> > >
> > > > Hey Guozhang,
> > > >
> > > > Thanks for the review:) Yes it is possible to combine them. Both
> > solution
> > > > will have the same performance. But I think the current solution will
> > > give
> > > > us simpler Java class design. Note that we will have to change Java
> API
> > > > (e.g. read() and write()) of OffsetCheckpoint class in order to
> > provide a
> > > > map from TopicPartition to a pair of integers when we write to
> > checkpoint
> > > > file. This makes this class less generic since this API is not used
> by
> > > log
> > > > recovery checkpoint and log cleaner checkpoint which are also using
> > > > OffsetCheckpoint class.
> > > >
> > > > Dong
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Sat, Jan 21, 2017 at 12:28 PM, Guozhang Wang 
> > > > wrote:
> > > >
> > > > > Hi Dong,
> > > > >
> > > > > Sorry for being late on reviewing this KIP. It LGTM overall, but
> I'm
> > > > > wondering if we can save adding the "replication-low-watermark-
> > > > checkpoint"
> > > > > file by just bumping up the version number of "replication-offset-
> > > > > checkpoint"
> > > > > to let it have two values for each partition, i.e.:
> > > > >
> > > > > 1  // version number
> > > > > [number of partitions]
> > > > > [topic name] [partition id] [lwm] [hwm]
> > > > >
> > > > >
> > > > > This will affects the upgrade path a bit, but I think 

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

2017-01-22 Thread Guozhang Wang
To me the distinction between recovery-checkpoint and
replication-checkpoint are different from the distinction between these two
hw checkpoint values: when broker starts up and act as the leader for a
partition, it can live without seeing the recovery checkpoint, but just
cannot rely on the existing last log segment and need to fetch from other
replicas; but if the replication-checkpoint file is missing, it is a
correctness issue, as it does not know from where to truncate its data, and
also how to respond to a fetch request. That is why I think we can separate
these two types of files, since the latter one is more important than the
previous one.

That being said, I do not want to recall another vote on this since it is
my bad not responding before the vote is called. Just wanted to point out
for the record that this approach may have some operational scenarios where
one of the replication files is missing and we need to treat them
specifically.


Guozhang


On Sun, Jan 22, 2017 at 1:56 PM, Dong Lin  wrote:

> Yeah, your solution of adding new APIs certainly works and I don't think
> that is an issue. On the other hand I don't think it is an issue to add a
> new checkpoint file as well since we already have multiple checkpoint
> files. The benefit of the new approach you mentioned is probably not an
> issue in the current approach since high watermark and low watermark works
> completely independently. Since there is no strong reason to choose either
> of them, I am inclined to choose the one that makes less format change and
> simpler in the Java API. The current approach seems better w.r.t this minor
> reason.
>
> If you are strong that we should use the new approach, I can do that as
> well. Please let me know if you think so, and I will need to ask
> Jun/Joel/Becket to vote on this again since this changes the interface of
> the KIP.
>
> On Sun, Jan 22, 2017 at 9:35 AM, Guozhang Wang  wrote:
>
> > I think this is less of an issue: we can use the same patterns as in the
> > request protocol, i.e.:
> >
> > write(Map[TP, Long]) // write the checkout point in v0 format
> > write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1 format
> >
> > CheckpointedOffsets read() // read the file relying on its version id
> >
> > class CheckpointedOffsets {
> >
> > Integer getVersion();
> > Long getFirstOffset();
> > Long getSecondOffset();   // would return NO_AVAILABLE with v0 format
> > }
> >
> >
> > As I think of it, another benefit is that we wont have a partition that
> > only have one of the watermarks in case of a failure in between writing
> two
> > files.
> >
> > Guozhang
> >
> > On Sun, Jan 22, 2017 at 12:03 AM, Dong Lin  wrote:
> >
> > > Hey Guozhang,
> > >
> > > Thanks for the review:) Yes it is possible to combine them. Both
> solution
> > > will have the same performance. But I think the current solution will
> > give
> > > us simpler Java class design. Note that we will have to change Java API
> > > (e.g. read() and write()) of OffsetCheckpoint class in order to
> provide a
> > > map from TopicPartition to a pair of integers when we write to
> checkpoint
> > > file. This makes this class less generic since this API is not used by
> > log
> > > recovery checkpoint and log cleaner checkpoint which are also using
> > > OffsetCheckpoint class.
> > >
> > > Dong
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Sat, Jan 21, 2017 at 12:28 PM, Guozhang Wang 
> > > wrote:
> > >
> > > > Hi Dong,
> > > >
> > > > Sorry for being late on reviewing this KIP. It LGTM overall, but I'm
> > > > wondering if we can save adding the "replication-low-watermark-
> > > checkpoint"
> > > > file by just bumping up the version number of "replication-offset-
> > > > checkpoint"
> > > > to let it have two values for each partition, i.e.:
> > > >
> > > > 1  // version number
> > > > [number of partitions]
> > > > [topic name] [partition id] [lwm] [hwm]
> > > >
> > > >
> > > > This will affects the upgrade path a bit, but I think not by large,
> and
> > > all
> > > > other logic will not be affected.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Wed, Jan 18, 2017 at 6:12 PM, Dong Lin 
> wrote:
> > > >
> > > > > Thanks to everyone who voted and provided feedback!
> > > > >
> > > > > This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and
> 2
> > > > > non-binding +1s (Radai, Mayuresh).
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the update. +1
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin 
> > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > >
> > > > > > > After some more thinking, I agree with you that it is better to
> > > > simply
> > > > > > > 

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

2017-01-22 Thread Dong Lin
Yeah, your solution of adding new APIs certainly works and I don't think
that is an issue. On the other hand I don't think it is an issue to add a
new checkpoint file as well since we already have multiple checkpoint
files. The benefit of the new approach you mentioned is probably not an
issue in the current approach since high watermark and low watermark works
completely independently. Since there is no strong reason to choose either
of them, I am inclined to choose the one that makes less format change and
simpler in the Java API. The current approach seems better w.r.t this minor
reason.

If you are strong that we should use the new approach, I can do that as
well. Please let me know if you think so, and I will need to ask
Jun/Joel/Becket to vote on this again since this changes the interface of
the KIP.

On Sun, Jan 22, 2017 at 9:35 AM, Guozhang Wang  wrote:

> I think this is less of an issue: we can use the same patterns as in the
> request protocol, i.e.:
>
> write(Map[TP, Long]) // write the checkout point in v0 format
> write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1 format
>
> CheckpointedOffsets read() // read the file relying on its version id
>
> class CheckpointedOffsets {
>
> Integer getVersion();
> Long getFirstOffset();
> Long getSecondOffset();   // would return NO_AVAILABLE with v0 format
> }
>
>
> As I think of it, another benefit is that we wont have a partition that
> only have one of the watermarks in case of a failure in between writing two
> files.
>
> Guozhang
>
> On Sun, Jan 22, 2017 at 12:03 AM, Dong Lin  wrote:
>
> > Hey Guozhang,
> >
> > Thanks for the review:) Yes it is possible to combine them. Both solution
> > will have the same performance. But I think the current solution will
> give
> > us simpler Java class design. Note that we will have to change Java API
> > (e.g. read() and write()) of OffsetCheckpoint class in order to provide a
> > map from TopicPartition to a pair of integers when we write to checkpoint
> > file. This makes this class less generic since this API is not used by
> log
> > recovery checkpoint and log cleaner checkpoint which are also using
> > OffsetCheckpoint class.
> >
> > Dong
> >
> >
> >
> >
> >
> >
> >
> > On Sat, Jan 21, 2017 at 12:28 PM, Guozhang Wang 
> > wrote:
> >
> > > Hi Dong,
> > >
> > > Sorry for being late on reviewing this KIP. It LGTM overall, but I'm
> > > wondering if we can save adding the "replication-low-watermark-
> > checkpoint"
> > > file by just bumping up the version number of "replication-offset-
> > > checkpoint"
> > > to let it have two values for each partition, i.e.:
> > >
> > > 1  // version number
> > > [number of partitions]
> > > [topic name] [partition id] [lwm] [hwm]
> > >
> > >
> > > This will affects the upgrade path a bit, but I think not by large, and
> > all
> > > other logic will not be affected.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Wed, Jan 18, 2017 at 6:12 PM, Dong Lin  wrote:
> > >
> > > > Thanks to everyone who voted and provided feedback!
> > > >
> > > > This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 2
> > > > non-binding +1s (Radai, Mayuresh).
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Thanks for the update. +1
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin 
> > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > After some more thinking, I agree with you that it is better to
> > > simply
> > > > > > throw OffsetOutOfRangeException and not update low_watermark if
> > > > > > offsetToPurge is larger than high_watermark.
> > > > > >
> > > > > > My use-case of allowing low_watermark > high_watermark in 2(b) is
> > to
> > > > > allow
> > > > > > user to purge all the data in the log even if that data is not
> > fully
> > > > > > replicated to followers. An offset higher than high_watermark may
> > be
> > > > > > returned to user either through producer's RecordMetadata, or
> > through
> > > > > > ListOffsetResponse if from_consumer option is false. However,
> this
> > > may
> > > > > > cause problem in case of unclean leader election or when consumer
> > > seeks
> > > > > to
> > > > > > the largest offset of the partition. It will complicate this KIP
> if
> > > we
> > > > > were
> > > > > > to address these two problems.
> > > > > >
> > > > > > At this moment I prefer to keep this KIP simple by requiring
> > > > > low_watermark
> > > > > > <= high_watermark. The caveat is that if user does want to purge
> > > *all*
> > > > > the
> > > > > > data that is already produced, then he needs to stop all
> producers
> > > that
> > > > > are
> > > > > > producing into this topic, wait long enough for all followers to
> > > catch
> > > > > up,
> > > > > > and then purge data 

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

2017-01-22 Thread Guozhang Wang
I think this is less of an issue: we can use the same patterns as in the
request protocol, i.e.:

write(Map[TP, Long]) // write the checkout point in v0 format
write(Map[TP, Pair[Long, Long]]) // write the checkout point in v1 format

CheckpointedOffsets read() // read the file relying on its version id

class CheckpointedOffsets {

Integer getVersion();
Long getFirstOffset();
Long getSecondOffset();   // would return NO_AVAILABLE with v0 format
}


As I think of it, another benefit is that we wont have a partition that
only have one of the watermarks in case of a failure in between writing two
files.

Guozhang

On Sun, Jan 22, 2017 at 12:03 AM, Dong Lin  wrote:

> Hey Guozhang,
>
> Thanks for the review:) Yes it is possible to combine them. Both solution
> will have the same performance. But I think the current solution will give
> us simpler Java class design. Note that we will have to change Java API
> (e.g. read() and write()) of OffsetCheckpoint class in order to provide a
> map from TopicPartition to a pair of integers when we write to checkpoint
> file. This makes this class less generic since this API is not used by log
> recovery checkpoint and log cleaner checkpoint which are also using
> OffsetCheckpoint class.
>
> Dong
>
>
>
>
>
>
>
> On Sat, Jan 21, 2017 at 12:28 PM, Guozhang Wang 
> wrote:
>
> > Hi Dong,
> >
> > Sorry for being late on reviewing this KIP. It LGTM overall, but I'm
> > wondering if we can save adding the "replication-low-watermark-
> checkpoint"
> > file by just bumping up the version number of "replication-offset-
> > checkpoint"
> > to let it have two values for each partition, i.e.:
> >
> > 1  // version number
> > [number of partitions]
> > [topic name] [partition id] [lwm] [hwm]
> >
> >
> > This will affects the upgrade path a bit, but I think not by large, and
> all
> > other logic will not be affected.
> >
> >
> > Guozhang
> >
> >
> >
> > On Wed, Jan 18, 2017 at 6:12 PM, Dong Lin  wrote:
> >
> > > Thanks to everyone who voted and provided feedback!
> > >
> > > This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 2
> > > non-binding +1s (Radai, Mayuresh).
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the update. +1
> > > >
> > > > Jun
> > > >
> > > > On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin 
> wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > After some more thinking, I agree with you that it is better to
> > simply
> > > > > throw OffsetOutOfRangeException and not update low_watermark if
> > > > > offsetToPurge is larger than high_watermark.
> > > > >
> > > > > My use-case of allowing low_watermark > high_watermark in 2(b) is
> to
> > > > allow
> > > > > user to purge all the data in the log even if that data is not
> fully
> > > > > replicated to followers. An offset higher than high_watermark may
> be
> > > > > returned to user either through producer's RecordMetadata, or
> through
> > > > > ListOffsetResponse if from_consumer option is false. However, this
> > may
> > > > > cause problem in case of unclean leader election or when consumer
> > seeks
> > > > to
> > > > > the largest offset of the partition. It will complicate this KIP if
> > we
> > > > were
> > > > > to address these two problems.
> > > > >
> > > > > At this moment I prefer to keep this KIP simple by requiring
> > > > low_watermark
> > > > > <= high_watermark. The caveat is that if user does want to purge
> > *all*
> > > > the
> > > > > data that is already produced, then he needs to stop all producers
> > that
> > > > are
> > > > > producing into this topic, wait long enough for all followers to
> > catch
> > > > up,
> > > > > and then purge data using the latest offset of this partition, i.e.
> > > > > high_watermark. We can revisit this if some strong use-case comes
> up
> > in
> > > > the
> > > > > future.
> > > > >
> > > > > I also updated the KIP to allow user to use offset -1L to indicate
> > > > > high_watermark in the PurgeRequest. In the future we can allow
> users
> > to
> > > > use
> > > > > offset -2L to indicate that they want to purge all data up to
> > > > logEndOffset.
> > > > >
> > > > > Thanks!
> > > > > Dong
> > > > >
> > > > >
> > > > > On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao 
> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > For 2(b), it seems a bit weird to allow highWatermark to be
> smaller
> > > > than
> > > > > > lowWatermark. Also, from the consumer's perspective, messages are
> > > > > available
> > > > > > only up to highWatermark. What if we simply throw
> > > > > OffsetOutOfRangeException
> > > > > > if offsetToPurge is larger than highWatermark?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin 
> > > wrote:
> > > 

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

2017-01-22 Thread Dong Lin
Hey Guozhang,

Thanks for the review:) Yes it is possible to combine them. Both solution
will have the same performance. But I think the current solution will give
us simpler Java class design. Note that we will have to change Java API
(e.g. read() and write()) of OffsetCheckpoint class in order to provide a
map from TopicPartition to a pair of integers when we write to checkpoint
file. This makes this class less generic since this API is not used by log
recovery checkpoint and log cleaner checkpoint which are also using
OffsetCheckpoint class.

Dong







On Sat, Jan 21, 2017 at 12:28 PM, Guozhang Wang  wrote:

> Hi Dong,
>
> Sorry for being late on reviewing this KIP. It LGTM overall, but I'm
> wondering if we can save adding the "replication-low-watermark-checkpoint"
> file by just bumping up the version number of "replication-offset-
> checkpoint"
> to let it have two values for each partition, i.e.:
>
> 1  // version number
> [number of partitions]
> [topic name] [partition id] [lwm] [hwm]
>
>
> This will affects the upgrade path a bit, but I think not by large, and all
> other logic will not be affected.
>
>
> Guozhang
>
>
>
> On Wed, Jan 18, 2017 at 6:12 PM, Dong Lin  wrote:
>
> > Thanks to everyone who voted and provided feedback!
> >
> > This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 2
> > non-binding +1s (Radai, Mayuresh).
> >
> > Thanks,
> > Dong
> >
> > On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the update. +1
> > >
> > > Jun
> > >
> > > On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin  wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > After some more thinking, I agree with you that it is better to
> simply
> > > > throw OffsetOutOfRangeException and not update low_watermark if
> > > > offsetToPurge is larger than high_watermark.
> > > >
> > > > My use-case of allowing low_watermark > high_watermark in 2(b) is to
> > > allow
> > > > user to purge all the data in the log even if that data is not fully
> > > > replicated to followers. An offset higher than high_watermark may be
> > > > returned to user either through producer's RecordMetadata, or through
> > > > ListOffsetResponse if from_consumer option is false. However, this
> may
> > > > cause problem in case of unclean leader election or when consumer
> seeks
> > > to
> > > > the largest offset of the partition. It will complicate this KIP if
> we
> > > were
> > > > to address these two problems.
> > > >
> > > > At this moment I prefer to keep this KIP simple by requiring
> > > low_watermark
> > > > <= high_watermark. The caveat is that if user does want to purge
> *all*
> > > the
> > > > data that is already produced, then he needs to stop all producers
> that
> > > are
> > > > producing into this topic, wait long enough for all followers to
> catch
> > > up,
> > > > and then purge data using the latest offset of this partition, i.e.
> > > > high_watermark. We can revisit this if some strong use-case comes up
> in
> > > the
> > > > future.
> > > >
> > > > I also updated the KIP to allow user to use offset -1L to indicate
> > > > high_watermark in the PurgeRequest. In the future we can allow users
> to
> > > use
> > > > offset -2L to indicate that they want to purge all data up to
> > > logEndOffset.
> > > >
> > > > Thanks!
> > > > Dong
> > > >
> > > >
> > > > On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao  wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > For 2(b), it seems a bit weird to allow highWatermark to be smaller
> > > than
> > > > > lowWatermark. Also, from the consumer's perspective, messages are
> > > > available
> > > > > only up to highWatermark. What if we simply throw
> > > > OffsetOutOfRangeException
> > > > > if offsetToPurge is larger than highWatermark?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin 
> > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thank you. Please see my answers below. The KIP is updated to
> > answer
> > > > > these
> > > > > > questions (see here
> > > > > >  > diffpagesbyversion.action
> > > ?
> > > > > > pageId=67636826=5=6>
> > > > > > ).
> > > > > >
> > > > > > 1. Yes, in this KIP we wait for all replicas. This is the same as
> > if
> > > > > > producer sends a messge with ack=all and isr=all_replicas. So it
> > > seems
> > > > > that
> > > > > > the comparison is OK?
> > > > > >
> > > > > > 2. Good point! I haven't thought about the case where the
> > > > user-specified
> > > > > > offset > logEndOffset. Please see answers below.
> > > > > >
> > > > > > a) If offsetToPurge < lowWatermark, the first condition
> > > > > > of DelayedOperationPurgatory will be satisfied immediately when
> > > broker
> > > > > > receives PurgeRequest. Broker will send PurgeResponse to admin
> > client
> > > > > > 

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

2017-01-21 Thread Guozhang Wang
Hi Dong,

Sorry for being late on reviewing this KIP. It LGTM overall, but I'm
wondering if we can save adding the "replication-low-watermark-checkpoint"
file by just bumping up the version number of "replication-offset-checkpoint"
to let it have two values for each partition, i.e.:

1  // version number
[number of partitions]
[topic name] [partition id] [lwm] [hwm]


This will affects the upgrade path a bit, but I think not by large, and all
other logic will not be affected.


Guozhang



On Wed, Jan 18, 2017 at 6:12 PM, Dong Lin  wrote:

> Thanks to everyone who voted and provided feedback!
>
> This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 2
> non-binding +1s (Radai, Mayuresh).
>
> Thanks,
> Dong
>
> On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:
>
> > Hi, Dong,
> >
> > Thanks for the update. +1
> >
> > Jun
> >
> > On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin  wrote:
> >
> > > Hi Jun,
> > >
> > > After some more thinking, I agree with you that it is better to simply
> > > throw OffsetOutOfRangeException and not update low_watermark if
> > > offsetToPurge is larger than high_watermark.
> > >
> > > My use-case of allowing low_watermark > high_watermark in 2(b) is to
> > allow
> > > user to purge all the data in the log even if that data is not fully
> > > replicated to followers. An offset higher than high_watermark may be
> > > returned to user either through producer's RecordMetadata, or through
> > > ListOffsetResponse if from_consumer option is false. However, this may
> > > cause problem in case of unclean leader election or when consumer seeks
> > to
> > > the largest offset of the partition. It will complicate this KIP if we
> > were
> > > to address these two problems.
> > >
> > > At this moment I prefer to keep this KIP simple by requiring
> > low_watermark
> > > <= high_watermark. The caveat is that if user does want to purge *all*
> > the
> > > data that is already produced, then he needs to stop all producers that
> > are
> > > producing into this topic, wait long enough for all followers to catch
> > up,
> > > and then purge data using the latest offset of this partition, i.e.
> > > high_watermark. We can revisit this if some strong use-case comes up in
> > the
> > > future.
> > >
> > > I also updated the KIP to allow user to use offset -1L to indicate
> > > high_watermark in the PurgeRequest. In the future we can allow users to
> > use
> > > offset -2L to indicate that they want to purge all data up to
> > logEndOffset.
> > >
> > > Thanks!
> > > Dong
> > >
> > >
> > > On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao  wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > For 2(b), it seems a bit weird to allow highWatermark to be smaller
> > than
> > > > lowWatermark. Also, from the consumer's perspective, messages are
> > > available
> > > > only up to highWatermark. What if we simply throw
> > > OffsetOutOfRangeException
> > > > if offsetToPurge is larger than highWatermark?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin 
> wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thank you. Please see my answers below. The KIP is updated to
> answer
> > > > these
> > > > > questions (see here
> > > > >  diffpagesbyversion.action
> > ?
> > > > > pageId=67636826=5=6>
> > > > > ).
> > > > >
> > > > > 1. Yes, in this KIP we wait for all replicas. This is the same as
> if
> > > > > producer sends a messge with ack=all and isr=all_replicas. So it
> > seems
> > > > that
> > > > > the comparison is OK?
> > > > >
> > > > > 2. Good point! I haven't thought about the case where the
> > > user-specified
> > > > > offset > logEndOffset. Please see answers below.
> > > > >
> > > > > a) If offsetToPurge < lowWatermark, the first condition
> > > > > of DelayedOperationPurgatory will be satisfied immediately when
> > broker
> > > > > receives PurgeRequest. Broker will send PurgeResponse to admin
> client
> > > > > immediately. The response maps this partition to the lowWatermark.
> > > > >
> > > > > This case is covered as the first condition of
> > > DelayedOperationPurgatory
> > > > in
> > > > > the current KIP.
> > > > >
> > > > > b) If highWatermark < offsetToPurge < logEndOffset, leader will
> send
> > > > > FetchResponse with low_watermark=offsetToPurge. Follower records
> the
> > > > > offsetToPurge as low_watermark and sends FetchRequest to the leader
> > > with
> > > > > the new low_watermark. Leader will then send PurgeResponse to admin
> > > > client
> > > > > which maps this partition to the new low_watermark. The data in the
> > > range
> > > > > [highWatermark, offsetToPurge] will still be appended from leader
> to
> > > > > followers but will not be exposed to consumers. And in a short
> period
> > > of
> > > > > time low_watermark on the follower will be higher than their
> > > > 

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

2017-01-18 Thread Dong Lin
Thanks to everyone who voted and provided feedback!

This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 2
non-binding +1s (Radai, Mayuresh).

Thanks,
Dong

On Wed, Jan 18, 2017 at 6:05 PM, Jun Rao  wrote:

> Hi, Dong,
>
> Thanks for the update. +1
>
> Jun
>
> On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin  wrote:
>
> > Hi Jun,
> >
> > After some more thinking, I agree with you that it is better to simply
> > throw OffsetOutOfRangeException and not update low_watermark if
> > offsetToPurge is larger than high_watermark.
> >
> > My use-case of allowing low_watermark > high_watermark in 2(b) is to
> allow
> > user to purge all the data in the log even if that data is not fully
> > replicated to followers. An offset higher than high_watermark may be
> > returned to user either through producer's RecordMetadata, or through
> > ListOffsetResponse if from_consumer option is false. However, this may
> > cause problem in case of unclean leader election or when consumer seeks
> to
> > the largest offset of the partition. It will complicate this KIP if we
> were
> > to address these two problems.
> >
> > At this moment I prefer to keep this KIP simple by requiring
> low_watermark
> > <= high_watermark. The caveat is that if user does want to purge *all*
> the
> > data that is already produced, then he needs to stop all producers that
> are
> > producing into this topic, wait long enough for all followers to catch
> up,
> > and then purge data using the latest offset of this partition, i.e.
> > high_watermark. We can revisit this if some strong use-case comes up in
> the
> > future.
> >
> > I also updated the KIP to allow user to use offset -1L to indicate
> > high_watermark in the PurgeRequest. In the future we can allow users to
> use
> > offset -2L to indicate that they want to purge all data up to
> logEndOffset.
> >
> > Thanks!
> > Dong
> >
> >
> > On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao  wrote:
> >
> > > Hi, Dong,
> > >
> > > For 2(b), it seems a bit weird to allow highWatermark to be smaller
> than
> > > lowWatermark. Also, from the consumer's perspective, messages are
> > available
> > > only up to highWatermark. What if we simply throw
> > OffsetOutOfRangeException
> > > if offsetToPurge is larger than highWatermark?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin  wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thank you. Please see my answers below. The KIP is updated to answer
> > > these
> > > > questions (see here
> > > >  ?
> > > > pageId=67636826=5=6>
> > > > ).
> > > >
> > > > 1. Yes, in this KIP we wait for all replicas. This is the same as if
> > > > producer sends a messge with ack=all and isr=all_replicas. So it
> seems
> > > that
> > > > the comparison is OK?
> > > >
> > > > 2. Good point! I haven't thought about the case where the
> > user-specified
> > > > offset > logEndOffset. Please see answers below.
> > > >
> > > > a) If offsetToPurge < lowWatermark, the first condition
> > > > of DelayedOperationPurgatory will be satisfied immediately when
> broker
> > > > receives PurgeRequest. Broker will send PurgeResponse to admin client
> > > > immediately. The response maps this partition to the lowWatermark.
> > > >
> > > > This case is covered as the first condition of
> > DelayedOperationPurgatory
> > > in
> > > > the current KIP.
> > > >
> > > > b) If highWatermark < offsetToPurge < logEndOffset, leader will send
> > > > FetchResponse with low_watermark=offsetToPurge. Follower records the
> > > > offsetToPurge as low_watermark and sends FetchRequest to the leader
> > with
> > > > the new low_watermark. Leader will then send PurgeResponse to admin
> > > client
> > > > which maps this partition to the new low_watermark. The data in the
> > range
> > > > [highWatermark, offsetToPurge] will still be appended from leader to
> > > > followers but will not be exposed to consumers. And in a short period
> > of
> > > > time low_watermark on the follower will be higher than their
> > > highWatermark.
> > > >
> > > > This case is also covered in the current KIP so no change is
> required.
> > > >
> > > > c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to
> > > admin
> > > > client immediately. The response maps this partition to
> > > > OffsetOutOfRangeException.
> > > >
> > > > This case is not covered by the current KIP. I just added this as the
> > > > second condition for the PurgeRequest to be removed from
> > > > DelayedOperationPurgatory (in the Proposed Change section). Since the
> > > > PurgeRequest is satisfied immediately when the leader receives it, it
> > > > actually won't be put into the DelayedOperationPurgatory.
> > > >
> > > > 3. Yes, lowWatermark will be used when smallest_offset is used in the
> > > > ListOffsetRequest. I just updated Proposed Change section to specify

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

2017-01-18 Thread Jun Rao
Hi, Dong,

Thanks for the update. +1

Jun

On Wed, Jan 18, 2017 at 1:44 PM, Dong Lin  wrote:

> Hi Jun,
>
> After some more thinking, I agree with you that it is better to simply
> throw OffsetOutOfRangeException and not update low_watermark if
> offsetToPurge is larger than high_watermark.
>
> My use-case of allowing low_watermark > high_watermark in 2(b) is to allow
> user to purge all the data in the log even if that data is not fully
> replicated to followers. An offset higher than high_watermark may be
> returned to user either through producer's RecordMetadata, or through
> ListOffsetResponse if from_consumer option is false. However, this may
> cause problem in case of unclean leader election or when consumer seeks to
> the largest offset of the partition. It will complicate this KIP if we were
> to address these two problems.
>
> At this moment I prefer to keep this KIP simple by requiring low_watermark
> <= high_watermark. The caveat is that if user does want to purge *all* the
> data that is already produced, then he needs to stop all producers that are
> producing into this topic, wait long enough for all followers to catch up,
> and then purge data using the latest offset of this partition, i.e.
> high_watermark. We can revisit this if some strong use-case comes up in the
> future.
>
> I also updated the KIP to allow user to use offset -1L to indicate
> high_watermark in the PurgeRequest. In the future we can allow users to use
> offset -2L to indicate that they want to purge all data up to logEndOffset.
>
> Thanks!
> Dong
>
>
> On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao  wrote:
>
> > Hi, Dong,
> >
> > For 2(b), it seems a bit weird to allow highWatermark to be smaller than
> > lowWatermark. Also, from the consumer's perspective, messages are
> available
> > only up to highWatermark. What if we simply throw
> OffsetOutOfRangeException
> > if offsetToPurge is larger than highWatermark?
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin  wrote:
> >
> > > Hi Jun,
> > >
> > > Thank you. Please see my answers below. The KIP is updated to answer
> > these
> > > questions (see here
> > >  > > pageId=67636826=5=6>
> > > ).
> > >
> > > 1. Yes, in this KIP we wait for all replicas. This is the same as if
> > > producer sends a messge with ack=all and isr=all_replicas. So it seems
> > that
> > > the comparison is OK?
> > >
> > > 2. Good point! I haven't thought about the case where the
> user-specified
> > > offset > logEndOffset. Please see answers below.
> > >
> > > a) If offsetToPurge < lowWatermark, the first condition
> > > of DelayedOperationPurgatory will be satisfied immediately when broker
> > > receives PurgeRequest. Broker will send PurgeResponse to admin client
> > > immediately. The response maps this partition to the lowWatermark.
> > >
> > > This case is covered as the first condition of
> DelayedOperationPurgatory
> > in
> > > the current KIP.
> > >
> > > b) If highWatermark < offsetToPurge < logEndOffset, leader will send
> > > FetchResponse with low_watermark=offsetToPurge. Follower records the
> > > offsetToPurge as low_watermark and sends FetchRequest to the leader
> with
> > > the new low_watermark. Leader will then send PurgeResponse to admin
> > client
> > > which maps this partition to the new low_watermark. The data in the
> range
> > > [highWatermark, offsetToPurge] will still be appended from leader to
> > > followers but will not be exposed to consumers. And in a short period
> of
> > > time low_watermark on the follower will be higher than their
> > highWatermark.
> > >
> > > This case is also covered in the current KIP so no change is required.
> > >
> > > c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to
> > admin
> > > client immediately. The response maps this partition to
> > > OffsetOutOfRangeException.
> > >
> > > This case is not covered by the current KIP. I just added this as the
> > > second condition for the PurgeRequest to be removed from
> > > DelayedOperationPurgatory (in the Proposed Change section). Since the
> > > PurgeRequest is satisfied immediately when the leader receives it, it
> > > actually won't be put into the DelayedOperationPurgatory.
> > >
> > > 3. Yes, lowWatermark will be used when smallest_offset is used in the
> > > ListOffsetRequest. I just updated Proposed Change section to specify
> > this.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Tue, Jan 17, 2017 at 6:53 PM, Jun Rao  wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the KIP. Looks good overall. Just a few more comments.
> > > >
> > > > 1."Note that the way broker handles PurgeRequest is similar to how it
> > > > handles ProduceRequest with ack = -1 and isr=all_replicas". It seems
> > that
> > > > the implementation is a bit different. In this KIP, we wait for all
> > > > replicas. 

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

2017-01-18 Thread Dong Lin
Hi Jun,

After some more thinking, I agree with you that it is better to simply
throw OffsetOutOfRangeException and not update low_watermark if
offsetToPurge is larger than high_watermark.

My use-case of allowing low_watermark > high_watermark in 2(b) is to allow
user to purge all the data in the log even if that data is not fully
replicated to followers. An offset higher than high_watermark may be
returned to user either through producer's RecordMetadata, or through
ListOffsetResponse if from_consumer option is false. However, this may
cause problem in case of unclean leader election or when consumer seeks to
the largest offset of the partition. It will complicate this KIP if we were
to address these two problems.

At this moment I prefer to keep this KIP simple by requiring low_watermark
<= high_watermark. The caveat is that if user does want to purge *all* the
data that is already produced, then he needs to stop all producers that are
producing into this topic, wait long enough for all followers to catch up,
and then purge data using the latest offset of this partition, i.e.
high_watermark. We can revisit this if some strong use-case comes up in the
future.

I also updated the KIP to allow user to use offset -1L to indicate
high_watermark in the PurgeRequest. In the future we can allow users to use
offset -2L to indicate that they want to purge all data up to logEndOffset.

Thanks!
Dong


On Wed, Jan 18, 2017 at 10:37 AM, Jun Rao  wrote:

> Hi, Dong,
>
> For 2(b), it seems a bit weird to allow highWatermark to be smaller than
> lowWatermark. Also, from the consumer's perspective, messages are available
> only up to highWatermark. What if we simply throw OffsetOutOfRangeException
> if offsetToPurge is larger than highWatermark?
>
> Thanks,
>
> Jun
>
> On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin  wrote:
>
> > Hi Jun,
> >
> > Thank you. Please see my answers below. The KIP is updated to answer
> these
> > questions (see here
> >  > pageId=67636826=5=6>
> > ).
> >
> > 1. Yes, in this KIP we wait for all replicas. This is the same as if
> > producer sends a messge with ack=all and isr=all_replicas. So it seems
> that
> > the comparison is OK?
> >
> > 2. Good point! I haven't thought about the case where the user-specified
> > offset > logEndOffset. Please see answers below.
> >
> > a) If offsetToPurge < lowWatermark, the first condition
> > of DelayedOperationPurgatory will be satisfied immediately when broker
> > receives PurgeRequest. Broker will send PurgeResponse to admin client
> > immediately. The response maps this partition to the lowWatermark.
> >
> > This case is covered as the first condition of DelayedOperationPurgatory
> in
> > the current KIP.
> >
> > b) If highWatermark < offsetToPurge < logEndOffset, leader will send
> > FetchResponse with low_watermark=offsetToPurge. Follower records the
> > offsetToPurge as low_watermark and sends FetchRequest to the leader with
> > the new low_watermark. Leader will then send PurgeResponse to admin
> client
> > which maps this partition to the new low_watermark. The data in the range
> > [highWatermark, offsetToPurge] will still be appended from leader to
> > followers but will not be exposed to consumers. And in a short period of
> > time low_watermark on the follower will be higher than their
> highWatermark.
> >
> > This case is also covered in the current KIP so no change is required.
> >
> > c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to
> admin
> > client immediately. The response maps this partition to
> > OffsetOutOfRangeException.
> >
> > This case is not covered by the current KIP. I just added this as the
> > second condition for the PurgeRequest to be removed from
> > DelayedOperationPurgatory (in the Proposed Change section). Since the
> > PurgeRequest is satisfied immediately when the leader receives it, it
> > actually won't be put into the DelayedOperationPurgatory.
> >
> > 3. Yes, lowWatermark will be used when smallest_offset is used in the
> > ListOffsetRequest. I just updated Proposed Change section to specify
> this.
> >
> > Thanks,
> > Dong
> >
> > On Tue, Jan 17, 2017 at 6:53 PM, Jun Rao  wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the KIP. Looks good overall. Just a few more comments.
> > >
> > > 1."Note that the way broker handles PurgeRequest is similar to how it
> > > handles ProduceRequest with ack = -1 and isr=all_replicas". It seems
> that
> > > the implementation is a bit different. In this KIP, we wait for all
> > > replicas. But in producer, acks=all means waiting for all in-sync
> > replicas.
> > >
> > > 2. Could you describe the behavior when the specified offsetToPurge is
> > (a)
> > > smaller than lowWatermark, (b) larger than highWatermark, but smaller
> > than
> > > log end offset, (c) larger than log end offset?
> > >
> > > 3. In the ListOffsetRequest, will 

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

2017-01-18 Thread Jun Rao
Hi, Dong,

For 2(b), it seems a bit weird to allow highWatermark to be smaller than
lowWatermark. Also, from the consumer's perspective, messages are available
only up to highWatermark. What if we simply throw OffsetOutOfRangeException
if offsetToPurge is larger than highWatermark?

Thanks,

Jun

On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin  wrote:

> Hi Jun,
>
> Thank you. Please see my answers below. The KIP is updated to answer these
> questions (see here
>  pageId=67636826=5=6>
> ).
>
> 1. Yes, in this KIP we wait for all replicas. This is the same as if
> producer sends a messge with ack=all and isr=all_replicas. So it seems that
> the comparison is OK?
>
> 2. Good point! I haven't thought about the case where the user-specified
> offset > logEndOffset. Please see answers below.
>
> a) If offsetToPurge < lowWatermark, the first condition
> of DelayedOperationPurgatory will be satisfied immediately when broker
> receives PurgeRequest. Broker will send PurgeResponse to admin client
> immediately. The response maps this partition to the lowWatermark.
>
> This case is covered as the first condition of DelayedOperationPurgatory in
> the current KIP.
>
> b) If highWatermark < offsetToPurge < logEndOffset, leader will send
> FetchResponse with low_watermark=offsetToPurge. Follower records the
> offsetToPurge as low_watermark and sends FetchRequest to the leader with
> the new low_watermark. Leader will then send PurgeResponse to admin client
> which maps this partition to the new low_watermark. The data in the range
> [highWatermark, offsetToPurge] will still be appended from leader to
> followers but will not be exposed to consumers. And in a short period of
> time low_watermark on the follower will be higher than their highWatermark.
>
> This case is also covered in the current KIP so no change is required.
>
> c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to admin
> client immediately. The response maps this partition to
> OffsetOutOfRangeException.
>
> This case is not covered by the current KIP. I just added this as the
> second condition for the PurgeRequest to be removed from
> DelayedOperationPurgatory (in the Proposed Change section). Since the
> PurgeRequest is satisfied immediately when the leader receives it, it
> actually won't be put into the DelayedOperationPurgatory.
>
> 3. Yes, lowWatermark will be used when smallest_offset is used in the
> ListOffsetRequest. I just updated Proposed Change section to specify this.
>
> Thanks,
> Dong
>
> On Tue, Jan 17, 2017 at 6:53 PM, Jun Rao  wrote:
>
> > Hi, Dong,
> >
> > Thanks for the KIP. Looks good overall. Just a few more comments.
> >
> > 1."Note that the way broker handles PurgeRequest is similar to how it
> > handles ProduceRequest with ack = -1 and isr=all_replicas". It seems that
> > the implementation is a bit different. In this KIP, we wait for all
> > replicas. But in producer, acks=all means waiting for all in-sync
> replicas.
> >
> > 2. Could you describe the behavior when the specified offsetToPurge is
> (a)
> > smaller than lowWatermark, (b) larger than highWatermark, but smaller
> than
> > log end offset, (c) larger than log end offset?
> >
> > 3. In the ListOffsetRequest, will lowWatermark be returned when the
> > smallest_offset option is used?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:
> >
> > > Hi all,
> > >
> > > It seems that there is no further concern with the KIP-107. At this
> point
> > > we would like to start the voting process. The KIP can be found at
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> > >
> > > Thanks,
> > > Dong
> > >
> >
>


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

2017-01-18 Thread Dong Lin
I forgot to mention this in response to question 2 c). If logEndOffset <
offsetToPurge, leader will set low_watermark to min(leader's logEndOffset,
max(low_watermark, offsetToPurge)) = logEndOffset. I made this

change in the KIP.


On Tue, Jan 17, 2017 at 9:54 PM, Dong Lin  wrote:

> Hi Jun,
>
> Thank you. Please see my answers below. The KIP is updated to answer these
> questions (see here
> 
> ).
>
> 1. Yes, in this KIP we wait for all replicas. This is the same as if
> producer sends a messge with ack=all and isr=all_replicas. So it seems that
> the comparison is OK?
>
> 2. Good point! I haven't thought about the case where the user-specified
> offset > logEndOffset. Please see answers below.
>
> a) If offsetToPurge < lowWatermark, the first condition
> of DelayedOperationPurgatory will be satisfied immediately when broker
> receives PurgeRequest. Broker will send PurgeResponse to admin client
> immediately. The response maps this partition to the lowWatermark.
>
> This case is covered as the first condition of DelayedOperationPurgatory
> in the current KIP.
>
> b) If highWatermark < offsetToPurge < logEndOffset, leader will send
> FetchResponse with low_watermark=offsetToPurge. Follower records the
> offsetToPurge as low_watermark and sends FetchRequest to the leader with
> the new low_watermark. Leader will then send PurgeResponse to admin client
> which maps this partition to the new low_watermark. The data in the range
> [highWatermark, offsetToPurge] will still be appended from leader to
> followers but will not be exposed to consumers. And in a short period of
> time low_watermark on the follower will be higher than their highWatermark.
>
> This case is also covered in the current KIP so no change is required.
>
> c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to
> admin client immediately. The response maps this partition to
> OffsetOutOfRangeException.
>
> This case is not covered by the current KIP. I just added this as the
> second condition for the PurgeRequest to be removed from
> DelayedOperationPurgatory (in the Proposed Change section). Since the
> PurgeRequest is satisfied immediately when the leader receives it, it
> actually won't be put into the DelayedOperationPurgatory.
>
> 3. Yes, lowWatermark will be used when smallest_offset is used in the
> ListOffsetRequest. I just updated Proposed Change section to specify this.
>
> Thanks,
> Dong
>
> On Tue, Jan 17, 2017 at 6:53 PM, Jun Rao  wrote:
>
>> Hi, Dong,
>>
>> Thanks for the KIP. Looks good overall. Just a few more comments.
>>
>> 1."Note that the way broker handles PurgeRequest is similar to how it
>> handles ProduceRequest with ack = -1 and isr=all_replicas". It seems that
>> the implementation is a bit different. In this KIP, we wait for all
>> replicas. But in producer, acks=all means waiting for all in-sync
>> replicas.
>>
>> 2. Could you describe the behavior when the specified offsetToPurge is (a)
>> smaller than lowWatermark, (b) larger than highWatermark, but smaller than
>> log end offset, (c) larger than log end offset?
>>
>> 3. In the ListOffsetRequest, will lowWatermark be returned when the
>> smallest_offset option is used?
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:
>>
>> > Hi all,
>> >
>> > It seems that there is no further concern with the KIP-107. At this
>> point
>> > we would like to start the voting process. The KIP can be found at
>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
>> > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>> >
>> > Thanks,
>> > Dong
>> >
>>
>
>


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

2017-01-17 Thread Dong Lin
Hi Jun,

Thank you. Please see my answers below. The KIP is updated to answer these
questions (see here

).

1. Yes, in this KIP we wait for all replicas. This is the same as if
producer sends a messge with ack=all and isr=all_replicas. So it seems that
the comparison is OK?

2. Good point! I haven't thought about the case where the user-specified
offset > logEndOffset. Please see answers below.

a) If offsetToPurge < lowWatermark, the first condition
of DelayedOperationPurgatory will be satisfied immediately when broker
receives PurgeRequest. Broker will send PurgeResponse to admin client
immediately. The response maps this partition to the lowWatermark.

This case is covered as the first condition of DelayedOperationPurgatory in
the current KIP.

b) If highWatermark < offsetToPurge < logEndOffset, leader will send
FetchResponse with low_watermark=offsetToPurge. Follower records the
offsetToPurge as low_watermark and sends FetchRequest to the leader with
the new low_watermark. Leader will then send PurgeResponse to admin client
which maps this partition to the new low_watermark. The data in the range
[highWatermark, offsetToPurge] will still be appended from leader to
followers but will not be exposed to consumers. And in a short period of
time low_watermark on the follower will be higher than their highWatermark.

This case is also covered in the current KIP so no change is required.

c) If logEndOffset < offsetToPurge, leader will send PurgeResponse to admin
client immediately. The response maps this partition to
OffsetOutOfRangeException.

This case is not covered by the current KIP. I just added this as the
second condition for the PurgeRequest to be removed from
DelayedOperationPurgatory (in the Proposed Change section). Since the
PurgeRequest is satisfied immediately when the leader receives it, it
actually won't be put into the DelayedOperationPurgatory.

3. Yes, lowWatermark will be used when smallest_offset is used in the
ListOffsetRequest. I just updated Proposed Change section to specify this.

Thanks,
Dong

On Tue, Jan 17, 2017 at 6:53 PM, Jun Rao  wrote:

> Hi, Dong,
>
> Thanks for the KIP. Looks good overall. Just a few more comments.
>
> 1."Note that the way broker handles PurgeRequest is similar to how it
> handles ProduceRequest with ack = -1 and isr=all_replicas". It seems that
> the implementation is a bit different. In this KIP, we wait for all
> replicas. But in producer, acks=all means waiting for all in-sync replicas.
>
> 2. Could you describe the behavior when the specified offsetToPurge is (a)
> smaller than lowWatermark, (b) larger than highWatermark, but smaller than
> log end offset, (c) larger than log end offset?
>
> 3. In the ListOffsetRequest, will lowWatermark be returned when the
> smallest_offset option is used?
>
> Thanks,
>
> Jun
>
>
> On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > It seems that there is no further concern with the KIP-107. At this point
> > we would like to start the voting process. The KIP can be found at
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >
> > Thanks,
> > Dong
> >
>


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

2017-01-17 Thread Jun Rao
Hi, Dong,

Thanks for the KIP. Looks good overall. Just a few more comments.

1."Note that the way broker handles PurgeRequest is similar to how it
handles ProduceRequest with ack = -1 and isr=all_replicas". It seems that
the implementation is a bit different. In this KIP, we wait for all
replicas. But in producer, acks=all means waiting for all in-sync replicas.

2. Could you describe the behavior when the specified offsetToPurge is (a)
smaller than lowWatermark, (b) larger than highWatermark, but smaller than
log end offset, (c) larger than log end offset?

3. In the ListOffsetRequest, will lowWatermark be returned when the
smallest_offset option is used?

Thanks,

Jun


On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:

> Hi all,
>
> It seems that there is no further concern with the KIP-107. At this point
> we would like to start the voting process. The KIP can be found at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>
> Thanks,
> Dong
>


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

2017-01-17 Thread Becket Qin
+1. Thanks for the KIP.

On Thu, Jan 12, 2017 at 10:33 AM, Joel Koshy  wrote:

> +1
>
> (for the record, I favor the rejected alternative of not awaiting low
> watermarks to go past the purge offset. I realize it offers a weaker
> guarantee but it is still very useful, easier to implement, slightly
> simpler API (no need to return a future) and you can still get access to
> the current low watermark via a fetch request; although it would be weird
> to include the low watermark on the purge response in this variation)
>
> On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > It seems that there is no further concern with the KIP-107. At this point
> > we would like to start the voting process. The KIP can be found at
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >
> > Thanks,
> > Dong
> >
>


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

2017-01-17 Thread Dong Lin
Hey Jun,

Do you have time to review the KIP again or vote for it?

Hey Ewen,

Can you also review the KIP again or vote for it? I have discussed with
Radai and Becket regarding your concern. We still think putting it in Admin
Client seems more intuitive because there is use-case where application
which manages topic or produces data may also want to purge data. It seems
weird if they need to create a consumer to do this.

Thanks,
Dong

On Thu, Jan 12, 2017 at 9:34 AM, Mayuresh Gharat  wrote:

> +1 (non-binding)
>
> Thanks,
>
> Mayuresh
>
> On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin  wrote:
>
> > Sorry for the duplicated email. It seems that gmail will put the voting
> > email in this thread if I simply replace DISCUSS with VOTE in the
> subject.
> >
> > On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin  wrote:
> >
> > > Hi all,
> > >
> > > It seems that there is no further concern with the KIP-107. At this
> point
> > > we would like to start the voting process. The KIP can be found at
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> > >
> > > Thanks,
> > > Dong
> > >
> >
>
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


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

2017-01-12 Thread Joel Koshy
+1

(for the record, I favor the rejected alternative of not awaiting low
watermarks to go past the purge offset. I realize it offers a weaker
guarantee but it is still very useful, easier to implement, slightly
simpler API (no need to return a future) and you can still get access to
the current low watermark via a fetch request; although it would be weird
to include the low watermark on the purge response in this variation)

On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:

> Hi all,
>
> It seems that there is no further concern with the KIP-107. At this point
> we would like to start the voting process. The KIP can be found at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>
> Thanks,
> Dong
>


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

2017-01-12 Thread Mayuresh Gharat
+1 (non-binding).

Thanks,

Mayuresh

On Wed, Jan 11, 2017 at 10:11 PM, radai  wrote:

> LGTM, +1
>
> On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > It seems that there is no further concern with the KIP-107. At this point
> > we would like to start the voting process. The KIP can be found at
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >
> > Thanks,
> > Dong
> >
>



-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125


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

2017-01-12 Thread Mayuresh Gharat
+1 (non-binding)

Thanks,

Mayuresh

On Wed, Jan 11, 2017 at 1:03 PM, Dong Lin  wrote:

> Sorry for the duplicated email. It seems that gmail will put the voting
> email in this thread if I simply replace DISCUSS with VOTE in the subject.
>
> On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin  wrote:
>
> > Hi all,
> >
> > It seems that there is no further concern with the KIP-107. At this point
> > we would like to start the voting process. The KIP can be found at
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
> >
> > Thanks,
> > Dong
> >
>



-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125


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

2017-01-11 Thread radai
LGTM, +1

On Wed, Jan 11, 2017 at 1:01 PM, Dong Lin  wrote:

> Hi all,
>
> It seems that there is no further concern with the KIP-107. At this point
> we would like to start the voting process. The KIP can be found at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>
> Thanks,
> Dong
>


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

2017-01-11 Thread Dong Lin
Sorry for the duplicated email. It seems that gmail will put the voting
email in this thread if I simply replace DISCUSS with VOTE in the subject.

On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin  wrote:

> Hi all,
>
> It seems that there is no further concern with the KIP-107. At this point
> we would like to start the voting process. The KIP can be found at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-107
> %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>
> Thanks,
> Dong
>