From f5d5f654db359af077088685e29fbe5ea69616cf Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Thu, 1 Jul 2021 22:05:03 +0100 Subject: [PATCH] KAFKA-12663: Update FindCoordinator to support batch lookups (KIP-699) (#10743) This implements KIP-699: https://cwiki.apache.org/confluence/display/KAFKA/KIP-699%3A+Update+FindCoordinator+to+resolve+multiple+Coordinators+at+a+time It updates FindCoordinator request and response to support resolving multiple coordinators at a time. If a broker does not support the new FindCoordinator version, clients can revert to the previous behaviour and use a request for each coordinator. Reviewers: David Jacot , Tom Bentley , Sanjana Kaundinya --- .../admin/ConsumerGroupDescription.java | 14 +- .../admin/DeleteConsumerGroupsResult.java | 11 +- .../admin/DescribeConsumerGroupsResult.java | 26 +- .../kafka/clients/admin/KafkaAdminClient.java | 607 ++---------------- .../internals/AbortTransactionHandler.java | 3 +- .../admin/internals/AdminApiDriver.java | 28 +- .../admin/internals/AdminApiFuture.java | 4 + .../admin/internals/AdminApiHandler.java | 5 +- .../AlterConsumerGroupOffsetsHandler.java | 157 +++++ .../ConsumerGroupOperationContext.java | 91 --- .../admin/internals/CoordinatorKey.java | 6 +- .../admin/internals/CoordinatorStrategy.java | 130 +++- .../DeleteConsumerGroupOffsetsHandler.java | 151 +++++ .../DeleteConsumerGroupsHandler.java | 134 ++++ .../DescribeConsumerGroupsHandler.java | 202 ++++++ .../internals/DescribeProducersHandler.java | 3 +- .../DescribeTransactionsHandler.java | 8 +- .../ListConsumerGroupOffsetsHandler.java | 148 +++++ .../internals/ListTransactionsHandler.java | 4 +- ...RemoveMembersFromConsumerGroupHandler.java | 133 ++++ .../internals/AbstractCoordinator.java | 57 +- .../internals/TransactionManager.java | 51 +- .../requests/FindCoordinatorRequest.java | 26 +- .../requests/FindCoordinatorResponse.java | 36 +- .../message/FindCoordinatorRequest.json | 10 +- .../message/FindCoordinatorResponse.json | 27 +- .../clients/admin/KafkaAdminClientTest.java | 388 +++++++---- .../AbortTransactionHandlerTest.java | 26 +- .../admin/internals/AdminApiDriverTest.java | 57 +- .../AllBrokersStrategyIntegrationTest.java | 19 +- .../AlterConsumerGroupOffsetsHandlerTest.java | 134 ++++ .../internals/CoordinatorStrategyTest.java | 181 +++++- ...DeleteConsumerGroupOffsetsHandlerTest.java | 152 +++++ .../DeleteConsumerGroupsHandlerTest.java | 128 ++++ .../DescribeConsumerGroupsHandlerTest.java | 192 ++++++ .../DescribeProducersHandlerTest.java | 10 +- .../DescribeTransactionsHandlerTest.java | 13 +- .../ListConsumerGroupOffsetsHandlerTest.java | 139 ++++ .../ListTransactionsHandlerTest.java | 6 +- ...veMembersFromConsumerGroupHandlerTest.java | 143 +++++ .../clients/consumer/KafkaConsumerTest.java | 30 +- .../internals/AbstractCoordinatorTest.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 2 +- .../clients/producer/KafkaProducerTest.java | 22 +- .../producer/internals/SenderTest.java | 12 +- .../internals/TransactionManagerTest.java | 9 +- .../common/requests/RequestResponseTest.java | 2 +- .../distributed/WorkerCoordinatorTest.java | 8 +- .../main/scala/kafka/server/KafkaApis.scala | 112 ++-- .../kafka/api/AuthorizerIntegrationTest.scala | 8 +- .../kafka/api/ConsumerBounceTest.scala | 2 +- .../admin/ConsumerGroupServiceTest.scala | 10 +- .../unit/kafka/server/KafkaApisTest.scala | 52 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- 54 files changed, 2874 insertions(+), 1059 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/ConsumerGroupOperationContext.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandler.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupDescription.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupDescription.java index 32bd1658174..3ae97549c2c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupDescription.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupDescription.java @@ -49,13 +49,13 @@ public class ConsumerGroupDescription { this(groupId, isSimpleConsumerGroup, members, partitionAssignor, state, coordinator, Collections.emptySet()); } - ConsumerGroupDescription(String groupId, - boolean isSimpleConsumerGroup, - Collection members, - String partitionAssignor, - ConsumerGroupState state, - Node coordinator, - Set authorizedOperations) { + public ConsumerGroupDescription(String groupId, + boolean isSimpleConsumerGroup, + Collection members, + String partitionAssignor, + ConsumerGroupState state, + Node coordinator, + Set authorizedOperations) { this.groupId = groupId == null ? "" : groupId; this.isSimpleConsumerGroup = isSimpleConsumerGroup; this.members = members == null ? Collections.emptyList() : diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupsResult.java index c7d7a5ab672..bf4c45c1b6d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupsResult.java @@ -16,10 +16,13 @@ */ package org.apache.kafka.clients.admin; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; import java.util.Collection; +import java.util.HashMap; import java.util.Map; /** @@ -29,9 +32,9 @@ import java.util.Map; */ @InterfaceStability.Evolving public class DeleteConsumerGroupsResult { - private final Map> futures; + private final Map> futures; - DeleteConsumerGroupsResult(final Map> futures) { + DeleteConsumerGroupsResult(final Map> futures) { this.futures = futures; } @@ -40,7 +43,9 @@ public class DeleteConsumerGroupsResult { * individual deletions. */ public Map> deletedGroups() { - return futures; + Map> deletedGroups = new HashMap<>(futures.size()); + futures.forEach((key, future) -> deletedGroups.put(key.idValue, future)); + return deletedGroups; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeConsumerGroupsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeConsumerGroupsResult.java index 2eddbba305d..a6881c449b8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeConsumerGroupsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeConsumerGroupsResult.java @@ -17,10 +17,11 @@ package org.apache.kafka.clients.admin; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; -import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -34,9 +35,9 @@ import java.util.concurrent.ExecutionException; @InterfaceStability.Evolving public class DescribeConsumerGroupsResult { - private final Map> futures; + private final Map> futures; - public DescribeConsumerGroupsResult(final Map> futures) { + public DescribeConsumerGroupsResult(final Map> futures) { this.futures = futures; } @@ -44,7 +45,9 @@ public class DescribeConsumerGroupsResult { * Return a map from group id to futures which yield group descriptions. */ public Map> describedGroups() { - return futures; + Map> describedGroups = new HashMap<>(); + futures.forEach((key, future) -> describedGroups.put(key.idValue, future)); + return describedGroups; } /** @@ -52,21 +55,18 @@ public class DescribeConsumerGroupsResult { */ public KafkaFuture> all() { return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply( - new KafkaFuture.BaseFunction>() { - @Override - public Map apply(Void v) { + nil -> { + Map descriptions = new HashMap<>(futures.size()); + futures.forEach((key, future) -> { try { - Map descriptions = new HashMap<>(futures.size()); - for (Map.Entry> entry : futures.entrySet()) { - descriptions.put(entry.getKey(), entry.getValue().get()); - } - return descriptions; + descriptions.put(key.idValue, future.get()); } catch (InterruptedException | ExecutionException e) { // This should be unreachable, since the KafkaFuture#allOf already ensured // that all of the futures completed successfully. throw new RuntimeException(e); } - } + }); + return descriptions; }); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index cf83a5272fb..09165ff5eec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -35,15 +35,20 @@ import org.apache.kafka.clients.admin.internals.AbortTransactionHandler; import org.apache.kafka.clients.admin.internals.AdminApiDriver; import org.apache.kafka.clients.admin.internals.AdminApiHandler; import org.apache.kafka.clients.admin.internals.AdminApiFuture; +import org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture; import org.apache.kafka.clients.admin.internals.AdminMetadataManager; import org.apache.kafka.clients.admin.internals.AllBrokersStrategy; -import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; +import org.apache.kafka.clients.admin.internals.AlterConsumerGroupOffsetsHandler; import org.apache.kafka.clients.admin.internals.CoordinatorKey; +import org.apache.kafka.clients.admin.internals.DeleteConsumerGroupOffsetsHandler; +import org.apache.kafka.clients.admin.internals.DeleteConsumerGroupsHandler; +import org.apache.kafka.clients.admin.internals.DescribeConsumerGroupsHandler; import org.apache.kafka.clients.admin.internals.DescribeProducersHandler; import org.apache.kafka.clients.admin.internals.DescribeTransactionsHandler; +import org.apache.kafka.clients.admin.internals.ListConsumerGroupOffsetsHandler; import org.apache.kafka.clients.admin.internals.ListTransactionsHandler; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; +import org.apache.kafka.clients.admin.internals.RemoveMembersFromConsumerGroupHandler; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.Cluster; @@ -71,7 +76,6 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.DisconnectException; -import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.KafkaStorageException; @@ -114,7 +118,6 @@ import org.apache.kafka.common.message.DeleteAclsRequestData.DeleteAclsFilter; import org.apache.kafka.common.message.DeleteAclsResponseData; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl; -import org.apache.kafka.common.message.DeleteGroupsRequestData; import org.apache.kafka.common.message.DeleteRecordsRequestData; import org.apache.kafka.common.message.DeleteRecordsRequestData.DeleteRecordsPartition; import org.apache.kafka.common.message.DeleteRecordsRequestData.DeleteRecordsTopic; @@ -126,9 +129,6 @@ import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicRe import org.apache.kafka.common.message.DescribeClusterRequestData; import org.apache.kafka.common.message.DescribeConfigsRequestData; import org.apache.kafka.common.message.DescribeConfigsResponseData; -import org.apache.kafka.common.message.DescribeGroupsRequestData; -import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; -import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.message.DescribeLogDirsRequestData; import org.apache.kafka.common.message.DescribeLogDirsRequestData.DescribableLogDirTopic; import org.apache.kafka.common.message.DescribeLogDirsResponseData; @@ -136,9 +136,7 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.ExpireDelegationTokenRequestData; -import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; -import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; @@ -147,15 +145,6 @@ import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartit import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.MetadataRequestData; -import org.apache.kafka.common.message.OffsetCommitRequestData; -import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition; -import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic; -import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; -import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic; -import org.apache.kafka.common.message.OffsetDeleteRequestData; -import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestPartition; -import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopic; -import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.message.UnregisterBrokerRequestData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; @@ -198,8 +187,6 @@ import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; import org.apache.kafka.common.requests.DeleteAclsRequest; import org.apache.kafka.common.requests.DeleteAclsResponse; -import org.apache.kafka.common.requests.DeleteGroupsRequest; -import org.apache.kafka.common.requests.DeleteGroupsResponse; import org.apache.kafka.common.requests.DeleteRecordsRequest; import org.apache.kafka.common.requests.DeleteRecordsResponse; import org.apache.kafka.common.requests.DeleteTopicsRequest; @@ -214,8 +201,6 @@ import org.apache.kafka.common.requests.DescribeConfigsRequest; import org.apache.kafka.common.requests.DescribeConfigsResponse; import org.apache.kafka.common.requests.DescribeDelegationTokenRequest; import org.apache.kafka.common.requests.DescribeDelegationTokenResponse; -import org.apache.kafka.common.requests.DescribeGroupsRequest; -import org.apache.kafka.common.requests.DescribeGroupsResponse; import org.apache.kafka.common.requests.DescribeLogDirsRequest; import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.apache.kafka.common.requests.DescribeUserScramCredentialsRequest; @@ -224,13 +209,8 @@ import org.apache.kafka.common.requests.ElectLeadersRequest; import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.ExpireDelegationTokenRequest; import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; -import org.apache.kafka.common.requests.FindCoordinatorRequest; -import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; -import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsRequest; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; -import org.apache.kafka.common.requests.LeaveGroupRequest; -import org.apache.kafka.common.requests.LeaveGroupResponse; import org.apache.kafka.common.requests.ListGroupsRequest; import org.apache.kafka.common.requests.ListGroupsResponse; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -239,12 +219,6 @@ import org.apache.kafka.common.requests.ListPartitionReassignmentsRequest; import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.requests.OffsetCommitRequest; -import org.apache.kafka.common.requests.OffsetCommitResponse; -import org.apache.kafka.common.requests.OffsetDeleteRequest; -import org.apache.kafka.common.requests.OffsetDeleteResponse; -import org.apache.kafka.common.requests.OffsetFetchRequest; -import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.RenewDelegationTokenRequest; import org.apache.kafka.common.requests.RenewDelegationTokenResponse; import org.apache.kafka.common.requests.UnregisterBrokerRequest; @@ -263,7 +237,6 @@ import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.security.InvalidKeyException; import java.security.NoSuchAlgorithmException; import java.time.Duration; @@ -1085,7 +1058,6 @@ public class KafkaAdminClient extends AdminClient { Iterator pendingIter = pendingCalls.iterator(); while (pendingIter.hasNext()) { Call call = pendingIter.next(); - // If the call is being retried, await the proper backoff before finding the node if (now < call.nextAllowedTryMs) { pollTimeout = Math.min(pollTimeout, call.nextAllowedTryMs - now); @@ -1525,10 +1497,6 @@ public class KafkaAdminClient extends AdminClient { return topicName == null || topicName.isEmpty(); } - private static boolean groupIdIsUnrepresentable(String groupId) { - return groupId == null; - } - // for testing int numPendingCalls() { return runnable.pendingCalls.size(); @@ -3110,20 +3078,6 @@ public class KafkaAdminClient extends AdminClient { return new DescribeDelegationTokenResult(tokensFuture); } - private void rescheduleFindCoordinatorTask(ConsumerGroupOperationContext context, Supplier nextCall, Call failedCall) { - log.info("Node {} is no longer the Coordinator. Retrying with new coordinator.", - context.node().orElse(null)); - // Requeue the task so that we can try with new coordinator - context.setNode(null); - - Call call = nextCall.get(); - call.tries = failedCall.tries + 1; - call.nextAllowedTryMs = calculateNextAllowedRetryMs(); - - Call findCoordinatorCall = getFindCoordinatorCall(context, nextCall); - runnable.call(findCoordinatorCall, time.milliseconds()); - } - private void rescheduleMetadataTask(MetadataOperationContext context, Supplier> nextCalls) { log.info("Retrying to fetch metadata."); // Requeue the task so that we can re-attempt fetching metadata @@ -3132,173 +3086,14 @@ public class KafkaAdminClient extends AdminClient { runnable.call(metadataCall, time.milliseconds()); } - private static Map> createFutures(Collection groupIds) { - return new HashSet<>(groupIds).stream().collect( - Collectors.toMap(groupId -> groupId, - groupId -> { - if (groupIdIsUnrepresentable(groupId)) { - KafkaFutureImpl future = new KafkaFutureImpl<>(); - future.completeExceptionally(new InvalidGroupIdException("The given group id '" + - groupId + "' cannot be represented in a request.")); - return future; - } else { - return new KafkaFutureImpl<>(); - } - } - )); - } - @Override public DescribeConsumerGroupsResult describeConsumerGroups(final Collection groupIds, final DescribeConsumerGroupsOptions options) { - - final Map> futures = createFutures(groupIds); - - // TODO: KAFKA-6788, we should consider grouping the request per coordinator and send one request with a list of - // all consumer groups this coordinator host - for (final Map.Entry> entry : futures.entrySet()) { - // skip sending request for those futures that already failed. - if (entry.getValue().isCompletedExceptionally()) - continue; - - final String groupId = entry.getKey(); - - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - ConsumerGroupOperationContext context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, futures.get(groupId)); - Call findCoordinatorCall = getFindCoordinatorCall(context, - () -> getDescribeConsumerGroupsCall(context)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - } - - return new DescribeConsumerGroupsResult(new HashMap<>(futures)); - } - - /** - * Returns a {@code Call} object to fetch the coordinator for a consumer group id. Takes another Call - * parameter to schedule action that need to be taken using the coordinator. The param is a Supplier - * so that it can be lazily created, so that it can use the results of find coordinator call in its - * construction. - * - * @param The type of return value of the KafkaFuture, like ConsumerGroupDescription, Void etc. - * @param The type of configuration option, like DescribeConsumerGroupsOptions, ListConsumerGroupsOptions etc - */ - private > Call getFindCoordinatorCall(ConsumerGroupOperationContext context, - Supplier nextCall) { - return new Call("findCoordinator", context.deadline(), new LeastLoadedNodeProvider()) { - @Override - FindCoordinatorRequest.Builder createRequest(int timeoutMs) { - return new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(CoordinatorType.GROUP.id()) - .setKey(context.groupId())); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; - - if (handleGroupRequestError(response.error(), context.future())) - return; - - context.setNode(response.node()); - - runnable.call(nextCall.get(), time.milliseconds()); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; - } - - private Call getDescribeConsumerGroupsCall( - ConsumerGroupOperationContext context) { - return new Call("describeConsumerGroups", - context.deadline(), - new ConstantNodeIdProvider(context.node().get().id())) { - @Override - DescribeGroupsRequest.Builder createRequest(int timeoutMs) { - return new DescribeGroupsRequest.Builder( - new DescribeGroupsRequestData() - .setGroups(Collections.singletonList(context.groupId())) - .setIncludeAuthorizedOperations(context.options().includeAuthorizedOperations())); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final DescribeGroupsResponse response = (DescribeGroupsResponse) abstractResponse; - - List describedGroups = response.data().groups(); - if (describedGroups.isEmpty()) { - context.future().completeExceptionally( - new InvalidGroupIdException("No consumer group found for GroupId: " + context.groupId())); - return; - } - - if (describedGroups.size() > 1 || - !describedGroups.get(0).groupId().equals(context.groupId())) { - String ids = Arrays.toString(describedGroups.stream().map(DescribedGroup::groupId).toArray()); - context.future().completeExceptionally(new InvalidGroupIdException( - "DescribeConsumerGroup request for GroupId: " + context.groupId() + " returned " + ids)); - return; - } - - final DescribedGroup describedGroup = describedGroups.get(0); - - // If coordinator changed since we fetched it, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) { - Call call = getDescribeConsumerGroupsCall(context); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - final Errors groupError = Errors.forCode(describedGroup.errorCode()); - if (handleGroupRequestError(groupError, context.future())) - return; - - final String protocolType = describedGroup.protocolType(); - if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) { - final List members = describedGroup.members(); - final List memberDescriptions = new ArrayList<>(members.size()); - final Set authorizedOperations = validAclOperations(describedGroup.authorizedOperations()); - for (DescribedGroupMember groupMember : members) { - Set partitions = Collections.emptySet(); - if (groupMember.memberAssignment().length > 0) { - final Assignment assignment = ConsumerProtocol. - deserializeAssignment(ByteBuffer.wrap(groupMember.memberAssignment())); - partitions = new HashSet<>(assignment.partitions()); - } - final MemberDescription memberDescription = new MemberDescription( - groupMember.memberId(), - Optional.ofNullable(groupMember.groupInstanceId()), - groupMember.clientId(), - groupMember.clientHost(), - new MemberAssignment(partitions)); - memberDescriptions.add(memberDescription); - } - final ConsumerGroupDescription consumerGroupDescription = - new ConsumerGroupDescription(context.groupId(), protocolType.isEmpty(), - memberDescriptions, - describedGroup.protocolData(), - ConsumerGroupState.parse(describedGroup.groupState()), - context.node().get(), - authorizedOperations); - context.future().complete(consumerGroupDescription); - } else { - context.future().completeExceptionally(new IllegalArgumentException( - String.format("GroupId %s is not a consumer group (%s).", - context.groupId(), protocolType))); - } - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture future = + DescribeConsumerGroupsHandler.newFuture(groupIds); + DescribeConsumerGroupsHandler handler = new DescribeConsumerGroupsHandler(options.includeAuthorizedOperations(), logContext); + invokeDriver(handler, future, options.timeoutMs); + return new DescribeConsumerGroupsResult(future.all()); } /** @@ -3354,16 +3149,6 @@ public class KafkaAdminClient extends AdminClient { .collect(Collectors.toSet()); } - private boolean handleGroupRequestError(Errors error, KafkaFutureImpl future) { - if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE) { - throw error.exception(); - } else if (error != Errors.NONE) { - future.completeExceptionally(error.exception()); - return true; - } - return false; - } - private final static class ListConsumerGroupsResults { private final List errors; private final HashMap listings; @@ -3495,135 +3280,20 @@ public class KafkaAdminClient extends AdminClient { @Override public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, final ListConsumerGroupOffsetsOptions options) { - final KafkaFutureImpl> groupOffsetListingFuture = new KafkaFutureImpl<>(); - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - - ConsumerGroupOperationContext, ListConsumerGroupOffsetsOptions> context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, groupOffsetListingFuture); - - Call findCoordinatorCall = getFindCoordinatorCall(context, - () -> getListConsumerGroupOffsetsCall(context)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - - return new ListConsumerGroupOffsetsResult(groupOffsetListingFuture); - } - - private Call getListConsumerGroupOffsetsCall(ConsumerGroupOperationContext, - ListConsumerGroupOffsetsOptions> context) { - return new Call("listConsumerGroupOffsets", context.deadline(), - new ConstantNodeIdProvider(context.node().get().id())) { - @Override - OffsetFetchRequest.Builder createRequest(int timeoutMs) { - // Set the flag to false as for admin client request, - // we don't need to wait for any pending offset state to clear. - return new OffsetFetchRequest.Builder(context.groupId(), false, context.options().topicPartitions(), false); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; - final Map groupOffsetsListing = new HashMap<>(); - - // If coordinator changed since we fetched it, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) { - Call call = getListConsumerGroupOffsetsCall(context); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - if (handleGroupRequestError(response.error(), context.future())) - return; - - for (Map.Entry entry : - response.responseData().entrySet()) { - final TopicPartition topicPartition = entry.getKey(); - OffsetFetchResponse.PartitionData partitionData = entry.getValue(); - final Errors error = partitionData.error; - - if (error == Errors.NONE) { - final Long offset = partitionData.offset; - final String metadata = partitionData.metadata; - final Optional leaderEpoch = partitionData.leaderEpoch; - // Negative offset indicates that the group has no committed offset for this partition - if (offset < 0) { - groupOffsetsListing.put(topicPartition, null); - } else { - groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); - } - } else { - log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); - } - } - context.future().complete(groupOffsetsListing); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture> future = + ListConsumerGroupOffsetsHandler.newFuture(groupId); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), logContext); + invokeDriver(handler, future, options.timeoutMs); + return new ListConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId))); } @Override public DeleteConsumerGroupsResult deleteConsumerGroups(Collection groupIds, DeleteConsumerGroupsOptions options) { - - final Map> futures = createFutures(groupIds); - - // TODO: KAFKA-6788, we should consider grouping the request per coordinator and send one request with a list of - // all consumer groups this coordinator host - for (final String groupId : groupIds) { - // skip sending request for those futures that already failed. - final KafkaFutureImpl future = futures.get(groupId); - if (future.isCompletedExceptionally()) - continue; - - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - ConsumerGroupOperationContext context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, future); - Call findCoordinatorCall = getFindCoordinatorCall(context, - () -> getDeleteConsumerGroupsCall(context)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - } - - return new DeleteConsumerGroupsResult(new HashMap<>(futures)); - } - - private Call getDeleteConsumerGroupsCall(ConsumerGroupOperationContext context) { - return new Call("deleteConsumerGroups", context.deadline(), new ConstantNodeIdProvider(context.node().get().id())) { - - @Override - DeleteGroupsRequest.Builder createRequest(int timeoutMs) { - return new DeleteGroupsRequest.Builder( - new DeleteGroupsRequestData() - .setGroupsNames(Collections.singletonList(context.groupId())) - ); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final DeleteGroupsResponse response = (DeleteGroupsResponse) abstractResponse; - - // If coordinator changed since we fetched it, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) { - Call call = getDeleteConsumerGroupsCall(context); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - final Errors groupError = response.get(context.groupId()); - if (handleGroupRequestError(groupError, context.future())) - return; - - context.future().complete(null); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture future = + DeleteConsumerGroupsHandler.newFuture(groupIds); + DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(logContext); + invokeDriver(handler, future, options.timeoutMs); + return new DeleteConsumerGroupsResult(future.all()); } @Override @@ -3631,83 +3301,11 @@ public class KafkaAdminClient extends AdminClient { String groupId, Set partitions, DeleteConsumerGroupOffsetsOptions options) { - final KafkaFutureImpl> future = new KafkaFutureImpl<>(); - - if (groupIdIsUnrepresentable(groupId)) { - future.completeExceptionally(new InvalidGroupIdException("The given group id '" + - groupId + "' cannot be represented in a request.")); - return new DeleteConsumerGroupOffsetsResult(future, partitions); - } - - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - ConsumerGroupOperationContext, DeleteConsumerGroupOffsetsOptions> context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, future); - - Call findCoordinatorCall = getFindCoordinatorCall(context, - () -> getDeleteConsumerGroupOffsetsCall(context, partitions)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - - return new DeleteConsumerGroupOffsetsResult(future, partitions); - } - - private Call getDeleteConsumerGroupOffsetsCall( - ConsumerGroupOperationContext, DeleteConsumerGroupOffsetsOptions> context, - Set partitions) { - return new Call("deleteConsumerGroupOffsets", context.deadline(), new ConstantNodeIdProvider(context.node().get().id())) { - - @Override - OffsetDeleteRequest.Builder createRequest(int timeoutMs) { - final OffsetDeleteRequestTopicCollection topics = new OffsetDeleteRequestTopicCollection(); - - partitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).forEach((topic, topicPartitions) -> { - topics.add( - new OffsetDeleteRequestTopic() - .setName(topic) - .setPartitions(topicPartitions.stream() - .map(tp -> new OffsetDeleteRequestPartition().setPartitionIndex(tp.partition())) - .collect(Collectors.toList()) - ) - ); - }); - - return new OffsetDeleteRequest.Builder( - new OffsetDeleteRequestData() - .setGroupId(context.groupId()) - .setTopics(topics) - ); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final OffsetDeleteResponse response = (OffsetDeleteResponse) abstractResponse; - - // If coordinator changed since we fetched it, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) { - Call call = getDeleteConsumerGroupOffsetsCall(context, partitions); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - // If the error is an error at the group level, the future is failed with it - final Errors groupError = Errors.forCode(response.data().errorCode()); - if (handleGroupRequestError(groupError, context.future())) - return; - - final Map partitions = new HashMap<>(); - response.data().topics().forEach(topic -> topic.partitions().forEach(partition -> partitions.put( - new TopicPartition(topic.name(), partition.partitionIndex()), - Errors.forCode(partition.errorCode()))) - ); - - context.future().complete(partitions); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture> future = + DeleteConsumerGroupOffsetsHandler.newFuture(groupId); + DeleteConsumerGroupOffsetsHandler handler = new DeleteConsumerGroupOffsetsHandler(groupId, partitions, logContext); + invokeDriver(handler, future, options.timeoutMs); + return new DeleteConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId)), partitions); } @Override @@ -3983,10 +3581,6 @@ public class KafkaAdminClient extends AdminClient { return new ListPartitionReassignmentsResult(partitionReassignmentsFuture); } - private long calculateNextAllowedRetryMs() { - return time.milliseconds() + retryBackoffMs; - } - private void handleNotControllerError(AbstractResponse response) throws ApiException { if (response.errorCounts().containsKey(Errors.NOT_CONTROLLER)) { handleNotControllerError(Errors.NOT_CONTROLLER); @@ -4034,152 +3628,28 @@ public class KafkaAdminClient extends AdminClient { @Override public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId, RemoveMembersFromConsumerGroupOptions options) { - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - - KafkaFutureImpl> future = new KafkaFutureImpl<>(); - - ConsumerGroupOperationContext, RemoveMembersFromConsumerGroupOptions> context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, future); - List members; if (options.removeAll()) { members = getMembersFromGroup(groupId); } else { members = options.members().stream().map(MemberToRemove::toMemberIdentity).collect(Collectors.toList()); } - Call findCoordinatorCall = getFindCoordinatorCall(context, () -> getRemoveMembersFromGroupCall(context, members)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - - return new RemoveMembersFromConsumerGroupResult(future, options.members()); - } - - private Call getRemoveMembersFromGroupCall(ConsumerGroupOperationContext, RemoveMembersFromConsumerGroupOptions> context, - List members) { - return new Call("leaveGroup", context.deadline(), new ConstantNodeIdProvider(context.node().get().id())) { - @Override - LeaveGroupRequest.Builder createRequest(int timeoutMs) { - return new LeaveGroupRequest.Builder(context.groupId(), members); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final LeaveGroupResponse response = (LeaveGroupResponse) abstractResponse; - - // If coordinator changed since we fetched it, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) { - Call call = getRemoveMembersFromGroupCall(context, members); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - if (handleGroupRequestError(response.topLevelError(), context.future())) - return; - - final Map memberErrors = new HashMap<>(); - for (MemberResponse memberResponse : response.memberResponses()) { - memberErrors.put(new MemberIdentity() - .setMemberId(memberResponse.memberId()) - .setGroupInstanceId(memberResponse.groupInstanceId()), - Errors.forCode(memberResponse.errorCode())); - } - context.future().complete(memberErrors); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture> future = + RemoveMembersFromConsumerGroupHandler.newFuture(groupId); + RemoveMembersFromConsumerGroupHandler handler = new RemoveMembersFromConsumerGroupHandler(groupId, members, logContext); + invokeDriver(handler, future, options.timeoutMs); + return new RemoveMembersFromConsumerGroupResult(future.get(CoordinatorKey.byGroupId(groupId)), options.members()); } @Override public AlterConsumerGroupOffsetsResult alterConsumerGroupOffsets(String groupId, Map offsets, AlterConsumerGroupOffsetsOptions options) { - final KafkaFutureImpl> future = new KafkaFutureImpl<>(); - - final long startFindCoordinatorMs = time.milliseconds(); - final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); - - ConsumerGroupOperationContext, AlterConsumerGroupOffsetsOptions> context = - new ConsumerGroupOperationContext<>(groupId, options, deadline, future); - - Call findCoordinatorCall = getFindCoordinatorCall(context, - () -> KafkaAdminClient.this.getAlterConsumerGroupOffsetsCall(context, offsets)); - runnable.call(findCoordinatorCall, startFindCoordinatorMs); - - return new AlterConsumerGroupOffsetsResult(future); - } - - private Call getAlterConsumerGroupOffsetsCall(ConsumerGroupOperationContext, - AlterConsumerGroupOffsetsOptions> context, - Map offsets) { - - return new Call("commitOffsets", context.deadline(), new ConstantNodeIdProvider(context.node().get().id())) { - - @Override - OffsetCommitRequest.Builder createRequest(int timeoutMs) { - List topics = new ArrayList<>(); - Map> offsetData = new HashMap<>(); - for (Map.Entry entry : offsets.entrySet()) { - String topic = entry.getKey().topic(); - OffsetAndMetadata oam = entry.getValue(); - offsetData.compute(topic, (key, value) -> { - if (value == null) { - value = new ArrayList<>(); - } - OffsetCommitRequestPartition partition = new OffsetCommitRequestPartition() - .setCommittedOffset(oam.offset()) - .setCommittedLeaderEpoch(oam.leaderEpoch().orElse(-1)) - .setCommittedMetadata(oam.metadata()) - .setPartitionIndex(entry.getKey().partition()); - value.add(partition); - return value; - }); - } - for (Map.Entry> entry : offsetData.entrySet()) { - OffsetCommitRequestTopic topic = new OffsetCommitRequestTopic() - .setName(entry.getKey()) - .setPartitions(entry.getValue()); - topics.add(topic); - } - OffsetCommitRequestData data = new OffsetCommitRequestData() - .setGroupId(context.groupId()) - .setTopics(topics); - return new OffsetCommitRequest.Builder(data); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - final OffsetCommitResponse response = (OffsetCommitResponse) abstractResponse; - - Map errorCounts = response.errorCounts(); - // 1) If coordinator changed since we fetched it, retry - // 2) If there is a coordinator error, retry - if (ConsumerGroupOperationContext.hasCoordinatorMoved(errorCounts) || - ConsumerGroupOperationContext.shouldRefreshCoordinator(errorCounts)) { - Call call = getAlterConsumerGroupOffsetsCall(context, offsets); - rescheduleFindCoordinatorTask(context, () -> call, this); - return; - } - - final Map partitions = new HashMap<>(); - for (OffsetCommitResponseTopic topic : response.data().topics()) { - for (OffsetCommitResponsePartition partition : topic.partitions()) { - TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); - Errors error = Errors.forCode(partition.errorCode()); - partitions.put(tp, error); - } - } - context.future().complete(partitions); - } - - @Override - void handleFailure(Throwable throwable) { - context.future().completeExceptionally(throwable); - } - }; + SimpleAdminApiFuture> future = + AlterConsumerGroupOffsetsHandler.newFuture(groupId); + AlterConsumerGroupOffsetsHandler handler = new AlterConsumerGroupOffsetsHandler(groupId, offsets, logContext); + invokeDriver(handler, future, options.timeoutMs); + return new AlterConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId))); } @Override @@ -4836,7 +4306,6 @@ public class KafkaAdminClient extends AdminClient { NodeProvider nodeProvider = spec.scope.destinationBrokerId().isPresent() ? new ConstantNodeIdProvider(spec.scope.destinationBrokerId().getAsInt()) : new LeastLoadedNodeProvider(); - return new Call(spec.name, spec.nextAllowedTryMs, spec.tries, spec.deadlineMs, nodeProvider) { @Override AbstractRequest.Builder createRequest(int timeoutMs) { @@ -4846,7 +4315,7 @@ public class KafkaAdminClient extends AdminClient { @Override void handleResponse(AbstractResponse response) { long currentTimeMs = time.milliseconds(); - driver.onResponse(currentTimeMs, spec, response); + driver.onResponse(currentTimeMs, spec, response, this.curNode()); maybeSendRequests(driver, currentTimeMs); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java index 90fda93a5fa..c25e4d8d3f4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.AbortTransactionSpec; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidProducerEpochException; @@ -93,7 +94,7 @@ public class AbortTransactionHandler implements AdminApiHandler handleResponse( - int brokerId, + Node broker, Set topicPartitions, AbstractResponse abstractResponse ) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java index a434f789e04..b5c9ff32f26 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -187,7 +189,7 @@ public class AdminApiDriver { * Check whether any requests need to be sent. This should be called immediately * after the driver is constructed and then again after each request returns * (i.e. after {@link #onFailure(long, RequestSpec, Throwable)} or - * {@link #onResponse(long, RequestSpec, AbstractResponse)}). + * {@link #onResponse(long, RequestSpec, AbstractResponse, Node)}). * * @return A list of requests that need to be sent */ @@ -204,14 +206,14 @@ public class AdminApiDriver { public void onResponse( long currentTimeMs, RequestSpec spec, - AbstractResponse response + AbstractResponse response, + Node node ) { clearInflightRequest(currentTimeMs, spec); if (spec.scope instanceof FulfillmentScope) { - int brokerId = ((FulfillmentScope) spec.scope).destinationBrokerId; AdminApiHandler.ApiResult result = handler.handleResponse( - brokerId, + node, spec.keys, response ); @@ -250,6 +252,13 @@ public class AdminApiDriver { .filter(future.lookupKeys()::contains) .collect(Collectors.toSet()); retryLookup(keysToUnmap); + + } else if (t instanceof NoBatchedFindCoordinatorsException) { + ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); + Set keysToUnmap = spec.keys.stream() + .filter(future.lookupKeys()::contains) + .collect(Collectors.toSet()); + retryLookup(keysToUnmap); } else { Map errors = spec.keys.stream().collect(Collectors.toMap( Function.identity(), @@ -267,7 +276,12 @@ public class AdminApiDriver { private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { RequestState requestState = requestStates.get(spec.scope); if (requestState != null) { - requestState.clearInflight(currentTimeMs); + // Only apply backoff if it's not a retry of a lookup request + if (spec.scope instanceof FulfillmentScope) { + requestState.clearInflight(currentTimeMs + retryBackoffMs); + } else { + requestState.clearInflight(currentTimeMs); + } } } @@ -384,9 +398,9 @@ public class AdminApiDriver { return inflightRequest.isPresent(); } - public void clearInflight(long currentTimeMs) { + public void clearInflight(long nextAllowedRetryMs) { this.inflightRequest = Optional.empty(); - this.nextAllowedRetryMs = currentTimeMs + retryBackoffMs; + this.nextAllowedRetryMs = nextAllowedRetryMs; } public void setInflight(RequestSpec spec) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiFuture.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiFuture.java index 00fabb3d1cb..8ddfb83604b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiFuture.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiFuture.java @@ -121,5 +121,9 @@ public interface AdminApiFuture { public Map> all() { return futures; } + + public KafkaFutureImpl get(K key) { + return futures.get(key); + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java index db3c68e5cc2..9f8d0ac5f07 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.common.Node; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; @@ -57,13 +58,13 @@ public interface AdminApiHandler { * Note that keys which received a retriable error should be left out of the * result. They will be retried automatically. * - * @param brokerId the brokerId that the associated request was sent to + * @param broker the broker that the associated request was sent to * @param keys the set of keys from the associated request * @param response the response received from the broker * * @return result indicating key completion, failure, and unmapping */ - ApiResult handleResponse(int brokerId, Set keys, AbstractResponse response); + ApiResult handleResponse(Node broker, Set keys, AbstractResponse response); /** * Get the lookup strategy that is responsible for finding the brokerId diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java new file mode 100644 index 00000000000..cd99b54c72a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java @@ -0,0 +1,157 @@ +/* + * 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 java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.OffsetCommitRequestData; +import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition; +import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic; +import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; +import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +public class AlterConsumerGroupOffsetsHandler implements AdminApiHandler> { + + private final CoordinatorKey groupId; + private final Map offsets; + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public AlterConsumerGroupOffsetsHandler( + String groupId, + Map offsets, + LogContext logContext + ) { + this.groupId = CoordinatorKey.byGroupId(groupId); + this.offsets = offsets; + this.log = logContext.logger(AlterConsumerGroupOffsetsHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + @Override + public String apiName() { + return "offsetCommit"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + public static AdminApiFuture.SimpleAdminApiFuture> newFuture( + String groupId + ) { + return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + } + + @Override + public OffsetCommitRequest.Builder buildRequest(int coordinatorId, Set keys) { + List topics = new ArrayList<>(); + Map> offsetData = new HashMap<>(); + for (Map.Entry entry : offsets.entrySet()) { + String topic = entry.getKey().topic(); + OffsetAndMetadata oam = entry.getValue(); + OffsetCommitRequestPartition partition = new OffsetCommitRequestPartition() + .setCommittedOffset(oam.offset()) + .setCommittedLeaderEpoch(oam.leaderEpoch().orElse(-1)) + .setCommittedMetadata(oam.metadata()) + .setPartitionIndex(entry.getKey().partition()); + offsetData.computeIfAbsent(topic, key -> new ArrayList<>()).add(partition); + } + for (Map.Entry> entry : offsetData.entrySet()) { + OffsetCommitRequestTopic topic = new OffsetCommitRequestTopic() + .setName(entry.getKey()) + .setPartitions(entry.getValue()); + topics.add(topic); + } + OffsetCommitRequestData data = new OffsetCommitRequestData() + .setGroupId(groupId.idValue) + .setTopics(topics); + return new OffsetCommitRequest.Builder(data); + } + + @Override + public ApiResult> handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + final OffsetCommitResponse response = (OffsetCommitResponse) abstractResponse; + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + Map partitions = new HashMap<>(); + for (OffsetCommitResponseTopic topic : response.data().topics()) { + for (OffsetCommitResponsePartition partition : topic.partitions()) { + TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); + Errors error = Errors.forCode(partition.errorCode()); + if (error != Errors.NONE) { + handleError(groupId, error, failed, unmapped); + } else { + partitions.put(tp, error); + } + } + } + if (failed.isEmpty() && unmapped.isEmpty()) + completed.put(groupId, partitions); + + return new ApiResult<>(completed, failed, unmapped); + } + + private void handleError( + CoordinatorKey groupId, + Errors error, + Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + log.error("Received authorization failure for group {} in `OffsetCommit` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + break; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + case NOT_COORDINATOR: + log.debug("OffsetCommit request for group {} returned error {}. Will retry", groupId, error); + unmapped.add(groupId); + break; + default: + log.error("Received unexpected error for group {} in `OffsetCommit` response", + groupId, error.exception()); + failed.put(groupId, error.exception( + "Received unexpected error for group " + groupId + " in `OffsetCommit` response")); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ConsumerGroupOperationContext.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ConsumerGroupOperationContext.java deleted file mode 100644 index 175c0855c97..00000000000 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ConsumerGroupOperationContext.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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 java.util.Map; -import java.util.Optional; - -import org.apache.kafka.clients.admin.AbstractOptions; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.AbstractResponse; - -/** - * Context class to encapsulate parameters of a call to find and use a consumer group coordinator. - * Some of the parameters are provided at construction and are immutable whereas others are provided - * as "Call" are completed and values are available, like node id of the coordinator. - * - * @param The type of return value of the KafkaFuture - * @param The type of configuration option. Different for different consumer group commands. - */ -public final class ConsumerGroupOperationContext> { - final private String groupId; - final private O options; - final private long deadline; - final private KafkaFutureImpl future; - private Optional node; - - public ConsumerGroupOperationContext(String groupId, - O options, - long deadline, - KafkaFutureImpl future) { - this.groupId = groupId; - this.options = options; - this.deadline = deadline; - this.future = future; - this.node = Optional.empty(); - } - - public String groupId() { - return groupId; - } - - public O options() { - return options; - } - - public long deadline() { - return deadline; - } - - public KafkaFutureImpl future() { - return future; - } - - public Optional node() { - return node; - } - - public void setNode(Node node) { - this.node = Optional.ofNullable(node); - } - - public static boolean hasCoordinatorMoved(AbstractResponse response) { - return hasCoordinatorMoved(response.errorCounts()); - } - - public static boolean hasCoordinatorMoved(Map errorCounts) { - return errorCounts.containsKey(Errors.NOT_COORDINATOR); - } - - public static boolean shouldRefreshCoordinator(Map errorCounts) { - return errorCounts.containsKey(Errors.COORDINATOR_LOAD_IN_PROGRESS) || - errorCounts.containsKey(Errors.COORDINATOR_NOT_AVAILABLE); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java index 774161196db..f61221b82c4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java @@ -24,7 +24,7 @@ public class CoordinatorKey { public final String idValue; public final FindCoordinatorRequest.CoordinatorType type; - private CoordinatorKey(String idValue, FindCoordinatorRequest.CoordinatorType type) { + private CoordinatorKey(FindCoordinatorRequest.CoordinatorType type, String idValue) { this.idValue = idValue; this.type = type; } @@ -52,11 +52,11 @@ public class CoordinatorKey { } public static CoordinatorKey byGroupId(String groupId) { - return new CoordinatorKey(groupId, FindCoordinatorRequest.CoordinatorType.GROUP); + return new CoordinatorKey(FindCoordinatorRequest.CoordinatorType.GROUP, groupId); } public static CoordinatorKey byTransactionalId(String transactionalId) { - return new CoordinatorKey(transactionalId, FindCoordinatorRequest.CoordinatorType.TRANSACTION); + return new CoordinatorKey(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java index 604218f3075..cb44ad40e87 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java @@ -17,42 +17,72 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.message.FindCoordinatorResponseData.Coordinator; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; public class CoordinatorStrategy implements AdminApiLookupStrategy { + + private static final ApiRequestScope BATCH_REQUEST_SCOPE = new ApiRequestScope() { }; + private final Logger log; + private final FindCoordinatorRequest.CoordinatorType type; + private Set unrepresentableKeys = Collections.emptySet(); + + boolean batch = true; public CoordinatorStrategy( + FindCoordinatorRequest.CoordinatorType type, LogContext logContext ) { + this.type = type; this.log = logContext.logger(CoordinatorStrategy.class); } @Override public ApiRequestScope lookupScope(CoordinatorKey key) { - // The `FindCoordinator` API does not support batched lookups, so we use a - // separate lookup context for each coordinator key we need to lookup - return new LookupRequestScope(key); + if (batch) { + return BATCH_REQUEST_SCOPE; + } else { + // If the `FindCoordinator` API does not support batched lookups, we use a + // separate lookup context for each coordinator key we need to lookup + return new LookupRequestScope(key); + } } @Override public FindCoordinatorRequest.Builder buildRequest(Set keys) { - CoordinatorKey key = requireSingleton(keys); - return new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKey(key.idValue) - .setKeyType(key.type.id()) - ); + unrepresentableKeys = keys.stream().filter(k -> k == null || !isRepresentableKey(k.idValue)).collect(Collectors.toSet()); + Set representableKeys = keys.stream().filter(k -> k != null && isRepresentableKey(k.idValue)).collect(Collectors.toSet()); + if (batch) { + ensureSameType(representableKeys); + FindCoordinatorRequestData data = new FindCoordinatorRequestData() + .setKeyType(type.id()) + .setCoordinatorKeys(representableKeys.stream().map(k -> k.idValue).collect(Collectors.toList())); + return new FindCoordinatorRequest.Builder(data); + } else { + CoordinatorKey key = requireSingletonAndType(representableKeys); + return new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKey(key.idValue) + .setKeyType(key.type.id()) + ); + } } @Override @@ -60,41 +90,85 @@ public class CoordinatorStrategy implements AdminApiLookupStrategy keys, AbstractResponse abstractResponse ) { - CoordinatorKey key = requireSingleton(keys); - FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; - Errors error = response.error(); + Map mappedKeys = new HashMap<>(); + Map failedKeys = new HashMap<>(); + for (CoordinatorKey key : unrepresentableKeys) { + failedKeys.put(key, new InvalidGroupIdException("The given group id '" + + key.idValue + "' cannot be represented in a request.")); + } + FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; + if (batch) { + for (Coordinator coordinator : response.data().coordinators()) { + CoordinatorKey key = (type == CoordinatorType.GROUP) + ? CoordinatorKey.byGroupId(coordinator.key()) + : CoordinatorKey.byTransactionalId(coordinator.key()); + handleError(Errors.forCode(coordinator.errorCode()), + key, + coordinator.nodeId(), + mappedKeys, + failedKeys); + } + } else { + CoordinatorKey key = requireSingletonAndType(keys); + Errors error = response.error(); + handleError(error, key, response.node().id(), mappedKeys, failedKeys); + } + return new LookupResult<>(failedKeys, mappedKeys); + } + + public void disableBatch() { + batch = false; + } + + private CoordinatorKey requireSingletonAndType(Set keys) { + if (keys.size() != 1) { + throw new IllegalArgumentException("Unexpected size of key set: expected 1, but got " + keys.size()); + } + CoordinatorKey key = keys.iterator().next(); + if (key.type != type) { + throw new IllegalArgumentException("Unexpected key type: expected key to be of type " + type + ", but got " + key.type); + } + return key; + } + + private void ensureSameType(Set keys) { + if (keys.size() < 1) { + throw new IllegalArgumentException("Unexpected size of key set: expected >= 1, but got " + keys.size()); + } + if (keys.stream().filter(k -> k.type == type).collect(Collectors.toSet()).size() != keys.size()) { + throw new IllegalArgumentException("Unexpected key set: expected all key to be of type " + type + ", but some key were not"); + } + } + + private static boolean isRepresentableKey(String groupId) { + return groupId != null; + } + + private void handleError(Errors error, CoordinatorKey key, int nodeId, Map mappedKeys, Map failedKeys) { switch (error) { case NONE: - return LookupResult.mapped(key, response.data().nodeId()); - + mappedKeys.put(key, nodeId); + break; case COORDINATOR_NOT_AVAILABLE: case COORDINATOR_LOAD_IN_PROGRESS: log.debug("FindCoordinator request for key {} returned topic-level error {}. Will retry", key, error); - return LookupResult.empty(); - + break; case GROUP_AUTHORIZATION_FAILED: - return LookupResult.failed(key, new GroupAuthorizationException("FindCoordinator request for groupId " + + failedKeys.put(key, new GroupAuthorizationException("FindCoordinator request for groupId " + "`" + key + "` failed due to authorization failure", key.idValue)); - + break; case TRANSACTIONAL_ID_AUTHORIZATION_FAILED: - return LookupResult.failed(key, new TransactionalIdAuthorizationException("FindCoordinator request for " + + failedKeys.put(key, new TransactionalIdAuthorizationException("FindCoordinator request for " + "transactionalId `" + key + "` failed due to authorization failure")); - + break; default: - return LookupResult.failed(key, error.exception("FindCoordinator request for key " + + failedKeys.put(key, error.exception("FindCoordinator request for key " + "`" + key + "` failed due to an unexpected error")); } } - private static CoordinatorKey requireSingleton(Set keys) { - if (keys.size() != 1) { - throw new IllegalArgumentException("Unexpected lookup key set"); - } - return keys.iterator().next(); - } - private static class LookupRequestScope implements ApiRequestScope { final CoordinatorKey key; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandler.java new file mode 100644 index 00000000000..7e8b549b323 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandler.java @@ -0,0 +1,151 @@ +/* + * 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 java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.OffsetDeleteRequestData; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestPartition; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopic; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.OffsetDeleteRequest; +import org.apache.kafka.common.requests.OffsetDeleteResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +public class DeleteConsumerGroupOffsetsHandler implements AdminApiHandler> { + + private final CoordinatorKey groupId; + private final Set partitions; + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public DeleteConsumerGroupOffsetsHandler( + String groupId, + Set partitions, + LogContext logContext + ) { + this.groupId = CoordinatorKey.byGroupId(groupId); + this.partitions = partitions; + this.log = logContext.logger(DeleteConsumerGroupOffsetsHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + @Override + public String apiName() { + return "offsetDelete"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + public static AdminApiFuture.SimpleAdminApiFuture> newFuture( + String groupId + ) { + return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + } + + @Override + public OffsetDeleteRequest.Builder buildRequest(int coordinatorId, Set keys) { + final OffsetDeleteRequestTopicCollection topics = new OffsetDeleteRequestTopicCollection(); + partitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).forEach((topic, topicPartitions) -> topics.add( + new OffsetDeleteRequestTopic() + .setName(topic) + .setPartitions(topicPartitions.stream() + .map(tp -> new OffsetDeleteRequestPartition().setPartitionIndex(tp.partition())) + .collect(Collectors.toList()) + ) + )); + + return new OffsetDeleteRequest.Builder( + new OffsetDeleteRequestData() + .setGroupId(groupId.idValue) + .setTopics(topics) + ); + } + + @Override + public ApiResult> handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + final OffsetDeleteResponse response = (OffsetDeleteResponse) abstractResponse; + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + final Errors error = Errors.forCode(response.data().errorCode()); + if (error != Errors.NONE) { + handleError(groupId, error, failed, unmapped); + } else { + final Map partitions = new HashMap<>(); + response.data().topics().forEach(topic -> + topic.partitions().forEach(partition -> { + Errors partitionError = Errors.forCode(partition.errorCode()); + if (!handleError(groupId, partitionError, failed, unmapped)) { + partitions.put(new TopicPartition(topic.name(), partition.partitionIndex()), partitionError); + } + }) + ); + if (!partitions.isEmpty()) + completed.put(groupId, partitions); + } + return new ApiResult<>(completed, failed, unmapped); + } + + private boolean handleError( + CoordinatorKey groupId, + Errors error, + Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + case GROUP_ID_NOT_FOUND: + case INVALID_GROUP_ID: + log.error("Received non retriable error for group {} in `DeleteConsumerGroupOffsets` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + return true; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + return true; + case NOT_COORDINATOR: + log.debug("DeleteConsumerGroupOffsets request for group {} returned error {}. Will retry", + groupId, error); + unmapped.add(groupId); + return true; + default: + return false; + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandler.java new file mode 100644 index 00000000000..c5d22053812 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandler.java @@ -0,0 +1,134 @@ +/* + * 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 java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.message.DeleteGroupsRequestData; +import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.DeleteGroupsRequest; +import org.apache.kafka.common.requests.DeleteGroupsResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +public class DeleteConsumerGroupsHandler implements AdminApiHandler { + + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public DeleteConsumerGroupsHandler( + LogContext logContext + ) { + this.log = logContext.logger(DeleteConsumerGroupsHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + @Override + public String apiName() { + return "deleteConsumerGroups"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + public static AdminApiFuture.SimpleAdminApiFuture newFuture( + Collection groupIds + ) { + return AdminApiFuture.forKeys(buildKeySet(groupIds)); + } + + private static Set buildKeySet(Collection groupIds) { + return groupIds.stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + @Override + public DeleteGroupsRequest.Builder buildRequest( + int coordinatorId, + Set keys + ) { + List groupIds = keys.stream().map(key -> key.idValue).collect(Collectors.toList()); + DeleteGroupsRequestData data = new DeleteGroupsRequestData() + .setGroupsNames(groupIds); + return new DeleteGroupsRequest.Builder(data); + } + + @Override + public ApiResult handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + DeleteGroupsResponse response = (DeleteGroupsResponse) abstractResponse; + Map completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + for (DeletableGroupResult deletedGroup : response.data().results()) { + CoordinatorKey groupIdKey = CoordinatorKey.byGroupId(deletedGroup.groupId()); + Errors error = Errors.forCode(deletedGroup.errorCode()); + if (error != Errors.NONE) { + handleError(groupIdKey, error, failed, unmapped); + continue; + } + + completed.put(groupIdKey, null); + } + return new ApiResult<>(completed, failed, unmapped); + } + + private void handleError( + CoordinatorKey groupId, + Errors error, + Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + log.error("Received authorization failure for group {} in `DeleteConsumerGroups` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + break; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + break; + case NOT_COORDINATOR: + log.debug("DeleteConsumerGroups request for group {} returned error {}. Will retry", + groupId, error); + unmapped.add(groupId); + break; + default: + log.error("Received unexpected error for group {} in `DeleteConsumerGroups` response", + groupId, error.exception()); + failed.put(groupId, error.exception()); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandler.java new file mode 100644 index 00000000000..8a94becef1a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandler.java @@ -0,0 +1,202 @@ +/* + * 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 java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.MemberAssignment; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; +import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.acl.AclOperation; +import org.apache.kafka.common.message.DescribeGroupsRequestData; +import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; +import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.DescribeGroupsRequest; +import org.apache.kafka.common.requests.DescribeGroupsResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; + +public class DescribeConsumerGroupsHandler implements AdminApiHandler { + + private final boolean includeAuthorizedOperations; + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public DescribeConsumerGroupsHandler( + boolean includeAuthorizedOperations, + LogContext logContext + ) { + this.includeAuthorizedOperations = includeAuthorizedOperations; + this.log = logContext.logger(DescribeConsumerGroupsHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + private static Set buildKeySet(Collection groupIds) { + return groupIds.stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + public static AdminApiFuture.SimpleAdminApiFuture newFuture( + Collection groupIds + ) { + return AdminApiFuture.forKeys(buildKeySet(groupIds)); + } + + @Override + public String apiName() { + return "describeGroups"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + @Override + public DescribeGroupsRequest.Builder buildRequest(int coordinatorId, Set keys) { + List groupIds = keys.stream().map(key -> { + if (key.type != FindCoordinatorRequest.CoordinatorType.GROUP) { + throw new IllegalArgumentException("Invalid transaction coordinator key " + key + + " when building `DescribeGroups` request"); + } + return key.idValue; + }).collect(Collectors.toList()); + DescribeGroupsRequestData data = new DescribeGroupsRequestData() + .setGroups(groupIds) + .setIncludeAuthorizedOperations(includeAuthorizedOperations); + return new DescribeGroupsRequest.Builder(data); + } + + @Override + public ApiResult handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + DescribeGroupsResponse response = (DescribeGroupsResponse) abstractResponse; + Map completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + for (DescribedGroup describedGroup : response.data().groups()) { + CoordinatorKey groupIdKey = CoordinatorKey.byGroupId(describedGroup.groupId()); + Errors error = Errors.forCode(describedGroup.errorCode()); + if (error != Errors.NONE) { + handleError(groupIdKey, error, failed, unmapped); + continue; + } + final String protocolType = describedGroup.protocolType(); + if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) { + final List members = describedGroup.members(); + final List memberDescriptions = new ArrayList<>(members.size()); + final Set authorizedOperations = validAclOperations(describedGroup.authorizedOperations()); + for (DescribedGroupMember groupMember : members) { + Set partitions = Collections.emptySet(); + if (groupMember.memberAssignment().length > 0) { + final Assignment assignment = ConsumerProtocol. + deserializeAssignment(ByteBuffer.wrap(groupMember.memberAssignment())); + partitions = new HashSet<>(assignment.partitions()); + } + final MemberDescription memberDescription = new MemberDescription( + groupMember.memberId(), + Optional.ofNullable(groupMember.groupInstanceId()), + groupMember.clientId(), + groupMember.clientHost(), + new MemberAssignment(partitions)); + memberDescriptions.add(memberDescription); + } + final ConsumerGroupDescription consumerGroupDescription = + new ConsumerGroupDescription(groupIdKey.idValue, protocolType.isEmpty(), + memberDescriptions, + describedGroup.protocolData(), + ConsumerGroupState.parse(describedGroup.groupState()), + coordinator, + authorizedOperations); + completed.put(groupIdKey, consumerGroupDescription); + } else { + failed.put(groupIdKey, new IllegalArgumentException( + String.format("GroupId %s is not a consumer group (%s).", + groupIdKey.idValue, protocolType))); + } + } + return new ApiResult<>(completed, failed, unmapped); + } + + private void handleError( + CoordinatorKey groupId, + Errors error, + Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + log.error("Received authorization failure for group {} in `DescribeGroups` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + break; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + break; + case NOT_COORDINATOR: + log.debug("DescribeGroups request for group {} returned error {}. Will retry", + groupId, error); + unmapped.add(groupId); + break; + default: + log.error("Received unexpected error for group {} in `DescribeGroups` response", + groupId, error.exception()); + failed.put(groupId, error.exception( + "Received unexpected error for group " + groupId + " in `DescribeGroups` response")); + } + } + + private Set validAclOperations(final int authorizedOperations) { + if (authorizedOperations == MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED) { + return null; + } + return Utils.from32BitField(authorizedOperations) + .stream() + .map(AclOperation::fromCode) + .filter(operation -> operation != AclOperation.UNKNOWN + && operation != AclOperation.ALL + && operation != AclOperation.ANY) + .collect(Collectors.toSet()); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java index de1061f124d..4b279d5c90c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java @@ -19,6 +19,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; import org.apache.kafka.clients.admin.ProducerState; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.TopicAuthorizationException; @@ -154,7 +155,7 @@ public class DescribeProducersHandler implements AdminApiHandler handleResponse( - int brokerId, + Node broker, Set keys, AbstractResponse abstractResponse ) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java index ac45f597f1e..d270145a423 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.TransactionDescription; import org.apache.kafka.clients.admin.TransactionState; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.errors.TransactionalIdNotFoundException; @@ -28,6 +29,7 @@ import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.DescribeTransactionsRequest; import org.apache.kafka.common.requests.DescribeTransactionsResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -49,7 +51,7 @@ public class DescribeTransactionsHandler implements AdminApiHandler newFuture( @@ -93,7 +95,7 @@ public class DescribeTransactionsHandler implements AdminApiHandler handleResponse( - int brokerId, + Node broker, Set keys, AbstractResponse abstractResponse ) { @@ -122,7 +124,7 @@ public class DescribeTransactionsHandler implements AdminApiHandler> { + + private final CoordinatorKey groupId; + private final List partitions; + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public ListConsumerGroupOffsetsHandler( + String groupId, + List partitions, + LogContext logContext + ) { + this.groupId = CoordinatorKey.byGroupId(groupId); + this.partitions = partitions; + this.log = logContext.logger(ListConsumerGroupOffsetsHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + public static AdminApiFuture.SimpleAdminApiFuture> newFuture( + String groupId + ) { + return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + } + + @Override + public String apiName() { + return "offsetFetch"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + @Override + public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set keys) { + // Set the flag to false as for admin client request, + // we don't need to wait for any pending offset state to clear. + return new OffsetFetchRequest.Builder(groupId.idValue, false, partitions, false); + } + + @Override + public ApiResult> handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + if (response.error() != Errors.NONE) { + handleError(groupId, response.error(), failed, unmapped); + } else { + final Map groupOffsetsListing = new HashMap<>(); + for (Map.Entry entry : + response.responseData().entrySet()) { + final TopicPartition topicPartition = entry.getKey(); + OffsetFetchResponse.PartitionData partitionData = entry.getValue(); + final Errors error = partitionData.error; + + if (error == Errors.NONE) { + final long offset = partitionData.offset; + final String metadata = partitionData.metadata; + final Optional leaderEpoch = partitionData.leaderEpoch; + // Negative offset indicates that the group has no committed offset for this partition + if (offset < 0) { + groupOffsetsListing.put(topicPartition, null); + } else { + groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + } + } else { + log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); + } + } + completed.put(groupId, groupOffsetsListing); + } + return new ApiResult<>(completed, failed, unmapped); + } + + private void handleError( + CoordinatorKey groupId, + Errors error, + Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + log.error("Received authorization failure for group {} in `OffsetFetch` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + break; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + break; + case NOT_COORDINATOR: + log.debug("OffsetFetch request for group {} returned error {}. Will retry", + groupId, error); + unmapped.add(groupId); + break; + default: + log.error("Received unexpected error for group {} in `OffsetFetch` response", + groupId, error.exception()); + failed.put(groupId, error.exception( + "Received unexpected error for group " + groupId + " in `OffsetFetch` response")); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java index d1b4ec2d5b8..d60580c85bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java @@ -19,6 +19,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.ListTransactionsOptions; import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; +import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.CoordinatorNotAvailableException; import org.apache.kafka.common.message.ListTransactionsRequestData; import org.apache.kafka.common.protocol.Errors; @@ -77,10 +78,11 @@ public class ListTransactionsHandler implements AdminApiHandler> handleResponse( - int brokerId, + Node broker, Set keys, AbstractResponse abstractResponse ) { + int brokerId = broker.id(); AllBrokersStrategy.BrokerKey key = requireSingleton(keys, brokerId); ListTransactionsResponse response = (ListTransactionsResponse) abstractResponse; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandler.java new file mode 100644 index 00000000000..c6af2d4a3db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandler.java @@ -0,0 +1,133 @@ +/* + * 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 java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; +import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.LeaveGroupRequest; +import org.apache.kafka.common.requests.LeaveGroupResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +public class RemoveMembersFromConsumerGroupHandler implements AdminApiHandler> { + + private final CoordinatorKey groupId; + private final List members; + private final Logger log; + private final AdminApiLookupStrategy lookupStrategy; + + public RemoveMembersFromConsumerGroupHandler( + String groupId, + List members, + LogContext logContext + ) { + this.groupId = CoordinatorKey.byGroupId(groupId); + this.members = members; + this.log = logContext.logger(RemoveMembersFromConsumerGroupHandler.class); + this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + } + + @Override + public String apiName() { + return "leaveGroup"; + } + + @Override + public AdminApiLookupStrategy lookupStrategy() { + return lookupStrategy; + } + + public static AdminApiFuture.SimpleAdminApiFuture> newFuture( + String groupId + ) { + return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + } + + @Override + public LeaveGroupRequest.Builder buildRequest(int coordinatorId, Set keys) { + return new LeaveGroupRequest.Builder(groupId.idValue, members); + } + + @Override + public ApiResult> handleResponse( + Node coordinator, + Set groupIds, + AbstractResponse abstractResponse + ) { + final LeaveGroupResponse response = (LeaveGroupResponse) abstractResponse; + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + + final Errors error = Errors.forCode(response.data().errorCode()); + if (error != Errors.NONE) { + handleError(groupId, error, failed, unmapped); + } else { + final Map memberErrors = new HashMap<>(); + for (MemberResponse memberResponse : response.memberResponses()) { + memberErrors.put(new MemberIdentity() + .setMemberId(memberResponse.memberId()) + .setGroupInstanceId(memberResponse.groupInstanceId()), + Errors.forCode(memberResponse.errorCode())); + + } + completed.put(groupId, memberErrors); + } + return new ApiResult<>(completed, failed, unmapped); + } + + private void handleError( + CoordinatorKey groupId, + Errors error, Map failed, + List unmapped + ) { + switch (error) { + case GROUP_AUTHORIZATION_FAILED: + log.error("Received authorization failure for group {} in `LeaveGroup` response", groupId, + error.exception()); + failed.put(groupId, error.exception()); + break; + case COORDINATOR_LOAD_IN_PROGRESS: + case COORDINATOR_NOT_AVAILABLE: + break; + case NOT_COORDINATOR: + log.debug("LeaveGroup request for group {} returned error {}. Will retry", + groupId, error); + unmapped.add(groupId); + break; + default: + log.error("Received unexpected error for group {} in `LeaveGroup` response", + groupId, error.exception()); + failed.put(groupId, error.exception( + "Received unexpected error for group " + groupId + " in `LeaveGroup` response")); + break; + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 9fbfe1f1ce3..4123914c6c9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -52,6 +52,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.HeartbeatRequest; import org.apache.kafka.common.requests.HeartbeatResponse; @@ -138,6 +139,7 @@ public abstract class AbstractCoordinator implements Closeable { private RequestFuture joinFuture = null; private RequestFuture findCoordinatorFuture = null; private volatile RuntimeException fatalFindCoordinatorException = null; + private volatile boolean batchFindCoordinator = true; private Generation generation = Generation.NO_GENERATION; private long lastRebalanceStartMs = -1L; private long lastRebalanceEndMs = -1L; @@ -813,34 +815,56 @@ public abstract class AbstractCoordinator implements Closeable { */ private RequestFuture sendFindCoordinatorRequest(Node node) { // initiate the group metadata request - log.debug("Sending FindCoordinator request to broker {}", node); - FindCoordinatorRequest.Builder requestBuilder = - new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(CoordinatorType.GROUP.id()) - .setKey(this.rebalanceConfig.groupId)); + log.debug("Sending FindCoordinator request to broker {} with batch={}", node, batchFindCoordinator); + FindCoordinatorRequestData data = new FindCoordinatorRequestData() + .setKeyType(CoordinatorType.GROUP.id()); + if (batchFindCoordinator) { + data.setCoordinatorKeys(Collections.singletonList(this.rebalanceConfig.groupId)); + } else { + data.setKey(this.rebalanceConfig.groupId); + } + FindCoordinatorRequest.Builder requestBuilder = new FindCoordinatorRequest.Builder(data); return client.send(node, requestBuilder) - .compose(new FindCoordinatorResponseHandler()); + .compose(new FindCoordinatorResponseHandler(batchFindCoordinator)); } private class FindCoordinatorResponseHandler extends RequestFutureAdapter { + private boolean batch; + FindCoordinatorResponseHandler(boolean batch) { + this.batch = batch; + } @Override public void onSuccess(ClientResponse resp, RequestFuture future) { log.debug("Received FindCoordinator response {}", resp); FindCoordinatorResponse findCoordinatorResponse = (FindCoordinatorResponse) resp.responseBody(); - Errors error = findCoordinatorResponse.error(); + if (batch && findCoordinatorResponse.data().coordinators().size() != 1) { + log.error("Group coordinator lookup failed: Invalid response containing more than a single coordinator"); + future.raise(new IllegalStateException("Group coordinator lookup failed: Invalid response containing more than a single coordinator")); + } + Errors error = batch + ? Errors.forCode(findCoordinatorResponse.data().coordinators().get(0).errorCode()) + : findCoordinatorResponse.error(); if (error == Errors.NONE) { synchronized (AbstractCoordinator.this) { + int nodeId = batch + ? findCoordinatorResponse.data().coordinators().get(0).nodeId() + : findCoordinatorResponse.data().nodeId(); + String host = batch + ? findCoordinatorResponse.data().coordinators().get(0).host() + : findCoordinatorResponse.data().host(); + int port = batch + ? findCoordinatorResponse.data().coordinators().get(0).port() + : findCoordinatorResponse.data().port(); // use MAX_VALUE - node.id as the coordinator id to allow separate connections // for the coordinator in the underlying network client layer - int coordinatorConnectionId = Integer.MAX_VALUE - findCoordinatorResponse.data().nodeId(); + int coordinatorConnectionId = Integer.MAX_VALUE - nodeId; AbstractCoordinator.this.coordinator = new Node( coordinatorConnectionId, - findCoordinatorResponse.data().host(), - findCoordinatorResponse.data().port()); + host, + port); log.info("Discovered group coordinator {}", coordinator); client.tryConnect(coordinator); heartbeat.resetSessionTimeout(); @@ -849,7 +873,10 @@ public abstract class AbstractCoordinator implements Closeable { } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId)); } else { - log.debug("Group coordinator lookup failed: {}", findCoordinatorResponse.data().errorMessage()); + String errorMessage = batch + ? findCoordinatorResponse.data().coordinators().get(0).errorMessage() + : findCoordinatorResponse.data().errorMessage(); + log.debug("Group coordinator lookup failed: {}", errorMessage); future.raise(error); } } @@ -858,6 +885,12 @@ public abstract class AbstractCoordinator implements Closeable { public void onFailure(RuntimeException e, RequestFuture future) { log.debug("FindCoordinator request failed due to {}", e.toString()); + if (e instanceof NoBatchedFindCoordinatorsException) { + batchFindCoordinator = false; + clearFindCoordinatorFuture(); + lookupCoordinator(); + return; + } if (!(e instanceof RetriableException)) { // Remember the exception if fatal so we can ensure it gets thrown by the main thread fatalFindCoordinatorException = e; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 0ac9ac57da1..2febc2ec89a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -58,6 +58,7 @@ import org.apache.kafka.common.requests.EndTxnRequest; import org.apache.kafka.common.requests.EndTxnResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; +import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.InitProducerIdRequest; import org.apache.kafka.common.requests.InitProducerIdResponse; @@ -71,6 +72,7 @@ import org.apache.kafka.common.utils.PrimitiveRef; import org.slf4j.Logger; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -97,6 +99,7 @@ public class TransactionManager { private final String transactionalId; private final int transactionTimeoutMs; private final ApiVersions apiVersions; + private boolean batchFindCoordinator = true; private static class TopicPartitionBookkeeper { @@ -1140,10 +1143,14 @@ public class TransactionManager { throw new IllegalStateException("Invalid coordinator type: " + type); } - FindCoordinatorRequest.Builder builder = new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(type.id()) - .setKey(coordinatorKey)); + FindCoordinatorRequestData data = new FindCoordinatorRequestData() + .setKeyType(type.id()); + if (batchFindCoordinator) { + data.setCoordinatorKeys(Collections.singletonList(coordinatorKey)); + } else { + data.setKey(coordinatorKey); + } + FindCoordinatorRequest.Builder builder = new FindCoordinatorRequest.Builder(data); enqueueRequest(new FindCoordinatorHandler(builder)); } @@ -1277,6 +1284,9 @@ public class TransactionManager { if (this.needsCoordinator()) lookupCoordinator(this.coordinatorType(), this.coordinatorKey()); reenqueue(); + } else if (response.versionMismatch() instanceof NoBatchedFindCoordinatorsException && response.requestHeader().apiKey() == ApiKeys.FIND_COORDINATOR) { + batchFindCoordinator = false; + reenqueue(); } else if (response.versionMismatch() != null) { fatalError(response.versionMismatch()); } else if (response.hasResponse()) { @@ -1524,11 +1534,29 @@ public class TransactionManager { @Override public void handleResponse(AbstractResponse response) { FindCoordinatorResponse findCoordinatorResponse = (FindCoordinatorResponse) response; - Errors error = findCoordinatorResponse.error(); CoordinatorType coordinatorType = CoordinatorType.forId(builder.data().keyType()); + if (batchFindCoordinator && findCoordinatorResponse.data().coordinators().size() != 1) { + log.error("Group coordinator lookup failed: Invalid response containing more than a single coordinator"); + fatalError(new IllegalStateException("Group coordinator lookup failed: Invalid response containing more than a single coordinator")); + } + String key = batchFindCoordinator + ? findCoordinatorResponse.data().coordinators().get(0).key() + : builder.data().key(); + Errors error = batchFindCoordinator + ? Errors.forCode(findCoordinatorResponse.data().coordinators().get(0).errorCode()) + : findCoordinatorResponse.error(); if (error == Errors.NONE) { - Node node = findCoordinatorResponse.node(); + int nodeId = batchFindCoordinator + ? findCoordinatorResponse.data().coordinators().get(0).nodeId() + : findCoordinatorResponse.data().nodeId(); + String host = batchFindCoordinator + ? findCoordinatorResponse.data().coordinators().get(0).host() + : findCoordinatorResponse.data().host(); + int port = batchFindCoordinator + ? findCoordinatorResponse.data().coordinators().get(0).port() + : findCoordinatorResponse.data().port(); + Node node = new Node(nodeId, host, port); switch (coordinatorType) { case GROUP: consumerGroupCoordinator = node; @@ -1543,12 +1571,15 @@ public class TransactionManager { reenqueue(); } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { fatalError(error.exception()); - } else if (findCoordinatorResponse.error() == Errors.GROUP_AUTHORIZATION_FAILED) { - abortableError(GroupAuthorizationException.forGroupId(builder.data().key())); + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { + abortableError(GroupAuthorizationException.forGroupId(key)); } else { + String errorMessage = batchFindCoordinator + ? findCoordinatorResponse.data().coordinators().get(0).errorMessage() + : findCoordinatorResponse.data().errorMessage(); fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to " + - "unexpected error: %s", coordinatorType, builder.data().key(), - findCoordinatorResponse.data().errorMessage()))); + "unexpected error: %s", coordinatorType, key, + errorMessage))); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java index f4491ad1ee5..30f98ba0007 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java @@ -43,6 +43,10 @@ public class FindCoordinatorRequest extends AbstractRequest { throw new UnsupportedVersionException("Cannot create a v" + version + " FindCoordinator request " + "because we require features supported only in 2 or later."); } + if (version < 4 && !data.coordinatorKeys().isEmpty()) { + throw new NoBatchedFindCoordinatorsException("Cannot create a v" + version + " FindCoordinator request " + + "because we require features supported only in 4 or later."); + } return new FindCoordinatorRequest(data, version); } @@ -56,6 +60,22 @@ public class FindCoordinatorRequest extends AbstractRequest { } } + /** + * Indicates that it is not possible to lookup coordinators in batches with FindCoordinator. Instead + * coordinators must be looked up one by one. + */ + public static class NoBatchedFindCoordinatorsException extends UnsupportedVersionException { + private static final long serialVersionUID = 1L; + + public NoBatchedFindCoordinatorsException(String message, Throwable cause) { + super(message, cause); + } + + public NoBatchedFindCoordinatorsException(String message) { + super(message); + } + } + private final FindCoordinatorRequestData data; private FindCoordinatorRequest(FindCoordinatorRequestData data, short version) { @@ -70,7 +90,11 @@ public class FindCoordinatorRequest extends AbstractRequest { response.setThrottleTimeMs(throttleTimeMs); } Errors error = Errors.forException(e); - return FindCoordinatorResponse.prepareResponse(error, Node.noNode()); + if (version() < 4) { + return FindCoordinatorResponse.prepareOldResponse(error, Node.noNode()); + } else { + return FindCoordinatorResponse.prepareErrorResponse(error, data.coordinatorKeys()); + } } public static FindCoordinatorRequest parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java index 11f3d488879..222f78ccb98 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java @@ -23,6 +23,9 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Map; @@ -86,7 +89,7 @@ public class FindCoordinatorResponse extends AbstractResponse { return version >= 2; } - public static FindCoordinatorResponse prepareResponse(Errors error, Node node) { + public static FindCoordinatorResponse prepareOldResponse(Errors error, Node node) { FindCoordinatorResponseData data = new FindCoordinatorResponseData(); data.setErrorCode(error.code()) .setErrorMessage(error.message()) @@ -95,4 +98,35 @@ public class FindCoordinatorResponse extends AbstractResponse { .setPort(node.port()); return new FindCoordinatorResponse(data); } + + public static FindCoordinatorResponse prepareResponse(Errors error, String key, Node node) { + FindCoordinatorResponseData data = new FindCoordinatorResponseData(); + data.setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setErrorCode(error.code()) + .setErrorMessage(error.message()) + .setKey(key) + .setHost(node.host()) + .setPort(node.port()) + .setNodeId(node.id()))); + return new FindCoordinatorResponse(data); + } + + public static FindCoordinatorResponse prepareErrorResponse(Errors error, List keys) { + FindCoordinatorResponseData data = new FindCoordinatorResponseData(); + List coordinators = new ArrayList<>(keys.size()); + for (String key : keys) { + FindCoordinatorResponseData.Coordinator coordinator = new FindCoordinatorResponseData.Coordinator() + .setErrorCode(error.code()) + .setErrorMessage(error.message()) + .setKey(key) + .setHost(Node.noNode().host()) + .setPort(Node.noNode().port()) + .setNodeId(Node.noNode().id()); + coordinators.add(coordinator); + } + data.setCoordinators(coordinators); + return new FindCoordinatorResponse(data); + } + } diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index cd5b77a3f83..a475cc27866 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -23,12 +23,16 @@ // Version 2 is the same as version 1. // // Version 3 is the first flexible version. - "validVersions": "0-3", + // + // Version 4 adds support for batching via CoordinatorKeys (KIP-699) + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ - { "name": "Key", "type": "string", "versions": "0+", + { "name": "Key", "type": "string", "versions": "0-3", "about": "The coordinator key." }, { "name": "KeyType", "type": "int8", "versions": "1+", "default": "0", "ignorable": false, - "about": "The coordinator key type. (Group, transaction, etc.)" } + "about": "The coordinator key type. (Group, transaction, etc.)" }, + { "name": "CoordinatorKeys", "type": "[]string", "versions": "4+", + "about": "The coordinator keys." } ] } diff --git a/clients/src/main/resources/common/message/FindCoordinatorResponse.json b/clients/src/main/resources/common/message/FindCoordinatorResponse.json index 996e8fb5fba..9309c0177d6 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorResponse.json +++ b/clients/src/main/resources/common/message/FindCoordinatorResponse.json @@ -22,20 +22,33 @@ // Starting in version 2, on quota violation, brokers send out responses before throttling. // // Version 3 is the first flexible version. - "validVersions": "0-3", + // + // Version 4 adds support for batching via Coordinators (KIP-699) + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", + { "name": "ErrorCode", "type": "int16", "versions": "0-3", "about": "The error code, or 0 if there was no error." }, - { "name": "ErrorMessage", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true, + { "name": "ErrorMessage", "type": "string", "versions": "1-3", "nullableVersions": "1-3", "ignorable": true, "about": "The error message, or null if there was no error." }, - { "name": "NodeId", "type": "int32", "versions": "0+", "entityType": "brokerId", + { "name": "NodeId", "type": "int32", "versions": "0-3", "entityType": "brokerId", "about": "The node id." }, - { "name": "Host", "type": "string", "versions": "0+", + { "name": "Host", "type": "string", "versions": "0-3", "about": "The host name." }, - { "name": "Port", "type": "int32", "versions": "0+", - "about": "The port." } + { "name": "Port", "type": "int32", "versions": "0-3", + "about": "The port." }, + { "name": "Coordinators", "type": "[]Coordinator", "versions": "4+", "about": "Each coordinator result in the response", "fields": [ + { "name": "Key", "type": "string", "versions": "4+", "about": "The coordinator key." }, + { "name": "NodeId", "type": "int32", "versions": "4+", "entityType": "brokerId", + "about": "The node id." }, + { "name": "Host", "type": "string", "versions": "4+", "about": "The host name." }, + { "name": "Port", "type": "int32", "versions": "4+", "about": "The port." }, + { "name": "ErrorCode", "type": "int16", "versions": "4+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "4+", "nullableVersions": "4+", "ignorable": true, + "about": "The error message, or null if there was no error." } + ]} ] } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 933d41ba6b3..446e22d17ff 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.admin.DeleteAclsResult.FilterResults; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; -import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; @@ -128,14 +127,12 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.quota.ClientQuotaFilterComponent; import org.apache.kafka.common.record.RecordVersion; -import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.AlterClientQuotasResponse; import org.apache.kafka.common.requests.AlterPartitionReassignmentsResponse; import org.apache.kafka.common.requests.AlterReplicaLogDirsResponse; @@ -255,6 +252,7 @@ import static org.junit.jupiter.api.Assertions.fail; @Timeout(120) public class KafkaAdminClientTest { private static final Logger log = LoggerFactory.getLogger(KafkaAdminClientTest.class); + private static final String GROUP_ID = "group-0"; @Test public void testDefaultApiTimeoutAndRequestTimeoutConflicts() { @@ -479,7 +477,15 @@ public class KafkaAdminClientTest { } private static FindCoordinatorResponse prepareFindCoordinatorResponse(Errors error, Node node) { - return FindCoordinatorResponse.prepareResponse(error, node); + return prepareFindCoordinatorResponse(error, GROUP_ID, node); + } + + private static FindCoordinatorResponse prepareFindCoordinatorResponse(Errors error, String key, Node node) { + return FindCoordinatorResponse.prepareResponse(error, key, node); + } + + private static FindCoordinatorResponse prepareOldFindCoordinatorResponse(Errors error, Node node) { + return FindCoordinatorResponse.prepareOldResponse(error, node); } private static MetadataResponse prepareMetadataResponse(Cluster cluster, Errors error) { @@ -2523,7 +2529,6 @@ public class KafkaAdminClientTest { AdminClientConfig.RETRIES_CONFIG, "0")) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); @@ -2532,7 +2537,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); - final AlterConsumerGroupOffsetsResult result = env.adminClient().alterConsumerGroupOffsets(groupId, offsets); + final AlterConsumerGroupOffsetsResult result = env.adminClient().alterConsumerGroupOffsets(GROUP_ID, offsets); TestUtils.assertFutureError(result.all(), TimeoutException.class); } @@ -2553,7 +2558,6 @@ public class KafkaAdminClientTest { AtomicLong firstAttemptTime = new AtomicLong(0); AtomicLong secondAttemptTime = new AtomicLong(0); - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); mockClient.prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); @@ -2572,7 +2576,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); - final KafkaFuture future = env.adminClient().alterConsumerGroupOffsets(groupId, offsets).all(); + final KafkaFuture future = env.adminClient().alterConsumerGroupOffsets(GROUP_ID, offsets).all(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting CommitOffsets first request failure"); TestUtils.waitForCondition(() -> ((KafkaAdminClient) env.adminClient()).numPendingCalls() == 1, "Failed to add retry CommitOffsets call on first failure"); @@ -2599,7 +2603,7 @@ public class KafkaAdminClientTest { DescribeGroupsResponseData data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NOT_COORDINATOR, "", "", @@ -2609,7 +2613,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new DescribeGroupsResponse(data)); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList("group-0")); + final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList(GROUP_ID)); TestUtils.assertFutureError(result.all(), TimeoutException.class); } @@ -2634,7 +2638,7 @@ public class KafkaAdminClientTest { DescribeGroupsResponseData data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NOT_COORDINATOR, "", "", @@ -2651,7 +2655,7 @@ public class KafkaAdminClientTest { data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NONE, "", ConsumerProtocol.PROTOCOL_TYPE, @@ -2665,7 +2669,7 @@ public class KafkaAdminClientTest { }, new DescribeGroupsResponse(data)); final KafkaFuture> future = - env.adminClient().describeConsumerGroups(singletonList("group-0")).all(); + env.adminClient().describeConsumerGroups(singletonList(GROUP_ID)).all(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting DescribeConsumerGroup first request failure"); TestUtils.waitForCondition(() -> ((KafkaAdminClient) env.adminClient()).numPendingCalls() == 1, "Failed to add retry DescribeConsumerGroup call on first failure"); @@ -2694,7 +2698,7 @@ public class KafkaAdminClientTest { //Retriable errors should be retried data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.COORDINATOR_LOAD_IN_PROGRESS, "", "", @@ -2705,7 +2709,7 @@ public class KafkaAdminClientTest { data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.COORDINATOR_NOT_AVAILABLE, "", "", @@ -2721,7 +2725,7 @@ public class KafkaAdminClientTest { */ data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NOT_COORDINATOR, "", "", @@ -2754,7 +2758,7 @@ public class KafkaAdminClientTest { expectedMemberDescriptions.add(convertToMemberDescriptions(memberTwo, new MemberAssignment(new HashSet<>(topicPartitions)))); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NONE, "", ConsumerProtocol.PROTOCOL_TYPE, @@ -2764,11 +2768,11 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new DescribeGroupsResponse(data)); - final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList("group-0")); - final ConsumerGroupDescription groupDescription = result.describedGroups().get("group-0").get(); + final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList(GROUP_ID)); + final ConsumerGroupDescription groupDescription = result.describedGroups().get(GROUP_ID).get(); assertEquals(1, result.describedGroups().size()); - assertEquals("group-0", groupDescription.groupId()); + assertEquals(GROUP_ID, groupDescription.groupId()); assertEquals(2, groupDescription.members().size()); assertEquals(expectedMemberDescriptions, groupDescription.members()); } @@ -2796,7 +2800,7 @@ public class KafkaAdminClientTest { DescribeGroupsResponseData group0Data = new DescribeGroupsResponseData(); group0Data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NONE, "", ConsumerProtocol.PROTOCOL_TYPE, @@ -2824,7 +2828,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new DescribeGroupsResponse(groupConnectData)); Collection groups = new HashSet<>(); - groups.add("group-0"); + groups.add(GROUP_ID); groups.add("group-connect-0"); final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(groups); assertEquals(2, result.describedGroups().size()); @@ -2842,7 +2846,7 @@ public class KafkaAdminClientTest { DescribeGroupsResponseData data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NONE, "", ConsumerProtocol.PROTOCOL_TYPE, @@ -2852,8 +2856,8 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new DescribeGroupsResponse(data)); - final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList("group-0")); - final ConsumerGroupDescription groupDescription = result.describedGroups().get("group-0").get(); + final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList(GROUP_ID)); + final ConsumerGroupDescription groupDescription = result.describedGroups().get(GROUP_ID).get(); assertNull(groupDescription.authorizedOperations()); } @@ -2869,7 +2873,7 @@ public class KafkaAdminClientTest { DescribeGroupsResponseData data = new DescribeGroupsResponseData(); data.groups().add(DescribeGroupsResponse.groupMetadata( - "group-0", + GROUP_ID, Errors.NONE, "", "non-consumer", @@ -2879,9 +2883,9 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new DescribeGroupsResponse(data)); - final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList("group-0")); + final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(singletonList(GROUP_ID)); - TestUtils.assertFutureError(result.describedGroups().get("group-0"), IllegalArgumentException.class); + TestUtils.assertFutureError(result.describedGroups().get(GROUP_ID), IllegalArgumentException.class); } } @@ -2898,7 +2902,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets("group-0"); + final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); TestUtils.assertFutureError(result.partitionsToOffsetAndMetadata(), TimeoutException.class); @@ -2946,13 +2950,72 @@ public class KafkaAdminClientTest { } } + @Test + public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { + // Retriable errors should be retried + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + + env.kafkaClient().prepareResponse( + new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + + /* + * We need to return two responses here, one for NOT_COORDINATOR call when calling list consumer offsets + * api using coordinator that has moved. This will retry whole operation. So we need to again respond with a + * FindCoordinatorResponse. + */ + env.kafkaClient().prepareResponse( + new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + new OffsetFetchResponse(Errors.NONE, Collections.emptyMap())); + + final ListConsumerGroupOffsetsResult errorResult1 = env.adminClient().listConsumerGroupOffsets(GROUP_ID); + + assertEquals(Collections.emptyMap(), errorResult1.partitionsToOffsetAndMetadata().get()); + } + } + + @Test + public void testListConsumerGroupOffsetsNonRetriableErrors() throws Exception { + // Non-retriable errors throw an exception + final List nonRetriableErrors = Arrays.asList( + Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + for (Errors error : nonRetriableErrors) { + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + new OffsetFetchResponse(error, Collections.emptyMap())); + + ListConsumerGroupOffsetsResult errorResult = env.adminClient().listConsumerGroupOffsets(GROUP_ID); + + TestUtils.assertFutureError(errorResult.partitionsToOffsetAndMetadata(), error.exception().getClass()); + } + } + } + @Test public void testListConsumerGroupOffsets() throws Exception { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); // Retriable FindCoordinatorResponse errors should be retried - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); @@ -2984,7 +3047,7 @@ public class KafkaAdminClientTest { Optional.empty(), "", Errors.NONE)); env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NONE, responseData)); - final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets("group-0"); + final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata().get(); assertEquals(4, partitionToOffsetAndMetadata.size()); @@ -3000,7 +3063,7 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupsNumRetries() throws Exception { final Cluster cluster = mockCluster(3, 0); final Time time = new MockTime(); - final List groupIds = singletonList("group-0"); + final List groupIds = singletonList("groupId"); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRIES_CONFIG, "0")) { @@ -3009,7 +3072,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final DeletableGroupResultCollection validResponse = new DeletableGroupResultCollection(); validResponse.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId("groupId") .setErrorCode(Errors.NOT_COORDINATOR.code())); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( new DeleteGroupsResponseData() @@ -3027,7 +3090,7 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupsRetryBackoff() throws Exception { MockTime time = new MockTime(); int retryBackoff = 100; - final List groupIds = singletonList("group-0"); + final List groupIds = singletonList(GROUP_ID); try (final AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, mockCluster(3, 0), @@ -3043,7 +3106,7 @@ public class KafkaAdminClientTest { DeletableGroupResultCollection validResponse = new DeletableGroupResultCollection(); validResponse.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId(GROUP_ID) .setErrorCode(Errors.NOT_COORDINATOR.code())); @@ -3056,7 +3119,7 @@ public class KafkaAdminClientTest { validResponse = new DeletableGroupResultCollection(); validResponse.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId(GROUP_ID) .setErrorCode(Errors.NONE.code())); mockClient.prepareResponse(body -> { @@ -3078,21 +3141,31 @@ public class KafkaAdminClientTest { } @Test - public void testDeleteConsumerGroups() throws Exception { - final List groupIds = singletonList("group-0"); + public void testDeleteConsumerGroupsWithOlderBroker() throws Exception { + final List groupIds = singletonList("groupId"); + ApiVersion findCoordinatorV3 = new ApiVersion() + .setApiKey(ApiKeys.FIND_COORDINATOR.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 3); + ApiVersion describeGroups = new ApiVersion() + .setApiKey(ApiKeys.DESCRIBE_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion(ApiKeys.DELETE_GROUPS.latestVersion()); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Arrays.asList(findCoordinatorV3, describeGroups))); + // dummy response for MockCLient to handle the UnsupportedVersionException correctly + env.kafkaClient().prepareResponse(null); //Retriable FindCoordinatorResponse errors should be retried - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final DeletableGroupResultCollection validResponse = new DeletableGroupResultCollection(); validResponse.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId("groupId") .setErrorCode(Errors.NONE.code())); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( new DeleteGroupsResponseData() @@ -3101,23 +3174,27 @@ public class KafkaAdminClientTest { final DeleteConsumerGroupsResult result = env.adminClient().deleteConsumerGroups(groupIds); - final KafkaFuture results = result.deletedGroups().get("group-0"); + final KafkaFuture results = result.deletedGroups().get("groupId"); assertNull(results.get()); + // dummy response for MockCLient to handle the UnsupportedVersionException correctly + env.kafkaClient().prepareResponse(null); //should throw error for non-retriable errors env.kafkaClient().prepareResponse( - prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode())); + prepareOldFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode())); final DeleteConsumerGroupsResult errorResult = env.adminClient().deleteConsumerGroups(groupIds); - TestUtils.assertFutureError(errorResult.deletedGroups().get("group-0"), GroupAuthorizationException.class); + TestUtils.assertFutureError(errorResult.deletedGroups().get("groupId"), GroupAuthorizationException.class); + // dummy response for MockCLient to handle the UnsupportedVersionException correctly + env.kafkaClient().prepareResponse(null); //Retriable errors should be retried env.kafkaClient().prepareResponse( - prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final DeletableGroupResultCollection errorResponse1 = new DeletableGroupResultCollection(); errorResponse1.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId("groupId") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( @@ -3126,7 +3203,7 @@ public class KafkaAdminClientTest { final DeletableGroupResultCollection errorResponse2 = new DeletableGroupResultCollection(); errorResponse2.add(new DeletableGroupResult() - .setGroupId("group-0") + .setGroupId("groupId") .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) ); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( @@ -3140,13 +3217,13 @@ public class KafkaAdminClientTest { */ final DeletableGroupResultCollection coordinatorMoved = new DeletableGroupResultCollection(); coordinatorMoved.add(new DeletableGroupResult() - .setGroupId("UnitTestError") + .setGroupId("groupId") .setErrorCode(Errors.NOT_COORDINATOR.code()) ); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( new DeleteGroupsResponseData() .setResults(coordinatorMoved))); - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse(new DeleteGroupsResponse( new DeleteGroupsResponseData() @@ -3154,7 +3231,7 @@ public class KafkaAdminClientTest { final DeleteConsumerGroupsResult errorResult1 = env.adminClient().deleteConsumerGroups(groupIds); - final KafkaFuture errorResults = errorResult1.deletedGroups().get("group-0"); + final KafkaFuture errorResults = errorResult1.deletedGroups().get("groupId"); assertNull(errorResults.get()); } } @@ -3175,7 +3252,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final DeleteConsumerGroupOffsetsResult result = env.adminClient() - .deleteConsumerGroupOffsets("group-0", Stream.of(tp1).collect(Collectors.toSet())); + .deleteConsumerGroupOffsets("groupId", Stream.of(tp1).collect(Collectors.toSet())); TestUtils.assertFutureError(result.all(), TimeoutException.class); } @@ -3212,7 +3289,7 @@ public class KafkaAdminClientTest { return true; }, prepareOffsetDeleteResponse("foo", 0, Errors.NONE)); - final KafkaFuture future = env.adminClient().deleteConsumerGroupOffsets("group-0", Stream.of(tp1).collect(Collectors.toSet())).all(); + final KafkaFuture future = env.adminClient().deleteConsumerGroupOffsets(GROUP_ID, Stream.of(tp1).collect(Collectors.toSet())).all(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting DeleteConsumerGroupOffsets first request failure"); TestUtils.waitForCondition(() -> ((KafkaAdminClient) env.adminClient()).numPendingCalls() == 1, "Failed to add retry DeleteConsumerGroupOffsets call on first failure"); @@ -3229,7 +3306,6 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupOffsets() throws Exception { // Happy path - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); final TopicPartition tp2 = new TopicPartition("bar", 0); final TopicPartition tp3 = new TopicPartition("foobar", 0); @@ -3262,7 +3338,7 @@ public class KafkaAdminClientTest { ); final DeleteConsumerGroupOffsetsResult errorResult = env.adminClient().deleteConsumerGroupOffsets( - groupId, Stream.of(tp1, tp2).collect(Collectors.toSet())); + GROUP_ID, Stream.of(tp1, tp2).collect(Collectors.toSet())); assertNull(errorResult.partitionResult(tp1).get()); TestUtils.assertFutureError(errorResult.all(), GroupSubscribedToTopicException.class); @@ -3275,7 +3351,6 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupOffsetsRetriableErrors() throws Exception { // Retriable errors should be retried - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { @@ -3305,7 +3380,7 @@ public class KafkaAdminClientTest { prepareOffsetDeleteResponse("foo", 0, Errors.NONE)); final DeleteConsumerGroupOffsetsResult errorResult1 = env.adminClient() - .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + .deleteConsumerGroupOffsets(GROUP_ID, Stream.of(tp1).collect(Collectors.toSet())); assertNull(errorResult1.all().get()); assertNull(errorResult1.partitionResult(tp1).get()); @@ -3316,7 +3391,6 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupOffsetsNonRetriableErrors() throws Exception { // Non-retriable errors throw an exception - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); final List nonRetriableErrors = Arrays.asList( Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND); @@ -3332,7 +3406,7 @@ public class KafkaAdminClientTest { prepareOffsetDeleteResponse(error)); DeleteConsumerGroupOffsetsResult errorResult = env.adminClient() - .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + .deleteConsumerGroupOffsets(GROUP_ID, Stream.of(tp1).collect(Collectors.toSet())); TestUtils.assertFutureError(errorResult.all(), error.exception().getClass()); TestUtils.assertFutureError(errorResult.partitionResult(tp1), error.exception().getClass()); @@ -3344,14 +3418,13 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupOffsetsFindCoordinatorRetriableErrors() throws Exception { // Retriable FindCoordinatorResponse errors should be retried - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse( - prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); @@ -3362,7 +3435,7 @@ public class KafkaAdminClientTest { prepareOffsetDeleteResponse("foo", 0, Errors.NONE)); final DeleteConsumerGroupOffsetsResult result = env.adminClient() - .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + .deleteConsumerGroupOffsets(GROUP_ID, Stream.of(tp1).collect(Collectors.toSet())); assertNull(result.all().get()); assertNull(result.partitionResult(tp1).get()); @@ -3373,7 +3446,6 @@ public class KafkaAdminClientTest { public void testDeleteConsumerGroupOffsetsFindCoordinatorNonRetriableErrors() throws Exception { // Non-retriable FindCoordinatorResponse errors throw an exception - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { @@ -3383,7 +3455,7 @@ public class KafkaAdminClientTest { prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode())); final DeleteConsumerGroupOffsetsResult errorResult = env.adminClient() - .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + .deleteConsumerGroupOffsets(GROUP_ID, Stream.of(tp1).collect(Collectors.toSet())); TestUtils.assertFutureError(errorResult.all(), GroupAuthorizationException.class); TestUtils.assertFutureError(errorResult.partitionResult(tp1), GroupAuthorizationException.class); @@ -3503,7 +3575,7 @@ public class KafkaAdminClientTest { Collection membersToRemove = singletonList(new MemberToRemove("instance-1")); final KafkaFuture future = env.adminClient().removeMembersFromConsumerGroup( - "groupId", new RemoveMembersFromConsumerGroupOptions(membersToRemove)).all(); + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove)).all(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting RemoveMembersFromGroup first request failure"); @@ -3517,20 +3589,99 @@ public class KafkaAdminClientTest { } } + @Test + public void testRemoveMembersFromGroupRetriableErrors() throws Exception { + // Retriable errors should be retried + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + new LeaveGroupResponse(new LeaveGroupResponseData() + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()))); + + env.kafkaClient().prepareResponse( + new LeaveGroupResponse(new LeaveGroupResponseData() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()))); + + /* + * We need to return two responses here, one for NOT_COORDINATOR call when calling remove member + * api using coordinator that has moved. This will retry whole operation. So we need to again respond with a + * FindCoordinatorResponse. + */ + env.kafkaClient().prepareResponse( + new LeaveGroupResponse(new LeaveGroupResponseData() + .setErrorCode(Errors.NOT_COORDINATOR.code()))); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + MemberResponse memberResponse = new MemberResponse() + .setGroupInstanceId("instance-1") + .setErrorCode(Errors.NONE.code()); + env.kafkaClient().prepareResponse( + new LeaveGroupResponse(new LeaveGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setMembers(Collections.singletonList(memberResponse)))); + + MemberToRemove memberToRemove = new MemberToRemove("instance-1"); + Collection membersToRemove = singletonList(memberToRemove); + + final RemoveMembersFromConsumerGroupResult result = env.adminClient().removeMembersFromConsumerGroup( + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove)); + + assertNull(result.all().get()); + assertNull(result.memberResult(memberToRemove).get()); + } + } + + @Test + public void testRemoveMembersFromGroupNonRetriableErrors() throws Exception { + // Non-retriable errors throw an exception + + final List nonRetriableErrors = Arrays.asList( + Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + for (Errors error : nonRetriableErrors) { + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + new LeaveGroupResponse(new LeaveGroupResponseData() + .setErrorCode(error.code()))); + + MemberToRemove memberToRemove = new MemberToRemove("instance-1"); + Collection membersToRemove = singletonList(memberToRemove); + + final RemoveMembersFromConsumerGroupResult result = env.adminClient().removeMembersFromConsumerGroup( + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove)); + + TestUtils.assertFutureError(result.all(), error.exception().getClass()); + TestUtils.assertFutureError(result.memberResult(memberToRemove), error.exception().getClass()); + } + } + } + @Test public void testRemoveMembersFromGroup() throws Exception { try (AdminClientUnitTestEnv env = mockClientEnv()) { final String instanceOne = "instance-1"; final String instanceTwo = "instance-2"; + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); // Retriable FindCoordinatorResponse errors should be retried - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); - env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); // Retriable errors should be retried - env.kafkaClient().prepareResponse(null, true); env.kafkaClient().prepareResponse(new LeaveGroupResponse(new LeaveGroupResponseData() .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()))); env.kafkaClient().prepareResponse(new LeaveGroupResponse(new LeaveGroupResponseData() @@ -3540,18 +3691,16 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse(new LeaveGroupResponse(new LeaveGroupResponseData() .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()))); - String groupId = "groupId"; Collection membersToRemove = Arrays.asList(new MemberToRemove(instanceOne), new MemberToRemove(instanceTwo)); final RemoveMembersFromConsumerGroupResult unknownErrorResult = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); MemberToRemove memberOne = new MemberToRemove(instanceOne); MemberToRemove memberTwo = new MemberToRemove(instanceTwo); - TestUtils.assertFutureError(unknownErrorResult.all(), UnknownServerException.class); TestUtils.assertFutureError(unknownErrorResult.memberResult(memberOne), UnknownServerException.class); TestUtils.assertFutureError(unknownErrorResult.memberResult(memberTwo), UnknownServerException.class); @@ -3570,7 +3719,7 @@ public class KafkaAdminClientTest { .setMembers(Arrays.asList(responseOne, responseTwo)))); final RemoveMembersFromConsumerGroupResult memberLevelErrorResult = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); @@ -3585,7 +3734,7 @@ public class KafkaAdminClientTest { .setMembers(Collections.singletonList(responseTwo)))); final RemoveMembersFromConsumerGroupResult missingMemberResult = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); @@ -3605,7 +3754,7 @@ public class KafkaAdminClientTest { )); final RemoveMembersFromConsumerGroupResult noErrorResult = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); assertNull(noErrorResult.all().get()); @@ -3616,7 +3765,7 @@ public class KafkaAdminClientTest { final List topicPartitions = Arrays.asList(1, 2, 3).stream().map(partition -> new TopicPartition("my_topic", partition)) .collect(Collectors.toList()); // construct the DescribeGroupsResponse - DescribeGroupsResponseData data = prepareDescribeGroupsResponseData(groupId, Arrays.asList(instanceOne, instanceTwo), topicPartitions); + DescribeGroupsResponseData data = prepareDescribeGroupsResponseData(GROUP_ID, Arrays.asList(instanceOne, instanceTwo), topicPartitions); // Return with partial failure for "removeAll" scenario // 1 prepare response for AdminClient.describeConsumerGroups @@ -3630,7 +3779,7 @@ public class KafkaAdminClientTest { Arrays.asList(responseOne, responseTwo)) )); final RemoveMembersFromConsumerGroupResult partialFailureResults = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions() ); ExecutionException exception = assertThrows(ExecutionException.class, () -> partialFailureResults.all().get()); @@ -3651,7 +3800,7 @@ public class KafkaAdminClientTest { )) )); final RemoveMembersFromConsumerGroupResult successResult = env.adminClient().removeMembersFromConsumerGroup( - groupId, + GROUP_ID, new RemoveMembersFromConsumerGroupOptions() ); assertNull(successResult.all().get()); @@ -3869,7 +4018,6 @@ public class KafkaAdminClientTest { public void testAlterConsumerGroupOffsets() throws Exception { // Happy path - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); final TopicPartition tp2 = new TopicPartition("bar", 0); final TopicPartition tp3 = new TopicPartition("foobar", 0); @@ -3889,7 +4037,7 @@ public class KafkaAdminClientTest { offsets.put(tp1, new OffsetAndMetadata(123L)); offsets.put(tp2, new OffsetAndMetadata(456L)); final AlterConsumerGroupOffsetsResult result = env.adminClient().alterConsumerGroupOffsets( - groupId, offsets); + GROUP_ID, offsets); assertNull(result.all().get()); assertNull(result.partitionResult(tp1).get()); @@ -3902,7 +4050,6 @@ public class KafkaAdminClientTest { public void testAlterConsumerGroupOffsetsRetriableErrors() throws Exception { // Retriable errors should be retried - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { @@ -3935,7 +4082,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); final AlterConsumerGroupOffsetsResult result1 = env.adminClient() - .alterConsumerGroupOffsets(groupId, offsets); + .alterConsumerGroupOffsets(GROUP_ID, offsets); assertNull(result1.all().get()); assertNull(result1.partitionResult(tp1).get()); @@ -3946,7 +4093,6 @@ public class KafkaAdminClientTest { public void testAlterConsumerGroupOffsetsNonRetriableErrors() throws Exception { // Non-retriable errors throw an exception - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); final List nonRetriableErrors = Arrays.asList( Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND); @@ -3963,7 +4109,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); AlterConsumerGroupOffsetsResult errorResult = env.adminClient() - .alterConsumerGroupOffsets(groupId, offsets); + .alterConsumerGroupOffsets(GROUP_ID, offsets); TestUtils.assertFutureError(errorResult.all(), error.exception().getClass()); TestUtils.assertFutureError(errorResult.partitionResult(tp1), error.exception().getClass()); @@ -3975,14 +4121,13 @@ public class KafkaAdminClientTest { public void testAlterConsumerGroupOffsetsFindCoordinatorRetriableErrors() throws Exception { // Retriable FindCoordinatorResponse errors should be retried - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse( - prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); @@ -3995,7 +4140,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); final AlterConsumerGroupOffsetsResult result = env.adminClient() - .alterConsumerGroupOffsets(groupId, offsets); + .alterConsumerGroupOffsets(GROUP_ID, offsets); assertNull(result.all().get()); assertNull(result.partitionResult(tp1).get()); @@ -4006,7 +4151,6 @@ public class KafkaAdminClientTest { public void testAlterConsumerGroupOffsetsFindCoordinatorNonRetriableErrors() throws Exception { // Non-retriable FindCoordinatorResponse errors throw an exception - final String groupId = "group-0"; final TopicPartition tp1 = new TopicPartition("foo", 0); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { @@ -4018,7 +4162,7 @@ public class KafkaAdminClientTest { Map offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(123L)); final AlterConsumerGroupOffsetsResult errorResult = env.adminClient() - .alterConsumerGroupOffsets(groupId, offsets); + .alterConsumerGroupOffsets(GROUP_ID, offsets); TestUtils.assertFutureError(errorResult.all(), GroupAuthorizationException.class); TestUtils.assertFutureError(errorResult.partitionResult(tp1), GroupAuthorizationException.class); @@ -5359,46 +5503,6 @@ public class KafkaAdminClientTest { } } - @Test - public void testHasCoordinatorMoved() { - Map errors = new HashMap<>(); - AbstractResponse response = new AbstractResponse(ApiKeys.OFFSET_COMMIT) { - @Override - public Map errorCounts() { - return errors; - } - - @Override - public ApiMessage data() { - return null; - } - - @Override - public int throttleTimeMs() { - return DEFAULT_THROTTLE_TIME; - } - }; - - assertFalse(ConsumerGroupOperationContext.hasCoordinatorMoved(response)); - - errors.put(Errors.NOT_COORDINATOR, 1); - assertTrue(ConsumerGroupOperationContext.hasCoordinatorMoved(response)); - } - - @Test - public void testShouldRefreshCoordinator() { - Map errorCounts = new HashMap<>(); - - assertFalse(ConsumerGroupOperationContext.shouldRefreshCoordinator(errorCounts)); - - errorCounts.put(Errors.COORDINATOR_LOAD_IN_PROGRESS, 1); - assertTrue(ConsumerGroupOperationContext.shouldRefreshCoordinator(errorCounts)); - - errorCounts.clear(); - errorCounts.put(Errors.COORDINATOR_NOT_AVAILABLE, 1); - assertTrue(ConsumerGroupOperationContext.shouldRefreshCoordinator(errorCounts)); - } - @Test public void testUnregisterBrokerSuccess() throws InterruptedException, ExecutionException { int nodeId = 1; @@ -5621,11 +5725,7 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse( request -> request instanceof FindCoordinatorRequest, - new FindCoordinatorResponse(new FindCoordinatorResponseData() - .setErrorCode(Errors.NONE.code()) - .setNodeId(coordinator.id()) - .setHost(coordinator.host()) - .setPort(coordinator.port())) + prepareFindCoordinatorResponse(Errors.NONE, transactionalId, coordinator) ); env.kafkaClient().prepareResponseFrom( @@ -5667,10 +5767,12 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse( request -> request instanceof FindCoordinatorRequest, new FindCoordinatorResponse(new FindCoordinatorResponseData() - .setErrorCode(Errors.NONE.code()) - .setNodeId(coordinator1.id()) - .setHost(coordinator1.host()) - .setPort(coordinator1.port())) + .setCoordinators(Arrays.asList(new FindCoordinatorResponseData.Coordinator() + .setKey(transactionalId) + .setErrorCode(Errors.NONE.code()) + .setNodeId(coordinator1.id()) + .setHost(coordinator1.host()) + .setPort(coordinator1.port())))) ); env.kafkaClient().prepareResponseFrom( @@ -5695,10 +5797,12 @@ public class KafkaAdminClientTest { env.kafkaClient().prepareResponse( request -> request instanceof FindCoordinatorRequest, new FindCoordinatorResponse(new FindCoordinatorResponseData() - .setErrorCode(Errors.NONE.code()) - .setNodeId(coordinator2.id()) - .setHost(coordinator2.host()) - .setPort(coordinator2.port())) + .setCoordinators(Arrays.asList(new FindCoordinatorResponseData.Coordinator() + .setKey(transactionalId) + .setErrorCode(Errors.NONE.code()) + .setNodeId(coordinator2.id()) + .setHost(coordinator2.host()) + .setPort(coordinator2.port())))) ); TransactionDescription expected = new TransactionDescription( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java index aac18cd50bc..78b33a7f59d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.AbortTransactionSpec; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidProducerEpochException; @@ -46,6 +47,7 @@ public class AbortTransactionHandlerTest { private final TopicPartition topicPartition = new TopicPartition("foo", 5); private final AbortTransactionSpec abortSpec = new AbortTransactionSpec( topicPartition, 12345L, (short) 15, 4321); + private final Node node = new Node(1, "host", 1234); @Test public void testInvalidBuildRequestCall() { @@ -79,11 +81,11 @@ public class AbortTransactionHandlerTest { public void testInvalidHandleResponseCall() { AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); - assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node, emptySet(), new WriteTxnMarkersResponse(response))); - assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node, mkSet(new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response))); - assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node, mkSet(topicPartition, new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response))); } @@ -92,43 +94,43 @@ public class AbortTransactionHandlerTest { AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = new WriteTxnMarkersResponseData.WritableTxnMarkerResult(); response.markers().add(markerResponse); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); markerResponse.setProducerId(abortSpec.producerId()); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult(); markerResponse.topics().add(topicResponse); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); topicResponse.setName(abortSpec.topicPartition().topic()); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult(); topicResponse.partitions().add(partitionResponse); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); partitionResponse.setPartitionIndex(abortSpec.topicPartition().partition()); topicResponse.setName(abortSpec.topicPartition().topic() + "random"); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); topicResponse.setName(abortSpec.topicPartition().topic()); markerResponse.setProducerId(abortSpec.producerId() + 1); - assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(node, singleton(topicPartition), new WriteTxnMarkersResponse(response))); } @@ -181,7 +183,7 @@ public class AbortTransactionHandlerTest { WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); response.markers().add(markerResponse); - return handler.handleResponse(1, singleton(abortSpec.topicPartition()), + return handler.handleResponse(node, singleton(abortSpec.topicPartition()), new WriteTxnMarkersResponse(response)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java index a78b73a0fb6..93a4fa84bea 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java @@ -19,6 +19,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.internals.AdminApiDriver.RequestSpec; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; import org.apache.kafka.clients.admin.internals.AdminApiLookupStrategy.LookupResult; +import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -26,6 +27,7 @@ import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.LogContext; @@ -35,6 +37,7 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -42,6 +45,7 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static org.apache.kafka.common.utils.Utils.mkSet; @@ -321,10 +325,46 @@ class AdminApiDriverTest { assertEquals(1, retryLookupSpecs.size()); RequestSpec retryLookupSpec = retryLookupSpecs.get(0); - assertEquals(ctx.time.milliseconds() + RETRY_BACKOFF_MS, retryLookupSpec.nextAllowedTryMs); + assertEquals(ctx.time.milliseconds(), retryLookupSpec.nextAllowedTryMs); assertEquals(1, retryLookupSpec.tries); } + @Test + public void testRetryLookupAndDisableBatchAfterNoBatchedFindCoordinatorsException() { + MockTime time = new MockTime(); + LogContext lc = new LogContext(); + Set groupIds = new HashSet<>(Arrays.asList("g1", "g2")); + DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(lc); + AdminApiFuture future = AdminApiFuture.forKeys( + groupIds.stream().map(g -> CoordinatorKey.byGroupId(g)).collect(Collectors.toSet())); + + AdminApiDriver driver = new AdminApiDriver<>( + handler, + future, + time.milliseconds() + API_TIMEOUT_MS, + RETRY_BACKOFF_MS, + new LogContext() + ); + + assertTrue(((CoordinatorStrategy) handler.lookupStrategy()).batch); + List> requestSpecs = driver.poll(); + // Expect CoordinatorStrategy to try resolving all coordinators in a single request + assertEquals(1, requestSpecs.size()); + + RequestSpec requestSpec = requestSpecs.get(0); + driver.onFailure(time.milliseconds(), requestSpec, new NoBatchedFindCoordinatorsException("message")); + assertFalse(((CoordinatorStrategy) handler.lookupStrategy()).batch); + + // Batching is now disabled, so we now have a request per groupId + List> retryLookupSpecs = driver.poll(); + assertEquals(groupIds.size(), retryLookupSpecs.size()); + // These new requests are treated a new requests and not retries + for (RequestSpec retryLookupSpec : retryLookupSpecs) { + assertEquals(0, retryLookupSpec.nextAllowedTryMs); + assertEquals(0, retryLookupSpec.tries); + } + } + @Test public void testCoalescedStaticAndDynamicFulfillment() { Map dynamicMapping = map( @@ -417,7 +457,7 @@ class AdminApiDriverTest { RequestSpec retrySpec = retrySpecs.get(0); assertEquals(1, retrySpec.tries); - assertEquals(ctx.time.milliseconds() + RETRY_BACKOFF_MS, retrySpec.nextAllowedTryMs); + assertEquals(ctx.time.milliseconds(), retrySpec.nextAllowedTryMs); } @Test @@ -433,7 +473,7 @@ class AdminApiDriverTest { RequestSpec requestSpec = requestSpecs.get(0); assertEquals(0, requestSpec.tries); assertEquals(0L, requestSpec.nextAllowedTryMs); - ctx.assertResponse(requestSpec, emptyFulfillment); + ctx.assertResponse(requestSpec, emptyFulfillment, Node.noNode()); List> retrySpecs = ctx.driver.poll(); assertEquals(1, retrySpecs.size()); @@ -580,7 +620,7 @@ class AdminApiDriverTest { // The response is just a placeholder. The result is all we are interested in MetadataResponse response = new MetadataResponse(new MetadataResponseData(), ApiKeys.METADATA.latestVersion()); - driver.onResponse(time.milliseconds(), requestSpec, response); + driver.onResponse(time.milliseconds(), requestSpec, response, Node.noNode()); result.mappedKeys.forEach((key, brokerId) -> { assertMappedKey(this, key, brokerId); @@ -593,7 +633,8 @@ class AdminApiDriverTest { private void assertResponse( RequestSpec requestSpec, - ApiResult result + ApiResult result, + Node node ) { int brokerId = requestSpec.scope.destinationBrokerId().orElseThrow(() -> new AssertionError("Fulfillment requests must specify a target brokerId")); @@ -606,7 +647,7 @@ class AdminApiDriverTest { MetadataResponse response = new MetadataResponse(new MetadataResponseData(), ApiKeys.METADATA.latestVersion()); - driver.onResponse(time.milliseconds(), requestSpec, response); + driver.onResponse(time.milliseconds(), requestSpec, response, node); result.unmappedKeys.forEach(key -> { assertUnmappedKey(this, key); @@ -649,7 +690,7 @@ class AdminApiDriverTest { assertLookupResponse(requestSpec, result); } else if (expectedRequests.containsKey(keys)) { ApiResult result = expectedRequests.get(keys); - assertResponse(requestSpec, result); + assertResponse(requestSpec, result, Node.noNode()); } else { fail("Unexpected request for keys " + keys); } @@ -723,7 +764,7 @@ class AdminApiDriverTest { } @Override - public ApiResult handleResponse(int brokerId, Set keys, AbstractResponse response) { + public ApiResult handleResponse(Node broker, Set keys, AbstractResponse response) { return Optional.ofNullable(expectedRequests.get(keys)).orElseThrow(() -> new AssertionError("Unexpected fulfillment request for keys " + keys) ); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AllBrokersStrategyIntegrationTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AllBrokersStrategyIntegrationTest.java index f4ec4680026..2b989058082 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AllBrokersStrategyIntegrationTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AllBrokersStrategyIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -96,7 +97,7 @@ public class AllBrokersStrategyIntegrationTest { AdminApiDriver.RequestSpec retrySpec = retrySpecs.get(0); assertEquals(AllBrokersStrategy.LOOKUP_KEYS, retrySpec.keys); - assertEquals(time.milliseconds() + RETRY_BACKOFF_MS, retrySpec.nextAllowedTryMs); + assertEquals(time.milliseconds(), retrySpec.nextAllowedTryMs); assertEquals(Collections.emptyList(), driver.poll()); } @@ -110,7 +111,7 @@ public class AllBrokersStrategyIntegrationTest { AdminApiDriver.RequestSpec lookupSpec = lookupSpecs.get(0); Set brokerIds = Utils.mkSet(1, 2); - driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(brokerIds)); + driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(brokerIds), Node.noNode()); assertTrue(result.all().isDone()); Map> brokerFutures = result.all().get(); @@ -123,7 +124,7 @@ public class AllBrokersStrategyIntegrationTest { int brokerId1 = requestSpec1.scope.destinationBrokerId().getAsInt(); assertTrue(brokerIds.contains(brokerId1)); - driver.onResponse(time.milliseconds(), requestSpec1, null); + driver.onResponse(time.milliseconds(), requestSpec1, null, Node.noNode()); KafkaFutureImpl future1 = brokerFutures.get(brokerId1); assertTrue(future1.isDone()); @@ -133,7 +134,7 @@ public class AllBrokersStrategyIntegrationTest { assertNotEquals(brokerId1, brokerId2); assertTrue(brokerIds.contains(brokerId2)); - driver.onResponse(time.milliseconds(), requestSpec2, null); + driver.onResponse(time.milliseconds(), requestSpec2, null, Node.noNode()); KafkaFutureImpl future2 = brokerFutures.get(brokerId2); assertTrue(future2.isDone()); assertEquals(Collections.emptyList(), driver.poll()); @@ -149,7 +150,7 @@ public class AllBrokersStrategyIntegrationTest { AdminApiDriver.RequestSpec lookupSpec = lookupSpecs.get(0); int brokerId = 1; - driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(Collections.singleton(brokerId))); + driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(Collections.singleton(brokerId)), Node.noNode()); assertTrue(result.all().isDone()); Map> brokerFutures = result.all().get(); @@ -169,7 +170,7 @@ public class AllBrokersStrategyIntegrationTest { assertEquals(time.milliseconds() + RETRY_BACKOFF_MS, retrySpec.nextAllowedTryMs); assertEquals(OptionalInt.of(brokerId), retrySpec.scope.destinationBrokerId()); - driver.onResponse(time.milliseconds(), retrySpec, null); + driver.onResponse(time.milliseconds(), retrySpec, null, new Node(brokerId, "host", 1234)); assertTrue(future.isDone()); assertEquals(brokerId, future.get()); assertEquals(Collections.emptyList(), driver.poll()); @@ -185,7 +186,7 @@ public class AllBrokersStrategyIntegrationTest { AdminApiDriver.RequestSpec lookupSpec = lookupSpecs.get(0); int brokerId = 1; - driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(Collections.singleton(brokerId))); + driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(Collections.singleton(brokerId)), Node.noNode()); assertTrue(result.all().isDone()); Map> brokerFutures = result.all().get(); @@ -232,11 +233,11 @@ public class AllBrokersStrategyIntegrationTest { @Override public ApiResult handleResponse( - int brokerId, + Node broker, Set keys, AbstractResponse response ) { - return ApiResult.completed(keys.iterator().next(), brokerId); + return ApiResult.completed(keys.iterator().next(), broker.id()); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandlerTest.java new file mode 100644 index 00000000000..c20107f67aa --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandlerTest.java @@ -0,0 +1,134 @@ +/* + * 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 static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.utils.LogContext; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class AlterConsumerGroupOffsetsHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId = "group-id"; + private final TopicPartition t0p0 = new TopicPartition("t0", 0); + private final TopicPartition t0p1 = new TopicPartition("t0", 1); + private final TopicPartition t1p0 = new TopicPartition("t1", 0); + private final TopicPartition t1p1 = new TopicPartition("t1", 1); + private final Map partitions = new HashMap<>(); + private final long offset = 1L; + private final Node node = new Node(1, "host", 1234); + + @BeforeEach + public void setUp() { + partitions.put(t0p0, new OffsetAndMetadata(offset)); + partitions.put(t0p1, new OffsetAndMetadata(offset)); + partitions.put(t1p0, new OffsetAndMetadata(offset)); + partitions.put(t1p1, new OffsetAndMetadata(offset)); + } + + @Test + public void testBuildRequest() { + AlterConsumerGroupOffsetsHandler handler = new AlterConsumerGroupOffsetsHandler(groupId, partitions, logContext); + OffsetCommitRequest request = handler.buildRequest(-1, singleton(CoordinatorKey.byGroupId(groupId))).build(); + assertEquals(groupId, request.data().groupId()); + assertEquals(2, request.data().topics().size()); + assertEquals(2, request.data().topics().get(0).partitions().size()); + assertEquals(offset, request.data().topics().get(0).partitions().get(0).committedOffset()); + } + + @Test + public void testSuccessfulHandleResponse() { + AlterConsumerGroupOffsetsHandler handler = new AlterConsumerGroupOffsetsHandler(groupId, partitions, logContext); + Map responseData = Collections.singletonMap(t0p0, Errors.NONE); + OffsetCommitResponse response = new OffsetCommitResponse(0, responseData); + ApiResult> result = handler.handleResponse(node, singleton(CoordinatorKey.byGroupId(groupId)), response); + assertCompleted(result, responseData); + } + + @Test + public void testRetriableHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); + assertUnmapped(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + assertUnmapped(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); + assertFailed(UnknownServerException.class, handleWithError(Errors.UNKNOWN_SERVER_ERROR)); + } + + private AdminApiHandler.ApiResult> handleWithError( + Errors error + ) { + AlterConsumerGroupOffsetsHandler handler = new AlterConsumerGroupOffsetsHandler(groupId, partitions, logContext); + Map responseData = Collections.singletonMap(t0p0, error); + OffsetCommitResponse response = new OffsetCommitResponse(0, responseData); + return handler.handleResponse(node, singleton(CoordinatorKey.byGroupId(groupId)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult> result, + Map expected + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + assertEquals(expected, result.completedKeys.get(key)); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult> result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java index ea4239c4f58..dd83c6be199 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java @@ -25,7 +25,12 @@ import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import static java.util.Collections.emptyMap; import static java.util.Collections.singleton; @@ -38,8 +43,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class CoordinatorStrategyTest { @Test - public void testBuildLookupRequest() { - CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext()); + public void testBuildOldLookupRequest() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + strategy.disableBatch(); FindCoordinatorRequest.Builder request = strategy.buildRequest(singleton( CoordinatorKey.byGroupId("foo"))); assertEquals("foo", request.data().key()); @@ -47,8 +53,30 @@ public class CoordinatorStrategyTest { } @Test - public void testBuildLookupRequestRequiresOneKey() { - CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext()); + public void testBuildLookupRequest() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + FindCoordinatorRequest.Builder request = strategy.buildRequest(new HashSet<>(Arrays.asList( + CoordinatorKey.byGroupId("foo"), + CoordinatorKey.byGroupId("bar")))); + assertEquals("", request.data().key()); + assertEquals(2, request.data().coordinatorKeys().size()); + assertEquals(CoordinatorType.GROUP, CoordinatorType.forId(request.data().keyType())); + } + + @Test + public void testBuildLookupRequestNonRepresentable() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + FindCoordinatorRequest.Builder request = strategy.buildRequest(new HashSet<>(Arrays.asList( + CoordinatorKey.byGroupId("foo"), + null))); + assertEquals("", request.data().key()); + assertEquals(1, request.data().coordinatorKeys().size()); + } + + @Test + public void testBuildOldLookupRequestRequiresOneKey() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + strategy.disableBatch(); assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Collections.emptySet())); CoordinatorKey group1 = CoordinatorKey.byGroupId("foo"); @@ -57,11 +85,38 @@ public class CoordinatorStrategyTest { } @Test - public void testHandleResponseRequiresOneKey() { + public void testBuildOldLookupRequestRequiresAtLeastOneKey() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + strategy.disableBatch(); + + assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest( + new HashSet<>(Arrays.asList(CoordinatorKey.byTransactionalId("txnid"))))); + } + + @Test + public void testBuildLookupRequestRequiresAtLeastOneKey() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + + assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Collections.emptySet())); + } + + @Test + public void testBuildLookupRequestRequiresKeySameType() { + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + + assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest( + new HashSet<>(Arrays.asList( + CoordinatorKey.byGroupId("group"), + CoordinatorKey.byTransactionalId("txnid"))))); + } + + @Test + public void testHandleOldResponseRequiresOneKey() { FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(Errors.NONE.code()); FindCoordinatorResponse response = new FindCoordinatorResponse(responseData); - CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext()); + CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext()); + strategy.disableBatch(); assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(Collections.emptySet(), response)); @@ -72,7 +127,7 @@ public class CoordinatorStrategyTest { } @Test - public void testSuccessfulCoordinatorLookup() { + public void testSuccessfulOldCoordinatorLookup() { CoordinatorKey group = CoordinatorKey.byGroupId("foo"); FindCoordinatorResponseData responseData = new FindCoordinatorResponseData() @@ -81,11 +136,54 @@ public class CoordinatorStrategyTest { .setPort(9092) .setNodeId(1); - AdminApiLookupStrategy.LookupResult result = runLookup(group, responseData); + AdminApiLookupStrategy.LookupResult result = runOldLookup(group, responseData); assertEquals(singletonMap(group, 1), result.mappedKeys); assertEquals(emptyMap(), result.failedKeys); } + @Test + public void testSuccessfulCoordinatorLookup() { + CoordinatorKey group1 = CoordinatorKey.byGroupId("foo"); + CoordinatorKey group2 = CoordinatorKey.byGroupId("bar"); + + FindCoordinatorResponseData responseData = new FindCoordinatorResponseData() + .setCoordinators(Arrays.asList( + new FindCoordinatorResponseData.Coordinator() + .setKey("foo") + .setErrorCode(Errors.NONE.code()) + .setHost("localhost") + .setPort(9092) + .setNodeId(1), + new FindCoordinatorResponseData.Coordinator() + .setKey("bar") + .setErrorCode(Errors.NONE.code()) + .setHost("localhost") + .setPort(9092) + .setNodeId(2))); + + AdminApiLookupStrategy.LookupResult result = runLookup(new HashSet<>(Arrays.asList(group1, group2)), responseData); + Map expectedResult = new HashMap<>(); + expectedResult.put(group1, 1); + expectedResult.put(group2, 2); + assertEquals(expectedResult, result.mappedKeys); + assertEquals(emptyMap(), result.failedKeys); + } + + @Test + public void testRetriableOldCoordinatorLookup() { + testRetriableOldCoordinatorLookup(Errors.COORDINATOR_LOAD_IN_PROGRESS); + testRetriableOldCoordinatorLookup(Errors.COORDINATOR_NOT_AVAILABLE); + } + + private void testRetriableOldCoordinatorLookup(Errors error) { + CoordinatorKey group = CoordinatorKey.byGroupId("foo"); + FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code()); + AdminApiLookupStrategy.LookupResult result = runOldLookup(group, responseData); + + assertEquals(emptyMap(), result.failedKeys); + assertEquals(emptyMap(), result.mappedKeys); + } + @Test public void testRetriableCoordinatorLookup() { testRetriableCoordinatorLookup(Errors.COORDINATOR_LOAD_IN_PROGRESS); @@ -93,12 +191,50 @@ public class CoordinatorStrategyTest { } private void testRetriableCoordinatorLookup(Errors error) { - CoordinatorKey group = CoordinatorKey.byGroupId("foo"); - FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code()); - AdminApiLookupStrategy.LookupResult result = runLookup(group, responseData); + CoordinatorKey group1 = CoordinatorKey.byGroupId("foo"); + CoordinatorKey group2 = CoordinatorKey.byGroupId("bar"); + FindCoordinatorResponseData responseData = new FindCoordinatorResponseData() + .setCoordinators(Arrays.asList( + new FindCoordinatorResponseData.Coordinator() + .setKey("foo") + .setErrorCode(error.code()), + new FindCoordinatorResponseData.Coordinator() + .setKey("bar") + .setErrorCode(Errors.NONE.code()) + .setHost("localhost") + .setPort(9092) + .setNodeId(2))); + AdminApiLookupStrategy.LookupResult result = runLookup(new HashSet<>(Arrays.asList(group1, group2)), responseData); assertEquals(emptyMap(), result.failedKeys); + assertEquals(singletonMap(group2, 2), result.mappedKeys); + } + + @Test + public void testFatalErrorOldLookupResponses() { + CoordinatorKey group = CoordinatorKey.byTransactionalId("foo"); + assertFatalOldLookup(group, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); + assertFatalOldLookup(group, Errors.UNKNOWN_SERVER_ERROR); + + Throwable throwable = assertFatalOldLookup(group, Errors.GROUP_AUTHORIZATION_FAILED); + assertTrue(throwable instanceof GroupAuthorizationException); + GroupAuthorizationException exception = (GroupAuthorizationException) throwable; + assertEquals("foo", exception.groupId()); + } + + public Throwable assertFatalOldLookup( + CoordinatorKey key, + Errors error + ) { + FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code()); + AdminApiLookupStrategy.LookupResult result = runOldLookup(key, responseData); + assertEquals(emptyMap(), result.mappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + + Throwable throwable = result.failedKeys.get(key); + assertTrue(error.exception().getClass().isInstance(throwable)); + return throwable; } @Test @@ -117,8 +253,12 @@ public class CoordinatorStrategyTest { CoordinatorKey key, Errors error ) { - FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code()); - AdminApiLookupStrategy.LookupResult result = runLookup(key, responseData); + FindCoordinatorResponseData responseData = new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setKey(key.idValue) + .setErrorCode(error.code()))); + AdminApiLookupStrategy.LookupResult result = runLookup(singleton(key), responseData); assertEquals(emptyMap(), result.mappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); @@ -128,13 +268,24 @@ public class CoordinatorStrategyTest { return throwable; } - private AdminApiLookupStrategy.LookupResult runLookup( + private AdminApiLookupStrategy.LookupResult runOldLookup( CoordinatorKey key, FindCoordinatorResponseData responseData ) { - CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext()); + CoordinatorStrategy strategy = new CoordinatorStrategy(key.type, new LogContext()); + strategy.disableBatch(); FindCoordinatorResponse response = new FindCoordinatorResponse(responseData); return strategy.handleResponse(singleton(key), response); } + private AdminApiLookupStrategy.LookupResult runLookup( + Set keys, + FindCoordinatorResponseData responseData + ) { + CoordinatorStrategy strategy = new CoordinatorStrategy(keys.iterator().next().type, new LogContext()); + strategy.buildRequest(keys); + FindCoordinatorResponse response = new FindCoordinatorResponse(responseData); + return strategy.handleResponse(keys, response); + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java new file mode 100644 index 00000000000..439b37733d9 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java @@ -0,0 +1,152 @@ +/* + * 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 static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.OffsetDeleteResponseData; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.OffsetDeleteRequest; +import org.apache.kafka.common.requests.OffsetDeleteResponse; +import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.Test; + +public class DeleteConsumerGroupOffsetsHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId = "group-id"; + private final TopicPartition t0p0 = new TopicPartition("t0", 0); + private final TopicPartition t0p1 = new TopicPartition("t0", 1); + private final TopicPartition t1p0 = new TopicPartition("t1", 0); + private final Set tps = new HashSet<>(Arrays.asList(t0p0, t0p1, t1p0)); + + @Test + public void testBuildRequest() { + DeleteConsumerGroupOffsetsHandler handler = new DeleteConsumerGroupOffsetsHandler(groupId, tps, logContext); + OffsetDeleteRequest request = handler.buildRequest(1, singleton(CoordinatorKey.byGroupId(groupId))).build(); + assertEquals(groupId, request.data().groupId()); + assertEquals(2, request.data().topics().size()); + assertEquals(2, request.data().topics().find("t0").partitions().size()); + assertEquals(1, request.data().topics().find("t1").partitions().size()); + } + + @Test + public void testSuccessfulHandleResponse() { + Map responseData = Collections.singletonMap(t0p0, Errors.NONE); + assertCompleted(handleWithError(Errors.NONE), responseData); + } + + @Test + public void testUnmappedHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); + } + + @Test + public void testRetriableHandleResponse() { + assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + assertRetriable(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); + assertFailed(GroupIdNotFoundException.class, handleWithError(Errors.GROUP_ID_NOT_FOUND)); + assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID)); + } + + private OffsetDeleteResponse buildResponse(Errors error) { + OffsetDeleteResponse response = new OffsetDeleteResponse( + new OffsetDeleteResponseData() + .setThrottleTimeMs(0) + .setTopics(new OffsetDeleteResponseTopicCollection(singletonList( + new OffsetDeleteResponseTopic() + .setName("t0") + .setPartitions(new OffsetDeleteResponsePartitionCollection(singletonList( + new OffsetDeleteResponsePartition() + .setPartitionIndex(0) + .setErrorCode(error.code()) + ).iterator())) + ).iterator()))); + return response; + } + + private AdminApiHandler.ApiResult> handleWithError( + Errors error + ) { + DeleteConsumerGroupOffsetsHandler handler = new DeleteConsumerGroupOffsetsHandler(groupId, tps, logContext); + OffsetDeleteResponse response = buildResponse(error); + return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + } + + private void assertRetriable( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult> result, + Map expected + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + assertEquals(expected, result.completedKeys.get(key)); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult> result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java new file mode 100644 index 00000000000..30a65f4c8fc --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java @@ -0,0 +1,128 @@ +/* + * 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 static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.DeleteGroupsResponseData; +import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult; +import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResultCollection; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.DeleteGroupsRequest; +import org.apache.kafka.common.requests.DeleteGroupsResponse; +import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.Test; + +public class DeleteConsumerGroupsHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId1 = "group-id1"; + + @Test + public void testBuildRequest() { + DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(logContext); + DeleteGroupsRequest request = handler.buildRequest(1, singleton(CoordinatorKey.byGroupId(groupId1))).build(); + assertEquals(1, request.data().groupsNames().size()); + assertEquals(groupId1, request.data().groupsNames().get(0)); + } + + @Test + public void testSuccessfulHandleResponse() { + assertCompleted(handleWithError(Errors.NONE)); + } + + @Test + public void testUnmappedHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); + } + + @Test + public void testRetriableHandleResponse() { + assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + assertRetriable(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); + assertFailed(GroupIdNotFoundException.class, handleWithError(Errors.GROUP_ID_NOT_FOUND)); + assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID)); + } + + private DeleteGroupsResponse buildResponse(Errors error) { + DeleteGroupsResponse response = new DeleteGroupsResponse( + new DeleteGroupsResponseData() + .setResults(new DeletableGroupResultCollection(singletonList( + new DeletableGroupResult() + .setErrorCode(error.code()) + .setGroupId(groupId1)).iterator()))); + return response; + } + + private AdminApiHandler.ApiResult handleWithError( + Errors error + ) { + DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(logContext); + DeleteGroupsResponse response = buildResponse(error); + return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId1)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId1)), result.unmappedKeys); + } + + private void assertRetriable( + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId1); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId1); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java new file mode 100644 index 00000000000..fe260436137 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java @@ -0,0 +1,192 @@ +/* + * 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 static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.MemberAssignment; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; +import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.DescribeGroupsResponseData; +import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; +import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.DescribeGroupsRequest; +import org.apache.kafka.common.requests.DescribeGroupsResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; +import org.junit.jupiter.api.Test; + +public class DescribeConsumerGroupsHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId1 = "group-id1"; + private final String groupId2 = "group-id2"; + private final Set groupIds = new HashSet<>(Arrays.asList(groupId1, groupId2)); + private final Set keys = groupIds.stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + private final Node coordinator = new Node(1, "host", 1234); + private final Set tps = new HashSet<>(Arrays.asList( + new TopicPartition("foo", 0), new TopicPartition("bar", 1))); + + @Test + public void testBuildRequest() { + DescribeConsumerGroupsHandler handler = new DescribeConsumerGroupsHandler(false, logContext); + DescribeGroupsRequest request = handler.buildRequest(1, keys).build(); + assertEquals(2, request.data().groups().size()); + assertFalse(request.data().includeAuthorizedOperations()); + + handler = new DescribeConsumerGroupsHandler(true, logContext); + request = handler.buildRequest(1, keys).build(); + assertEquals(2, request.data().groups().size()); + assertTrue(request.data().includeAuthorizedOperations()); + } + + @Test + public void testInvalidBuildRequest() { + DescribeConsumerGroupsHandler handler = new DescribeConsumerGroupsHandler(false, logContext); + assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1, singleton(CoordinatorKey.byTransactionalId("tId")))); + } + + @Test + public void testSuccessfulHandleResponse() { + Collection members = singletonList(new MemberDescription( + "memberId", + "clientId", + "host", + new MemberAssignment(tps))); + ConsumerGroupDescription expected = new ConsumerGroupDescription( + groupId1, + true, + members, + "assignor", + ConsumerGroupState.STABLE, + coordinator); + assertCompleted(handleWithError(Errors.NONE, ""), expected); + } + + @Test + public void testUnmappedHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR, "")); + } + + @Test + public void testRetriableHandleResponse() { + assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS, "")); + assertRetriable(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE, "")); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED, "")); + assertFailed(GroupIdNotFoundException.class, handleWithError(Errors.GROUP_ID_NOT_FOUND, "")); + assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID, "")); + assertFailed(IllegalArgumentException.class, handleWithError(Errors.NONE, "custom-protocol")); + } + + private DescribeGroupsResponse buildResponse(Errors error, String protocolType) { + DescribeGroupsResponse response = new DescribeGroupsResponse( + new DescribeGroupsResponseData() + .setGroups(singletonList( + new DescribedGroup() + .setErrorCode(error.code()) + .setGroupId(groupId1) + .setGroupState(ConsumerGroupState.STABLE.toString()) + .setProtocolType(protocolType) + .setProtocolData("assignor") + .setAuthorizedOperations(Utils.to32BitField(emptySet())) + .setMembers(singletonList( + new DescribedGroupMember() + .setClientHost("host") + .setClientId("clientId") + .setMemberId("memberId") + .setMemberAssignment(ConsumerProtocol.serializeAssignment( + new Assignment(new ArrayList<>(tps))).array()) + ))))); + return response; + } + + private AdminApiHandler.ApiResult handleWithError( + Errors error, + String protocolType + ) { + DescribeConsumerGroupsHandler handler = new DescribeConsumerGroupsHandler(true, logContext); + DescribeGroupsResponse response = buildResponse(error, protocolType); + return handler.handleResponse(coordinator, singleton(CoordinatorKey.byGroupId(groupId1)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId1)), result.unmappedKeys); + } + + private void assertRetriable( + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult result, + ConsumerGroupDescription expected + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId1); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + assertEquals(expected, result.completedKeys.get(CoordinatorKey.byGroupId(groupId1))); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId1); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java index d0ddb9960d7..8daed06ddf1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; @@ -63,6 +64,7 @@ public class DescribeProducersHandlerTest { new LogContext() ); } + @Test public void testBrokerIdSetInOptions() { @@ -192,14 +194,14 @@ public class DescribeProducersHandlerTest { DescribeProducersOptions options = new DescribeProducersOptions().brokerId(1); DescribeProducersHandler handler = newHandler(options); - int brokerId = 3; PartitionResponse partitionResponse = sampleProducerState(topicPartition); DescribeProducersResponse response = describeProducersResponse( singletonMap(topicPartition, partitionResponse) ); + Node node = new Node(3, "host", 1); ApiResult result = - handler.handleResponse(brokerId, mkSet(topicPartition), response); + handler.handleResponse(node, mkSet(topicPartition), response); assertEquals(mkSet(topicPartition), result.completedKeys.keySet()); assertEquals(emptyMap(), result.failedKeys); @@ -238,9 +240,9 @@ public class DescribeProducersHandlerTest { Errors error ) { DescribeProducersHandler handler = newHandler(options); - int brokerId = options.brokerId().orElse(3); DescribeProducersResponse response = buildResponseWithError(topicPartition, error); - return handler.handleResponse(brokerId, mkSet(topicPartition), response); + Node node = new Node(options.brokerId().orElse(3), "host", 1); + return handler.handleResponse(node, mkSet(topicPartition), response); } private DescribeProducersResponse buildResponseWithError( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java index 0db9cbb6917..04eac89a373 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.TransactionDescription; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DescribeTransactionsResponseData; import org.apache.kafka.common.protocol.Errors; @@ -40,6 +41,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class DescribeTransactionsHandlerTest { private final LogContext logContext = new LogContext(); + private final Node node = new Node(1, "host", 1234); @Test public void testBuildRequest() { @@ -57,7 +59,6 @@ public class DescribeTransactionsHandlerTest { @Test public void testHandleSuccessfulResponse() { - int brokerId = 1; String transactionalId1 = "foo"; String transactionalId2 = "bar"; @@ -74,12 +75,12 @@ public class DescribeTransactionsHandlerTest { .setTransactionStates(asList(transactionState1, transactionState2))); ApiResult result = handler.handleResponse( - brokerId, keys, response); + node, keys, response); assertEquals(keys, result.completedKeys.keySet()); - assertMatchingTransactionState(brokerId, transactionState1, + assertMatchingTransactionState(node.id(), transactionState1, result.completedKeys.get(CoordinatorKey.byTransactionalId(transactionalId1))); - assertMatchingTransactionState(brokerId, transactionState2, + assertMatchingTransactionState(node.id(), transactionState2, result.completedKeys.get(CoordinatorKey.byTransactionalId(transactionalId2))); } @@ -135,8 +136,6 @@ public class DescribeTransactionsHandlerTest { String transactionalId, Errors error ) { - int brokerId = 1; - CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); Set keys = mkSet(key); @@ -147,7 +146,7 @@ public class DescribeTransactionsHandlerTest { DescribeTransactionsResponse response = new DescribeTransactionsResponse(new DescribeTransactionsResponseData() .setTransactionStates(singletonList(transactionState))); - ApiResult result = handler.handleResponse(brokerId, keys, response); + ApiResult result = handler.handleResponse(node, keys, response); assertEquals(emptyMap(), result.completedKeys); return result; } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java new file mode 100644 index 00000000000..5c989406451 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java @@ -0,0 +1,139 @@ +/* + * 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 static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.OffsetFetchRequest; +import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; +import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.Test; + +public class ListConsumerGroupOffsetsHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId = "group-id"; + private final TopicPartition t0p0 = new TopicPartition("t0", 0); + private final TopicPartition t0p1 = new TopicPartition("t0", 1); + private final TopicPartition t1p0 = new TopicPartition("t1", 0); + private final TopicPartition t1p1 = new TopicPartition("t1", 1); + private final List tps = Arrays.asList(t0p0, t0p1, t1p0, t1p1); + + @Test + public void testBuildRequest() { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + OffsetFetchRequest request = handler.buildRequest(1, singleton(CoordinatorKey.byGroupId(groupId))).build(); + assertEquals(groupId, request.data().groupId()); + assertEquals(2, request.data().topics().size()); + assertEquals(2, request.data().topics().get(0).partitionIndexes().size()); + assertEquals(2, request.data().topics().get(1).partitionIndexes().size()); + } + + @Test + public void testSuccessfulHandleResponse() { + Map expected = new HashMap<>(); + assertCompleted(handleWithError(Errors.NONE), expected); + } + + @Test + public void testUnmappedHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); + } + + @Test + public void testRetriableHandleResponse() { + assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + assertRetriable(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); + assertFailed(GroupIdNotFoundException.class, handleWithError(Errors.GROUP_ID_NOT_FOUND)); + assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID)); + } + + private OffsetFetchResponse buildResponse(Errors error) { + Map responseData = new HashMap<>(); + OffsetFetchResponse response = new OffsetFetchResponse(error, responseData); + return response; + } + + private AdminApiHandler.ApiResult> handleWithError( + Errors error + ) { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + OffsetFetchResponse response = buildResponse(error); + return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + } + + private void assertRetriable( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult> result, + Map expected + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + assertEquals(expected, result.completedKeys.get(CoordinatorKey.byGroupId(groupId))); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult> result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java index 799eb40ce8c..a8923d1aa2c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; import org.apache.kafka.clients.admin.internals.AllBrokersStrategy.BrokerKey; +import org.apache.kafka.common.Node; import org.apache.kafka.common.message.ListTransactionsResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ListTransactionsRequest; @@ -43,6 +44,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; public class ListTransactionsHandlerTest { private final LogContext logContext = new LogContext(); + private final Node node = new Node(1, "host", 1234); @Test public void testBuildRequestWithoutFilters() { @@ -89,7 +91,7 @@ public class ListTransactionsHandlerTest { ListTransactionsHandler handler = new ListTransactionsHandler(options, logContext); ListTransactionsResponse response = sampleListTransactionsResponse1(); ApiResult> result = handler.handleResponse( - brokerId, singleton(brokerKey), response); + node, singleton(brokerKey), response); assertEquals(singleton(brokerKey), result.completedKeys.keySet()); assertExpectedTransactions(response.data().transactionStates(), result.completedKeys.get(brokerKey)); } @@ -134,7 +136,7 @@ public class ListTransactionsHandlerTest { ListTransactionsResponse response = new ListTransactionsResponse( new ListTransactionsResponseData().setErrorCode(error.code()) ); - return handler.handleResponse(brokerId, singleton(brokerKey), response); + return handler.handleResponse(node, singleton(brokerKey), response); } private ListTransactionsResponse sampleListTransactionsResponse1() { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java new file mode 100644 index 00000000000..0ffa43b4c0a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java @@ -0,0 +1,143 @@ +/* + * 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 static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; +import org.apache.kafka.common.message.LeaveGroupResponseData; +import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.LeaveGroupRequest; +import org.apache.kafka.common.requests.LeaveGroupResponse; +import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.Test; + +public class RemoveMembersFromConsumerGroupHandlerTest { + + private final LogContext logContext = new LogContext(); + private final String groupId = "group-id"; + private final MemberIdentity m1 = new MemberIdentity() + .setMemberId("m1") + .setGroupInstanceId("m1-gii"); + private final MemberIdentity m2 = new MemberIdentity() + .setMemberId("m2") + .setGroupInstanceId("m2-gii"); + private final List members = Arrays.asList(m1, m2); + + @Test + public void testBuildRequest() { + RemoveMembersFromConsumerGroupHandler handler = new RemoveMembersFromConsumerGroupHandler(groupId, members, logContext); + LeaveGroupRequest request = handler.buildRequest(1, singleton(CoordinatorKey.byGroupId(groupId))).build(); + assertEquals(groupId, request.data().groupId()); + assertEquals(2, request.data().members().size()); + } + + @Test + public void testSuccessfulHandleResponse() { + Map responseData = Collections.singletonMap(m1, Errors.NONE); + assertCompleted(handleWithError(Errors.NONE), responseData); + } + + @Test + public void testUnmappedHandleResponse() { + assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); + } + + @Test + public void testRetriableHandleResponse() { + assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + assertRetriable(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + @Test + public void testFailedHandleResponse() { + assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); + assertFailed(UnknownServerException.class, handleWithError(Errors.UNKNOWN_SERVER_ERROR)); + } + + private LeaveGroupResponse buildResponse(Errors error) { + LeaveGroupResponse response = new LeaveGroupResponse( + new LeaveGroupResponseData() + .setErrorCode(error.code()) + .setMembers(singletonList( + new MemberResponse() + .setErrorCode(error.code()) + .setMemberId("m1") + .setGroupInstanceId("m1-gii")))); + return response; + } + + private AdminApiHandler.ApiResult> handleWithError( + Errors error + ) { + RemoveMembersFromConsumerGroupHandler handler = new RemoveMembersFromConsumerGroupHandler(groupId, members, logContext); + LeaveGroupResponse response = buildResponse(error); + return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + } + + private void assertUnmapped( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + } + + private void assertRetriable( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + } + + private void assertCompleted( + AdminApiHandler.ApiResult> result, + Map expected + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.completedKeys.keySet()); + assertEquals(expected, result.completedKeys.get(key)); + } + + private void assertFailed( + Class expectedExceptionType, + AdminApiHandler.ApiResult> result + ) { + CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(key), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 4c528bc9826..ab8a9503b39 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -603,7 +603,7 @@ public class KafkaConsumerTest { // Since we would enable the heartbeat thread after received join-response which could // send the sync-group on behalf of the consumer if it is enqueued, we may still complete // the rebalance and send out the fetch; in order to avoid it we do not prepare sync response here. - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, memberId, leaderId, Errors.NONE), coordinator); @@ -731,7 +731,7 @@ public class KafkaConsumerTest { true, groupId, groupInstanceId, false); consumer.assign(singletonList(tp0)); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // lookup committed offset and find nothing @@ -755,7 +755,7 @@ public class KafkaConsumerTest { true, groupId, groupInstanceId, false); consumer.assign(singletonList(tp0)); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, 539L), Errors.NONE), coordinator); @@ -780,7 +780,7 @@ public class KafkaConsumerTest { true, groupId, groupInstanceId, false); consumer.assign(singletonList(tp0)); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, -1L), Errors.NONE), coordinator); @@ -829,7 +829,7 @@ public class KafkaConsumerTest { consumer.assign(singletonList(tp0)); // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // fetch offset for one topic @@ -885,7 +885,7 @@ public class KafkaConsumerTest { time, client, subscription, metadata, assignor, true, groupId, groupInstanceId, true); consumer.assign(singletonList(tp0)); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(offsetResponse( @@ -911,7 +911,7 @@ public class KafkaConsumerTest { consumer.assign(Arrays.asList(tp0, tp1)); // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // fetch offset for one topic @@ -1387,7 +1387,7 @@ public class KafkaConsumerTest { KafkaConsumer consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId); // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // manual assignment @@ -1444,7 +1444,7 @@ public class KafkaConsumerTest { KafkaConsumer consumer = newConsumer(time, client, subscription, metadata, assignor, false, groupInstanceId); // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // manual assignment @@ -1498,7 +1498,7 @@ public class KafkaConsumerTest { KafkaConsumer consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId); // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // manual assignment @@ -1692,7 +1692,7 @@ public class KafkaConsumerTest { KafkaConsumer consumer = newConsumer(time, client, subscription, metadata, assignor, false, groupInstanceId); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); @@ -1732,7 +1732,7 @@ public class KafkaConsumerTest { client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator, true); // should try and find the new coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); // rejoin group client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, memberId, leaderId, Errors.NONE), coordinator); @@ -1920,7 +1920,7 @@ public class KafkaConsumerTest { consumer.subscribe(singleton(topic), getExceptionConsumerRebalanceListener()); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, memberId, leaderId, Errors.NONE), coordinator); client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator); @@ -2234,7 +2234,7 @@ public class KafkaConsumerTest { private Node prepareRebalance(MockClient client, Node node, final Set subscribedTopics, ConsumerPartitionAssignor assignor, List partitions, Node coordinator) { if (coordinator == null) { // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); } @@ -2259,7 +2259,7 @@ public class KafkaConsumerTest { private Node prepareRebalance(MockClient client, Node node, ConsumerPartitionAssignor assignor, List partitions, Node coordinator) { if (coordinator == null) { // lookup coordinator - client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, node), node); + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index c0606f147ec..b97fb789faf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -1343,7 +1343,7 @@ public class AbstractCoordinatorTest { } private FindCoordinatorResponse groupCoordinatorResponse(Node node, Errors error) { - return FindCoordinatorResponse.prepareResponse(error, node); + return FindCoordinatorResponse.prepareResponse(error, GROUP_ID, node); } private HeartbeatResponse heartbeatResponse(Errors error) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 7310f852e91..9eb173692a3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -3029,7 +3029,7 @@ public abstract class ConsumerCoordinatorTest { } private FindCoordinatorResponse groupCoordinatorResponse(Node node, Errors error) { - return FindCoordinatorResponse.prepareResponse(error, node); + return FindCoordinatorResponse.prepareResponse(error, groupId, node); } private HeartbeatResponse heartbeatResponse(Errors error) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 4f44dfbf01d..f055e12c231 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -801,14 +801,14 @@ public class KafkaProducerTest { client.prepareResponse( request -> request instanceof FindCoordinatorRequest && ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), - FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", host1)); assertThrows(TimeoutException.class, producer::initTransactions); client.prepareResponse( request -> request instanceof FindCoordinatorRequest && ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), - FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); @@ -834,7 +834,7 @@ public class KafkaProducerTest { Node node = metadata.fetch().nodes().get(0); client.throttle(node, 5000); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); try (Producer producer = kafkaProducer(configs, new StringSerializer(), @@ -856,7 +856,7 @@ public class KafkaProducerTest { MockClient client = new MockClient(time, metadata); client.updateMetadata(initialUpdateResponse); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); client.prepareResponse(endTxnResponse(Errors.NONE)); @@ -885,10 +885,10 @@ public class KafkaProducerTest { Node node = metadata.fetch().nodes().get(0); client.throttle(node, 5000); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); client.prepareResponse(addOffsetsToTxnResponse(Errors.NONE)); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); String groupId = "group"; client.prepareResponse(request -> ((TxnOffsetCommitRequest) request).data().groupId().equals(groupId), @@ -924,10 +924,10 @@ public class KafkaProducerTest { Node node = metadata.fetch().nodes().get(0); client.throttle(node, 5000); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); client.prepareResponse(addOffsetsToTxnResponse(Errors.NONE)); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); String groupId = "group"; String memberId = "member"; int generationId = 5; @@ -980,7 +980,7 @@ public class KafkaProducerTest { Node node = metadata.fetch().nodes().get(0); client.throttle(node, 5000); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); try (Producer producer = kafkaProducer(configs, new StringSerializer(), @@ -1191,7 +1191,7 @@ public class KafkaProducerTest { ExecutorService executorService = Executors.newSingleThreadExecutor(); CountDownLatch assertionDoneLatch = new CountDownLatch(1); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "this-is-a-transactional-id", host1)); executorService.submit(() -> { assertThrows(KafkaException.class, producer::initTransactions); assertionDoneLatch.countDown(); @@ -1220,7 +1220,7 @@ public class KafkaProducerTest { ExecutorService executorService = Executors.newSingleThreadExecutor(); CountDownLatch assertionDoneLatch = new CountDownLatch(1); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "this-is-a-transactional-id", host1)); executorService.submit(() -> { assertThrows(KafkaException.class, producer::initTransactions); assertionDoneLatch.countDown(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index cfc720e37e1..34e4f180e78 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -580,7 +580,7 @@ public class SenderTest { // Verify producerId after the sender is run to process responses. MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWith(1, Collections.emptyMap()); client.respond(metadataUpdate); - prepareFindCoordinatorResponse(Errors.NONE); + prepareFindCoordinatorResponse(Errors.NONE, "testInitProducerIdWithPendingMetadataRequest"); prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch); waitForProducerId(transactionManager, producerIdAndEpoch); } @@ -630,13 +630,13 @@ public class SenderTest { Node node = metadata.fetch().nodes().get(0); client.delayReady(node, REQUEST_TIMEOUT + 20); - prepareFindCoordinatorResponse(Errors.NONE); + prepareFindCoordinatorResponse(Errors.NONE, "testNodeNotReady"); sender.runOnce(); sender.runOnce(); assertNotNull(transactionManager.coordinator(CoordinatorType.TRANSACTION), "Coordinator not found"); client.throttle(node, REQUEST_TIMEOUT + 20); - prepareFindCoordinatorResponse(Errors.NONE); + prepareFindCoordinatorResponse(Errors.NONE, "Coordinator not found"); prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch); waitForProducerId(transactionManager, producerIdAndEpoch); } @@ -3192,7 +3192,7 @@ public class SenderTest { private void doInitTransactions(TransactionManager transactionManager, ProducerIdAndEpoch producerIdAndEpoch) { transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE); + prepareFindCoordinatorResponse(Errors.NONE, transactionManager.transactionalId()); sender.runOnce(); sender.runOnce(); @@ -3201,9 +3201,9 @@ public class SenderTest { assertTrue(transactionManager.hasProducerId()); } - private void prepareFindCoordinatorResponse(Errors error) { + private void prepareFindCoordinatorResponse(Errors error, String txnid) { Node node = metadata.fetch().nodes().get(0); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(error, node)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(error, txnid, node)); } private void prepareInitProducerResponse(Errors error, long producerId, short producerEpoch) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 6a4afda15ce..6c1e2fdf1e7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -3319,10 +3319,13 @@ public class TransactionManagerTest { final String coordinatorKey) { client.prepareResponse(body -> { FindCoordinatorRequest findCoordinatorRequest = (FindCoordinatorRequest) body; - assertEquals(CoordinatorType.forId(findCoordinatorRequest.data().keyType()), coordinatorType); - assertEquals(findCoordinatorRequest.data().key(), coordinatorKey); + assertEquals(coordinatorType, CoordinatorType.forId(findCoordinatorRequest.data().keyType())); + String key = findCoordinatorRequest.data().coordinatorKeys().isEmpty() + ? findCoordinatorRequest.data().key() + : findCoordinatorRequest.data().coordinatorKeys().get(0); + assertEquals(coordinatorKey, key); return true; - }, FindCoordinatorResponse.prepareResponse(error, brokerNode), shouldDisconnect); + }, FindCoordinatorResponse.prepareResponse(error, coordinatorKey, brokerNode), shouldDisconnect); } private void prepareInitPidResponse(Errors error, boolean shouldDisconnect, long producerId, short producerEpoch) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 0208c05ab9c..2fabba1843d 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -1192,7 +1192,7 @@ public class RequestResponseTest { private FindCoordinatorResponse createFindCoordinatorResponse() { Node node = new Node(10, "host1", 2014); - return FindCoordinatorResponse.prepareResponse(Errors.NONE, node); + return FindCoordinatorResponse.prepareOldResponse(Errors.NONE, node); } private FetchRequest createFetchRequest(int version, FetchMetadata metadata, List toForget) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index a407b4d4e65..51232b6e6b6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -244,7 +244,7 @@ public class WorkerCoordinatorTest { final String consumerId = "leader"; - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, node)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // normal join group @@ -281,7 +281,7 @@ public class WorkerCoordinatorTest { final String memberId = "member"; - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, node)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // normal join group @@ -319,7 +319,7 @@ public class WorkerCoordinatorTest { final String memberId = "member"; - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, node)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // config mismatch results in assignment error @@ -347,7 +347,7 @@ public class WorkerCoordinatorTest { PowerMock.replayAll(); - client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, node)); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // join the group once diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7c35a4451fa..aaaa808877a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -1331,28 +1331,44 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleFindCoordinatorRequest(request: RequestChannel.Request): Unit = { + val version = request.header.apiVersion + if (version < 4) { + handleFindCoordinatorRequestLessThanV4(request) + } else { + handleFindCoordinatorRequestV4AndAbove(request) + } + } + + private def handleFindCoordinatorRequestV4AndAbove(request: RequestChannel.Request): Unit = { val findCoordinatorRequest = request.body[FindCoordinatorRequest] - if (findCoordinatorRequest.data.keyType == CoordinatorType.GROUP.id && - !authHelper.authorize(request.context, DESCRIBE, GROUP, findCoordinatorRequest.data.key)) - requestHelper.sendErrorResponseMaybeThrottle(request, Errors.GROUP_AUTHORIZATION_FAILED.exception) - else if (findCoordinatorRequest.data.keyType == CoordinatorType.TRANSACTION.id && - !authHelper.authorize(request.context, DESCRIBE, TRANSACTIONAL_ID, findCoordinatorRequest.data.key)) - requestHelper.sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) - else { - val (partition, internalTopicName) = CoordinatorType.forId(findCoordinatorRequest.data.keyType) match { - case CoordinatorType.GROUP => - (groupCoordinator.partitionFor(findCoordinatorRequest.data.key), GROUP_METADATA_TOPIC_NAME) + val coordinators = findCoordinatorRequest.data.coordinatorKeys.asScala.map { key => + val (error, node) = getCoordinator(request, findCoordinatorRequest.data.keyType, key) + new FindCoordinatorResponseData.Coordinator() + .setKey(key) + .setErrorCode(error.code) + .setHost(node.host) + .setNodeId(node.id) + .setPort(node.port) + } + def createResponse(requestThrottleMs: Int): AbstractResponse = { + val response = new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(coordinators.asJava) + .setThrottleTimeMs(requestThrottleMs)) + trace("Sending FindCoordinator response %s for correlation id %d to client %s." + .format(response, request.header.correlationId, request.header.clientId)) + response + } + requestHelper.sendResponseMaybeThrottle(request, createResponse) + } - case CoordinatorType.TRANSACTION => - (txnCoordinator.partitionFor(findCoordinatorRequest.data.key), TRANSACTION_STATE_TOPIC_NAME) - } + private def handleFindCoordinatorRequestLessThanV4(request: RequestChannel.Request): Unit = { + val findCoordinatorRequest = request.body[FindCoordinatorRequest] - val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName), request.context.listenerName) - def createFindCoordinatorResponse(error: Errors, - node: Node, - requestThrottleMs: Int): FindCoordinatorResponse = { - new FindCoordinatorResponse( + val (error, node) = getCoordinator(request, findCoordinatorRequest.data.keyType, findCoordinatorRequest.data.key) + def createResponse(requestThrottleMs: Int): AbstractResponse = { + val responseBody = new FindCoordinatorResponse( new FindCoordinatorResponseData() .setErrorCode(error.code) .setErrorMessage(error.message()) @@ -1360,36 +1376,56 @@ class KafkaApis(val requestChannel: RequestChannel, .setHost(node.host) .setPort(node.port) .setThrottleTimeMs(requestThrottleMs)) + trace("Sending FindCoordinator response %s for correlation id %d to client %s." + .format(responseBody, request.header.correlationId, request.header.clientId)) + responseBody + } + if (error == Errors.NONE) { + requestHelper.sendResponseMaybeThrottle(request, createResponse) + } else { + requestHelper.sendErrorResponseMaybeThrottle(request, error.exception) + } + } + + private def getCoordinator(request: RequestChannel.Request, keyType: Byte, key: String): (Errors, Node) = { + if (keyType == CoordinatorType.GROUP.id && + !authHelper.authorize(request.context, DESCRIBE, GROUP, key)) + (Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode) + else if (keyType == CoordinatorType.TRANSACTION.id && + !authHelper.authorize(request.context, DESCRIBE, TRANSACTIONAL_ID, key)) + (Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, Node.noNode) + else { + val (partition, internalTopicName) = CoordinatorType.forId(keyType) match { + case CoordinatorType.GROUP => + (groupCoordinator.partitionFor(key), GROUP_METADATA_TOPIC_NAME) + + case CoordinatorType.TRANSACTION => + (txnCoordinator.partitionFor(key), TRANSACTION_STATE_TOPIC_NAME) } + val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName), request.context.listenerName) + if (topicMetadata.headOption.isEmpty) { val controllerMutationQuota = quotas.controllerMutation.newPermissiveQuotaFor(request) autoTopicCreationManager.createTopics(Seq(internalTopicName).toSet, controllerMutationQuota, None) - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createFindCoordinatorResponse( - Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs)) + (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) } else { - def createResponse(requestThrottleMs: Int): AbstractResponse = { - val responseBody = if (topicMetadata.head.errorCode != Errors.NONE.code) { - createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs) - } else { - val coordinatorEndpoint = topicMetadata.head.partitions.asScala - .find(_.partitionIndex == partition) - .filter(_.leaderId != MetadataResponse.NO_LEADER_ID) - .flatMap(metadata => metadataCache. + if (topicMetadata.head.errorCode != Errors.NONE.code) { + (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) + } else { + val coordinatorEndpoint = topicMetadata.head.partitions.asScala + .find(_.partitionIndex == partition) + .filter(_.leaderId != MetadataResponse.NO_LEADER_ID) + .flatMap(metadata => metadataCache. getAliveBrokerNode(metadata.leaderId, request.context.listenerName)) - coordinatorEndpoint match { - case Some(endpoint) => - createFindCoordinatorResponse(Errors.NONE, endpoint, requestThrottleMs) - case _ => - createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs) - } + coordinatorEndpoint match { + case Some(endpoint) => + (Errors.NONE, endpoint) + case _ => + (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) } - trace("Sending FindCoordinator response %s for correlation id %d to client %s." - .format(responseBody, request.header.correlationId, request.header.clientId)) - responseBody } - requestHelper.sendResponseMaybeThrottle(request, createResponse) } } } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 0204b8917b0..bbdb94bf48c 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -162,7 +162,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { .responses.find(topic) .partitionResponses.asScala.find(_.index == part).get .errorCode - ) + ) }), ApiKeys.FETCH -> ((resp: requests.FetchResponse) => Errors.forCode(resp.responseData.asScala.find(_._1 == tp).get._2.errorCode)), ApiKeys.LIST_OFFSETS -> ((resp: ListOffsetsResponse) => { @@ -176,7 +176,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode( resp.data.topics().get(0).partitions().get(0).errorCode)), ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.error), - ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => resp.error), + ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => { + Errors.forCode(resp.data.coordinators.asScala.find(g => group == g.key).head.errorCode) + }), ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error), ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error), ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => Errors.forCode(resp.data.errorCode)), @@ -384,7 +386,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id) - .setKey(group)).build() + .setCoordinatorKeys(Collections.singletonList(group))).build() } private def createUpdateMetadataRequest = { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index d2cdf082e39..90748e7ff88 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -261,7 +261,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { private def findCoordinator(group: String): Int = { val request = new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id) - .setKey(group)).build() + .setCoordinatorKeys(Collections.singletonList(group))).build() var nodeId = -1 TestUtils.waitUntilTrue(() => { val response = connectAndReceive[FindCoordinatorResponse](request) diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala index 3b3b781417b..c9b2f0be6d8 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala @@ -32,6 +32,8 @@ import org.mockito.Mockito._ import org.mockito.ArgumentMatcher import scala.jdk.CollectionConverters._ +import org.apache.kafka.clients.admin.internals.CoordinatorKey +import org.apache.kafka.common.internals.KafkaFutureImpl class ConsumerGroupServiceTest { @@ -107,8 +109,10 @@ class ConsumerGroupServiceTest { topicPartitionOffsets => topicPartitionOffsets != null && topicPartitionOffsets.keySet.asScala.equals(expectedPartitions) } + val future = new KafkaFutureImpl[ConsumerGroupDescription]() + future.complete(consumerGroupDescription) when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any())) - .thenReturn(new DescribeConsumerGroupsResult(Collections.singletonMap(group, KafkaFuture.completedFuture(consumerGroupDescription)))) + .thenReturn(new DescribeConsumerGroupsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future))) when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(group), any())) .thenReturn(AdminClientTestUtils.listConsumerGroupOffsetsResult(commitedOffsets)) when(admin.listOffsets( @@ -184,7 +188,9 @@ class ConsumerGroupServiceTest { classOf[RangeAssignor].getName, groupState, new Node(1, "localhost", 9092)) - new DescribeConsumerGroupsResult(Collections.singletonMap(group, KafkaFuture.completedFuture(description))) + val future = new KafkaFutureImpl[ConsumerGroupDescription]() + future.complete(description) + new DescribeConsumerGroupsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future)) } private def listGroupOffsetsResult: ListConsumerGroupOffsetsResult = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index f9db5b375c6..2789cefc15d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -81,6 +81,7 @@ import org.mockito.{ArgumentMatchers, Mockito} import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ +import java.util.Arrays class KafkaApisTest { @@ -891,12 +892,32 @@ class KafkaApisTest { testFindCoordinatorWithTopicCreation(CoordinatorType.TRANSACTION, hasEnoughLiveBrokers = false) } + @Test + def testOldFindCoordinatorAutoTopicCreationForOffsetTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.GROUP, version = 3) + } + + @Test + def testOldFindCoordinatorAutoTopicCreationForTxnTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.TRANSACTION, version = 3) + } + + @Test + def testOldFindCoordinatorNotEnoughBrokersForOffsetTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.GROUP, hasEnoughLiveBrokers = false, version = 3) + } + + @Test + def testOldFindCoordinatorNotEnoughBrokersForTxnTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.TRANSACTION, hasEnoughLiveBrokers = false, version = 3) + } + private def testFindCoordinatorWithTopicCreation(coordinatorType: CoordinatorType, - hasEnoughLiveBrokers: Boolean = true): Unit = { + hasEnoughLiveBrokers: Boolean = true, + version: Short = ApiKeys.FIND_COORDINATOR.latestVersion): Unit = { val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) - val requestHeader = new RequestHeader(ApiKeys.FIND_COORDINATOR, ApiKeys.FIND_COORDINATOR.latestVersion, - clientId, 0) + val requestHeader = new RequestHeader(ApiKeys.FIND_COORDINATOR, version, clientId, 0) val numBrokersNeeded = 3 @@ -927,12 +948,18 @@ class KafkaApisTest { throw new IllegalStateException(s"Unknown coordinator type $coordinatorType") } - val findCoordinatorRequest = new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKeyType(coordinatorType.id()) - .setKey(groupId) - ).build(requestHeader.apiVersion) - val request = buildRequest(findCoordinatorRequest) + val findCoordinatorRequestBuilder = if (version >= 4) { + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(coordinatorType.id()) + .setCoordinatorKeys(Arrays.asList(groupId))) + } else { + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(coordinatorType.id()) + .setKey(groupId)) + } + val request = buildRequest(findCoordinatorRequestBuilder.build(requestHeader.apiVersion)) val capturedResponse = expectNoThrottling(request) @@ -945,7 +972,12 @@ class KafkaApisTest { overrideProperties = topicConfigOverride).handleFindCoordinatorRequest(request) val response = capturedResponse.getValue.asInstanceOf[FindCoordinatorResponse] - assertEquals(Errors.COORDINATOR_NOT_AVAILABLE, response.error()) + if (version >= 4) { + assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code, response.data.coordinators.get(0).errorCode) + assertEquals(groupId, response.data.coordinators.get(0).key) + } else { + assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code, response.data.errorCode) + } assertTrue(capturedRequest.getValue.isEmpty) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 1f04f08e8d9..5764282f43b 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -327,7 +327,7 @@ class RequestQuotaTest extends BaseRequestTest { new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id) - .setKey("test-group")) + .setCoordinatorKeys(Collections.singletonList("test-group"))) case ApiKeys.JOIN_GROUP => new JoinGroupRequest.Builder(