showuon commented on a change in pull request #11019: URL: https://github.com/apache/kafka/pull/11019#discussion_r667672477
########## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandler.java ########## @@ -100,51 +100,115 @@ public String apiName() { final OffsetDeleteResponse response = (OffsetDeleteResponse) abstractResponse; Map<CoordinatorKey, Map<TopicPartition, Errors>> completed = new HashMap<>(); Map<CoordinatorKey, Throwable> failed = new HashMap<>(); - List<CoordinatorKey> unmapped = new ArrayList<>(); + final Set<CoordinatorKey> groupsToUnmap = new HashSet<>(); + final Set<CoordinatorKey> groupsToRetry = new HashSet<>(); final Errors error = Errors.forCode(response.data().errorCode()); if (error != Errors.NONE) { - handleError(groupId, error, failed, unmapped); + handleGroupError(groupId, error, failed, groupsToUnmap, groupsToRetry); } else { - final Map<TopicPartition, Errors> partitions = new HashMap<>(); - response.data().topics().forEach(topic -> - topic.partitions().forEach(partition -> { - Errors partitionError = Errors.forCode(partition.errorCode()); - if (!handleError(groupId, partitionError, failed, unmapped)) { - partitions.put(new TopicPartition(topic.name(), partition.partitionIndex()), partitionError); + final Map<TopicPartition, Errors> partitionResults = new HashMap<>(); + response.data().topics().forEach(topic -> + topic.partitions().forEach(partitionoffsetDeleteResponse -> { + Errors partitionError = Errors.forCode(partitionoffsetDeleteResponse.errorCode()); + TopicPartition topicPartition = new TopicPartition(topic.name(), partitionoffsetDeleteResponse.partitionIndex()); + if (partitionError != Errors.NONE) { + handlePartitionError(groupId, partitionError, topicPartition, groupsToUnmap, groupsToRetry); } + + partitionResults.put(new TopicPartition(topic.name(), partitionoffsetDeleteResponse.partitionIndex()), partitionError); Review comment: put every error into partitionResults, as the log logic did -- 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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org