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


##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.KafkaAdminClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.DeleteShareGroupOffsetsRequestData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.DeleteShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.DeleteShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This class is the handler for {@link 
KafkaAdminClient#deleteShareGroupOffsets(String, Set, 
DeleteShareGroupOffsetsOptions)} call
+ */
+public class DeleteShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+
+    private final Logger log;
+
+    private final Set<TopicPartition> partitions;
+
+    private final CoordinatorStrategy lookupStrategy;
+
+
+    public DeleteShareGroupOffsetsHandler(String groupId, Set<TopicPartition> 
partitions, LogContext logContext) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.partitions = partitions;
+        this.log = logContext.logger(DeleteShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "deleteShareGroupOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(String groupId) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    private void validateKeys(Set<CoordinatorKey> groupIds) {
+        if (!groupIds.equals(Collections.singleton(groupId))) {
+            throw new IllegalArgumentException("Received unexpected group ids 
" + groupIds +
+                " (expected only " + Collections.singleton(groupId) + ")");
+        }
+    }
+
+    @Override
+    DeleteShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<CoordinatorKey> groupIds) {
+        validateKeys(groupIds);
+
+        final 
List<DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic> 
topics =
+            new ArrayList<>();
+        
partitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).forEach((topic,
 topicPartitions) -> topics.add(
+            new 
DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic()
+                .setTopicName(topic)
+                .setPartitions(topicPartitions.stream()
+                    .map(TopicPartition::partition)
+                    .collect(Collectors.toList())
+                )
+        ));
+
+        return new DeleteShareGroupOffsetsRequest.Builder(
+            new DeleteShareGroupOffsetsRequestData()
+                .setGroupId(groupId.idValue)
+                .setTopics(topics),
+            true
+        );
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(
+        Node coordinator,
+        Set<CoordinatorKey> groupIds,
+        AbstractResponse abstractResponse
+    ) {
+        validateKeys(groupIds);
+
+        final DeleteShareGroupOffsetsResponse response = 
(DeleteShareGroupOffsetsResponse) abstractResponse;
+
+        final Errors groupError = Errors.forCode(response.data().errorCode());
+
+        if (groupError != Errors.NONE) {
+            final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
+            final Map<CoordinatorKey, Throwable> groupsFailed = new 
HashMap<>();
+            handleGroupError(groupId, groupError, groupsFailed, groupsToUnmap);
+
+            return new ApiResult<>(Collections.emptyMap(), groupsFailed, new 
ArrayList<>(groupsToUnmap));
+        } else {
+            final Map<TopicPartition, Errors> partitionResults = new 
HashMap<>();
+            response.data().responses().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    partitionResults.put(
+                        new TopicPartition(topic.topicName(), 
partition.partitionIndex()),
+                        Errors.forCode(partition.errorCode())

Review Comment:
   Ah, but AlterShareGroupOffsetsHandler is in a draft PR which is not yet 
reviewed, and DeleteConsumerGroupOffsetsHandler is built upon the 
`OffsetDelete` RPC which is relatively old and lacks an `ErrorMessage` field.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.KafkaAdminClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.DeleteShareGroupOffsetsRequestData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.DeleteShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.DeleteShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This class is the handler for {@link 
KafkaAdminClient#deleteShareGroupOffsets(String, Set, 
DeleteShareGroupOffsetsOptions)} call
+ */
+public class DeleteShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+
+    private final Logger log;
+
+    private final Set<TopicPartition> partitions;
+
+    private final CoordinatorStrategy lookupStrategy;
+
+    public DeleteShareGroupOffsetsHandler(String groupId, Set<TopicPartition> 
partitions, LogContext logContext) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.partitions = partitions;
+        this.log = logContext.logger(DeleteShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "deleteShareGroupOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(String groupId) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    private void validateKeys(Set<CoordinatorKey> groupIds) {
+        if (!groupIds.equals(Collections.singleton(groupId))) {
+            throw new IllegalArgumentException("Received unexpected group ids 
" + groupIds +
+                " (expected only " + Collections.singleton(groupId) + ")");
+        }
+    }
+
+    @Override
+    DeleteShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<CoordinatorKey> groupIds) {
+        validateKeys(groupIds);
+
+        final 
List<DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic> 
topics =
+            new ArrayList<>();
+        
partitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).forEach((topic,
 topicPartitions) -> topics.add(
+            new 
DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic()
+                .setTopicName(topic)
+                .setPartitions(topicPartitions.stream()
+                    .map(TopicPartition::partition)
+                    .collect(Collectors.toList())
+                )
+        ));
+
+        return new DeleteShareGroupOffsetsRequest.Builder(
+            new DeleteShareGroupOffsetsRequestData()
+                .setGroupId(groupId.idValue)
+                .setTopics(topics),
+            true
+        );
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(
+        Node coordinator,
+        Set<CoordinatorKey> groupIds,
+        AbstractResponse abstractResponse
+    ) {
+        validateKeys(groupIds);
+
+        final DeleteShareGroupOffsetsResponse response = 
(DeleteShareGroupOffsetsResponse) abstractResponse;
+
+        final Errors groupError = Errors.forCode(response.data().errorCode());
+
+        if (groupError != Errors.NONE) {
+            final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
+            final Map<CoordinatorKey, Throwable> groupsFailed = new 
HashMap<>();
+            handleGroupError(groupId, groupError, groupsFailed, groupsToUnmap);
+
+            return new ApiResult<>(Collections.emptyMap(), groupsFailed, new 
ArrayList<>(groupsToUnmap));
+        } else {
+            final Map<TopicPartition, Errors> partitionResults = new 
HashMap<>();
+            response.data().responses().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    partitionResults.put(
+                        new TopicPartition(topic.topicName(), 
partition.partitionIndex()),
+                        Errors.forCode(partition.errorCode())
+                    )
+                )
+            );
+
+            return ApiResult.completed(groupId, partitionResults);
+        }
+    }
+
+    private void handleGroupError(
+        CoordinatorKey groupId,
+        Errors error,
+        Map<CoordinatorKey, Throwable> failed,
+        Set<CoordinatorKey> groupsToUnmap
+    ) {
+        switch (error) {
+            case COORDINATOR_LOAD_IN_PROGRESS:
+            case REBALANCE_IN_PROGRESS:
+                // If the coordinator is in the middle of loading, then we 
just need to retry
+                log.debug("DeleteShareGroupOffsets request for group id {} 
failed because the coordinator" +
+                    " is still in the process of loading state. Will retry.", 
groupId.idValue);
+                break;
+            case COORDINATOR_NOT_AVAILABLE:
+            case NOT_COORDINATOR:
+                // If the coordinator is unavailable or there was a 
coordinator change, then we unmap
+                // the key so that we retry the `FindCoordinator` request
+                log.debug("DeleteShareGroupOffsets request for group id {} 
returned error {}. Will rediscover the coordinator and retry.",
+                    groupId.idValue, error);
+                groupsToUnmap.add(groupId);
+                break;
+            case GROUP_ID_NOT_FOUND:
+            case NON_EMPTY_GROUP:
+            case INVALID_REQUEST:
+            case UNKNOWN_SERVER_ERROR:
+            case KAFKA_STORAGE_ERROR:
+            case GROUP_AUTHORIZATION_FAILED:

Review Comment:
   You need to handle `TOPIC_AUTHORIZATION_FAILED` too.



##########
clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java:
##########
@@ -1399,6 +1399,11 @@ public synchronized ListShareGroupOffsetsResult 
listShareGroupOffsets(Map<String
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
+    @Override
+    public synchronized  DeleteShareGroupOffsetsResult 
deleteShareGroupOffsets(String groupId, Set<TopicPartition> partitions, 
DeleteShareGroupOffsetsOptions options) {

Review Comment:
   nit: Extra space.



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