chia7712 commented on code in PR #20049:
URL: https://github.com/apache/kafka/pull/20049#discussion_r2192904850


##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -87,30 +92,38 @@ public String apiName() {
     }
 
     @Override
-    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+    public ApiResult<CoordinatorKey, Map<TopicPartition, ApiException>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+        validateKeys(keys);
+
         AlterShareGroupOffsetsResponse response = 
(AlterShareGroupOffsetsResponse) abstractResponse;
-        final Map<TopicPartition, Errors> partitionResults = new HashMap<>();
         final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
         final Set<CoordinatorKey> groupsToRetry = new HashSet<>();
-
-        for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic topic : 
response.data().responses()) {
-            for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition 
partition : topic.partitions()) {
-                TopicPartition topicPartition = new 
TopicPartition(topic.topicName(), partition.partitionIndex());
-                Errors error = Errors.forCode(partition.errorCode());
-
-                if (error != Errors.NONE) {
-                    handleError(
-                        groupId,
-                        topicPartition,
-                        error,
-                        partitionResults,
-                        groupsToUnmap,
-                        groupsToRetry
-                    );
-                } else {
-                    partitionResults.put(topicPartition, error);
+        final Map<TopicPartition, ApiException> partitionResults = new 
HashMap<>();
+
+        if (response.data().errorCode() != Errors.NONE.code()) {
+            final Errors topLevelError = 
Errors.forCode(response.data().errorCode());
+            final String topLevelErrorMessage = response.data().errorMessage();
+
+            offsets.forEach((topicPartition, offset) ->
+                handleError(
+                    groupId,
+                    topicPartition,
+                    topLevelError,
+                    topLevelErrorMessage,
+                    partitionResults,
+                    groupsToUnmap,
+                    groupsToRetry
+                ));
+        } else {
+            response.data().responses().forEach(topic -> 
topic.partitions().forEach(partition -> {
+                if (partition.errorCode() != Errors.NONE.code()) {
+                    final Errors partitionError = 
Errors.forCode(partition.errorCode());

Review Comment:
   nit: we can reuse the `Errors`



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -87,30 +92,38 @@ public String apiName() {
     }
 
     @Override
-    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+    public ApiResult<CoordinatorKey, Map<TopicPartition, ApiException>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+        validateKeys(keys);
+
         AlterShareGroupOffsetsResponse response = 
(AlterShareGroupOffsetsResponse) abstractResponse;
-        final Map<TopicPartition, Errors> partitionResults = new HashMap<>();
         final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
         final Set<CoordinatorKey> groupsToRetry = new HashSet<>();
-
-        for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic topic : 
response.data().responses()) {
-            for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition 
partition : topic.partitions()) {
-                TopicPartition topicPartition = new 
TopicPartition(topic.topicName(), partition.partitionIndex());
-                Errors error = Errors.forCode(partition.errorCode());
-
-                if (error != Errors.NONE) {
-                    handleError(
-                        groupId,
-                        topicPartition,
-                        error,
-                        partitionResults,
-                        groupsToUnmap,
-                        groupsToRetry
-                    );
-                } else {
-                    partitionResults.put(topicPartition, error);
+        final Map<TopicPartition, ApiException> partitionResults = new 
HashMap<>();
+
+        if (response.data().errorCode() != Errors.NONE.code()) {
+            final Errors topLevelError = 
Errors.forCode(response.data().errorCode());
+            final String topLevelErrorMessage = response.data().errorMessage();
+
+            offsets.forEach((topicPartition, offset) ->
+                handleError(
+                    groupId,
+                    topicPartition,
+                    topLevelError,
+                    topLevelErrorMessage,
+                    partitionResults,
+                    groupsToUnmap,
+                    groupsToRetry
+                ));
+        } else {
+            response.data().responses().forEach(topic -> 
topic.partitions().forEach(partition -> {
+                if (partition.errorCode() != Errors.NONE.code()) {
+                    final Errors partitionError = 
Errors.forCode(partition.errorCode());

Review Comment:
   nit: the debug message should use placeholder for `partitionErrorMessage`



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -87,30 +92,38 @@ public String apiName() {
     }
 
     @Override
-    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+    public ApiResult<CoordinatorKey, Map<TopicPartition, ApiException>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+        validateKeys(keys);
+
         AlterShareGroupOffsetsResponse response = 
(AlterShareGroupOffsetsResponse) abstractResponse;
-        final Map<TopicPartition, Errors> partitionResults = new HashMap<>();
         final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
         final Set<CoordinatorKey> groupsToRetry = new HashSet<>();
-
-        for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic topic : 
response.data().responses()) {
-            for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition 
partition : topic.partitions()) {
-                TopicPartition topicPartition = new 
TopicPartition(topic.topicName(), partition.partitionIndex());
-                Errors error = Errors.forCode(partition.errorCode());
-
-                if (error != Errors.NONE) {
-                    handleError(
-                        groupId,
-                        topicPartition,
-                        error,
-                        partitionResults,
-                        groupsToUnmap,
-                        groupsToRetry
-                    );
-                } else {
-                    partitionResults.put(topicPartition, error);
+        final Map<TopicPartition, ApiException> partitionResults = new 
HashMap<>();
+
+        if (response.data().errorCode() != Errors.NONE.code()) {
+            final Errors topLevelError = 
Errors.forCode(response.data().errorCode());
+            final String topLevelErrorMessage = response.data().errorMessage();
+
+            offsets.forEach((topicPartition, offset) ->
+                handleError(
+                    groupId,
+                    topicPartition,
+                    topLevelError,
+                    topLevelErrorMessage,
+                    partitionResults,
+                    groupsToUnmap,
+                    groupsToRetry
+                ));
+        } else {
+            response.data().responses().forEach(topic -> 
topic.partitions().forEach(partition -> {
+                if (partition.errorCode() != Errors.NONE.code()) {
+                    final Errors partitionError = 
Errors.forCode(partition.errorCode());

Review Comment:
   for example:
   ```java
                   var partitionError = Errors.forCode(partition.errorCode());
                   var partitionErrorMessage = partition.errorMessage();
                   if (partitionError != Errors.NONE) {
                       log.debug("AlterShareGroupOffsets request for group id 
{} and topic-partition {}-{} failed and returned error {}. {}",
                           groupId.idValue, topic.topicName(), 
partition.partitionIndex(), partitionError, partitionErrorMessage);
                   }
                   partitionResults.put(new TopicPartition(topic.topicName(), 
partition.partitionIndex()), partitionError.exception(partitionErrorMessage));
   ```



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

Reply via email to