chirag-wadhwa5 commented on code in PR #19478:
URL: https://github.com/apache/kafka/pull/19478#discussion_r2046501920


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -1257,38 +1257,103 @@ private CompletableFuture<Map<String, Errors>> 
persisterDeleteToGroupIdErrorMap(
         });
     }
 
-    private CompletableFuture<DeleteShareGroupOffsetsResponseData> 
persistDeleteShareGroupOffsets(
-        DeleteShareGroupStateParameters deleteStateRequestParameters,
-        
List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> 
errorTopicResponseList
+    private CompletableFuture<DeleteShareGroupOffsetsResponseData> 
completeDeleteShareGroupOffsets(
+        String groupId,
+        Map<Uuid, String> successTopics,
+        
List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> 
errorTopicResponses
     ) {
-        return persister.deleteState(deleteStateRequestParameters)
-            .thenCompose(result -> {
-                if (result == null || result.topicsData() == null) {
-                    log.error("Result is null for the delete share group 
state");
-                    Exception exception = new IllegalStateException("Result is 
null for the delete share group state");
-                    return CompletableFuture.completedFuture(
-                        
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(exception))
-                    );
-                }
-                result.topicsData().forEach(topicData ->
-                    errorTopicResponseList.add(
-                        new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic()
-                            .setTopicId(topicData.topicId())
-                            
.setTopicName(metadataImage.topics().topicIdToNameView().get(topicData.topicId()))
-                            .setPartitions(topicData.partitions().stream().map(
-                                partitionData -> new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition()
-                                    
.setPartitionIndex(partitionData.partition())
-                                    .setErrorMessage(partitionData.errorCode() 
== Errors.NONE.code() ? null : 
Errors.forCode(partitionData.errorCode()).message())
-                                    .setErrorCode(partitionData.errorCode())
-                            ).toList())
-                    )
-                );
+        return runtime.scheduleWriteOperation(
+            "complete-delete-share-group-offsets",
+            topicPartitionFor(groupId),
+            Duration.ofMillis(config.offsetCommitTimeoutMs()),
+            coordinator -> 
coordinator.completeDeleteShareGroupOffsets(groupId, successTopics, 
errorTopicResponses)
+        ).exceptionally(exception -> handleOperationException(
+            "complete-delete-share-group-offsets",
+            groupId,
+            exception,
+            (error, __) -> 
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(error),
+            log
+        ));
+    }
 
-                return CompletableFuture.completedFuture(
-                    new DeleteShareGroupOffsetsResponseData()
-                        .setResponses(errorTopicResponseList)
+    private CompletableFuture<DeleteShareGroupOffsetsResponseData> 
handleDeleteShareGroupStateResult(
+        String groupId,
+        DeleteShareGroupStateResult result,
+        
List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> 
errorTopicResponses
+    ) {
+        if (result == null || result.topicsData() == null) {
+            log.error("Result is null for the delete share group state");
+            Exception exception = new IllegalStateException("Result is null 
for the delete share group state");
+            return CompletableFuture.completedFuture(
+                
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(exception))
+            );
+        }
+        Map<Uuid, String> successTopics = new HashMap<>();
+        result.topicsData().forEach(topicData -> {
+            Optional<PartitionErrorData> errItem = 
topicData.partitions().stream()
+                .filter(errData -> errData.errorCode() != Errors.NONE.code())
+                .findAny();
+
+            if (errItem.isPresent()) {
+                errorTopicResponses.add(
+                    new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic()
+                        .setTopicId(topicData.topicId())
+                        
.setTopicName(metadataImage.topics().topicIdToNameView().get(topicData.topicId()))
+                        
.setErrorMessage(Errors.forCode(errItem.get().errorCode()).message())
+                        .setErrorCode(errItem.get().errorCode())
                 );
-            }).exceptionally(throwable -> {
+            } else {
+                successTopics.put(
+                    topicData.topicId(),
+                    
metadataImage.topics().topicIdToNameView().get(topicData.topicId())
+                );
+            }
+        });
+
+        // If there are no topics for which persister delete state request 
succeeded, then we can return directly from here
+        if (successTopics.isEmpty()) {
+            return CompletableFuture.completedFuture(
+                new DeleteShareGroupOffsetsResponseData()
+                    .setResponses(errorTopicResponses)
+            );
+        }
+
+        return completeDeleteShareGroupOffsets(groupId, successTopics, 
errorTopicResponses);
+    }
+
+    private CompletableFuture<DeleteShareGroupOffsetsResponseData> 
persistDeleteShareGroupOffsets(

Review Comment:
   Thanks for the review. The 2 methods defined in this PR in group coordinator 
shard (`initiateDeleteShareGroupOffsets`, `completeDeleteShareGroupOffsets`) 
would not correspond to the method referred to in this comment. I have changed 
this method name to `deleteState` which is method in share coordinator shard 
and is called eventually to delete the share group state



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