Thanks Boyang for the proposal! I made a pass over the list and here are
some thoughts:

0) Although this is not part of the public API, I think we should make sure
that the suggested pattern (i.e. user can always call abortTxn() when
handling non-fatal errors) are indeed supported. E.g. if the txn is already
aborted by the broker side, then users can still call abortTxn which would
not throw another exception but just be treated as a no-op.

1) *ConcurrentTransactionsException*: I think this error can also be
returned but not documented yet. This should be a non-fatal error.

2) *InvalidTxnStateException*: this error is returned from broker when txn
state transition failed (e.g. it is trying to transit to complete-commit
while the current state is not prepare-commit). This error could indicates
a bug on the client internal code or the broker code, OR a user error --- a
similar error is ConcurrentTransactionsException, i.e. if Kafka is bug-free
these exceptions would only be returned if users try to do something wrong,
e.g. calling abortTxn right after a commitTxn, etc. So I'm thinking it
should be a non-fatal error instead of a fatal error, wdyt?

3) *KafkaException*: case i "indicates fatal transactional sequence
(Fatal)", this is a bit conflicting with the *OutOfSequenceException* that
is treated as non-fatal. I guess your proposal is that
OutOfOrderSequenceException would be treated either as fatal with
transactional producer, or non-fatal with idempotent producer, is that
right? If the producer is only configured with idempotency but not
transaction, then throwing a TransactionStateCorruptedException for
non-fatal errors would be confusing since users are not using transactions
at all.. So I suggest we always throw OutOfSequenceException as-is (i.e.
not wrapped) no matter how the producer is configured, and let the caller
decide how to handle it based on whether it is only idempotent or
transactional itself.

4) Besides all the txn APIs, the `send()` callback / future can also throw
txn-related exceptions, I think this KIP should also cover this API as well?

5) This is related to 1/2) above: sometimes those non-fatal errors like
ConcurrentTxn or InvalidTxnState are not due to the state being corrupted
at the broker side, but maybe users are doing something wrong. So I'm
wondering if we should further distinguish those non-fatal errors between
a) those that are caused by Kafka itself, e.g. a broker timed out and
aborted a txn and later an endTxn request is received, and b) the user's
API call pattern is incorrect, causing the request to be rejected with an
error code from the broker. *TransactionStateCorruptedException* feels to
me more like for case a), but not case b).


Guozhang


On Wed, Dec 2, 2020 at 4:50 PM Boyang Chen <reluctanthero...@gmail.com>
wrote:

> Thanks Matthias, I think your proposal makes sense as well, on the pro side
> we could have a universally agreed exception type to be caught by the user,
> without having an extra layer on top of the actual exceptions. I could see
> some issue on downsides:
>
> 1. The exception hierarchy will be more complex. Right now we have
> RetriableException type, if we are going to add a
> `ProducerRetriableException` type, we have to put this new interface as the
> parent of the RetriableException, because not all thrown non-fatal
> exceptions are `retriable` in general for producer, for example
> <
> https://github.com/apache/kafka/blob/e275742f850af4a1b79b0d1bd1ac9a1d2e89c64e/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L745
> >.
> We could have an empty interface `ProducerRetriableException` to let all
> the thrown exceptions implement for sure, but it's a bit unorthodox in the
> way we deal with exceptions in general.
>
> 2. There are cases where we throw a KafkaException wrapping another
> KafkaException as either fatal or non-fatal. If we use an interface to
> solve #1, it is also required to implement another bloated exception class
> which could replace KafkaException type, as we couldn't mark KafkaException
> as retriable for sure.
>
> 3. In terms of the encapsulation, wrapping means we could limit the scope
> of affection to the producer only, which is important since we don't want
> shared exception types to implement a producer-related interface, such
> as UnknownTopicOrPartitionException.
>
> Best,
> Boyang
>
> On Wed, Dec 2, 2020 at 3:38 PM Matthias J. Sax <mj...@apache.org> wrote:
>
> > Thanks for the KIP Boyang!
> >
> > Overall, categorizing exceptions makes a lot of sense. As I don't know
> > the producer internals well enough, I cannot comment on the
> > categorization in detail though.
> >
> > What I am wondering is, if we should introduce an exception interface
> > that non-fatal exception would implement instead of creating a new class
> > that will wrap non-fatal exceptions? What would be the pros/cons for
> > both designs?
> >
> >
> > -Matthias
> >
> >
> > On 12/2/20 11:35 AM, Boyang Chen wrote:
> > > Hey there,
> > >
> > > I would like to start a discussion thread for KIP-691:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > >
> > > The KIP is aiming to simplify the exception handling logic for
> > > transactional Producer users by classifying fatal and non-fatal
> > exceptions
> > > and throw them correspondingly for easier catch and retry. Let me know
> > what
> > > you think.
> > >
> > > Best,
> > > Boyang
> > >
> >
>


-- 
-- Guozhang

Reply via email to