AndrewJSchofield commented on code in PR #18976: URL: https://github.com/apache/kafka/pull/18976#discussion_r1981601768
########## clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java: ########## @@ -9439,4 +9443,106 @@ public void testAlterShareGroupOffsetsWithErrorInOnePartition() throws Exception assertNull(result.partitionResult(barPartition0).get()); } } + + @Test + public void testDeleteShareGroupOffsetsOptionsWithBatchedApi() throws Exception { + final Cluster cluster = mockCluster(3, 0); + final Time time = new MockTime(); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, + AdminClientConfig.RETRIES_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + final Set<TopicPartition> partitions = Collections.singleton(new TopicPartition("A", 0)); + final DeleteShareGroupOffsetsOptions options = new DeleteShareGroupOffsetsOptions(); + + env.adminClient().deleteShareGroupOffsets(GROUP_ID, partitions, options); + + final MockClient mockClient = env.kafkaClient(); + waitForRequest(mockClient, ApiKeys.DELETE_SHARE_GROUP_OFFSETS); + + ClientRequest clientRequest = mockClient.requests().peek(); + assertNotNull(clientRequest); + DeleteShareGroupOffsetsRequestData data = ((DeleteShareGroupOffsetsRequest.Builder) clientRequest.requestBuilder()).build().data(); + assertEquals(GROUP_ID, data.groupId()); + assertEquals(1, data.topics().size()); + assertEquals(Collections.singletonList("A"), + data.topics().stream().map(DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic::topicName).collect(Collectors.toList())); + } + } + + @Test + public void testDeleteShareGroupOffsets() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + DeleteShareGroupOffsetsResponseData data = new DeleteShareGroupOffsetsResponseData().setResponses( + List.of( + new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic().setTopicName("foo").setPartitions(List.of(new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition().setPartitionIndex(0), new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition().setPartitionIndex(1))), + new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic().setTopicName("bar").setPartitions(List.of(new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition().setPartitionIndex(0))) + ) + ); + + TopicPartition fooTopicPartition0 = new TopicPartition("foo", 0); + TopicPartition fooTopicPartition1 = new TopicPartition("foo", 1); + TopicPartition barPartition0 = new TopicPartition("bar", 0); + TopicPartition zooTopicPartition0 = new TopicPartition("zoo", 0); + + env.kafkaClient().prepareResponse(new DeleteShareGroupOffsetsResponse(data)); + final DeleteShareGroupOffsetsResult result = env.adminClient().deleteShareGroupOffsets(GROUP_ID, Set.of(fooTopicPartition0, fooTopicPartition1, barPartition0)); + + assertNull(result.all().get()); + assertNull(result.partitionResult(fooTopicPartition0).get()); + assertNull(result.partitionResult(fooTopicPartition1).get()); + assertNull(result.partitionResult(barPartition0).get()); + TestUtils.assertFutureThrows(IllegalArgumentException.class, result.partitionResult(zooTopicPartition0)); + } + } + + @Test + public void testDeleteShareGroupOffsetsEmpty() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + DeleteShareGroupOffsetsResponseData data = new DeleteShareGroupOffsetsResponseData().setResponses( + Collections.emptyList() + ); + env.kafkaClient().prepareResponse(new DeleteShareGroupOffsetsResponse(data)); + + final DeleteShareGroupOffsetsResult result = env.adminClient().deleteShareGroupOffsets(GROUP_ID, Collections.emptySet()); + assertDoesNotThrow(() -> result.all().get()); + } + } + + @Test Review Comment: Could we also have a test for a group-level error? ########## clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java: ########## @@ -0,0 +1,184 @@ +/* + * 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()); + 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, Errors> 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(); Review Comment: It's good that the partition error message is logged, but you could even get it back into the exception that the user sees in the KafkaFuture with a bit more work. You'd have to change to `Map<TopicPartition, Throwable>` or similar, but it's not too tricky to construct an exception from an `Errors` that has a specific message. ########## core/src/main/scala/kafka/server/KafkaApis.scala: ########## @@ -3474,8 +3477,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 { + responseData.responses.forEach { topic => { + deleteShareGroupOffsetsResponseTopics.add(topic) + }} + val deleteShareGroupStateResponse = new DeleteShareGroupOffsetsResponse(new DeleteShareGroupOffsetsResponseData() + .setResponses(deleteShareGroupOffsetsResponseTopics)) Review Comment: nit: Indentation on this line. ########## clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupOffsetsResult.java: ########## @@ -0,0 +1,88 @@ +/* + * 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<>(); Review Comment: I think it would be preferable to throw an exception directly for the situation where the supplied partition was not in the original request, rather than returning a failed future. I expect this would only occur if the application code had a bug and a simple exception is simpler to deal with because there's no need to unwrap the exception. -- 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