Hi Gokul, Thanks, I think it makes sense to use a separate exception type. +1 on Sophie's suggestion for `TransactionAbortedException`.
Extending from `RetriableException` seems reasonable as well. I guess the only question is whether it's safe to catch it as a `RetriableException` and apply common retry logic. For a transactional producer, my expectation is that the application would abort the transaction and retry it. However, if the transaction is already being aborted, maybe it would be better to skip the abort. It might be helpful to have an example which shows how we expect applications to handle this. Thanks, Jason On Thu, Aug 27, 2020 at 6:25 PM Sophie Blee-Goldman <sop...@confluent.io> wrote: > Hey Gokul, thanks for taking up this KIP! > > I agree with Matthias that directly extending KafkaException may not be > ideal, > and we should instead extend APIException or RetriableException. Of the > two, > I think APIException would be more appropriate. My understanding is that > RetriableException is generally reserved for internally retriable > exceptions > whereas APIException is used for pseudo-fatal exceptions that require some > user input as to how to proceed (eg ProducerFencedException) > > I also agree that the name could be a bit more concise. My personal vote > would be for "TransactionAbortedException" which seems a bit more > grammatically aligned with the other exceptions in Kafka. > > Cheers, > Sophie > > On Thu, Aug 27, 2020 at 6:01 PM Matthias J. Sax <mj...@apache.org> wrote: > > > Thanks for the KIP. Looks good overall. > > > > However, I am wondering if the new exception should extend > > `KafkaException`? It seems, extending `ApiException` or maybe even > > `RetriableException` might be better? > > > > About the name itself. I would prefer something simpler like > > `AbortedTransactionException`. > > > > Thoughts? > > > > > > -Matthias > > > > > > On 8/27/20 10:24 AM, Gokul Srinivas wrote: > > > Hello all, > > > > > > I would like to propose the following KIP to throw a new non-fatal > > > exception whilst aborting transactions with non-flushed data. This will > > > help users distinguish non-fatal errors and potentially retry the > batch. > > > > > > *Issue *- https://issues.apache.org/jira/browse/KAFKA-10186 > > > <https://issues.apache.org/jira/browse/KAFKA-10186> > > > > > > *KIP *- > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-654:+Aborted+transaction+with+non-flushed+data+should+throw+a+non-fatal+exception > > > < > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-654:+Aborted+transaction+with+non-flushed+data+should+throw+a+non-fatal+exception > > > > > > > > > > > > Please let me know how best we can proceed with this. > > > > > > -Gokul > > > > > > > > > > >