@Dong

Those are fair points. Both approaches require some fuzziness to reset the
offset in these pathological scenarios and we cannot guarantee
at-least-once delivery either way unless we have the full history of leader
epochs that were consumed. The KIP-101 logic may actually be more accurate
than using timestamps because it does not depend on the messages which are
written after the unclean leader election. The case we're talking about
should be extremely rare in practice anyway. I also agree that we may not
want to add new machinery if it only helps the old message format. Ok,
let's go ahead and drop the timestamp.

@Guozhang

* My current understanding is that, with unclean leader election turned on,
> exactly-once is out of the window since we cannot guarantee that all
> committed message markers will not be lost. And hence there is no need to
> have special handling logic for LOG_TRUNCATED or OOR error codes with
> read.committed turned on. Is that right?


Yes, that's right. EoS and unclean leader election don't mix well. It may
be worth considering separately whether we should try to reconcile the
transaction log following an unclean leader election. At least we may be
able to prevent dangling transactions from blocking consumers. This KIP
does not address this problem.

* MINOR: "if the epoch is greater than the minimum expected epoch, that the
> new epoch does not begin at an earlier offset than the fetch offset.  In
> the latter case, the leader can respond with a new LOG_TRUNCATION error
> code" should it be "does not begin at a later offset than the fetch
> offset"?


I think the comment is correct, though the phrasing may be confusing. We
know truncation has occurred if there exists a larger epoch with a starting
offset that is lower than the fetch offset. Let me try to rephrase this.

Thanks,
Jason

On Wed, Jun 27, 2018 at 9:17 AM, Guozhang Wang <wangg...@gmail.com> wrote:

> Jason, thanks for the KIP. A few comments:
>
> * I think Dong's question about whether to use timestamp-based approach
> v.s. start-offset-of-first-larger-epoch is valid; more specifically, with
> timestamp-based approach we may still be reseting to an offset falling into
> the truncated interval, and hence we may still miss some data, i.e. not
> guaranteeing at-least-once still. With the
> start-offset-of-first-larger-epoch, I'm not sure if it will guarantee no
> valid data is missed when we have consecutive log truncations (maybe we
> need to look back into details of KIP-101 to figure it out). If the latter
> can indeed guarantee at least once, we could consider using that approach.
>
> * My current understanding is that, with unclean leader election turned on,
> exactly-once is out of the window since we cannot guarantee that all
> committed message markers will not be lost. And hence there is no need to
> have special handling logic for LOG_TRUNCATED or OOR error codes with
> read.committed turned on. Is that right?
>
> * MINOR: "if the epoch is greater than the minimum expected epoch, that the
> new epoch does not begin at an earlier offset than the fetch offset.  In
> the latter case, the leader can respond with a new LOG_TRUNCATION error
> code" should it be "does not begin at a later offset than the fetch
> offset"?
>
>
>
> Guozhang
>
>
> On Tue, Jun 26, 2018 at 6:51 PM, Dong Lin <lindon...@gmail.com> wrote:
>
> > Hey Jason,
> >
> > Thanks for the explanation.
> >
> > Please correct me if this is wrong. The "unknown truncation offset"
> > scenario happens when consumer does not have the full leaderEpoch ->
> offset
> > mapping. In this case we can still use the KIP-101-based approach to
> > truncate offset to "start offset of the first Leader Epoch larger than
> last
> > epoch of the consumer" but it may be inaccurate. So the KIP chooses to
> use
> > the timestamp-based approach which is also best-effort.
> >
> > If this understanding is correct, for "closest" offset reset policy and
> > "unknown truncation offset" scenario, I am wondering whether it maybe
> > better to replace timestamp-based approach with KIP-101 based approach.
> In
> > comparison to timestamp-based approach, the KIP-101-based approach seems
> to
> > simplify the API a bit since user does not need to understand timestamp.
> > Similar to the timestamp-based approach, both approaches are best-effort
> > and do not guarantee that consumer can consume all messages. It is not
> like
> > KIP-279 which guarantees that follower broker can consume all messages
> from
> > the leader.
> >
> > Then it seems that the remaining difference is mostly about accuracy,
> i.e.
> > how much message will be duplicated or missed in the "unknown truncation
> > offset" scenario. Not sure either one is clearly better than the other.
> > Note that there are two scenarios mentioned in KIP-279 which are not
> > addressed by KIP-101. Both scenarios require quick leadership change
> > between brokers, which seems to suggest that the offset based obtained
> > by "start
> > offset of the first Leader Epoch larger than last epoch of the consumer"
> > under these two scenarios may be very close to the offset obtained by the
> > message timestamp. Does this sound reasonable?
> >
> > Good point that users on v1 format can get benefit with timestamp based
> > approach. On the other hand it seems like a short term benefit for users
> > who have not migrated. I am just not sure whether it is more important
> than
> > designing a better API.
> >
> > Also, for both "latest" and "earliest" reset policy, do you think it
> would
> > make sense to also use the KIP-101 based approach to truncate offset for
> > the "unknown truncation offset" scenario?
> >
> >
> > Thanks,
> > Dong
> >
>
>
>
> --
> -- Guozhang
>

Reply via email to