Copilot commented on code in PR #19642:
URL: https://github.com/apache/kafka/pull/19642#discussion_r2077902387


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1512,64 +1513,71 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, 
RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            Errors responseError = 
response.groupLevelError(rebalanceConfig.groupId);
-            if (responseError != Errors.NONE) {
-                log.debug("Offset fetch failed: {}", responseError.message());
+            var group = response.group(rebalanceConfig.groupId);
+            var groupError = Errors.forCode(group.errorCode());
 
-                if (responseError == Errors.COORDINATOR_NOT_AVAILABLE ||
-                    responseError == Errors.NOT_COORDINATOR) {
+            if (groupError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", groupError.message());
+
+                if (groupError == Errors.COORDINATOR_NOT_AVAILABLE ||
+                    groupError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(responseError);
-                    future.raise(responseError);
-                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) 
{
+                    markCoordinatorUnknown(groupError);
+                    future.raise(groupError);
+                } else if (groupError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     
future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
-                } else if (responseError.exception() instanceof 
RetriableException) {
+                } else if (groupError.exception() instanceof 
RetriableException) {
                     // retry
-                    future.raise(responseError);
+                    future.raise(groupError);
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch 
offset response: " + responseError.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch 
offset response: " + groupError.message()));
                 }
                 return;
             }
 
-            Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetFetchResponse.PartitionData> 
responseData =
-                response.partitionDataMap(rebalanceConfig.groupId);
-            Map<TopicPartition, OffsetAndMetadata> offsets = new 
HashMap<>(responseData.size());
-            Set<TopicPartition> unstableTxnOffsetTopicPartitions = new 
HashSet<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> 
entry : responseData.entrySet()) {
-                TopicPartition tp = entry.getKey();
-                OffsetFetchResponse.PartitionData partitionData = 
entry.getValue();
-                if (partitionData.hasError()) {
-                    Errors error = partitionData.error;
-                    log.debug("Failed to fetch offset for partition {}: {}", 
tp, error.message());
-
-                    if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
-                        future.raise(new KafkaException("Topic or Partition " 
+ tp + " does not exist"));
-                        return;
-                    } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
-                        if (unauthorizedTopics == null) {
-                            unauthorizedTopics = new HashSet<>();
+            var offsets = new HashMap<TopicPartition, OffsetAndMetadata>();
+            var unstableTxnOffsetTopicPartitions = new 
HashSet<TopicPartition>();
+            var unauthorizedTopics = new HashSet<String>();
+
+            for (var topic : group.topics()) {

Review Comment:
   [nitpick] Consider extracting the logic that iterates over group.topics() 
and converts each partition's data into an OffsetAndMetadata instance into a 
separate helper method to reduce duplication and improve readability.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java:
##########
@@ -60,221 +55,127 @@
 public class OffsetFetchResponse extends AbstractResponse {
     public static final long INVALID_OFFSET = -1L;
     public static final String NO_METADATA = "";
-    public static final PartitionData UNKNOWN_PARTITION = new 
PartitionData(INVALID_OFFSET,
-                                                                            
Optional.empty(),
-                                                                            
NO_METADATA,
-                                                                            
Errors.UNKNOWN_TOPIC_OR_PARTITION);
-    public static final PartitionData UNAUTHORIZED_PARTITION = new 
PartitionData(INVALID_OFFSET,
-                                                                               
  Optional.empty(),
-                                                                               
  NO_METADATA,
-                                                                               
  Errors.TOPIC_AUTHORIZATION_FAILED);
+
+    // We only need to track the partition errors returned in version 1. This
+    // is used to identify group level errors when the response is normalized.
     private static final List<Errors> PARTITION_ERRORS = Arrays.asList(
-        Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED);
+        Errors.UNKNOWN_TOPIC_OR_PARTITION,
+        Errors.TOPIC_AUTHORIZATION_FAILED
+    );
 
+    private final short version;
     private final OffsetFetchResponseData data;
-    private final Errors error;
-    private final Map<String, Errors> groupLevelErrors = new HashMap<>();
-
-    public static final class PartitionData {
-        public final long offset;
-        public final String metadata;
-        public final Errors error;
-        public final Optional<Integer> leaderEpoch;
+    // Lazily initialized when OffsetFetchResponse#group is called.
+    private Map<String, OffsetFetchResponseData.OffsetFetchResponseGroup> 
groups = null;
 
-        public PartitionData(long offset,
-                             Optional<Integer> leaderEpoch,
-                             String metadata,
-                             Errors error) {
-            this.offset = offset;
-            this.leaderEpoch = leaderEpoch;
-            this.metadata = metadata;
-            this.error = error;
-        }
-
-        public boolean hasError() {
-            return this.error != Errors.NONE;
-        }
-
-        @Override
-        public boolean equals(Object other) {
-            if (!(other instanceof PartitionData))
-                return false;
-            PartitionData otherPartition = (PartitionData) other;
-            return Objects.equals(this.offset, otherPartition.offset)
-                   && Objects.equals(this.leaderEpoch, 
otherPartition.leaderEpoch)
-                   && Objects.equals(this.metadata, otherPartition.metadata)
-                   && Objects.equals(this.error, otherPartition.error);
-        }
-
-        @Override
-        public String toString() {
-            return "PartitionData("
-                       + "offset=" + offset
-                       + ", leaderEpoch=" + 
leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH)
-                       + ", metadata=" + metadata
-                       + ", error='" + error.toString()
-                       + ")";
-        }
+    public static class Builder {
+        private final List<OffsetFetchResponseGroup> groups;
 
-        @Override
-        public int hashCode() {
-            return Objects.hash(offset, leaderEpoch, metadata, error);
+        public Builder(OffsetFetchResponseGroup group) {
+            this(List.of(group));
         }
-    }
-
-    /**
-     * Constructor without throttle time.
-     * @param error Potential coordinator or group level error code (for api 
version 2 and later)
-     * @param responseData Fetched offset information grouped by 
topic-partition
-     */
-    public OffsetFetchResponse(Errors error, Map<TopicPartition, 
PartitionData> responseData) {
-        this(DEFAULT_THROTTLE_TIME, error, responseData);
-    }
 
-    /**
-     * Constructor with throttle time for version 0 to 7
-     * @param throttleTimeMs The time in milliseconds that this response was 
throttled
-     * @param error Potential coordinator or group level error code (for api 
version 2 and later)
-     * @param responseData Fetched offset information grouped by 
topic-partition
-     */
-    public OffsetFetchResponse(int throttleTimeMs, Errors error, 
Map<TopicPartition, PartitionData> responseData) {
-        super(ApiKeys.OFFSET_FETCH);
-        Map<String, OffsetFetchResponseTopic> offsetFetchResponseTopicMap = 
new HashMap<>();
-        for (Map.Entry<TopicPartition, PartitionData> entry : 
responseData.entrySet()) {
-            String topicName = entry.getKey().topic();
-            OffsetFetchResponseTopic topic = 
offsetFetchResponseTopicMap.getOrDefault(
-                topicName, new OffsetFetchResponseTopic().setName(topicName));
-            PartitionData partitionData = entry.getValue();
-            topic.partitions().add(new OffsetFetchResponsePartition()
-                                       
.setPartitionIndex(entry.getKey().partition())
-                                       
.setErrorCode(partitionData.error.code())
-                                       
.setCommittedOffset(partitionData.offset)
-                                       .setCommittedLeaderEpoch(
-                                           
partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH))
-                                       .setMetadata(partitionData.metadata)
-            );
-            offsetFetchResponseTopicMap.put(topicName, topic);
+        public Builder(List<OffsetFetchResponseGroup> groups) {
+            this.groups = groups;
         }
 
-        this.data = new OffsetFetchResponseData()
-            .setTopics(new ArrayList<>(offsetFetchResponseTopicMap.values()))
-            .setErrorCode(error.code())
-            .setThrottleTimeMs(throttleTimeMs);
-        this.error = error;
-    }
+        public OffsetFetchResponse build(short version) {
+            var data = new OffsetFetchResponseData();
 
-    /**
-     * Constructor with throttle time for version 8 and above.
-     * @param throttleTimeMs The time in milliseconds that this response was 
throttled
-     * @param errors Potential coordinator or group level error code
-     * @param responseData Fetched offset information grouped by 
topic-partition and by group
-     */
-    public OffsetFetchResponse(int throttleTimeMs,
-                               Map<String, Errors> errors,
-                               Map<String, Map<TopicPartition, PartitionData>> 
responseData) {
-        super(ApiKeys.OFFSET_FETCH);
-        List<OffsetFetchResponseGroup> groupList = new ArrayList<>();
-        for (Entry<String, Map<TopicPartition, PartitionData>> entry : 
responseData.entrySet()) {
-            String groupName = entry.getKey();
-            Map<TopicPartition, PartitionData> partitionDataMap = 
entry.getValue();
-            Map<String, OffsetFetchResponseTopics> 
offsetFetchResponseTopicsMap = new HashMap<>();
-            for (Entry<TopicPartition, PartitionData> partitionEntry : 
partitionDataMap.entrySet()) {
-                String topicName = partitionEntry.getKey().topic();
-                OffsetFetchResponseTopics topic =
-                    offsetFetchResponseTopicsMap.getOrDefault(topicName,
-                        new OffsetFetchResponseTopics().setName(topicName));
-                PartitionData partitionData = partitionEntry.getValue();
-                topic.partitions().add(new OffsetFetchResponsePartitions()
-                    .setPartitionIndex(partitionEntry.getKey().partition())
-                    .setErrorCode(partitionData.error.code())
-                    .setCommittedOffset(partitionData.offset)
-                    .setCommittedLeaderEpoch(
-                        
partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH))
-                    .setMetadata(partitionData.metadata));
-                offsetFetchResponseTopicsMap.put(topicName, topic);
-            }
-            groupList.add(new OffsetFetchResponseGroup()
-                .setGroupId(groupName)
-                .setTopics(new 
ArrayList<>(offsetFetchResponseTopicsMap.values()))
-                .setErrorCode(errors.get(groupName).code()));
-            groupLevelErrors.put(groupName, errors.get(groupName));
-        }
-        this.data = new OffsetFetchResponseData()
-            .setGroups(groupList)
-            .setThrottleTimeMs(throttleTimeMs);
-        this.error = null;
-    }
-
-    public OffsetFetchResponse(List<OffsetFetchResponseGroup> groups, short 
version) {
-        super(ApiKeys.OFFSET_FETCH);
-        data = new OffsetFetchResponseData();
-
-        if (version >= 8) {
-            data.setGroups(groups);
-            error = null;
+            if (version >= BATCH_MIN_VERSION) {
+                data.setGroups(groups);
+            } else {
+                if (groups.size() != 1) {
+                    throw new UnsupportedVersionException(
+                        "Version " + version + " of OffsetFetchResponse only 
supports one group."
+                    );
+                }
 
-            for (OffsetFetchResponseGroup group : data.groups()) {
-                this.groupLevelErrors.put(group.groupId(), 
Errors.forCode(group.errorCode()));
-            }
-        } else {
-            if (groups.size() != 1) {
-                throw new UnsupportedVersionException(
-                    "Version " + version + " of OffsetFetchResponse only 
supports one group."
-                );
+                OffsetFetchResponseGroup group = groups.get(0);
+                data.setErrorCode(group.errorCode());
+
+                group.topics().forEach(topic -> {
+                    OffsetFetchResponseTopic newTopic = new 
OffsetFetchResponseTopic().setName(topic.name());
+                    data.topics().add(newTopic);
+
+                    topic.partitions().forEach(partition -> {
+                        OffsetFetchResponsePartition newPartition;
+
+                        if (version < 
TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION && group.errorCode() != 
Errors.NONE.code()) {
+                            // Versions prior to version 2 do not support a 
top level error. Therefore,
+                            // we put it at the partition level.
+                            newPartition = new OffsetFetchResponsePartition()
+                                .setPartitionIndex(partition.partitionIndex())
+                                .setErrorCode(group.errorCode())
+                                .setCommittedOffset(INVALID_OFFSET)
+                                .setMetadata(NO_METADATA)
+                                
.setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH);
+                        } else {
+                            newPartition = new OffsetFetchResponsePartition()
+                                .setPartitionIndex(partition.partitionIndex())
+                                .setErrorCode(partition.errorCode())
+                                
.setCommittedOffset(partition.committedOffset())
+                                .setMetadata(partition.metadata())
+                                
.setCommittedLeaderEpoch(partition.committedLeaderEpoch());
+                        }
+
+                        newTopic.partitions().add(newPartition);
+                    });
+                });
             }
 
-            OffsetFetchResponseGroup group = groups.get(0);
-            data.setErrorCode(group.errorCode());
-            error = Errors.forCode(group.errorCode());
-
-            group.topics().forEach(topic -> {
-                OffsetFetchResponseTopic newTopic = new 
OffsetFetchResponseTopic().setName(topic.name());
-                data.topics().add(newTopic);
-
-                topic.partitions().forEach(partition -> {
-                    OffsetFetchResponsePartition newPartition;
-
-                    if (version < 2 && group.errorCode() != 
Errors.NONE.code()) {
-                        // Versions prior to version 2 do not support a top 
level error. Therefore,
-                        // we put it at the partition level.
-                        newPartition = new OffsetFetchResponsePartition()
-                            .setPartitionIndex(partition.partitionIndex())
-                            .setErrorCode(group.errorCode())
-                            .setCommittedOffset(INVALID_OFFSET)
-                            .setMetadata(NO_METADATA)
-                            
.setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH);
-                    } else {
-                        newPartition = new OffsetFetchResponsePartition()
-                            .setPartitionIndex(partition.partitionIndex())
-                            .setErrorCode(partition.errorCode())
-                            .setCommittedOffset(partition.committedOffset())
-                            .setMetadata(partition.metadata())
-                            
.setCommittedLeaderEpoch(partition.committedLeaderEpoch());
-                    }
-
-                    newTopic.partitions().add(newPartition);
-                });
-            });
+            return new OffsetFetchResponse(data, version);
         }
     }
 
     public OffsetFetchResponse(OffsetFetchResponseData data, short version) {
         super(ApiKeys.OFFSET_FETCH);
         this.data = data;
-        // for version 2 and later use the top-level error code (in 
ERROR_CODE_KEY_NAME) from the response.
-        // for older versions there is no top-level error in the response and 
all errors are partition errors,
-        // so if there is a group or coordinator error at the partition level 
use that as the top-level error.
-        // this way clients can depend on the top-level error regardless of 
the offset fetch version.
-        // we return the error differently starting with version 8, so we will 
only populate the
-        // error field if we are between version 2 and 7. if we are in version 
8 or greater, then
-        // we will populate the map of group id to error codes.
-        if (version < 8) {
-            this.error = version >= 2 ? Errors.forCode(data.errorCode()) : 
topLevelError(data);
+        this.version = version;
+    }
+
+    public OffsetFetchResponseData.OffsetFetchResponseGroup group(String 
groupId) {
+        if (version < BATCH_MIN_VERSION) {
+            // for version 2 and later use the top-level error code from the 
response.
+            // for older versions there is no top-level error in the response 
and all errors are partition errors,
+            // so if there is a group or coordinator error at the partition 
level use that as the top-level error.
+            // this way clients can depend on the top-level error regardless 
of the offset fetch version.
+            // we return the error differently starting with version 8, so we 
will only populate the
+            // error field if we are between version 2 and 7. if we are in 
version 8 or greater, then
+            // we will populate the map of group id to error codes.

Review Comment:
   [nitpick] Consider adding a comment in the group() method explaining the 
version-specific logic, particularly the decision branches based on 
TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION and BATCH_MIN_VERSION, to aid 
future maintainers.
   ```suggestion
           // The logic in this method is version-specific due to changes in 
the OffsetFetch API over time.
           // For versions earlier than BATCH_MIN_VERSION (version 8), the 
response does not include a map of group IDs
           // to error codes. Instead, errors are either top-level or 
partition-level, and we need to infer the top-level
           // error for older versions.
           if (version < BATCH_MIN_VERSION) {
               // For versions 2 through 7, use the top-level error code from 
the response if available.
               // For versions earlier than 
TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION (version 6), there is no explicit
               // top-level error field in the response. In these cases, we 
derive the top-level error from partition errors.
               // This ensures that clients can rely on a consistent top-level 
error regardless of the version.
   ```



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