[
https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17764443#comment-17764443
]
Justine Olshan commented on KAFKA-15459:
----------------------------------------
Hey Tom --
The KIP specifies some of this but it could do a better job and I plan to
expand on it soon.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
{{We will bump the ProduceRequest/Response version to indicate the client is
using the new protocol that doesn’t require adding partitions to transactions
and will implicitly do so. The bump will also support a new error
{{{}ABORTABLE_ERROR{}}}. }}{{{{ABORTABLE_ERROR}} can be returned on any failure
server-side so that the server can indicate to the client that the transaction
should be aborted.}}
I'd be happy to add something similar for retriable errors. I think there may
be some work to bump multiple Produce APIs and that may require another KIP to
address the ones beyond the apis touched in KIP-890.
> Convert coordinator retriable errors to a known producer response error.
> ------------------------------------------------------------------------
>
> Key: KAFKA-15459
> URL: https://issues.apache.org/jira/browse/KAFKA-15459
> Project: Kafka
> Issue Type: Sub-task
> Affects Versions: 3.6.0
> Reporter: Justine Olshan
> Assignee: Justine Olshan
> Priority: Blocker
> Fix For: 3.6.0
>
>
> KIP-890 Part 1 tries to address hanging transactions on old clients. Thus,
> the produce version can not be bumped and no new errors can be added.
> Currently we use the java client's notion of retriable and abortable errors
> -- retriable errors are defined as such by extending the retriable error
> class, fatal errors are defined explicitly, and abortable errors are the
> remaining. However, many other clients treat non specified errors as fatal
> and that means many retriable errors kill the application. This is not ideal.
> While reviewing [https://github.com/apache/kafka/pull/14370] I added some of
> the documentation for the returned errors in the produce response as well.
> There were concerns about the new errors:
> * {@link Errors#COORDINATOR_LOAD_IN_PROGRESS}
> * {@link Errors#COORDINATOR_NOT_AVAILABLE}
> * {@link Errors#INVALID_TXN_STATE}
> * {@link Errors#INVALID_PRODUCER_ID_MAPPING}
> * {@link Errors#CONCURRENT_TRANSACTIONS}
> The coordinator load, not available, and concurrent transactions errors
> should be retriable.
> The invalid txn state and pid mapping errors should be abortable.
> This is how older java clients handle the errors, but it is unclear how other
> clients handle them. It seems that rdkafka (for example) treats the abortable
> errors as fatal instead. The coordinator errors are retriable but not the
> concurrent transactions error. Generally anything not specified otherwise is
> fatal.
> It seems acceptable for the abortable errors to be fatal on some clients
> since the error is likely on a zombie producer or in a state that may be
> harder to recover from. However, for the retriable errors, we can return
> NOT_ENOUGH_REPLICAS which is a known retriable response. We can use the
> produce api's response string to specify the real cause of the error for
> debugging.
> There were trade-offs between making the older clients work and for clarity
> in errors. This seems to be the best compromise.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)