[jira] [Commented] (KAFKA-15459) Convert coordinator retriable errors to a known producer response error.
[ https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764459#comment-17764459 ] Artem Livshits commented on KAFKA-15459: > Taken to its logical conclusion it would seem we only need a single error > code to represent all retriable errors. This is one of the options for allowing better cross-version compatibility that we discussed. In the end of the day the important property of an error is the way it's being handled by the client and/or application and the specific source of the error is mostly for diagnostic purposes. So the error code could be more like a class of errors that are handled uniformly regardless of source. For example, one class of errors would represent a transient condition in the system that prevents executing the request at this moment but would likely recover shortly (i.e. the client should just retry), another class of errors could represent transient condition due to stale metadata (i.e. the client should refresh metadata and retry), etc. KIP-890 part 2 takes a step in that direction by separating errors into retriable (client can handle transparently without raising them to the application), abortable (application would need to abort the current transaction to get a known good state) and fatal (application would need to do some initialization steps to get to a known good state). > what was considered Originally, it was considered that just letting COORDINATOR_NOT_AVAILABLE and other errors to be present in the produce response would be satisfactory (at least for the official Kafka client). But after some discussion, it was concluded that it's hard to definitively say that all versions of all possible clients would be able to handle new errors (the errors themselves are not new, but previously couldn't happen in produce requests). So basically there are 3 options: # Require a new client (version bump) for this functionality. This a non-starter, because the purpose of KIP-890 is to close the gaps for older clients as well. # Let the new errors pass through. This was the original solution, but the concern is that it could break some old clients, which would cause disruption and the verification would be disabled by the admin, which would kind of defeat the purpose. # Map the new errors to an existing error with similar error handling. With the current flat system of each error having its own error code it becomes less intuitive, but it has the least risk of breaking clients. > 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
[jira] [Commented] (KAFKA-15459) Convert coordinator retriable errors to a known producer response error.
[ https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
[jira] [Commented] (KAFKA-15459) Convert coordinator retriable errors to a known producer response error.
[ https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764442#comment-17764442 ] Tom Bentley commented on KAFKA-15459: - [~jolshan] thanks for the explanation. Is there somewhere which elaborates on part 2? If error responses indicated their retriability (rather than it being inferred from the error code, and thus embedded within client libraries) it would enable better compatibility in the future for things like this, but I suspect your part 2 plans are simpler than that. > 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 > > > 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. > 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)
[jira] [Commented] (KAFKA-15459) Convert coordinator retriable errors to a known producer response error.
[ https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764440#comment-17764440 ] Justine Olshan commented on KAFKA-15459: Hey [~tombentley] thanks for taking a look. The struggle here is that we want to support old clients. Adding errors safely requires a produce request version bump that I was hoping to avoid for part 1 since it addresses older clients. We are stuck between having specific errors for Java clients that are handled correctly (ie we retry) or specific fatal errors for cases that should not be fatal. We opted for a middle ground of non-specific error, but a message in the response to specify. I agree that newer clients should be able to support specific errors and in part 2 we can make this a lot better with the error story and have a single error code for all retriable errors. I've had conversations with [~hachikuji] and [~alivshits] about simplifying into retriable, abortable, and fatal errors. I hope to tackle this in part 2 and I can make a Jira for that as well. Let me know if there is any other information you would like to get a better idea of the decision. I will also update the KIP to explain this. > 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 > > > 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. > 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)
[jira] [Commented] (KAFKA-15459) Convert coordinator retriable errors to a known producer response error.
[ https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764437#comment-17764437 ] Tom Bentley commented on KAFKA-15459: - Is this _really_ the best compromise? AFAICS the linked PR and issue don't contain enough information to know what was considered. The loss of specific error codes seems like a big disadvantage to me. Taken to its logical conclusion it would seem we only need a single error code to represent all retriable errors. > 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 > > > 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. > 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)