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/confluence/display/KAFKA/KIP-107
>> > >> > > %3A+Add+purgeDataBefore%28%29+API+in+AdminClient.
>> > >> > >
>> > >> > > Thanks,
>> > >> > > Dong
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >>
>> > >> --
>> > >> -Regards,
>> > >> Mayuresh R. Gharat
>> > >> (862) 250-7125
>> > >>
>> > >
>> > >
>> >
>>
>
>

Reply via email to