AndrewJSchofield commented on code in PR #18939:
URL: https://github.com/apache/kafka/pull/18939#discussion_r1961464616


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -851,59 +873,25 @@ private void handleShareAcknowledgeSuccess(Node 
fetchTarget,
                         // We retry the request until the timer expires, 
unless we are closing.
                         acknowledgeRequestState.moveAllToIncompleteAcks();
                     } else {
-                        
response.data().responses().forEach(shareAcknowledgeTopicResponse -> 
shareAcknowledgeTopicResponse.partitions().forEach(partitionData -> {
-                            TopicIdPartition tip = new 
TopicIdPartition(shareAcknowledgeTopicResponse.topicId(),
-                                    partitionData.partitionIndex(),
-                                    
metadata.topicNames().get(shareAcknowledgeTopicResponse.topicId()));
-
-                            
acknowledgeRequestState.handleAcknowledgeErrorCode(tip, response.error());
-                        }));
+                        
acknowledgeRequestState.processPendingInFlightAcknowledgements(response.error().exception());
                         acknowledgeRequestState.processingComplete();
                     }
+

Review Comment:
   nit: I would not put a blank line before a closing brace like this.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -97,6 +99,7 @@ public class ShareConsumeRequestManager implements 
RequestManager, MemberStateLi
     private final CompletableFuture<Void> closeFuture;
     private boolean isAcknowledgementCommitCallbackRegistered = false;
     private final Map<IdAndPartition, String> topicNamesMap = new HashMap<>();
+    private static final InvalidRecordStateException INVALID_RESPONSE = new 
InvalidRecordStateException("Acknowledgement not successful due to invalid 
response from broker");

Review Comment:
   I suggest having a static constant string for the message, but actually 
instantiating the exception at the time of use. Otherwise, you'll find that the 
stack trace for the exception is for the static initialization of the exception 
instance, and that's probably not what you want.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -851,59 +873,25 @@ private void handleShareAcknowledgeSuccess(Node 
fetchTarget,
                         // We retry the request until the timer expires, 
unless we are closing.
                         acknowledgeRequestState.moveAllToIncompleteAcks();
                     } else {
-                        
response.data().responses().forEach(shareAcknowledgeTopicResponse -> 
shareAcknowledgeTopicResponse.partitions().forEach(partitionData -> {
-                            TopicIdPartition tip = new 
TopicIdPartition(shareAcknowledgeTopicResponse.topicId(),
-                                    partitionData.partitionIndex(),
-                                    
metadata.topicNames().get(shareAcknowledgeTopicResponse.topicId()));
-
-                            
acknowledgeRequestState.handleAcknowledgeErrorCode(tip, response.error());
-                        }));
+                        
acknowledgeRequestState.processPendingInFlightAcknowledgements(response.error().exception());
                         acknowledgeRequestState.processingComplete();
                     }
+
                 } else {
                     AtomicBoolean shouldRetry = new AtomicBoolean(false);
                     // Check all partition level error codes
                     
response.data().responses().forEach(shareAcknowledgeTopicResponse -> 
shareAcknowledgeTopicResponse.partitions().forEach(partitionData -> {
                         Errors partitionError = 
Errors.forCode(partitionData.errorCode());
-                        TopicIdPartition tip = new 
TopicIdPartition(shareAcknowledgeTopicResponse.topicId(),
-                                partitionData.partitionIndex(),
-                                
metadata.topicNames().get(shareAcknowledgeTopicResponse.topicId()));
-                        if (partitionError.exception() != null) {
-                            boolean retry = false;
-
-                            if (partitionError == 
Errors.NOT_LEADER_OR_FOLLOWER || partitionError == Errors.FENCED_LEADER_EPOCH) {
-                                // If the leader has changed, there's no point 
in retrying the operation because the acquisition locks
-                                // will have been released.
-                                TopicPartition tp = new 
TopicPartition(metadata.topicNames().get(shareAcknowledgeTopicResponse.topicId()),
 partitionData.partitionIndex());
-
-                                log.debug("For {}, received error {}, with 
leaderIdAndEpoch {}", tp, partitionError, partitionData.currentLeader());
-                                if (partitionData.currentLeader().leaderId() 
!= -1 && partitionData.currentLeader().leaderEpoch() != -1) {
-                                    partitionsWithUpdatedLeaderInfo.put(tp, 
new Metadata.LeaderIdAndEpoch(
-                                        
Optional.of(partitionData.currentLeader().leaderId()), 
Optional.of(partitionData.currentLeader().leaderEpoch())));
-                                }
-                            } else if (partitionError.exception() instanceof 
RetriableException) {
-                                retry = true;
-                            }
-
-                            if (retry) {
-                                // Move to incomplete acknowledgements to retry
-                                
acknowledgeRequestState.moveToIncompleteAcks(tip);
-                                shouldRetry.set(true);
-                            } else {
-                                
metricsManager.recordFailedAcknowledgements(acknowledgeRequestState.getInFlightAcknowledgementsCount(tip));
-                                
acknowledgeRequestState.handleAcknowledgeErrorCode(tip, partitionError);
-                            }
-                        } else {
-                            
acknowledgeRequestState.handleAcknowledgeErrorCode(tip, partitionError);
+                        TopicIdPartition tip = 
createTopicIdPartition(shareAcknowledgeTopicResponse.topicId(), 
partitionData.partitionIndex());
+                        if (tip == null) {
+                            log.error("Invalid topic name in SA-2 response");

Review Comment:
   "SA-2 response" looks like you've left some testing code in here. I expect 
it's a mistake.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -811,6 +829,7 @@ private void handleShareFetchFailure(Node fetchTarget,
                     }
                 }
             }));
+

Review Comment:
   nit: Blank line before a closing brace doesn't match the usual style.



-- 
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