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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -8078,6 +8092,73 @@ public 
List<DeleteShareGroupStateRequestData.DeleteStateData> sharePartitionsEli
         return deleteShareGroupStateRequestTopicsData;
     }
 
+    public Map.Entry<AlterShareGroupOffsetsResponseData, 
InitializeShareGroupStateParameters> completeAlterShareGroupOffsets(
+        String groupId,
+        AlterShareGroupOffsetsRequestData alterShareGroupOffsetsRequest,
+        List<CoordinatorRecord> records
+    ) {
+        Group group = groups.get(groupId);
+        
List<AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic> 
alterShareGroupOffsetsResponseTopics = new ArrayList<>();
+
+        Map<Uuid, Set<Integer>> initializingTopics = new HashMap<>();
+        Map<Uuid, Map<Integer, Long>> offsetByTopicPartitions = new 
HashMap<>();
+
+        alterShareGroupOffsetsRequest.topics().forEach(topic -> {
+            TopicImage topicImage = 
metadataImage.topics().getTopic(topic.topicName());
+            if (topicImage != null) {
+                Uuid topicId = topicImage.id();
+                Set<Integer> existingPartitions = new 
HashSet<>(topicImage.partitions().keySet());
+                
List<AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition>
 partitions = new ArrayList<>();
+                topic.partitions().forEach(partition -> {
+                    if 
(existingPartitions.contains(partition.partitionIndex())) {
+                        partitions.add(
+                            new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
+                                .setPartitionIndex(partition.partitionIndex())
+                                .setErrorCode(Errors.NONE.code()));
+                        offsetByTopicPartitions.computeIfAbsent(topicId, k -> 
new HashMap<>()).put(partition.partitionIndex(), partition.startOffset());
+                    } else {
+                        partitions.add(
+                            new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
+                                .setPartitionIndex(partition.partitionIndex())
+                                
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
+                                
.setErrorMessage(Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
+                    }
+                });
+
+                initializingTopics.put(topicId, topic.partitions().stream()
+                    
.map(AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition::partitionIndex)
+                    .filter(existingPartitions::contains)
+                    .collect(Collectors.toSet()));
+
+                alterShareGroupOffsetsResponseTopics.add(
+                    new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
+                        .setTopicName(topic.topicName())

Review Comment:
   I would expect you to set topic ID at this point. In some error situations, 
you do not know the topic ID. Here you do and passing it back from here seems 
to make sense to me.



##########
clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java:
##########
@@ -58,17 +54,25 @@ public String toString() {
 
     @Override
     public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        
List<AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic> 
results = new ArrayList<>();
-        data.topics().forEach(
-            topicResult -> results.add(new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
-                .setTopicName(topicResult.topicName())
-                .setPartitions(topicResult.partitions().stream()
-                    .map(partitionData -> new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
-                        .setPartitionIndex(partitionData.partitionIndex())
-                        .setErrorCode(Errors.forException(e).code()))
-                    .collect(Collectors.toList()))));
-        return new AlterShareGroupOffsetsResponse(new 
AlterShareGroupOffsetsResponseData()
-            .setResponses(results));
+        Errors errors = Errors.forException(e);
+        return new 
AlterShareGroupOffsetsResponse(getErrorResponse(throttleTimeMs, errors));
+    }
+
+    public static AlterShareGroupOffsetsResponseData getErrorResponse(int 
throttleTimeMs, Errors error) {
+        return new AlterShareGroupOffsetsResponseData()
+            .setThrottleTimeMs(throttleTimeMs)
+            .setErrorCode(error.code())
+            .setErrorMessage(error.message());
+    }
+
+    public static AlterShareGroupOffsetsResponseData getErrorResponse(Errors 
errors) {

Review Comment:
   And here.



##########
clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java:
##########
@@ -58,17 +54,25 @@ public String toString() {
 
     @Override
     public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        
List<AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic> 
results = new ArrayList<>();
-        data.topics().forEach(
-            topicResult -> results.add(new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
-                .setTopicName(topicResult.topicName())
-                .setPartitions(topicResult.partitions().stream()
-                    .map(partitionData -> new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
-                        .setPartitionIndex(partitionData.partitionIndex())
-                        .setErrorCode(Errors.forException(e).code()))
-                    .collect(Collectors.toList()))));
-        return new AlterShareGroupOffsetsResponse(new 
AlterShareGroupOffsetsResponseData()
-            .setResponses(results));
+        Errors errors = Errors.forException(e);

Review Comment:
   nit: This is an error, not errors. `Errors error` would be better.



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