AndrewJSchofield commented on code in PR #18976: URL: https://github.com/apache/kafka/pull/18976#discussion_r2031381756
########## 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)); + } else if (groups == null || groups.isEmpty()) { + log.error("Describe share group resulted in empty response for group {}", groupId); + future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.GROUP_ID_NOT_FOUND)); + } else if (groups.get(0).errorCode() != Errors.NONE.code()) { + log.error("Failed to describe the share group {}", groupId); + future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.GROUP_ID_NOT_FOUND)); Review Comment: In this case, the list `groups` may contain an exception which includes the error code, and error message. Overwriting it with a plain `GROUP_ID_NOT_FOUND` with its default message will throw away useful information. For example, this is the case where we try to delete share group offsets for a group which is not a share group. ########## clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java: ########## @@ -0,0 +1,185 @@ +/* + * 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.errors.ApiException; +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, ApiException>> { + + 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, ApiException>> 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, ApiException>> handleResponse( + Node coordinator, + Set<CoordinatorKey> groupIds, + AbstractResponse abstractResponse + ) { + validateKeys(groupIds); + + final DeleteShareGroupOffsetsResponse response = (DeleteShareGroupOffsetsResponse) abstractResponse; + + final Errors groupError = Errors.forCode(response.data().errorCode()); + final String groupErrorMessage = response.data().errorMessage(); + + if (groupError != Errors.NONE) { + final Set<CoordinatorKey> groupsToUnmap = new HashSet<>(); + final Map<CoordinatorKey, Throwable> groupsFailed = new HashMap<>(); + handleGroupError(groupId, groupError, groupErrorMessage, groupsFailed, groupsToUnmap); + + return new ApiResult<>(Collections.emptyMap(), groupsFailed, new ArrayList<>(groupsToUnmap)); + } else { + final Map<TopicPartition, ApiException> partitionResults = new HashMap<>(); + response.data().responses().forEach(topic -> + topic.partitions().forEach(partition -> { + if (partition.errorCode() != Errors.NONE.code()) { + final Errors partitionError = Errors.forCode(partition.errorCode()); + final String partitionErrorMessage = partition.errorMessage(); + log.debug("DeleteShareGroupOffsets request for group id {}, topic {} and partition {} failed and returned error {}." + partitionErrorMessage, + groupId.idValue, topic.topicName(), partition.partitionIndex(), partitionError); + } + partitionResults.put( + new TopicPartition(topic.topicName(), partition.partitionIndex()), + Errors.forCode(partition.errorCode()).exception(partition.errorMessage()) + ); + }) + ); + + return ApiResult.completed(groupId, partitionResults); + } + } + + private void handleGroupError( + CoordinatorKey groupId, + Errors error, + String errorMessage, + 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. " + errorMessage, 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. " + errorMessage, + groupId.idValue, error); + groupsToUnmap.add(groupId); + break; + case INVALID_GROUP_ID: + case GROUP_ID_NOT_FOUND: + case NON_EMPTY_GROUP: + case INVALID_REQUEST: + case UNKNOWN_SERVER_ERROR: + case KAFKA_STORAGE_ERROR: + case GROUP_AUTHORIZATION_FAILED: + case TOPIC_AUTHORIZATION_FAILED: Review Comment: I know I asked you to handle `TOPIC_AUTHORIZATION_FAILED`. I was incorrect. It's not a group-level error code, so it should not be in this list. I think that just removing the `case TOPIC_AUTHORIZATION_FAILED` will be sufficient. ########## clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java: ########## @@ -0,0 +1,185 @@ +/* + * 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.errors.ApiException; +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, ApiException>> { + + 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, ApiException>> 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, ApiException>> handleResponse( + Node coordinator, + Set<CoordinatorKey> groupIds, + AbstractResponse abstractResponse + ) { + validateKeys(groupIds); + + final DeleteShareGroupOffsetsResponse response = (DeleteShareGroupOffsetsResponse) abstractResponse; + + final Errors groupError = Errors.forCode(response.data().errorCode()); + final String groupErrorMessage = response.data().errorMessage(); + + if (groupError != Errors.NONE) { + final Set<CoordinatorKey> groupsToUnmap = new HashSet<>(); + final Map<CoordinatorKey, Throwable> groupsFailed = new HashMap<>(); + handleGroupError(groupId, groupError, groupErrorMessage, groupsFailed, groupsToUnmap); + + return new ApiResult<>(Collections.emptyMap(), groupsFailed, new ArrayList<>(groupsToUnmap)); + } else { + final Map<TopicPartition, ApiException> partitionResults = new HashMap<>(); + response.data().responses().forEach(topic -> + topic.partitions().forEach(partition -> { + if (partition.errorCode() != Errors.NONE.code()) { + final Errors partitionError = Errors.forCode(partition.errorCode()); + final String partitionErrorMessage = partition.errorMessage(); + log.debug("DeleteShareGroupOffsets request for group id {}, topic {} and partition {} failed and returned error {}." + partitionErrorMessage, + groupId.idValue, topic.topicName(), partition.partitionIndex(), partitionError); + } + partitionResults.put( + new TopicPartition(topic.topicName(), partition.partitionIndex()), + Errors.forCode(partition.errorCode()).exception(partition.errorMessage()) + ); + }) + ); + + return ApiResult.completed(groupId, partitionResults); + } + } + + private void handleGroupError( + CoordinatorKey groupId, + Errors error, + String errorMessage, + 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. " + errorMessage, 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. " + errorMessage, + groupId.idValue, error); + groupsToUnmap.add(groupId); + break; + case INVALID_GROUP_ID: + case GROUP_ID_NOT_FOUND: + case NON_EMPTY_GROUP: + case INVALID_REQUEST: + case UNKNOWN_SERVER_ERROR: + case KAFKA_STORAGE_ERROR: + case GROUP_AUTHORIZATION_FAILED: + case TOPIC_AUTHORIZATION_FAILED: + log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. " + errorMessage, groupId.idValue, error); + failed.put(groupId, error.exception()); Review Comment: This should be `error.exception(errorMessage)`. ########## clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupOffsetsResult.java: ########## @@ -0,0 +1,80 @@ +/* + * 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 { Review Comment: I'd like this class to more strongly follow `DeleteConsumerGroupOffsetsResult`. ########## core/src/main/scala/kafka/server/KafkaApis.scala: ########## @@ -3623,8 +3626,59 @@ class KafkaApis(val requestChannel: RequestChannel, def handleDeleteShareGroupOffsetsRequest(request: RequestChannel.Request): Unit = { val deleteShareGroupOffsetsRequest = request.body[DeleteShareGroupOffsetsRequest] - requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) - CompletableFuture.completedFuture[Unit](()) + + val groupId = deleteShareGroupOffsetsRequest.data.groupId + + if (!isShareGroupProtocolEnabled) { + requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.UNSUPPORTED_VERSION.exception)) + return + } else if (!authHelper.authorize(request.context, DELETE, GROUP, groupId)) { + requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.GROUP_AUTHORIZATION_FAILED.exception)) + return + } + + val deleteShareGroupOffsetsResponseTopics: util.List[DeleteShareGroupOffsetsResponseTopic] = new util.ArrayList[DeleteShareGroupOffsetsResponseTopic]() + + val authorizedTopics: util.List[DeleteShareGroupOffsetsRequestTopic] = + new util.ArrayList[DeleteShareGroupOffsetsRequestTopic] + + deleteShareGroupOffsetsRequest.data.topics.forEach{ topic => + if (!authHelper.authorize(request.context, READ, TOPIC, topic.topicName)) { + deleteShareGroupOffsetsResponseTopics.add( + new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic() + .setTopicName(topic.topicName) + .setPartitions(topic.partitions.map(partition => { + new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition() + .setPartitionIndex(partition) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message()) + }).toList.asJava) + ) + } else { + authorizedTopics.add(topic) + } + } + + if (authorizedTopics.isEmpty) { + requestHelper.sendMaybeThrottle(request, new DeleteShareGroupOffsetsResponse(new DeleteShareGroupOffsetsResponseData())) + return + } + + groupCoordinator.deleteShareGroupOffsets( + request.context, + new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics) + ).handle[Unit] {(responseData, exception) => { + if (exception != null) { + requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception)) + } else { Review Comment: It is possible that there's a request-level error in here. So I think you need something like ``` } else if (responseData.errorCode() != Errors.NONE.code) { val deleteShareGroupOffsetsResponse = new DeleteShareGroupOffsetsResponse(responseData) requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsResponse) ``` and then of course, you'll need to build the error response containing the error code and error message at the top level. ########## clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java: ########## @@ -0,0 +1,185 @@ +/* + * 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.errors.ApiException; +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, ApiException>> { + + 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, ApiException>> 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, ApiException>> handleResponse( + Node coordinator, + Set<CoordinatorKey> groupIds, + AbstractResponse abstractResponse + ) { + validateKeys(groupIds); + + final DeleteShareGroupOffsetsResponse response = (DeleteShareGroupOffsetsResponse) abstractResponse; + + final Errors groupError = Errors.forCode(response.data().errorCode()); + final String groupErrorMessage = response.data().errorMessage(); + + if (groupError != Errors.NONE) { + final Set<CoordinatorKey> groupsToUnmap = new HashSet<>(); + final Map<CoordinatorKey, Throwable> groupsFailed = new HashMap<>(); + handleGroupError(groupId, groupError, groupErrorMessage, groupsFailed, groupsToUnmap); + + return new ApiResult<>(Collections.emptyMap(), groupsFailed, new ArrayList<>(groupsToUnmap)); + } else { + final Map<TopicPartition, ApiException> partitionResults = new HashMap<>(); + response.data().responses().forEach(topic -> + topic.partitions().forEach(partition -> { + if (partition.errorCode() != Errors.NONE.code()) { + final Errors partitionError = Errors.forCode(partition.errorCode()); + final String partitionErrorMessage = partition.errorMessage(); + log.debug("DeleteShareGroupOffsets request for group id {}, topic {} and partition {} failed and returned error {}." + partitionErrorMessage, + groupId.idValue, topic.topicName(), partition.partitionIndex(), partitionError); + } + partitionResults.put( + new TopicPartition(topic.topicName(), partition.partitionIndex()), + Errors.forCode(partition.errorCode()).exception(partition.errorMessage()) + ); + }) + ); + + return ApiResult.completed(groupId, partitionResults); + } + } + + private void handleGroupError( + CoordinatorKey groupId, + Errors error, + String errorMessage, + 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. " + errorMessage, 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. " + errorMessage, + groupId.idValue, error); + groupsToUnmap.add(groupId); + break; + case INVALID_GROUP_ID: + case GROUP_ID_NOT_FOUND: + case NON_EMPTY_GROUP: + case INVALID_REQUEST: + case UNKNOWN_SERVER_ERROR: + case KAFKA_STORAGE_ERROR: + case GROUP_AUTHORIZATION_FAILED: + case TOPIC_AUTHORIZATION_FAILED: + log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. " + errorMessage, groupId.idValue, error); + failed.put(groupId, error.exception()); + break; + default: + log.error("DeleteShareGroupOffsets request for group id {} failed due to unexpected error {}. " + errorMessage, groupId.idValue, error); + failed.put(groupId, error.exception()); Review Comment: This should be `error.exception(errorMessage)`. -- 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