You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Gokul Srinivas <ap...@nym3r0s.cc> on 2020/08/27 17:24:23 UTC

[DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

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


Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Gokul Srinivas <ap...@nym3r0s.cc>.
All,

As the vote has passed, I have raised a PR here:
https://github.com/apache/kafka/pull/9280

Please help review.

Thanks,
Gokul

On 04-09-2020 00:48, Gokul Srinivas wrote:
> Appreciate the help!
>
> On 04-09-2020 00:46, Sophie Blee-Goldman wrote:
>> Yep, you can go ahead and call for a vote on the KIP
>>
>> On Thu, Sep 3, 2020 at 12:09 PM Gokul Srinivas <ap...@nym3r0s.cc> 
>> wrote:
>>
>>     Sophie,
>>
>>     That sounds fair. I've updated the KIP to state the same message for
>>     backward compatibility to existing (albeit hacky) solutions.
>>
>>     As this is my first ever contribution - is the next step to
>>     initiate the
>>     voting on this KIP?
>>
>>     -Gokul
>>
>>     On 04-09-2020 00:34, Sophie Blee-Goldman wrote:
>>     > I think the current proposal looks good to me. One minor
>>     suggestion I have
>>     > is to consider keeping the same error message:
>>     >
>>     > Failing batch since transaction was aborted
>>     >
>>     >
>>     > When we were running into this issue in Streams and accidentally
>>     rethrowing
>>     > the KafkaException as fatal, we ended up checking the specific
>>     error message
>>     > of the KafkaException and swallowing the exception if it was
>>     equivalent to
>>     > the
>>     > above. Obviously this was pretty hacky (hence the motivation for
>>     this KIP)
>>     > and
>>     > luckily we found a way around this, but it makes me wonder if any
>>     > applications
>>     > out there might be doing the same. So maybe we should reuse the
>>     old error
>>     > message just in case?
>>     >
>>     > Besides that, this KIP LGTM
>>     >
>>     > On Thu, Sep 3, 2020 at 5:23 AM Gokul Srinivas
>>     <ap...@nym3r0s.cc> wrote:
>>     >
>>     >> All,
>>     >>
>>     >> Gentle reminder - any comments on the line of thinking I
>>     mentioned in
>>     >> the last email? I've updated the Exception to be named
>>     >> "TransactionAbortedException" on the KIP confluence page.
>>     >>
>>     >> -Gokul
>>     >>
>>     >> On 01-09-2020 18:34, Gokul Srinivas wrote:
>>     >>> Matthias, Sophie, Jason,
>>     >>>
>>     >>> Took another pass at understanding the internals and it seems
>>     to me
>>     >>> like we should be extending the `ApiException` rather than the
>>     >>> `RetriableException`.
>>     >>>
>>     >>> The check in question
>>     >>> =====================
>>     >>>
>>     >>> Do we abort any undrained batches that are present on this
>>     transaction
>>     >>> if the transaction is in an aborting state? And, if we do,
>>     what would
>>     >>> be the reason sent back to the user for aborting these batches?
>>     >>>
>>     >>> Logic for this
>>     >>> ==============
>>     >>>
>>     >>> If the transaction `isAborting` and `hasAbortableError` and the
>>     >>> `lastError()` is not empty -> then there has been some error 
>> which
>>     >>> will cause / has caused the transaction to abort and this *is* a
>>     >>> runtime exception. This same exception should be sent back to
>>     the user.
>>     >>>
>>     >>> If the transaction `isAborting` and `lastError()` is empty ->
>>     then for
>>     >>> some unknown reason (maybe even user initiated, according to the
>>     >>> tests), the transaction manager has started to abort the
>>     transaction.
>>     >>> In this case, the newly proposed exception should be sent back
>>     to the
>>     >>> user.
>>     >>>
>>     >>> Reasoning
>>     >>> =========
>>     >>>
>>     >>> Prima facie - I do not think this is a `RetriableException`.
>>     >>>
>>     >>> If the user has chosen to abort this transaction, then it
>>     would be up
>>     >>> to the user to choose whether to retry the exception, in which
>>     case it
>>     >>> is /*not*/ a `RetriableException`.
>>     >>>
>>     >>> If there is a case where the transaction manager has no error,
>>     but has
>>     >>> started to abort the exception, we still do not retry the
>>     transaction,
>>     >>> rather we abort any undrained batches - in which case, it is
>>     /*still
>>     >>> not*/ a `RetriableException`.
>>     >>>
>>     >>> Does that sound right?
>>     >>>
>>     >>> -Gokul
>>     >>>
>>     >>> On 29-08-2020 01:17, Jason Gustafson wrote:
>>     >>>> 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
>>     >>>> <sophie@confluent.io <ma...@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
>>     <mjsax@apache.org <ma...@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>
>>     >>>>>>> <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>
>>     >>>>>>> <
>>     >>
>> 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
>>     >>>>>>>
>>     >>>>>>>
>>
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Gokul Srinivas <ap...@nym3r0s.cc>.
Appreciate the help!

On 04-09-2020 00:46, Sophie Blee-Goldman wrote:
> Yep, you can go ahead and call for a vote on the KIP
>
> On Thu, Sep 3, 2020 at 12:09 PM Gokul Srinivas <ap...@nym3r0s.cc> wrote:
>
>     Sophie,
>
>     That sounds fair. I've updated the KIP to state the same message for
>     backward compatibility to existing (albeit hacky) solutions.
>
>     As this is my first ever contribution - is the next step to
>     initiate the
>     voting on this KIP?
>
>     -Gokul
>
>     On 04-09-2020 00:34, Sophie Blee-Goldman wrote:
>     > I think the current proposal looks good to me. One minor
>     suggestion I have
>     > is to consider keeping the same error message:
>     >
>     > Failing batch since transaction was aborted
>     >
>     >
>     > When we were running into this issue in Streams and accidentally
>     rethrowing
>     > the KafkaException as fatal, we ended up checking the specific
>     error message
>     > of the KafkaException and swallowing the exception if it was
>     equivalent to
>     > the
>     > above. Obviously this was pretty hacky (hence the motivation for
>     this KIP)
>     > and
>     > luckily we found a way around this, but it makes me wonder if any
>     > applications
>     > out there might be doing the same. So maybe we should reuse the
>     old error
>     > message just in case?
>     >
>     > Besides that, this KIP LGTM
>     >
>     > On Thu, Sep 3, 2020 at 5:23 AM Gokul Srinivas
>     <ap...@nym3r0s.cc> wrote:
>     >
>     >> All,
>     >>
>     >> Gentle reminder - any comments on the line of thinking I
>     mentioned in
>     >> the last email? I've updated the Exception to be named
>     >> "TransactionAbortedException" on the KIP confluence page.
>     >>
>     >> -Gokul
>     >>
>     >> On 01-09-2020 18:34, Gokul Srinivas wrote:
>     >>> Matthias, Sophie, Jason,
>     >>>
>     >>> Took another pass at understanding the internals and it seems
>     to me
>     >>> like we should be extending the `ApiException` rather than the
>     >>> `RetriableException`.
>     >>>
>     >>> The check in question
>     >>> =====================
>     >>>
>     >>> Do we abort any undrained batches that are present on this
>     transaction
>     >>> if the transaction is in an aborting state? And, if we do,
>     what would
>     >>> be the reason sent back to the user for aborting these batches?
>     >>>
>     >>> Logic for this
>     >>> ==============
>     >>>
>     >>> If the transaction `isAborting` and `hasAbortableError` and the
>     >>> `lastError()` is not empty -> then there has been some error which
>     >>> will cause / has caused the transaction to abort and this *is* a
>     >>> runtime exception. This same exception should be sent back to
>     the user.
>     >>>
>     >>> If the transaction `isAborting` and `lastError()` is empty ->
>     then for
>     >>> some unknown reason (maybe even user initiated, according to the
>     >>> tests), the transaction manager has started to abort the
>     transaction.
>     >>> In this case, the newly proposed exception should be sent back
>     to the
>     >>> user.
>     >>>
>     >>> Reasoning
>     >>> =========
>     >>>
>     >>> Prima facie - I do not think this is a `RetriableException`.
>     >>>
>     >>> If the user has chosen to abort this transaction, then it
>     would be up
>     >>> to the user to choose whether to retry the exception, in which
>     case it
>     >>> is /*not*/ a `RetriableException`.
>     >>>
>     >>> If there is a case where the transaction manager has no error,
>     but has
>     >>> started to abort the exception, we still do not retry the
>     transaction,
>     >>> rather we abort any undrained batches - in which case, it is
>     /*still
>     >>> not*/ a `RetriableException`.
>     >>>
>     >>> Does that sound right?
>     >>>
>     >>> -Gokul
>     >>>
>     >>> On 29-08-2020 01:17, Jason Gustafson wrote:
>     >>>> 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
>     >>>> <sophie@confluent.io <ma...@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
>     <mjsax@apache.org <ma...@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>
>     >>>>>>> <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>
>     >>>>>>> <
>     >>
>     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
>     >>>>>>>
>     >>>>>>>
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Sophie Blee-Goldman <so...@confluent.io>.
Yep, you can go ahead and call for a vote on the KIP

On Thu, Sep 3, 2020 at 12:09 PM Gokul Srinivas <ap...@nym3r0s.cc> wrote:

> Sophie,
>
> That sounds fair. I've updated the KIP to state the same message for
> backward compatibility to existing (albeit hacky) solutions.
>
> As this is my first ever contribution - is the next step to initiate the
> voting on this KIP?
>
> -Gokul
>
> On 04-09-2020 00:34, Sophie Blee-Goldman wrote:
> > I think the current proposal looks good to me. One minor suggestion I
> have
> > is to consider keeping the same error message:
> >
> > Failing batch since transaction was aborted
> >
> >
> > When we were running into this issue in Streams and accidentally
> rethrowing
> > the KafkaException as fatal, we ended up checking the specific error
> message
> > of the KafkaException and swallowing the exception if it was equivalent
> to
> > the
> > above. Obviously this was pretty hacky (hence the motivation for this
> KIP)
> > and
> > luckily we found a way around this, but it makes me wonder if any
> > applications
> > out there might be doing the same. So maybe we should reuse the old error
> > message just in case?
> >
> > Besides that, this KIP LGTM
> >
> > On Thu, Sep 3, 2020 at 5:23 AM Gokul Srinivas <ap...@nym3r0s.cc> wrote:
> >
> >> All,
> >>
> >> Gentle reminder - any comments on the line of thinking I mentioned in
> >> the last email? I've updated the Exception to be named
> >> "TransactionAbortedException" on the KIP confluence page.
> >>
> >> -Gokul
> >>
> >> On 01-09-2020 18:34, Gokul Srinivas wrote:
> >>> Matthias, Sophie, Jason,
> >>>
> >>> Took another pass at understanding the internals and it seems to me
> >>> like we should be extending the `ApiException` rather than the
> >>> `RetriableException`.
> >>>
> >>> The check in question
> >>> =====================
> >>>
> >>> Do we abort any undrained batches that are present on this transaction
> >>> if the transaction is in an aborting state? And, if we do, what would
> >>> be the reason sent back to the user for aborting these batches?
> >>>
> >>> Logic for this
> >>> ==============
> >>>
> >>> If the transaction `isAborting` and `hasAbortableError` and the
> >>> `lastError()` is not empty -> then there has been some error which
> >>> will cause / has caused the transaction to abort and this *is* a
> >>> runtime exception. This same exception should be sent back to the user.
> >>>
> >>> If the transaction `isAborting` and `lastError()` is empty -> then for
> >>> some unknown reason (maybe even user initiated, according to the
> >>> tests), the transaction manager has started to abort the transaction.
> >>> In this case, the newly proposed exception should be sent back to the
> >>> user.
> >>>
> >>> Reasoning
> >>> =========
> >>>
> >>> Prima facie - I do not think this is a `RetriableException`.
> >>>
> >>> If the user has chosen to abort this transaction, then it would be up
> >>> to the user to choose whether to retry the exception, in which case it
> >>> is /*not*/ a `RetriableException`.
> >>>
> >>> If there is a case where the transaction manager has no error, but has
> >>> started to abort the exception, we still do not retry the transaction,
> >>> rather we abort any undrained batches - in which case, it is /*still
> >>> not*/ a `RetriableException`.
> >>>
> >>> Does that sound right?
> >>>
> >>> -Gokul
> >>>
> >>> On 29-08-2020 01:17, Jason Gustafson wrote:
> >>>> 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
> >>>> <so...@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
> >>>>>>>
> >>>>>>>
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Gokul Srinivas <ap...@nym3r0s.cc>.
Sophie,

That sounds fair. I've updated the KIP to state the same message for 
backward compatibility to existing (albeit hacky) solutions.

As this is my first ever contribution - is the next step to initiate the 
voting on this KIP?

-Gokul

On 04-09-2020 00:34, Sophie Blee-Goldman wrote:
> I think the current proposal looks good to me. One minor suggestion I have
> is to consider keeping the same error message:
>
> Failing batch since transaction was aborted
>
>
> When we were running into this issue in Streams and accidentally rethrowing
> the KafkaException as fatal, we ended up checking the specific error message
> of the KafkaException and swallowing the exception if it was equivalent to
> the
> above. Obviously this was pretty hacky (hence the motivation for this KIP)
> and
> luckily we found a way around this, but it makes me wonder if any
> applications
> out there might be doing the same. So maybe we should reuse the old error
> message just in case?
>
> Besides that, this KIP LGTM
>
> On Thu, Sep 3, 2020 at 5:23 AM Gokul Srinivas <ap...@nym3r0s.cc> wrote:
>
>> All,
>>
>> Gentle reminder - any comments on the line of thinking I mentioned in
>> the last email? I've updated the Exception to be named
>> "TransactionAbortedException" on the KIP confluence page.
>>
>> -Gokul
>>
>> On 01-09-2020 18:34, Gokul Srinivas wrote:
>>> Matthias, Sophie, Jason,
>>>
>>> Took another pass at understanding the internals and it seems to me
>>> like we should be extending the `ApiException` rather than the
>>> `RetriableException`.
>>>
>>> The check in question
>>> =====================
>>>
>>> Do we abort any undrained batches that are present on this transaction
>>> if the transaction is in an aborting state? And, if we do, what would
>>> be the reason sent back to the user for aborting these batches?
>>>
>>> Logic for this
>>> ==============
>>>
>>> If the transaction `isAborting` and `hasAbortableError` and the
>>> `lastError()` is not empty -> then there has been some error which
>>> will cause / has caused the transaction to abort and this *is* a
>>> runtime exception. This same exception should be sent back to the user.
>>>
>>> If the transaction `isAborting` and `lastError()` is empty -> then for
>>> some unknown reason (maybe even user initiated, according to the
>>> tests), the transaction manager has started to abort the transaction.
>>> In this case, the newly proposed exception should be sent back to the
>>> user.
>>>
>>> Reasoning
>>> =========
>>>
>>> Prima facie - I do not think this is a `RetriableException`.
>>>
>>> If the user has chosen to abort this transaction, then it would be up
>>> to the user to choose whether to retry the exception, in which case it
>>> is /*not*/ a `RetriableException`.
>>>
>>> If there is a case where the transaction manager has no error, but has
>>> started to abort the exception, we still do not retry the transaction,
>>> rather we abort any undrained batches - in which case, it is /*still
>>> not*/ a `RetriableException`.
>>>
>>> Does that sound right?
>>>
>>> -Gokul
>>>
>>> On 29-08-2020 01:17, Jason Gustafson wrote:
>>>> 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
>>>> <so...@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
>>>>>>>
>>>>>>>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Sophie Blee-Goldman <so...@confluent.io>.
I think the current proposal looks good to me. One minor suggestion I have
is to consider keeping the same error message:

Failing batch since transaction was aborted


When we were running into this issue in Streams and accidentally rethrowing
the KafkaException as fatal, we ended up checking the specific error message
of the KafkaException and swallowing the exception if it was equivalent to
the
above. Obviously this was pretty hacky (hence the motivation for this KIP)
and
luckily we found a way around this, but it makes me wonder if any
applications
out there might be doing the same. So maybe we should reuse the old error
message just in case?

Besides that, this KIP LGTM

On Thu, Sep 3, 2020 at 5:23 AM Gokul Srinivas <ap...@nym3r0s.cc> wrote:

> All,
>
> Gentle reminder - any comments on the line of thinking I mentioned in
> the last email? I've updated the Exception to be named
> "TransactionAbortedException" on the KIP confluence page.
>
> -Gokul
>
> On 01-09-2020 18:34, Gokul Srinivas wrote:
> > Matthias, Sophie, Jason,
> >
> > Took another pass at understanding the internals and it seems to me
> > like we should be extending the `ApiException` rather than the
> > `RetriableException`.
> >
> > The check in question
> > =====================
> >
> > Do we abort any undrained batches that are present on this transaction
> > if the transaction is in an aborting state? And, if we do, what would
> > be the reason sent back to the user for aborting these batches?
> >
> > Logic for this
> > ==============
> >
> > If the transaction `isAborting` and `hasAbortableError` and the
> > `lastError()` is not empty -> then there has been some error which
> > will cause / has caused the transaction to abort and this *is* a
> > runtime exception. This same exception should be sent back to the user.
> >
> > If the transaction `isAborting` and `lastError()` is empty -> then for
> > some unknown reason (maybe even user initiated, according to the
> > tests), the transaction manager has started to abort the transaction.
> > In this case, the newly proposed exception should be sent back to the
> > user.
> >
> > Reasoning
> > =========
> >
> > Prima facie - I do not think this is a `RetriableException`.
> >
> > If the user has chosen to abort this transaction, then it would be up
> > to the user to choose whether to retry the exception, in which case it
> > is /*not*/ a `RetriableException`.
> >
> > If there is a case where the transaction manager has no error, but has
> > started to abort the exception, we still do not retry the transaction,
> > rather we abort any undrained batches - in which case, it is /*still
> > not*/ a `RetriableException`.
> >
> > Does that sound right?
> >
> > -Gokul
> >
> > On 29-08-2020 01:17, Jason Gustafson wrote:
> >> 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
> >> <so...@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
> >>>>>
> >>>>>
> >>>>
> >
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Gokul Srinivas <ap...@nym3r0s.cc>.
All,

Gentle reminder - any comments on the line of thinking I mentioned in 
the last email? I've updated the Exception to be named 
"TransactionAbortedException" on the KIP confluence page.

-Gokul

On 01-09-2020 18:34, Gokul Srinivas wrote:
> Matthias, Sophie, Jason,
>
> Took another pass at understanding the internals and it seems to me 
> like we should be extending the `ApiException` rather than the 
> `RetriableException`.
>
> The check in question
> =====================
>
> Do we abort any undrained batches that are present on this transaction 
> if the transaction is in an aborting state? And, if we do, what would 
> be the reason sent back to the user for aborting these batches?
>
> Logic for this
> ==============
>
> If the transaction `isAborting` and `hasAbortableError` and the 
> `lastError()` is not empty -> then there has been some error which 
> will cause / has caused the transaction to abort and this *is* a 
> runtime exception. This same exception should be sent back to the user.
>
> If the transaction `isAborting` and `lastError()` is empty -> then for 
> some unknown reason (maybe even user initiated, according to the 
> tests), the transaction manager has started to abort the transaction. 
> In this case, the newly proposed exception should be sent back to the 
> user.
>
> Reasoning
> =========
>
> Prima facie - I do not think this is a `RetriableException`.
>
> If the user has chosen to abort this transaction, then it would be up 
> to the user to choose whether to retry the exception, in which case it 
> is /*not*/ a `RetriableException`.
>
> If there is a case where the transaction manager has no error, but has 
> started to abort the exception, we still do not retry the transaction, 
> rather we abort any undrained batches - in which case, it is /*still 
> not*/ a `RetriableException`.
>
> Does that sound right?
>
> -Gokul
>
> On 29-08-2020 01:17, Jason Gustafson wrote:
>> 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 
>> <so...@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
>>>>>
>>>>>
>>>>
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Gokul Srinivas <ap...@nym3r0s.cc>.
Matthias, Sophie, Jason,

Took another pass at understanding the internals and it seems to me like 
we should be extending the `ApiException` rather than the 
`RetriableException`.

The check in question
=====================

Do we abort any undrained batches that are present on this transaction 
if the transaction is in an aborting state? And, if we do, what would be 
the reason sent back to the user for aborting these batches?

Logic for this
==============

If the transaction `isAborting` and `hasAbortableError` and the 
`lastError()` is not empty -> then there has been some error which will 
cause / has caused the transaction to abort and this *is* a runtime 
exception. This same exception should be sent back to the user.

If the transaction `isAborting` and `lastError()` is empty -> then for 
some unknown reason (maybe even user initiated, according to the tests), 
the transaction manager has started to abort the transaction. In this 
case, the newly proposed exception should be sent back to the user.

Reasoning
=========

Prima facie - I do not think this is a `RetriableException`.

If the user has chosen to abort this transaction, then it would be up to 
the user to choose whether to retry the exception, in which case it is 
/*not*/ a `RetriableException`.

If there is a case where the transaction manager has no error, but has 
started to abort the exception, we still do not retry the transaction, 
rather we abort any undrained batches - in which case, it is /*still 
not*/ a `RetriableException`.

Does that sound right?

-Gokul

On 29-08-2020 01:17, Jason Gustafson wrote:
> 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 <so...@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
>>>>
>>>>
>>>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Jason Gustafson <ja...@confluent.io>.
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 <so...@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
> > >
> > >
> >
> >
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by Sophie Blee-Goldman <so...@confluent.io>.
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
> >
> >
>
>

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

Posted by "Matthias J. Sax" <mj...@apache.org>.
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
> 
>