You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "lucasbru (via GitHub)" <gi...@apache.org> on 2023/06/05 14:02:24 UTC

[GitHub] [kafka] lucasbru opened a new pull request, #13811: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

lucasbru opened a new pull request, #13811:
URL: https://github.com/apache/kafka/pull/13811

   This PR fixes three issues:
   
   * `InvalidProducerEpochException` was not handled consistently. `InvalidProducerEpochException` used to be able to be return via both transactional response and produce response, but as of KIP-588 (2.7+), transactional responses should not return `InvalidProducerEpochException` anymore, only produce responses can. It can happen that older brokers may still return `InvalidProducerEpochException` for transactional responses; these must be converted to the newer `ProducerFencedException`. This conversion wasn't done for `TxnOffsetCommit` (sent to the group coordinator).
   
   `InvalidTxnStateException` was double-wrapped in `KafkaException`, whereas other exceptions are usually wrapped only once. Furthermore, `InvalidTxnStateException` was not handled at all for in `AddOffsetsToTxn` response, where it should be a possible error as well, according to API documentation.
   
    - According to API documentation, `UNSUPPORTED_FOR_MESSAGE_FORMAT` is not possible for `TxnOffsetCommit`, but it looks like it is, and it is being handled there, so I updated the API documentation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1220507827


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1376,7 +1376,7 @@ public void handleResponse(AbstractResponse response) {
                     fatalError(error.exception());
                     return;
                 } else if (error == Errors.INVALID_TXN_STATE) {
-                    fatalError(new KafkaException(error.exception()));

Review Comment:
   As mentioned, in the other instance, this probably won't be returned to the client, but in the case that it was -- could there be a client that expects the wrapped error? At this point, it's probably not an issue, but just curious the implication.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan merged pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan merged PR #13811:
URL: https://github.com/apache/kafka/pull/13811


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1218377141


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1661,6 +1663,10 @@ public void handleResponse(AbstractResponse response) {
                         || error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // If the topic is unknown or the coordinator is loading, retry with the current coordinator
                     continue;
+                } else if (error == Errors.INVALID_PRODUCER_EPOCH

Review Comment:
   what was the reasoning for the ordering change here? We've moved these errors up, but left the auth and unsupported message format errors lower.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] lucasbru commented on pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "lucasbru (via GitHub)" <gi...@apache.org>.
lucasbru commented on PR #13811:
URL: https://github.com/apache/kafka/pull/13811#issuecomment-1578303825

   > Hey Lucas -- thanks for the PR. Just wanted to confirm -- these changes are in line with what is proposed as part of KIP-691? It looks to me that is the case, but wanted to confirm.
   
   Yes, exactly. This is one of the changes to clean up exceptions in preparation for KIP-691


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] lucasbru commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "lucasbru (via GitHub)" <gi...@apache.org>.
lucasbru commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1221565214


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1376,7 +1376,7 @@ public void handleResponse(AbstractResponse response) {
                     fatalError(error.exception());
                     return;
                 } else if (error == Errors.INVALID_TXN_STATE) {
-                    fatalError(new KafkaException(error.exception()));

Review Comment:
   I think, for one, it is rather unlikely that a client matches exactly the wrapped exception. 
   
   The accepted KIP states: "For customized EOS use case, user needs to change their exception catching logic ..... However, all the thrown exceptions' base type would still be KafkaException, so the effect should be minimal." 
   
   So I would say these kinds of backward incompatible changes around exceptions were expected when the KIP was accepted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] lucasbru commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "lucasbru (via GitHub)" <gi...@apache.org>.
lucasbru commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1219320732


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1661,6 +1663,10 @@ public void handleResponse(AbstractResponse response) {
                         || error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // If the topic is unknown or the coordinator is loading, retry with the current coordinator
                     continue;
+                } else if (error == Errors.INVALID_PRODUCER_EPOCH

Review Comment:
   Hmm, not sure. There shouldn't be any functional difference, so I moved it back down.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on PR #13811:
URL: https://github.com/apache/kafka/pull/13811#issuecomment-1577188413

   Hey Lucas -- thanks for the PR. Just wanted to confirm -- these changes are in line with what is proposed as part of KIP-691? It looks to me that is the case, but wanted to confirm.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] lucasbru commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "lucasbru (via GitHub)" <gi...@apache.org>.
lucasbru commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1221557911


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1599,6 +1599,8 @@ public void handleResponse(AbstractResponse response) {
                 fatalError(Errors.PRODUCER_FENCED.exception());
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
                 fatalError(error.exception());
+            } else if (error == Errors.INVALID_TXN_STATE) {

Review Comment:
   Thanks for digging up the details!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1220504060


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1599,6 +1599,8 @@ public void handleResponse(AbstractResponse response) {
                 fatalError(Errors.PRODUCER_FENCED.exception());
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
                 fatalError(error.exception());
+            } else if (error == Errors.INVALID_TXN_STATE) {

Review Comment:
   I went down a very long rabbit hole to find that we only returned INVALID_TXN_STATE for a short time from this request: https://github.com/apache/kafka/commit/1f2451d4e7e3766540d3650d177e304fcddf49b8 (here's the commit that removed this error). We never removed it from the errors returned I suppose because some really old broker could return it. 
   
   Having this here doesn't hurt, I was just curious about the history behind it. 😅 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13811: KAFKA-14278: Fix InvalidProducerEpochException and InvalidTxnStateException handling in producer clients

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13811:
URL: https://github.com/apache/kafka/pull/13811#discussion_r1220504060


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1599,6 +1599,8 @@ public void handleResponse(AbstractResponse response) {
                 fatalError(Errors.PRODUCER_FENCED.exception());
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
                 fatalError(error.exception());
+            } else if (error == Errors.INVALID_TXN_STATE) {

Review Comment:
   I went down a very long rabbit hole to find that we only returned INVALID_TXN_STATE for a short time from this request.
    https://github.com/apache/kafka/commit/1f2451d4e7e3766540d3650d177e304fcddf49b8 (here's the commit that removed this error). 
   
   We never removed it from the errors returned I suppose because some really old broker could return it. 
   
   Having this here doesn't hurt, I was just curious about the history behind it. 😅 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org