Hi, Jason,

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

1. "As the consumer is fetching from a partition, it will keep a small
cache of the recent epochs that were fetched for each partition. " Do we
need to cache more than one leader epoch? Also, during consumer failover,
initially, only the the last epoch will be available.

2. "This KIP has implications for the consumer's offset reset policy, which
defines what the consumer should do if its fetch offset becomes out of
range. With this KIP, the only case in which this is possible is if the
consumer fetches from an offset earlier than the log start offset. ". If
the fetch epoch matches that in the leader, but the offset is larger than
the leader's HW, should we still treat it as offset out of range?

3. "We propose in this KIP to change the behavior for both the "earliest"
and "latest" reset modes to do this automatically as long as the message
format supports lookup by leader epoch.  ". It will probably be useful to
indicate to the user that a reset has happened. So, it's probably useful to
at least log this in the client.

4. "If the user ignores the exception, we will continue fetching from the
current offset, but we will drop the last fetched offset metadata from the
new FetchRequest so that we do not get the same log truncation error." Is
it better to do this or keep failing?

5. LogTruncationException:  Should we add an error code for that?

6. "We have added fields for the leader epoch and the timestamp." It seems
that we only added the leader epoch?

Jun


On Mon, Jun 25, 2018 at 9:17 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey All,
>
> I wrote up a KIP to handle one more edge case in the replication protocol
> and to support better handling of truncation in the consumer when unclean
> leader election is enabled. Let me know what you think.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 320%3A+Allow+fetchers+to+detect+and+handle+log+truncation
>
> Thanks to Anna Povzner and Dong Lin for initial feedback.
>
> Thanks,
> Jason
>

Reply via email to