Hey Justine,

About the consequences: the consequences will be like when we did not have
the fix made in `KAFKA-9279`: silent loss of data! Obviously, when the user
intentionally chose to ignore errors, that would not be silent any more.
Right?
Of course, considering all types of `ApiException`s would be too broad. But
are the exceptions caught in `catch(ApiException e)` of the `doSend()`
method also too broad?

-Alieh

On Tue, Jul 2, 2024 at 9:45 PM Justine Olshan <jols...@confluent.io.invalid>
wrote:

> Hey Alieh,
>
> If we want to allow any error to be ignored we should probably run through
> all the errors to make sure they make sense.
> I just want to feel confident that we aren't just making a decision without
> considering the consequences carefully.
>
> Justine
>
> On Tue, Jul 2, 2024 at 12:30 PM Alieh Saeedi <asae...@confluent.io.invalid
> >
> wrote:
>
> > Hey Justine,
> >
> > yes we talked about `RecordTooLargeException` as an example, but did we
> > ever limit ourselves to only this specific exception? I think neither in
> > the KIP nor in the PR.  As Chris mentioned, this KIP is going to undo
> what
> > we have done in `KAFKA-9279` in case 1) the user is in a transaction and
> 2)
> > he decides to ignore the errors in which the record was not even added to
> > the batch. Yes, and we suggested some methods for undoing or, in fact,
> > moving back the transaction from the error state in `flush` or in
> > `commitTnx` and we finally came to the idea of not even doing the changes
> > (better than undoing) in `send`.
> >
> > Bests,
> > Alieh
> >
> > On Tue, Jul 2, 2024 at 8:03 PM Justine Olshan
> <jols...@confluent.io.invalid
> > >
> > wrote:
> >
> > > Hey folks,
> > >
> > > I understand where you are coming from by asking for specific use
> cases.
> > My
> > > understanding based on previous conversations was that there were a few
> > > different errors that have been seen.
> > > One example I heard some information about was when the record was too
> > > large and it fails the batch. Besides that, I'm not really sure if
> there
> > > are cases in mind, though it is fair to ask on those and bring them up.
> > >
> > > > Does a record qualify as a poison pill if it targets a topic that
> > > doesn't exist? Or if it targets a topic that the producer principal
> lacks
> > > ACLs for? What if it fails broker-side validation (e.g., has a null key
> > for
> > > a compacted topic)?
> > >
> > > I think there was some parallel work with addressing the
> > > UnknownTopicOrPartitionError in another way. As for the other checks,
> > acls,
> > > validation etc. I am not aware of that being in Alieh's scope, but we
> > > should be clear about exactly what we are doing.
> > >
> > > All errors that fall into ApiException seems too broad to me.
> > >
> > > Justine
> > >
> > > On Tue, Jul 2, 2024 at 10:51 AM Alieh Saeedi
> > <asae...@confluent.io.invalid
> > > >
> > > wrote:
> > >
> > > > Hey Chris,
> > > > thanks for sharing your concerns.
> > > >
> > > > 1) About the language of KIP (or maybe later in Javadocs): Is that
> > > alright
> > > > if I write all errors that fall into the `ApiException` category
> thrown
> > > > (actually returned) by Producer?
> > > > 2) About future expansion: do you have any better suggestions for
> enum
> > > > names? Do you think `IGNORE_API_EXEPTIONS` or something like that is
> a
> > > > "better/more accurate" one?
> > > >
> > > > Bests,
> > > > Alieh
> > > >
> > > > On Tue, Jul 2, 2024 at 7:29 PM Chris Egerton <chr...@aiven.io.invalid
> >
> > > > wrote:
> > > >
> > > > > Hi Alieh and Justine,
> > > > >
> > > > > I'm concerned that we're settling on a definition of "poison pill"
> > > that's
> > > > > easiest to tackle right now but may lead to shortcomings down the
> > > road. I
> > > > > understand the relationship between this KIP and KAFKA-9279, and I
> > can
> > > > > totally get behind the desire to keep things small, focused, and
> > simple
> > > > in
> > > > > the name of avoiding bugs. However, what I don't think is clear at
> > all
> > > is
> > > > > what the "specific circumstances" are that Justine mentioned. I
> had a
> > > > > drastically different idea of what the intended behavioral change
> > would
> > > > be
> > > > > before looking at the draft PR.
> > > > >
> > > > > I would like 1) for us to be clearer about the categories of errors
> > > that
> > > > we
> > > > > want to cover with this new API (especially since we'll have to
> find
> > a
> > > > > clear, succinct way to document this for users), and 2) to make
> sure
> > > that
> > > > > if we do try to expand this API in the future, that we won't be
> > painted
> > > > > into a corner.
> > > > >
> > > > > For item 1, hopefully we can agree that the language in the KIP
> > > > > for IGNORE_SEND_ERRORS ("The records causing irrecoverable errors
> are
> > > > > excluded from the batch and the transaction is committed
> > > successfully.")
> > > > is
> > > > > pretty vague. If we start using the phrase "poison pill record"
> that
> > > > could
> > > > > help, but IMO more detail would still be needed. We know that we
> want
> > > to
> > > > > include records that are so large that they can be immediately
> > rejected
> > > > by
> > > > > the producer. But there are other cases that users might expect to
> be
> > > > > handled. Does a record qualify as a poison pill if it targets a
> topic
> > > > that
> > > > > doesn't exist? Or if it targets a topic that the producer principal
> > > lacks
> > > > > ACLs for? What if it fails broker-side validation (e.g., has a null
> > key
> > > > for
> > > > > a compacted topic)?
> > > > >
> > > > > For item 2, this really depends on how narrow the scope of what
> we're
> > > > doing
> > > > > right now is. If we only handle a subset of the examples I laid out
> > > above
> > > > > that could possibly be considered poison pills with this KIP, do we
> > > want
> > > > to
> > > > > lock ourselves in to never addressing more in the future, or can we
> > > > choose
> > > > > an API (probably just enum names would be the only important
> decision
> > > > here)
> > > > > that leaves room for more later?
> > > > >
> > > > > Best,
> > > > >
> > > > > Chris
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jul 2, 2024 at 12:28 PM Justine Olshan
> > > > > <jols...@confluent.io.invalid>
> > > > > wrote:
> > > > >
> > > > > > Chris and Alieh,
> > > > > >
> > > > > > My understanding is that this KIP is really only trying to solve
> an
> > > > issue
> > > > > > of a "poison pill" record that fails send().
> > > > > > We've talked a lot about having a generic framework for all
> errors,
> > > > but I
> > > > > > don't think that is what this KIP is trying to do. Essentially
> the
> > > > > request
> > > > > > is to undo the change from KAFKA-9279
> > > > > > <https://issues.apache.org/jira/browse/KAFKA-9279> but under
> > > specific
> > > > > > circumstances that are controlled. I really am concerned about
> > > opening
> > > > > new
> > > > > > avenues for bugs with EOS and hesitate to handle any other types
> of
> > > > > errors.
> > > > > > I think if we all agree on the problem that we are trying to
> solve,
> > > it
> > > > is
> > > > > > easier to agree on solutions.
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Mon, Jul 1, 2024 at 2:20 AM Alieh Saeedi
> > > > <asae...@confluent.io.invalid
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Matthias,
> > > > > > > Thanks for the valid points you mentioned. I updated the KIP
> and
> > > the
> > > > PR
> > > > > > > with:
> > > > > > > 1) mentioning that the new overloaded `send` throws
> > > > > > `IllegalStateException`
> > > > > > > if the user tries to ignore `send()` errors outside of a
> > > transaction.
> > > > > > > 2) the default implementation in `Producer` interface throws an
> > > > > > > `UnsupportedOperationException`
> > > > > > >
> > > > > > > Hi Chris,
> > > > > > > Thanks for the feedback. I tried to clarify the points you
> > listed:
> > > > > > > -------> we've narrowed the scope from any error that might
> take
> > > > place
> > > > > > with
> > > > > > > producing a record to Kafka, to only the ones that are thrown
> > > > directly
> > > > > > from
> > > > > > > Producer::send;
> > > > > > >
> > > > > > > From the very beginning and even since KIP-1038, the main
> purpose
> > > was
> > > > > to
> > > > > > > have "more flexibility," or, in other words, "giving the user
> the
> > > > > > > authority" to handle some specific exceptions thrown from the
> > > > > `Producer`.
> > > > > > > Due to the specific cases we had in mind, KIP-1038 was
> discarded
> > > and
> > > > we
> > > > > > > decided to not define a `CustomExceptionHandler` for `Producer`
> > and
> > > > > > instead
> > > > > > > treat the `send` failures in a different way. The main issue is
> > > that
> > > > > > `send`
> > > > > > > makes a transition to error state, which is undoable. In fact,
> > one
> > > > > single
> > > > > > > poison pill record makes the whole batch fail. The former
> > > suggestions
> > > > > > that
> > > > > > > you agreed with have been all about un-doing this transition in
> > > > `flush`
> > > > > > or
> > > > > > > `commit`. The new suggestion is to un-do (or better, NOT do) in
> > > > `send`
> > > > > > due
> > > > > > > to the reasons listed in the discussions above.
> > > > > > > Moreover, I would say that having such a large scope as you
> > > mentioned
> > > > > is
> > > > > > > impossible. In the best case, we may have control over the
> > > > `Producer`.
> > > > > > What
> > > > > > > shall we do with the broker? The `any error that might take
> place
> > > > with
> > > > > > > producing a record to Kafka` is too much, I think.
> > > > > > >
> > > > > > > -------> is this all we want to handle, and will it prevent us
> > from
> > > > > > > handling more in the future in an intuitive way?
> > > > > > >
> > > > > > > I think yes. This is all we want. Other sorts of errors such as
> > > > having
> > > > > > > problem with partition addition, producer fenced exception, etc
> > > seem
> > > > to
> > > > > > be
> > > > > > > more serious issues. The intention was to handle problems
> created
> > > by
> > > > > > > (maybe) a single poison pill record. BTW, I do not see any
> > > obstacles
> > > > to
> > > > > > > future changes.
> > > > > > >
> > > > > > > Bests,
> > > > > > > Alieh
> > > > > > >
> > > > > > > On Sat, Jun 29, 2024 at 3:03 AM Chris Egerton
> > > > <chr...@aiven.io.invalid
> > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Ah, sorry--spoke too soon. The PR doesn't show that errors
> > thrown
> > > > > from
> > > > > > > > Producer::send are handled, but instead, ApiException
> instances
> > > > that
> > > > > > are
> > > > > > > > caught inside KafkaProducer::doSend and are handled by
> > returning
> > > an
> > > > > > > > already-failed future are. I think the same question still
> > > applies
> > > > > (is
> > > > > > > this
> > > > > > > > all we want to handle, and will it prevent us from handling
> > more
> > > in
> > > > > the
> > > > > > > > future in an intuitive way), though.
> > > > > > > >
> > > > > > > > On Fri, Jun 28, 2024 at 8:57 PM Chris Egerton <
> chr...@aiven.io
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Alieh,
> > > > > > > > >
> > > > > > > > > This KIP has evolved a lot since I last looked at it, but
> the
> > > > > changes
> > > > > > > > seem
> > > > > > > > > well thought-out both in semantics and API. One clarifying
> > > > > question I
> > > > > > > > have
> > > > > > > > > is that it looks based on the draft PR that we've narrowed
> > the
> > > > > scope
> > > > > > > from
> > > > > > > > > any error that might take place with producing a record to
> > > Kafka,
> > > > > to
> > > > > > > only
> > > > > > > > > the ones that are thrown directly from Producer::send; is
> > that
> > > > the
> > > > > > > > intended
> > > > > > > > > behavior here? And if so, do you have thoughts on how we
> > might
> > > > > > design a
> > > > > > > > > follow-up KIP that would catch all errors (including ones
> > > > reported
> > > > > > > > > asynchronously instead of synchronously)? I'd like it if we
> > > could
> > > > > > leave
> > > > > > > > the
> > > > > > > > > door open for that without painting ourselves into too much
> > of
> > > a
> > > > > > corner
> > > > > > > > > with the API design for this KIP.
> > > > > > > > >
> > > > > > > > > Cheers,
> > > > > > > > >
> > > > > > > > > Chris
> > > > > > > > >
> > > > > > > > > On Fri, Jun 28, 2024 at 6:31 PM Matthias J. Sax <
> > > > mj...@apache.org>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Thanks Alieh,
> > > > > > > > >>
> > > > > > > > >> it seems this KIP can just pick between a couple of
> > tradeoffs.
> > > > > > Adding
> > > > > > > an
> > > > > > > > >> overloaded `send()` as the KIP propose makes sense to me
> and
> > > > seems
> > > > > > to
> > > > > > > > >> provides the cleanest solution compare to there options we
> > > > > > discussed.
> > > > > > > > >>
> > > > > > > > >> Given the explicit name of the passed-in option that
> > > highlights
> > > > > that
> > > > > > > the
> > > > > > > > >> option is for TX only make is pretty clear and avoids the
> > > issue
> > > > of
> > > > > > > > >> `flush()` ambiguity.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> Nit: We should make clear on the KIP though, that the new
> > > > `send()`
> > > > > > > > >> overload would throw an `IllegalStateException` if TX are
> > not
> > > > used
> > > > > > > > >> (similar to other TX methods like initTx(), etc)
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> About the `Producer` interface, I am not sure how this was
> > > done
> > > > in
> > > > > > the
> > > > > > > > >> past (eg, KIP-266 added `Consumer.poll(Duration)` w/o a
> > > default
> > > > > > > > >> implementation), if we need a default implementation for
> > > > backward
> > > > > > > > >> compatibility or not? If we do want to add one, I think it
> > > would
> > > > > be
> > > > > > > > >> appropriate to throw an `UnsupportedOperationException` by
> > > > > default,
> > > > > > > > >> instead of just keeping the default impl empty?
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> My points are rather minor, and should not block this KIP
> > > > though.
> > > > > > > > >> Overall LGTM.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> -Matthias
> > > > > > > > >>
> > > > > > > > >> On 6/27/24 1:28 PM, Alieh Saeedi wrote:
> > > > > > > > >> > Hi Justine,
> > > > > > > > >> >
> > > > > > > > >> > Thanks for the suggestion.
> > > > > > > > >> > Making applications to validate every single record is
> not
> > > the
> > > > > > best
> > > > > > > > way,
> > > > > > > > >> > from an efficiency point of view.
> > > > > > > > >> > Moreover, between changing the behavior of the Producer
> in
> > > > > `send`
> > > > > > > and
> > > > > > > > >> > `commitTnx`, the former seems more reasonable and clean.
> > > > > > > > >> >
> > > > > > > > >> > Bests,
> > > > > > > > >> > Alieh
> > > > > > > > >> >
> > > > > > > > >> > On Thu, Jun 27, 2024 at 8:14 PM Justine Olshan
> > > > > > > > >> <jols...@confluent.io.invalid>
> > > > > > > > >> > wrote:
> > > > > > > > >> >
> > > > > > > > >> >> Hey Alieh,
> > > > > > > > >> >>
> > > > > > > > >> >> I see there are two options now. So folks will be
> > > discussing
> > > > > the
> > > > > > > > >> approaches
> > > > > > > > >> >> and deciding the best way forward before we vote?
> > > > > > > > >> >> I do think there could be a problem with the approach
> on
> > > > commit
> > > > > > if
> > > > > > > we
> > > > > > > > >> get
> > > > > > > > >> >> stuck on an earlier error and have more records
> > > (potentially
> > > > on
> > > > > > new
> > > > > > > > >> >> partitions) to commit as the current PR is implemented.
> > > > > > > > >> >>
> > > > > > > > >> >> I guess this takes us back to the question of whether
> the
> > > > error
> > > > > > > > should
> > > > > > > > >> be
> > > > > > > > >> >> cleared on send.
> > > > > > > > >> >>
> > > > > > > > >> >> (And I guess at the back of my mind, I'm wondering if
> > there
> > > > is
> > > > > a
> > > > > > > way
> > > > > > > > >> we can
> > > > > > > > >> >> validate the "posion pill" records application side
> > before
> > > we
> > > > > > even
> > > > > > > > try
> > > > > > > > >> to
> > > > > > > > >> >> send them)
> > > > > > > > >> >>
> > > > > > > > >> >> Justine
> > > > > > > > >> >>
> > > > > > > > >> >> On Wed, Jun 26, 2024 at 4:38 PM Alieh Saeedi
> > > > > > > > >> <asae...@confluent.io.invalid
> > > > > > > > >> >>>
> > > > > > > > >> >> wrote:
> > > > > > > > >> >>
> > > > > > > > >> >>> Hi Justine,
> > > > > > > > >> >>>
> > > > > > > > >> >>> I did not update the KIP with `TxnSendOption` since I
> > > > thought
> > > > > > it'd
> > > > > > > > be
> > > > > > > > >> >>> better discussed here beforehand.
> > > > > > > > >> >>> right now, there are 2 PRs:
> > > > > > > > >> >>> - the PR that implements the current version of the
> KIP:
> > > > > > > > >> >>> https://github.com/apache/kafka/pull/16332
> > > > > > > > >> >>> - the POC PR that clarifies the `TxnSendOption`:
> > > > > > > > >> >>> https://github.com/apache/kafka/pull/16465
> > > > > > > > >> >>>
> > > > > > > > >> >>> Bests,
> > > > > > > > >> >>> Alieh
> > > > > > > > >> >>>
> > > > > > > > >> >>> On Thu, Jun 27, 2024 at 12:42 AM Justine Olshan
> > > > > > > > >> >>> <jols...@confluent.io.invalid> wrote:
> > > > > > > > >> >>>
> > > > > > > > >> >>>> Hey Alieh,
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> I think I am a little confused. Are the 3 points
> above
> > > > > > addressed
> > > > > > > by
> > > > > > > > >> the
> > > > > > > > >> >>> KIP
> > > > > > > > >> >>>> or did something change? The PR seems to not include
> > this
> > > > > > change
> > > > > > > > and
> > > > > > > > >> >>> still
> > > > > > > > >> >>>> has the CommitOption as well.
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> Thanks,
> > > > > > > > >> >>>> Justine
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> On Wed, Jun 26, 2024 at 2:15 PM Alieh Saeedi
> > > > > > > > >> >>> <asae...@confluent.io.invalid
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>> wrote:
> > > > > > > > >> >>>>
> > > > > > > > >> >>>>> Hi all,
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> Looking at the PR <
> > > > > https://github.com/apache/kafka/pull/16332
> > > > > > >
> > > > > > > > >> >>>>> corresponding to the KIP, there are some points
> worthy
> > > of
> > > > > > > mention:
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> 1) clearing the error ends up dirty/messy code in
> > > > > > > > >> >> `TransactionManager`.
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> 2) By clearing the error, we are actually doing an
> > > illegal
> > > > > > > > >> transition
> > > > > > > > >> >>>> from
> > > > > > > > >> >>>>> `ABORTABLE_ERROR` to `IN_TRANSACTION` which is
> > > > conceptually
> > > > > > not
> > > > > > > > >> >>>> acceptable.
> > > > > > > > >> >>>>> This can be the root cause of some issues, with
> > perhaps
> > > > > > further
> > > > > > > > >> >> future
> > > > > > > > >> >>>>> changes by others.
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> 3) If the poison pill record `r1` causes a
> transition
> > to
> > > > the
> > > > > > > error
> > > > > > > > >> >>> state
> > > > > > > > >> >>>>> and then the next record `r2` requires adding a
> > > partition
> > > > to
> > > > > > the
> > > > > > > > >> >>>>> transaction, the action fails due to being in the
> > error
> > > > > state.
> > > > > > > In
> > > > > > > > >> >> this
> > > > > > > > >> >>>>> case, clearing errors during
> > > `commitTnx(CLEAR_SEND_ERROR)`
> > > > > is
> > > > > > > too
> > > > > > > > >> >> late.
> > > > > > > > >> >>>>> However, this case can NOT be the main concern as
> soon
> > > as
> > > > > > > KIP-890
> > > > > > > > is
> > > > > > > > >> >>>> fully
> > > > > > > > >> >>>>> implemented.
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> My suggestion is to solve the problem where it
> arises.
> > > If
> > > > > the
> > > > > > > > >> >>> transition
> > > > > > > > >> >>>> to
> > > > > > > > >> >>>>> the error state does not happen during `send()`, we
> do
> > > not
> > > > > > need
> > > > > > > to
> > > > > > > > >> >>> clear
> > > > > > > > >> >>>>> the error later. Therefore, instead of
> `CommitOption`,
> > > we
> > > > > can
> > > > > > > > define
> > > > > > > > >> >> a
> > > > > > > > >> >>>>> `TxnSendOption` and prevent the `send()` method from
> > > going
> > > > > to
> > > > > > > the
> > > > > > > > >> >> error
> > > > > > > > >> >>>>> state in case 1) we're in a transaction and 2) the
> > user
> > > > > asked
> > > > > > > for
> > > > > > > > >> >>>>> IGONRE_SEND_ERRORS. For more clarity, you can take a
> > > look
> > > > at
> > > > > > the
> > > > > > > > POC
> > > > > > > > >> >> PR
> > > > > > > > >> >>>>> <https://github.com/apache/kafka/pull/16465>.
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>> Cheers,
> > > > > > > > >> >>>>> Alieh
> > > > > > > > >> >>>>>
> > > > > > > > >> >>>>
> > > > > > > > >> >>>
> > > > > > > > >> >>
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to