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 <j...@confluent.io> 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 <lindon...@gmail.com> 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 <ism...@juma.me.uk> 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<Map<TopicPartition, > > > PurgeDataResult>> purgeDataBefore(Map<TopicPartition, Long> > > > 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 <lindon...@gmail.com> 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 <j...@confluent.io> 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 <lindon...@gmail.com> > > 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 <lindon...@gmail.com> > > > 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 <ism...@juma.me.uk > > > > > > 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 < > lindon...@gmail.com> > > > > > 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 < > > lindon...@gmail.com > > > > > > > > > > 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 < > > > lindon...@gmail.com > > > > > > > > > > > >> 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 < > > > > > lindon...@gmail.com> > > > > > > >> > 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/confl > > > uence/display/KAFKA/KIP-107 > > > > > > >> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient. > > > > > > >> > >> > > > > > > > > >> > >> > > Thanks, > > > > > > >> > >> > > Dong > > > > > > >> > >> > > > > > > > > >> > >> > > > > > > > >> > >> > > > > > > >> > >> > > > > > > >> > >> > > > > > > >> > >> -- > > > > > > >> > >> -Regards, > > > > > > >> > >> Mayuresh R. Gharat > > > > > > >> > >> (862) 250-7125 > > > > > > >> > >> > > > > > > >> > > > > > > > > >> > > > > > > > > >> > > > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >