The other thing I forgot to mention is that resetting the offset using the leader epoch is only available with the latest message format. By supporting reset by timestamp, users on the v1 format can still get some benefit from this KIP.
-Jason On Tue, Jun 26, 2018 at 11:47 AM, Jason Gustafson <ja...@confluent.io> wrote: > Hey Dong, > > Thanks for the comments. > > - The KIP says that, with auto.offset.reset="closest", timestamp is used to >> find offset if truncation offset is unknown. It seems that if consumer >> knows the timestamp of the last message, then the consumer should also >> know >> the (offset, leaderEpoch) of the last message which can then be used for >> find the truncation offset. Can you explain why truncation offset is >> unknown in this case? > > > The intent of the new reset policy is to automatically locate the closest > offset within the limits of Kafka log semantics. Unlike replicas, > consumers do not know the full history of leader epochs that have been > previously read. In some scenarios, they may not be able to precisely find > the offset where the log diverged after a sequence of unclean leader > elections (see KIP-279 for more detail). It seemed unfortunate in these > cases to have to resort to the coarse-grained resetting using either the > earliest or latest offset. Using the timestamp, we can find a more accurate > reset point and minimize the amount of loss or duplication. > > - How does consumer differentiates between "Offset out of rnage (too low)" >> and "Offset out of range (unknown truncation offset)", i.e. the two >> columns >> in table provided in the KIP? > > > We know when an offset is too low because we have the start offset of the > log from the fetch response. Following this KIP, that should really be the > only time we get an OutOfRange error (other than buggy application code). > The other two cases are distinguished based on whether we are able to find > the right offset of divergence. > > - It is probably a typo. Maybe fix "This is not the last The" in the >> Proposed Section. > > > Thanks. Magnus noticed this too and I fixed it earlier this morning. Good > to know who's actually reading the proposal! > > -Jason > > > > On Tue, Jun 26, 2018 at 11:09 AM, Dong Lin <lindon...@gmail.com> wrote: > >> Hey Jason, >> >> Thanks for the KIP! It is pretty useful. >> >> At high level the new set of reset policies may be a bit complicated and >> confusing to users. I am wondering whether we can simplify it. A few >> questions below: >> >> - The KIP says that, with auto.offset.reset="closest", timestamp is used >> to >> find offset if truncation offset is unknown. It seems that if consumer >> knows the timestamp of the last message, then the consumer should also >> know >> the (offset, leaderEpoch) of the last message which can then be used for >> find the truncation offset. Can you explain why truncation offset is >> unknown in this case? >> >> - How does consumer differentiates between "Offset out of rnage (too low)" >> and "Offset out of range (unknown truncation offset)", i.e. the two >> columns >> in table provided in the KIP? >> >> - It is probably a typo. Maybe fix "This is not the last The" in the >> Proposed Section. >> >> >> Thanks, >> Dong >> >> 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 >> > >> > >