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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -1243,6 +1248,48 @@ private CompletableFuture<Map<String, Errors>> 
persisterDeleteToGroupIdErrorMap(
         });
     }
 
+    private void populateDeleteShareGroupOffsetsFuture(
+        DeleteShareGroupOffsetsRequestData requestData,
+        CompletableFuture<DeleteShareGroupOffsetsResponseData> future,
+        Map<Uuid, String> requestTopicIdToNameMapping,
+        List<DeleteShareGroupStateRequestData.DeleteStateData> 
deleteShareGroupStateRequestTopicsData,
+        
List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> 
deleteShareGroupOffsetsResponseTopicList
+
+    ) {
+        DeleteShareGroupStateRequestData deleteShareGroupStateRequestData = 
new DeleteShareGroupStateRequestData()
+            .setGroupId(requestData.groupId())
+            .setTopics(deleteShareGroupStateRequestTopicsData);
+
+        
persister.deleteState(DeleteShareGroupStateParameters.from(deleteShareGroupStateRequestData))
+            .whenComplete((result, error) -> {
+                if (error != null) {
+                    log.error("Failed to delete share group state");
+                    future.completeExceptionally(error);
+                    return;
+                }
+                if (result == null || result.topicsData() == null) {
+                    log.error("Result is null for the delete share group 
state");
+                    future.completeExceptionally(new 
IllegalStateException("Result is null for the delete share group state"));
+                    return;
+                }
+                result.topicsData().forEach(topicData ->
+                    deleteShareGroupOffsetsResponseTopicList.add(new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic()
+                        .setTopicId(topicData.topicId())
+                        
.setTopicName(requestTopicIdToNameMapping.get(topicData.topicId()))
+                        .setPartitions(topicData.partitions().stream().map(
+                            partitionData -> new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition()
+                                .setPartitionIndex(partitionData.partition())
+                                
.setErrorMessage(Errors.forCode(partitionData.errorCode()).message())

Review Comment:
   This seemingly innocuous line has a small problem. If the error code is 
NONE, which it generally is, the error message of `"NONE"` will be set, when 
really we want `null`. The code helpfully does a `toString()` which is not what 
we want for the protocol response. I suggest
   
   ```
   partitionData.errorCode() == Errors.NONE.code() ? null : 
Errors.forCode(partitionData.errorCode()).message()
   ```
   
   A bit scruffy, I'm afraid.



##########
clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupOffsetsResult.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The result of the {@link Admin#deleteShareGroupOffsets(String, Set, 
DeleteShareGroupOffsetsOptions)} call.
+ * <p>
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DeleteShareGroupOffsetsResult {
+
+    private final KafkaFuture<Map<TopicPartition, ApiException>> future;
+    private final Set<TopicPartition> partitions;
+
+    DeleteShareGroupOffsetsResult(KafkaFuture<Map<TopicPartition, 
ApiException>> future, Set<TopicPartition> partitions) {
+        this.future = future;
+        this.partitions = partitions;
+    }
+
+    /**
+     * Return a future which succeeds only if all the deletions succeed.
+     * * If not, the first partition error shall be returned.

Review Comment:
   nit: Extra `*`



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -1508,6 +1555,110 @@ private 
CompletableFuture<DescribeShareGroupOffsetsResponseData.DescribeShareGro
         return future;
     }
 
+    /**
+     * See {@link GroupCoordinator#deleteShareGroupOffsets(RequestContext, 
DeleteShareGroupOffsetsRequestData)}.
+     */
+    @Override
+    public CompletableFuture<DeleteShareGroupOffsetsResponseData> 
deleteShareGroupOffsets(
+        RequestContext context,
+        DeleteShareGroupOffsetsRequestData requestData
+    ) {
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.COORDINATOR_NOT_AVAILABLE));
+        }
+
+        if (metadataImage == null) {
+            return CompletableFuture.completedFuture(
+                
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.COORDINATOR_NOT_AVAILABLE));
+        }
+
+        String groupId = requestData.groupId();
+
+        if (!isGroupIdNotEmpty(groupId)) {
+            return CompletableFuture.completedFuture(
+                
DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.INVALID_GROUP_ID));
+        }
+
+        Map<Uuid, String> requestTopicIdToNameMapping = new HashMap<>();
+        List<DeleteShareGroupStateRequestData.DeleteStateData> 
deleteShareGroupStateRequestTopicsData = new 
ArrayList<>(requestData.topics().size());
+        
List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> 
deleteShareGroupOffsetsResponseTopicList = new 
ArrayList<>(requestData.topics().size());
+
+        requestData.topics().forEach(topic -> {
+            Uuid topicId = 
metadataImage.topics().topicNameToIdView().get(topic.topicName());
+            if (topicId != null) {
+                requestTopicIdToNameMapping.put(topicId, topic.topicName());
+                deleteShareGroupStateRequestTopicsData.add(new 
DeleteShareGroupStateRequestData.DeleteStateData()
+                    .setTopicId(topicId)
+                    .setPartitions(
+                        topic.partitions().stream().map(
+                            partitionIndex -> new 
DeleteShareGroupStateRequestData.PartitionData().setPartition(partitionIndex)
+                        ).toList()
+                    ));
+            } else {
+                deleteShareGroupOffsetsResponseTopicList.add(new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic()
+                    .setTopicName(topic.topicName())
+                    .setPartitions(topic.partitions().stream().map(
+                        partition -> new 
DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition()
+                            .setPartitionIndex(partition)
+                            
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
+                            
.setErrorMessage(Errors.UNKNOWN_TOPIC_OR_PARTITION.message())
+                    ).toList()));
+            }
+        });
+
+        // If the request for the persister is empty, just complete the 
operation right away.
+        if (deleteShareGroupStateRequestTopicsData.isEmpty()) {
+            return CompletableFuture.completedFuture(
+                new DeleteShareGroupOffsetsResponseData()
+                    .setResponses(deleteShareGroupOffsetsResponseTopicList));
+        }
+
+        CompletableFuture<DeleteShareGroupOffsetsResponseData> future = new 
CompletableFuture<>();
+
+        TopicPartition topicPartition = topicPartitionFor(groupId);
+
+        // This is done to make sure the provided group is empty. Offsets can 
be deleted only for an empty share group.
+        CompletableFuture<List<ShareGroupDescribeResponseData.DescribedGroup>> 
describeGroupFuture =
+            runtime.scheduleReadOperation(
+                "share-group-describe",
+                topicPartition,
+                (coordinator, lastCommittedOffset) -> 
coordinator.shareGroupDescribe(List.of(groupId), lastCommittedOffset)
+            ).exceptionally(exception -> handleOperationException(
+                "share-group-describe",
+                List.of(groupId),
+                exception,
+                (error, __) -> 
ShareGroupDescribeRequest.getErrorDescribedGroupList(List.of(groupId), error),
+                log
+            ));
+
+        describeGroupFuture.whenComplete((groups, throwable) -> {
+            if (throwable != null) {
+                log.error("Failed to describe the share group {}", groupId, 
throwable);
+                
future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.GROUP_ID_NOT_FOUND));

Review Comment:
   I suggest `Errors.forException(throwable)` here.



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