AndrewJSchofield commented on code in PR #18976: URL: https://github.com/apache/kafka/pull/18976#discussion_r1973409686
########## clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupOffsetsResult.java: ########## @@ -0,0 +1,97 @@ +/* + * 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.internals.KafkaFutureImpl; +import org.apache.kafka.common.protocol.Errors; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * 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, Errors>> future; + + DeleteShareGroupOffsetsResult(KafkaFuture<Map<TopicPartition, Errors>> future) { + this.future = future; + } + + /** + * Return a future which succeeds only if all the deletions succeed. + */ + public KafkaFuture<Void> all() { + return this.future.thenApply(topicPartitionErrorsMap -> { + List<TopicPartition> partitionsFailed = topicPartitionErrorsMap.entrySet() + .stream() + .filter(e -> e.getValue() != Errors.NONE) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + for (Errors error : topicPartitionErrorsMap.values()) { + if (error != Errors.NONE) { + throw error.exception( + "Failed deleting share group offsets for the following partitions: " + partitionsFailed); + } + } + return null; + }); + } + + /** + * Return a future which can be used to check the result for a given partition. + */ + public KafkaFuture<Void> partitionResult(final TopicPartition partition) { + final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>(); + + this.future.whenComplete((topicPartitions, throwable) -> { + if (throwable != null) { + result.completeExceptionally(throwable); + } else if (!topicPartitions.containsKey(partition)) { + result.completeExceptionally(new IllegalArgumentException( + "Delete offset for partition \"" + partition + "\" was not attempted")); + } else { + final Errors error = topicPartitions.get(partition); + if (error == Errors.NONE) { + result.complete(null); + } else { + result.completeExceptionally(error.exception()); + } + } + }); + + return result; + } + + /** Review Comment: I wonder why this method exists. It's not in the KIP (which could perhaps be an omission) and it's not in `DeleteConsumerGroupOffsetsResult`. ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -1206,6 +1210,93 @@ public CompletableFuture<DescribeShareGroupOffsetsResponseData.DescribeShareGrou 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)); + } + + 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)); + } + + DeleteShareGroupStateRequestData deleteShareGroupStateRequestData = new DeleteShareGroupStateRequestData() + .setGroupId(requestData.groupId()) + .setTopics(deleteShareGroupStateRequestTopicsData); + CompletableFuture<DeleteShareGroupOffsetsResponseData> future = new CompletableFuture<>(); + persister.deleteState(DeleteShareGroupStateParameters.from(deleteShareGroupStateRequestData)) + .whenComplete((result, error) -> { + if (error != null) { + log.error("Failed to delete share partitions"); Review Comment: I think "delete share partitions" would be better as "delete share group state". ########## 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: This is discarding the error message. ########## 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; + Review Comment: nit: Extra blank line -- 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