Thanks for updating the KIP. It's much clearer now what you propose. I have a bunch of question about the proposal:


(100) nit (typo / missing word?):

We would new error types



(101) `TransactionAbortableException`, `ProducerFencedException`, and `UnknownProducerIdException` are listed twice in the tables.



(102) You introduce a new exception `AbortableTransactionException` which will only be extended by `TransactionAbortableException`. Given that the existing TransactionAbortableException is not thrown by the producer right now (no passed into the `Callback`), it seem if the producer starts to throw/return the exiting `TransactionAbortableException` or the new `AbortableTransactionException` is would be an incompatible change?



(103) It's unclear which method would throw the new `AbortableTransactionException` and/or if this new exception might be passe into the producer's send `Callback`.



Btw: KIP-890 does not mention `TransactionAbortableException`... Does KIP-890 need an update? KIP-890 only mentions a new error code TRANSACTION_ABORTABLE -- or is this an implicit introduction of TransactionAbortableException -- I am not familiar with the details how core KIPs are written?



(104) The KIP does not explicitly say, which of the new exceptions are actually user facing? It seems only AbortableTransactionException, ApplicationRecoverableTransactionException, and InvalidConfiguationTransactionException are exception which user will be able to catch (or handle vie the `Callback`), while ProducerRetriableTransactionException and ProducerRefreshRetriableTransactionException won't be thrown/return by the producer into the app code?



(105) `IllegalStateException` and `RuntimeException` which are Java exceptions are listed in the table of `ApplicationRecoverableTransactionException`: I think it is not valid to list them, as we cannot change their super-class.



(106) `UnknownMemberIdException`, `IllegalGenerationException`, and `CorrelationIdMismatchException` are listed in the table of `ApplicationRecoverableTransactionException` but it seems they are not thrown/returned by the producer atm. If we start to throw/return either of them it seem to be a backward incompatible change.



(106) Similarly to 105, `InvalidRecordException` and `InvalidRequiredAcksException` are listed in the table of `InvalidConfiguationTransactionException` but they seem not to be user facing.




-Matthias


On 7/25/24 8:50 AM, Kaushik Raina wrote:
Additionally,
- We will be depreciating KIP-691 in favour of KIP-1050.


On Fri, Jun 21, 2024 at 12:20 PM Kaushik Raina <kra...@confluent.io> wrote:

Thanks Matthias for feedback
- We have updates KIP and grouped exceptions
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions#KIP1050:ConsistenterrorhandlingforTransactions-ExceptionTable

- Regarding compatibility,  all changes in KIP are expected to be *backword
compatible*.  We have updated KIP to make it clear.


On Tue, Jun 11, 2024 at 4:50 AM Matthias J. Sax <mj...@apache.org> wrote:

Thanks for this KIP. Great to see it. I would assume it will make
KIP-691 unnecessary?

I don't think I fully understand the proposal yet. It's clear, that you
propose to add new sub-classed to group existing exceptions. But it's
not clear to me, which of the existing exceptions (which implement
ApiException directly right now) will get a new parent class and go into
the same group. You only list `InvalidProducerEpochException` which gets
`AbortableTransactionException` as new parent. It would help a lot, if
you could list out explicitly, which existing exceptions are grouped
together via which sub-class.

It should be sufficient to just add a list for each group. For the newly
added exception classes, I would also omit all constructors etc and just
add a comment about it -- having constructors listed out does not add
much value to the KIP itself but makes it harder to read (it's
effectively noise we can avoid IMHO).



I am also wondering about compatibility? If I read the section
correctly, you actually propose to introduce a non-backward-compatible
change?

Based on type of exception thrown, user needs to change their exception
catching logic to take actions against their exception handling.

Ie, an application cannot be upgrade w/o code changes? I am not sure if
this is acceptable?

I think it would be much better (not sure if feasible) to keep the old
behavior and let users opt-in / enable the new semantics via a config.
If the new behavior is disabled, we could log a WARN that the app should
upgrade to work with the new semantics, and we would only enforce the
new behavior in a later major release.

Thoughts?



-Matthias






On 6/7/24 4:06 AM, Kaushik Raina wrote:
Thank you Andrew for feedback

1. We are suggesting to only update subclasses of
o.a.k.common.errors.ApiException, which are used in transactions. All
such
subclasses are mentioned in Exception table
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions#KIP1050:ConsistenterrorhandlingforTransactions-ExceptionTable


2. "Producer-Recoverable" corresponds to the AbortableException. I have
updated comments on each exception type.

3. Yes, it's correct that by adding a "Retriable" exception, it
simplifies
the determination of which errors can be retried internally. In the
Exception
table
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions#KIP1050:ConsistenterrorhandlingforTransactions-ExceptionTable

mentioned
in the "Proposed Changes" section, the "Expected Handling" column
signifies
the handling for each error type. Please let me know if any further
clarification is needed.

4a. Yes, that is correct. For clarity, only one constructor has been
mentioned in the KIP. An ellipsis has been added as a placeholder,
indicating that there are additional functions in the class but they are
not explicitly specified.
4b. Updated in the KIP.

5. TopicAuthorizationException extends "Invalid Configuration". "Invalid
Configuration" type can be resolved either by dynamically updating the
configuration, which does not require a restart, or by statically
updating
it by restarting the application. It is at the application's discretion
how
they want to handle each "Invalid Configuration" type.

I have added Client side handling example
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions#KIP1050:ConsistenterrorhandlingforTransactions-Clientsidecodeexample

in
KIP. Hope that helps.




Reply via email to