Hi Jun,

Thank you. Please see my answers below. The KIP is updated to answer these
questions (see here
<https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=67636826&selectedPageVersions=5&selectedPageVersions=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 <j...@confluent.io> 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 <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
> >
>

Reply via email to