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:

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

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

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

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

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

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

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

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

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

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

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

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

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

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 >

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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 >

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,

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

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

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

2017-01-11 Thread Dong Lin
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

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

2017-01-11 Thread Dong Lin
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

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

2017-01-11 Thread Dong Lin
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