lianetm commented on code in PR #15202:
URL: https://github.com/apache/kafka/pull/15202#discussion_r1462700895


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -536,12 +543,23 @@ public void onResponse(final ClientResponse response) {
                         continue;
                     }
 
-                    if (error == Errors.COORDINATOR_NOT_AVAILABLE ||
+                    if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                        
future.completeExceptionally(GroupAuthorizationException.forGroupId(groupId));
+                        return;
+                    } else if (error == Errors.COORDINATOR_NOT_AVAILABLE ||
                         error == Errors.NOT_COORDINATOR ||
                         error == Errors.REQUEST_TIMED_OUT) {
                         
coordinatorRequestManager.markCoordinatorUnknown(error.message(), 
currentTimeMs);
                         maybeRetry(currentTimeMs, error.exception());
                         return;
+                    } else if (error == Errors.FENCED_INSTANCE_ID) {
+                        log.info("OffsetCommit failed due to group instance id 
{} fenced: {}", groupInstanceId, error.message());
+                        future.completeExceptionally(new 
CommitFailedException());
+                        return;
+                    } else if (error == Errors.OFFSET_METADATA_TOO_LARGE ||
+                        error == Errors.INVALID_COMMIT_OFFSET_SIZE) {
+                        future.completeExceptionally(error.exception());
+                        return;
                     } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS ||
                         error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
                         // just retry

Review Comment:
   Agree, but the intention was to bring the retry changes in a next PR after 
this one, is that ok? That one will include the refactoring to move all the 
retries to the specific caller functions in this same manager, and keep the 
response handling only completing/failing the future.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to