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 <djacot@confluent.io>, Tom Bentley <tbentley@redhat.com>, Sanjana Kaundinya <skaundinya@gmail.com>
This commit is contained in:
Mickael Maison 2021-07-01 22:05:03 +01:00 committed by GitHub
parent c92d013fd4
commit f5d5f654db
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
54 changed files with 2874 additions and 1059 deletions

View File

@ -49,13 +49,13 @@ public class ConsumerGroupDescription {
this(groupId, isSimpleConsumerGroup, members, partitionAssignor, state, coordinator, Collections.emptySet());
}
ConsumerGroupDescription(String groupId,
boolean isSimpleConsumerGroup,
Collection<MemberDescription> members,
String partitionAssignor,
ConsumerGroupState state,
Node coordinator,
Set<AclOperation> authorizedOperations) {
public ConsumerGroupDescription(String groupId,
boolean isSimpleConsumerGroup,
Collection<MemberDescription> members,
String partitionAssignor,
ConsumerGroupState state,
Node coordinator,
Set<AclOperation> authorizedOperations) {
this.groupId = groupId == null ? "" : groupId;
this.isSimpleConsumerGroup = isSimpleConsumerGroup;
this.members = members == null ? Collections.emptyList() :

View File

@ -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<String, KafkaFuture<Void>> futures;
private final Map<CoordinatorKey, KafkaFutureImpl<Void>> futures;
DeleteConsumerGroupsResult(final Map<String, KafkaFuture<Void>> futures) {
DeleteConsumerGroupsResult(final Map<CoordinatorKey, KafkaFutureImpl<Void>> futures) {
this.futures = futures;
}
@ -40,7 +43,9 @@ public class DeleteConsumerGroupsResult {
* individual deletions.
*/
public Map<String, KafkaFuture<Void>> deletedGroups() {
return futures;
Map<String, KafkaFuture<Void>> deletedGroups = new HashMap<>(futures.size());
futures.forEach((key, future) -> deletedGroups.put(key.idValue, future));
return deletedGroups;
}
/**

View File

@ -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<String, KafkaFuture<ConsumerGroupDescription>> futures;
private final Map<CoordinatorKey, KafkaFutureImpl<ConsumerGroupDescription>> futures;
public DescribeConsumerGroupsResult(final Map<String, KafkaFuture<ConsumerGroupDescription>> futures) {
public DescribeConsumerGroupsResult(final Map<CoordinatorKey, KafkaFutureImpl<ConsumerGroupDescription>> 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<String, KafkaFuture<ConsumerGroupDescription>> describedGroups() {
return futures;
Map<String, KafkaFuture<ConsumerGroupDescription>> describedGroups = new HashMap<>();
futures.forEach((key, future) -> describedGroups.put(key.idValue, future));
return describedGroups;
}
/**
@ -52,21 +55,18 @@ public class DescribeConsumerGroupsResult {
*/
public KafkaFuture<Map<String, ConsumerGroupDescription>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply(
new KafkaFuture.BaseFunction<Void, Map<String, ConsumerGroupDescription>>() {
@Override
public Map<String, ConsumerGroupDescription> apply(Void v) {
nil -> {
Map<String, ConsumerGroupDescription> descriptions = new HashMap<>(futures.size());
futures.forEach((key, future) -> {
try {
Map<String, ConsumerGroupDescription> descriptions = new HashMap<>(futures.size());
for (Map.Entry<String, KafkaFuture<ConsumerGroupDescription>> 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;
});
}
}

View File

@ -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<Call> 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<Call> 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<List<Call>> 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 <T> Map<String, KafkaFutureImpl<T>> createFutures(Collection<String> groupIds) {
return new HashSet<>(groupIds).stream().collect(
Collectors.toMap(groupId -> groupId,
groupId -> {
if (groupIdIsUnrepresentable(groupId)) {
KafkaFutureImpl<T> 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<String> groupIds,
final DescribeConsumerGroupsOptions options) {
final Map<String, KafkaFutureImpl<ConsumerGroupDescription>> 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<String, KafkaFutureImpl<ConsumerGroupDescription>> 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<ConsumerGroupDescription, DescribeConsumerGroupsOptions> 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 <T> The type of return value of the KafkaFuture, like ConsumerGroupDescription, Void etc.
* @param <O> The type of configuration option, like DescribeConsumerGroupsOptions, ListConsumerGroupsOptions etc
*/
private <T, O extends AbstractOptions<O>> Call getFindCoordinatorCall(ConsumerGroupOperationContext<T, O> context,
Supplier<Call> 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<ConsumerGroupDescription, DescribeConsumerGroupsOptions> 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<DescribedGroup> 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<DescribedGroupMember> members = describedGroup.members();
final List<MemberDescription> memberDescriptions = new ArrayList<>(members.size());
final Set<AclOperation> authorizedOperations = validAclOperations(describedGroup.authorizedOperations());
for (DescribedGroupMember groupMember : members) {
Set<TopicPartition> 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<CoordinatorKey, ConsumerGroupDescription> 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<Throwable> errors;
private final HashMap<String, ConsumerGroupListing> listings;
@ -3495,135 +3280,20 @@ public class KafkaAdminClient extends AdminClient {
@Override
public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId,
final ListConsumerGroupOffsetsOptions options) {
final KafkaFutureImpl<Map<TopicPartition, OffsetAndMetadata>> groupOffsetListingFuture = new KafkaFutureImpl<>();
final long startFindCoordinatorMs = time.milliseconds();
final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs());
ConsumerGroupOperationContext<Map<TopicPartition, OffsetAndMetadata>, 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<Map<TopicPartition, OffsetAndMetadata>,
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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, OffsetFetchResponse.PartitionData> 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<Integer> 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<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> 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<String> groupIds, DeleteConsumerGroupsOptions options) {
final Map<String, KafkaFutureImpl<Void>> 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<Void> future = futures.get(groupId);
if (future.isCompletedExceptionally())
continue;
final long startFindCoordinatorMs = time.milliseconds();
final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs());
ConsumerGroupOperationContext<Void, DeleteConsumerGroupsOptions> 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<Void, DeleteConsumerGroupsOptions> 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<CoordinatorKey, Void> 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<TopicPartition> partitions,
DeleteConsumerGroupOffsetsOptions options) {
final KafkaFutureImpl<Map<TopicPartition, Errors>> 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<Map<TopicPartition, Errors>, 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<Map<TopicPartition, Errors>, DeleteConsumerGroupOffsetsOptions> context,
Set<TopicPartition> 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<TopicPartition, Errors> 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<CoordinatorKey, Map<TopicPartition, Errors>> 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<Map<MemberIdentity, Errors>> future = new KafkaFutureImpl<>();
ConsumerGroupOperationContext<Map<MemberIdentity, Errors>, RemoveMembersFromConsumerGroupOptions> context =
new ConsumerGroupOperationContext<>(groupId, options, deadline, future);
List<MemberIdentity> 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<Map<MemberIdentity, Errors>, RemoveMembersFromConsumerGroupOptions> context,
List<MemberIdentity> 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<MemberIdentity, Errors> 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<CoordinatorKey, Map<MemberIdentity, Errors>> 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<TopicPartition, OffsetAndMetadata> offsets,
AlterConsumerGroupOffsetsOptions options) {
final KafkaFutureImpl<Map<TopicPartition, Errors>> future = new KafkaFutureImpl<>();
final long startFindCoordinatorMs = time.milliseconds();
final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs());
ConsumerGroupOperationContext<Map<TopicPartition, Errors>, 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<Map<TopicPartition, Errors>,
AlterConsumerGroupOffsetsOptions> context,
Map<TopicPartition, OffsetAndMetadata> offsets) {
return new Call("commitOffsets", context.deadline(), new ConstantNodeIdProvider(context.node().get().id())) {
@Override
OffsetCommitRequest.Builder createRequest(int timeoutMs) {
List<OffsetCommitRequestTopic> topics = new ArrayList<>();
Map<String, List<OffsetCommitRequestPartition>> offsetData = new HashMap<>();
for (Map.Entry<TopicPartition, OffsetAndMetadata> 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<String, List<OffsetCommitRequestPartition>> 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<Errors, Integer> 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<TopicPartition, Errors> 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<CoordinatorKey, Map<TopicPartition, Errors>> 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);
}

View File

@ -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<TopicPartition,
@Override
public ApiResult<TopicPartition, Void> handleResponse(
int brokerId,
Node broker,
Set<TopicPartition> topicPartitions,
AbstractResponse abstractResponse
) {

View File

@ -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<K, V> {
* 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<K, V> {
public void onResponse(
long currentTimeMs,
RequestSpec<K> spec,
AbstractResponse response
AbstractResponse response,
Node node
) {
clearInflightRequest(currentTimeMs, spec);
if (spec.scope instanceof FulfillmentScope) {
int brokerId = ((FulfillmentScope) spec.scope).destinationBrokerId;
AdminApiHandler.ApiResult<K, V> result = handler.handleResponse(
brokerId,
node,
spec.keys,
response
);
@ -250,6 +252,13 @@ public class AdminApiDriver<K, V> {
.filter(future.lookupKeys()::contains)
.collect(Collectors.toSet());
retryLookup(keysToUnmap);
} else if (t instanceof NoBatchedFindCoordinatorsException) {
((CoordinatorStrategy) handler.lookupStrategy()).disableBatch();
Set<K> keysToUnmap = spec.keys.stream()
.filter(future.lookupKeys()::contains)
.collect(Collectors.toSet());
retryLookup(keysToUnmap);
} else {
Map<K, Throwable> errors = spec.keys.stream().collect(Collectors.toMap(
Function.identity(),
@ -267,7 +276,12 @@ public class AdminApiDriver<K, V> {
private void clearInflightRequest(long currentTimeMs, RequestSpec<K> 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<K, V> {
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<K> spec) {

View File

@ -121,5 +121,9 @@ public interface AdminApiFuture<K, V> {
public Map<K, KafkaFutureImpl<V>> all() {
return futures;
}
public KafkaFutureImpl<V> get(K key) {
return futures.get(key);
}
}
}

View File

@ -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<K, V> {
* 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<K, V> handleResponse(int brokerId, Set<K> keys, AbstractResponse response);
ApiResult<K, V> handleResponse(Node broker, Set<K> keys, AbstractResponse response);
/**
* Get the lookup strategy that is responsible for finding the brokerId

View File

@ -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<CoordinatorKey, Map<TopicPartition, Errors>> {
private final CoordinatorKey groupId;
private final Map<TopicPartition, OffsetAndMetadata> offsets;
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
public AlterConsumerGroupOffsetsHandler(
String groupId,
Map<TopicPartition, OffsetAndMetadata> 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<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, Map<TopicPartition, Errors>> newFuture(
String groupId
) {
return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
}
@Override
public OffsetCommitRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
List<OffsetCommitRequestTopic> topics = new ArrayList<>();
Map<String, List<OffsetCommitRequestPartition>> offsetData = new HashMap<>();
for (Map.Entry<TopicPartition, OffsetAndMetadata> 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<String, List<OffsetCommitRequestPartition>> 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<CoordinatorKey, Map<TopicPartition, Errors>> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
final OffsetCommitResponse response = (OffsetCommitResponse) abstractResponse;
Map<CoordinatorKey, Map<TopicPartition, Errors>> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> unmapped = new ArrayList<>();
Map<TopicPartition, Errors> 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<CoordinatorKey, Throwable> failed,
List<CoordinatorKey> 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"));
}
}
}

View File

@ -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 <T> The type of return value of the KafkaFuture
* @param <O> The type of configuration option. Different for different consumer group commands.
*/
public final class ConsumerGroupOperationContext<T, O extends AbstractOptions<O>> {
final private String groupId;
final private O options;
final private long deadline;
final private KafkaFutureImpl<T> future;
private Optional<Node> node;
public ConsumerGroupOperationContext(String groupId,
O options,
long deadline,
KafkaFutureImpl<T> 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<T> future() {
return future;
}
public Optional<Node> 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<Errors, Integer> errorCounts) {
return errorCounts.containsKey(Errors.NOT_COORDINATOR);
}
public static boolean shouldRefreshCoordinator(Map<Errors, Integer> errorCounts) {
return errorCounts.containsKey(Errors.COORDINATOR_LOAD_IN_PROGRESS) ||
errorCounts.containsKey(Errors.COORDINATOR_NOT_AVAILABLE);
}
}

View File

@ -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);
}
}

View File

@ -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<CoordinatorKey> {
private static final ApiRequestScope BATCH_REQUEST_SCOPE = new ApiRequestScope() { };
private final Logger log;
private final FindCoordinatorRequest.CoordinatorType type;
private Set<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKe
Set<CoordinatorKey> keys,
AbstractResponse abstractResponse
) {
CoordinatorKey key = requireSingleton(keys);
FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse;
Errors error = response.error();
Map<CoordinatorKey, Integer> mappedKeys = new HashMap<>();
Map<CoordinatorKey, Throwable> 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<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKey, Integer> mappedKeys, Map<CoordinatorKey, Throwable> 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<CoordinatorKey> 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;

View File

@ -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<CoordinatorKey, Map<TopicPartition, Errors>> {
private final CoordinatorKey groupId;
private final Set<TopicPartition> partitions;
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
public DeleteConsumerGroupOffsetsHandler(
String groupId,
Set<TopicPartition> 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<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, Map<TopicPartition, Errors>> newFuture(
String groupId
) {
return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
}
@Override
public OffsetDeleteRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> 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<CoordinatorKey, Map<TopicPartition, Errors>> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
final OffsetDeleteResponse response = (OffsetDeleteResponse) abstractResponse;
Map<CoordinatorKey, Map<TopicPartition, Errors>> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> unmapped = new ArrayList<>();
final Errors error = Errors.forCode(response.data().errorCode());
if (error != Errors.NONE) {
handleError(groupId, error, failed, unmapped);
} else {
final Map<TopicPartition, Errors> 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<CoordinatorKey, Throwable> failed,
List<CoordinatorKey> 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;
}
}
}

View File

@ -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<CoordinatorKey, Void> {
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> 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<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, Void> newFuture(
Collection<String> groupIds
) {
return AdminApiFuture.forKeys(buildKeySet(groupIds));
}
private static Set<CoordinatorKey> buildKeySet(Collection<String> groupIds) {
return groupIds.stream()
.map(CoordinatorKey::byGroupId)
.collect(Collectors.toSet());
}
@Override
public DeleteGroupsRequest.Builder buildRequest(
int coordinatorId,
Set<CoordinatorKey> keys
) {
List<String> groupIds = keys.stream().map(key -> key.idValue).collect(Collectors.toList());
DeleteGroupsRequestData data = new DeleteGroupsRequestData()
.setGroupsNames(groupIds);
return new DeleteGroupsRequest.Builder(data);
}
@Override
public ApiResult<CoordinatorKey, Void> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
DeleteGroupsResponse response = (DeleteGroupsResponse) abstractResponse;
Map<CoordinatorKey, Void> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> 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<CoordinatorKey, Throwable> failed,
List<CoordinatorKey> 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());
}
}
}

View File

@ -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<CoordinatorKey, ConsumerGroupDescription> {
private final boolean includeAuthorizedOperations;
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> 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<CoordinatorKey> buildKeySet(Collection<String> groupIds) {
return groupIds.stream()
.map(CoordinatorKey::byGroupId)
.collect(Collectors.toSet());
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, ConsumerGroupDescription> newFuture(
Collection<String> groupIds
) {
return AdminApiFuture.forKeys(buildKeySet(groupIds));
}
@Override
public String apiName() {
return "describeGroups";
}
@Override
public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
@Override
public DescribeGroupsRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
List<String> 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<CoordinatorKey, ConsumerGroupDescription> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
DescribeGroupsResponse response = (DescribeGroupsResponse) abstractResponse;
Map<CoordinatorKey, ConsumerGroupDescription> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> 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<DescribedGroupMember> members = describedGroup.members();
final List<MemberDescription> memberDescriptions = new ArrayList<>(members.size());
final Set<AclOperation> authorizedOperations = validAclOperations(describedGroup.authorizedOperations());
for (DescribedGroupMember groupMember : members) {
Set<TopicPartition> 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<CoordinatorKey, Throwable> failed,
List<CoordinatorKey> 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<AclOperation> 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());
}
}

View File

@ -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<TopicPartition,
@Override
public ApiResult<TopicPartition, PartitionProducerState> handleResponse(
int brokerId,
Node broker,
Set<TopicPartition> keys,
AbstractResponse abstractResponse
) {

View File

@ -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<CoordinatorK
LogContext logContext
) {
this.log = logContext.logger(DescribeTransactionsHandler.class);
this.lookupStrategy = new CoordinatorStrategy(logContext);
this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.TRANSACTION, logContext);
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, TransactionDescription> newFuture(
@ -93,7 +95,7 @@ public class DescribeTransactionsHandler implements AdminApiHandler<CoordinatorK
@Override
public ApiResult<CoordinatorKey, TransactionDescription> handleResponse(
int brokerId,
Node broker,
Set<CoordinatorKey> keys,
AbstractResponse abstractResponse
) {
@ -122,7 +124,7 @@ public class DescribeTransactionsHandler implements AdminApiHandler<CoordinatorK
OptionalLong.of(transactionState.transactionStartTimeMs());
completed.put(transactionalIdKey, new TransactionDescription(
brokerId,
broker.id(),
TransactionState.parse(transactionState.transactionState()),
transactionState.producerId(),
transactionState.producerEpoch(),

View File

@ -0,0 +1,148 @@
/*
* 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.Optional;
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.protocol.Errors;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.OffsetFetchRequest;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;
public class ListConsumerGroupOffsetsHandler implements AdminApiHandler<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> {
private final CoordinatorKey groupId;
private final List<TopicPartition> partitions;
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
public ListConsumerGroupOffsetsHandler(
String groupId,
List<TopicPartition> 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<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> newFuture(
String groupId
) {
return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
}
@Override
public String apiName() {
return "offsetFetch";
}
@Override
public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
@Override
public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> 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<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse;
Map<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> unmapped = new ArrayList<>();
if (response.error() != Errors.NONE) {
handleError(groupId, response.error(), failed, unmapped);
} else {
final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> 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<Integer> 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<CoordinatorKey,
Throwable> failed,
List<CoordinatorKey> 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"));
}
}
}

View File

@ -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<AllBrokersStrate
@Override
public ApiResult<AllBrokersStrategy.BrokerKey, Collection<TransactionListing>> handleResponse(
int brokerId,
Node broker,
Set<AllBrokersStrategy.BrokerKey> keys,
AbstractResponse abstractResponse
) {
int brokerId = broker.id();
AllBrokersStrategy.BrokerKey key = requireSingleton(keys, brokerId);
ListTransactionsResponse response = (ListTransactionsResponse) abstractResponse;

View File

@ -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<CoordinatorKey, Map<MemberIdentity, Errors>> {
private final CoordinatorKey groupId;
private final List<MemberIdentity> members;
private final Logger log;
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
public RemoveMembersFromConsumerGroupHandler(
String groupId,
List<MemberIdentity> 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<CoordinatorKey> lookupStrategy() {
return lookupStrategy;
}
public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, Map<MemberIdentity, Errors>> newFuture(
String groupId
) {
return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
}
@Override
public LeaveGroupRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
return new LeaveGroupRequest.Builder(groupId.idValue, members);
}
@Override
public ApiResult<CoordinatorKey, Map<MemberIdentity, Errors>> handleResponse(
Node coordinator,
Set<CoordinatorKey> groupIds,
AbstractResponse abstractResponse
) {
final LeaveGroupResponse response = (LeaveGroupResponse) abstractResponse;
Map<CoordinatorKey, Map<MemberIdentity, Errors>> completed = new HashMap<>();
Map<CoordinatorKey, Throwable> failed = new HashMap<>();
List<CoordinatorKey> unmapped = new ArrayList<>();
final Errors error = Errors.forCode(response.data().errorCode());
if (error != Errors.NONE) {
handleError(groupId, error, failed, unmapped);
} else {
final Map<MemberIdentity, Errors> 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<CoordinatorKey,
Throwable> failed,
List<CoordinatorKey> 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;
}
}
}

View File

@ -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<ByteBuffer> joinFuture = null;
private RequestFuture<Void> 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<Void> 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<ClientResponse, Void> {
private boolean batch;
FindCoordinatorResponseHandler(boolean batch) {
this.batch = batch;
}
@Override
public void onSuccess(ClientResponse resp, RequestFuture<Void> 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<Void> 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;

View File

@ -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)));
}
}
}

View File

@ -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) {

View File

@ -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<String> keys) {
FindCoordinatorResponseData data = new FindCoordinatorResponseData();
List<FindCoordinatorResponseData.Coordinator> 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);
}
}

View File

@ -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." }
]
}

View File

@ -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." }
]}
]
}

View File

@ -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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
offsets.put(tp1, new OffsetAndMetadata(123L));
final KafkaFuture<Void> future = env.adminClient().alterConsumerGroupOffsets(groupId, offsets).all();
final KafkaFuture<Void> 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<Map<String, ConsumerGroupDescription>> 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<String> 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<Errors> 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<TopicPartition, OffsetAndMetadata> 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<String> groupIds = singletonList("group-0");
final List<String> 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<String> groupIds = singletonList("group-0");
final List<String> 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<String> groupIds = singletonList("group-0");
public void testDeleteConsumerGroupsWithOlderBroker() throws Exception {
final List<String> 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<Void> results = result.deletedGroups().get("group-0");
final KafkaFuture<Void> 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<Void> errorResults = errorResult1.deletedGroups().get("group-0");
final KafkaFuture<Void> 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<Void> future = env.adminClient().deleteConsumerGroupOffsets("group-0", Stream.of(tp1).collect(Collectors.toSet())).all();
final KafkaFuture<Void> 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<Errors> 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<MemberToRemove> membersToRemove = singletonList(new MemberToRemove("instance-1"));
final KafkaFuture<Void> 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<MemberToRemove> 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<Errors> 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<MemberToRemove> 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<MemberToRemove> 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<TopicPartition> 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<TopicPartition, OffsetAndMetadata> 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<Errors> nonRetriableErrors = Arrays.asList(
Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND);
@ -3963,7 +4109,7 @@ public class KafkaAdminClientTest {
Map<TopicPartition, OffsetAndMetadata> 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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, OffsetAndMetadata> 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, Integer> errors = new HashMap<>();
AbstractResponse response = new AbstractResponse(ApiKeys.OFFSET_COMMIT) {
@Override
public Map<Errors, Integer> 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<Errors, Integer> 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(

View File

@ -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));
}

View File

@ -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<String> 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<String> groupIds = new HashSet<>(Arrays.asList("g1", "g2"));
DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(lc);
AdminApiFuture<CoordinatorKey, Void> future = AdminApiFuture.forKeys(
groupIds.stream().map(g -> CoordinatorKey.byGroupId(g)).collect(Collectors.toSet()));
AdminApiDriver<CoordinatorKey, Void> driver = new AdminApiDriver<>(
handler,
future,
time.milliseconds() + API_TIMEOUT_MS,
RETRY_BACKOFF_MS,
new LogContext()
);
assertTrue(((CoordinatorStrategy) handler.lookupStrategy()).batch);
List<RequestSpec<CoordinatorKey>> requestSpecs = driver.poll();
// Expect CoordinatorStrategy to try resolving all coordinators in a single request
assertEquals(1, requestSpecs.size());
RequestSpec<CoordinatorKey> 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<RequestSpec<CoordinatorKey>> retryLookupSpecs = driver.poll();
assertEquals(groupIds.size(), retryLookupSpecs.size());
// These new requests are treated a new requests and not retries
for (RequestSpec<CoordinatorKey> retryLookupSpec : retryLookupSpecs) {
assertEquals(0, retryLookupSpec.nextAllowedTryMs);
assertEquals(0, retryLookupSpec.tries);
}
}
@Test
public void testCoalescedStaticAndDynamicFulfillment() {
Map<String, String> dynamicMapping = map(
@ -417,7 +457,7 @@ class AdminApiDriverTest {
RequestSpec<String> 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<String> requestSpec = requestSpecs.get(0);
assertEquals(0, requestSpec.tries);
assertEquals(0L, requestSpec.nextAllowedTryMs);
ctx.assertResponse(requestSpec, emptyFulfillment);
ctx.assertResponse(requestSpec, emptyFulfillment, Node.noNode());
List<RequestSpec<String>> 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<String> requestSpec,
ApiResult<String, Long> result
ApiResult<String, Long> 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<String, Long> 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<K, V> handleResponse(int brokerId, Set<K> keys, AbstractResponse response) {
public ApiResult<K, V> handleResponse(Node broker, Set<K> keys, AbstractResponse response) {
return Optional.ofNullable(expectedRequests.get(keys)).orElseThrow(() ->
new AssertionError("Unexpected fulfillment request for keys " + keys)
);

View File

@ -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<AllBrokersStrategy.BrokerKey> 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<AllBrokersStrategy.BrokerKey> lookupSpec = lookupSpecs.get(0);
Set<Integer> 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<Integer, KafkaFutureImpl<Integer>> 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<Integer> 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<Integer> future2 = brokerFutures.get(brokerId2);
assertTrue(future2.isDone());
assertEquals(Collections.emptyList(), driver.poll());
@ -149,7 +150,7 @@ public class AllBrokersStrategyIntegrationTest {
AdminApiDriver.RequestSpec<AllBrokersStrategy.BrokerKey> 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<Integer, KafkaFutureImpl<Integer>> 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<AllBrokersStrategy.BrokerKey> 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<Integer, KafkaFutureImpl<Integer>> brokerFutures = result.all().get();
@ -232,11 +233,11 @@ public class AllBrokersStrategyIntegrationTest {
@Override
public ApiResult<AllBrokersStrategy.BrokerKey, Integer> handleResponse(
int brokerId,
Node broker,
Set<AllBrokersStrategy.BrokerKey> keys,
AbstractResponse response
) {
return ApiResult.completed(keys.iterator().next(), brokerId);
return ApiResult.completed(keys.iterator().next(), broker.id());
}
@Override

View File

@ -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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, Errors> responseData = Collections.singletonMap(t0p0, Errors.NONE);
OffsetCommitResponse response = new OffsetCommitResponse(0, responseData);
ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 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<CoordinatorKey, Map<TopicPartition, Errors>> handleWithError(
Errors error
) {
AlterConsumerGroupOffsetsHandler handler = new AlterConsumerGroupOffsetsHandler(groupId, partitions, logContext);
Map<TopicPartition, Errors> 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<CoordinatorKey, Map<TopicPartition, Errors>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> result,
Map<TopicPartition, Errors> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 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)));
}
}

View File

@ -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<CoordinatorKey> result = runLookup(group, responseData);
AdminApiLookupStrategy.LookupResult<CoordinatorKey> 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<CoordinatorKey> result = runLookup(new HashSet<>(Arrays.asList(group1, group2)), responseData);
Map<CoordinatorKey, Integer> 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<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKey> 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<CoordinatorKey> result = runLookup(key, responseData);
FindCoordinatorResponseData responseData = new FindCoordinatorResponseData()
.setCoordinators(Collections.singletonList(
new FindCoordinatorResponseData.Coordinator()
.setKey(key.idValue)
.setErrorCode(error.code())));
AdminApiLookupStrategy.LookupResult<CoordinatorKey> 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<CoordinatorKey> runLookup(
private AdminApiLookupStrategy.LookupResult<CoordinatorKey> 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<CoordinatorKey> runLookup(
Set<CoordinatorKey> 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);
}
}

View File

@ -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<TopicPartition> 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<TopicPartition, Errors> 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<CoordinatorKey, Map<TopicPartition, Errors>> 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<CoordinatorKey, Map<TopicPartition, Errors>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys);
}
private void assertRetriable(
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> result,
Map<TopicPartition, Errors> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 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)));
}
}

View File

@ -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<CoordinatorKey, Void> 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<CoordinatorKey, Void> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId1)), result.unmappedKeys);
}
private void assertRetriable(
AdminApiHandler.ApiResult<CoordinatorKey, Void> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, Void> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Void> 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)));
}
}

View File

@ -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<String> groupIds = new HashSet<>(Arrays.asList(groupId1, groupId2));
private final Set<CoordinatorKey> keys = groupIds.stream()
.map(CoordinatorKey::byGroupId)
.collect(Collectors.toSet());
private final Node coordinator = new Node(1, "host", 1234);
private final Set<TopicPartition> 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<MemberDescription> 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<CoordinatorKey, ConsumerGroupDescription> 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<CoordinatorKey, ConsumerGroupDescription> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId1)), result.unmappedKeys);
}
private void assertRetriable(
AdminApiHandler.ApiResult<CoordinatorKey, ConsumerGroupDescription> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, ConsumerGroupDescription> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, ConsumerGroupDescription> 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)));
}
}

View File

@ -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<TopicPartition, PartitionProducerState> 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(

View File

@ -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<CoordinatorKey, TransactionDescription> 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<CoordinatorKey> keys = mkSet(key);
@ -147,7 +146,7 @@ public class DescribeTransactionsHandlerTest {
DescribeTransactionsResponse response = new DescribeTransactionsResponse(new DescribeTransactionsResponseData()
.setTransactionStates(singletonList(transactionState)));
ApiResult<CoordinatorKey, TransactionDescription> result = handler.handleResponse(brokerId, keys, response);
ApiResult<CoordinatorKey, TransactionDescription> result = handler.handleResponse(node, keys, response);
assertEquals(emptyMap(), result.completedKeys);
return result;
}

View File

@ -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<TopicPartition> 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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, PartitionData> responseData = new HashMap<>();
OffsetFetchResponse response = new OffsetFetchResponse(error, responseData);
return response;
}
private AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> 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<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys);
}
private void assertRetriable(
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> result,
Map<TopicPartition, OffsetAndMetadata> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> 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)));
}
}

View File

@ -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<BrokerKey, Collection<TransactionListing>> 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() {

View File

@ -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<MemberIdentity> 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<MemberIdentity, Errors> 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<CoordinatorKey, Map<MemberIdentity, Errors>> 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<CoordinatorKey, Map<MemberIdentity, Errors>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys);
}
private void assertRetriable(
AdminApiHandler.ApiResult<CoordinatorKey, Map<MemberIdentity, Errors>> result
) {
assertEquals(emptySet(), result.completedKeys.keySet());
assertEquals(emptySet(), result.failedKeys.keySet());
assertEquals(emptyList(), result.unmappedKeys);
}
private void assertCompleted(
AdminApiHandler.ApiResult<CoordinatorKey, Map<MemberIdentity, Errors>> result,
Map<MemberIdentity, Errors> 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<? extends Throwable> expectedExceptionType,
AdminApiHandler.ApiResult<CoordinatorKey, Map<MemberIdentity, Errors>> 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)));
}
}

View File

@ -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<String, String> 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<String, String> 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<String, String> 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<String, String> 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<String> subscribedTopics, ConsumerPartitionAssignor assignor, List<TopicPartition> 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<TopicPartition> 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());
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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<String, String> 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<String, String> 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();

View File

@ -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) {

View File

@ -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) {

View File

@ -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<TopicPartition> toForget) {

View File

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

View File

@ -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)
}
}
}

View File

@ -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 = {

View File

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

View File

@ -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 = {

View File

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

View File

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