From 548ee7f34a85690a4324a26777528e5213bd2ef0 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 28 Aug 2020 17:36:53 -0700 Subject: [PATCH 01/17] KAFKA-10442; Add support for the `DescribeProducers` API --- bin/kafka-transactions.sh | 17 + .../org/apache/kafka/clients/admin/Admin.java | 6 + .../admin/DescribeProducersOptions.java | 41 ++ .../admin/DescribeProducersResult.java | 118 +++++ .../kafka/clients/admin/KafkaAdminClient.java | 80 +++- .../admin/internals/CoordinatorKey.java | 53 +++ .../internals/CoordinatorRequestDriver.java | 122 +++++ .../DescribeProducersRequestDriver.java | 151 ++++++ .../internals/MetadataRequestDriver.java | 161 +++++++ .../admin/internals/RequestDriver.java | 446 ++++++++++++++++++ .../apache/kafka/common/protocol/ApiKeys.java | 22 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../requests/DescribeGroupsRequest.java | 2 +- .../requests/DescribeProducersRequest.java | 103 ++++ .../requests/DescribeProducersResponse.java | 66 +++ .../common/requests/MetadataResponse.java | 2 +- .../kafka/common/utils/CollectionUtils.java | 33 +- .../org/apache/kafka/common/utils/Utils.java | 6 + .../message/DescribeProducersRequest.json | 31 ++ .../message/DescribeProducersResponse.json | 45 ++ .../kafka/clients/admin/MockAdminClient.java | 9 +- .../admin/internals/AdminRequestUtil.java | 55 +++ .../CoordinatorRequestDriverTest.java | 217 +++++++++ .../DescribeProducersRequestDriverTest.java | 357 ++++++++++++++ .../internals/MetadataRequestDriverTest.java | 306 ++++++++++++ .../common/requests/RequestResponseTest.java | 37 ++ .../kafka/admin/TransactionCommand.scala | 179 +++++++ .../main/scala/kafka/cluster/Partition.scala | 18 + core/src/main/scala/kafka/log/Log.scala | 14 + .../main/scala/kafka/server/KafkaApis.scala | 42 +- .../scala/kafka/server/ReplicaManager.scala | 12 +- 32 files changed, 2726 insertions(+), 29 deletions(-) create mode 100755 bin/kafka-transactions.sh create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java create mode 100644 clients/src/main/resources/common/message/DescribeProducersRequest.json create mode 100644 clients/src/main/resources/common/message/DescribeProducersResponse.json create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminRequestUtil.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java create mode 100644 core/src/main/scala/kafka/admin/TransactionCommand.scala diff --git a/bin/kafka-transactions.sh b/bin/kafka-transactions.sh new file mode 100755 index 0000000000000..0bb79c6fd8e31 --- /dev/null +++ b/bin/kafka-transactions.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# 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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.TransactionCommand "$@" diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 53ed92afd8888..da17a17c30527 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1215,6 +1215,12 @@ default AlterClientQuotasResult alterClientQuotas(Collection entries, AlterClientQuotasOptions options); + default DescribeProducersResult describeProducers(Collection partitions) { + return describeProducers(partitions, new DescribeProducersOptions()); + } + + DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options); + /** * Describe all SASL/SCRAM credentials. * diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java new file mode 100644 index 0000000000000..2480195e3c147 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; +import java.util.OptionalInt; + +/** + * Options for {@link Admin#describeProducers(Collection)}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class DescribeProducersOptions extends AbstractOptions { + private OptionalInt brokerId = OptionalInt.empty(); + + public DescribeProducersOptions setBrokerId(int brokerId) { + this.brokerId = OptionalInt.of(brokerId); + return this; + } + + public OptionalInt brokerId() { + return brokerId; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java new file mode 100644 index 0000000000000..006909d78e142 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalLong; +import java.util.concurrent.ExecutionException; + +@InterfaceStability.Evolving +public class DescribeProducersResult { + + private final Map> futures; + + DescribeProducersResult(Map> futures) { + this.futures = futures; + } + + public KafkaFuture partitionResult(final TopicPartition partition) { + KafkaFuture future = futures.get(partition); + if (future == null) { + throw new IllegalArgumentException("Topic partition " + partition + + " was not included in the request"); + } + return future; + } + + public KafkaFuture> all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])) + .thenApply(nil -> { + Map results = new HashMap<>(futures.size()); + for (Map.Entry> entry : futures.entrySet()) { + try { + results.put(entry.getKey(), entry.getValue().get()); + } catch (InterruptedException | ExecutionException e) { + // This should be unreachable, because allOf ensured that all the futures completed successfully. + throw new RuntimeException(e); + } + } + return results; + }); + } + + public static class PartitionProducerState { + private final List activeProducers; + + public PartitionProducerState(List activeProducers) { + this.activeProducers = activeProducers; + } + + public List activeProducers() { + return activeProducers; + } + } + + public static class ProducerState { + private final long producerId; + private final int producerEpoch; + private final int lastSequence; + private final long lastTimestamp; + private final OptionalLong currentTransactionStartOffset; + + public ProducerState( + long producerId, + int producerEpoch, + int lastSequence, + long lastTimestamp, + OptionalLong currentTransactionStartOffset + ) { + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.lastSequence = lastSequence; + this.lastTimestamp = lastTimestamp; + this.currentTransactionStartOffset = currentTransactionStartOffset; + } + + public long producerId() { + return producerId; + } + + public int producerEpoch() { + return producerEpoch; + } + + public int lastSequence() { + return lastSequence; + } + + public long lastTimestamp() { + return lastTimestamp; + } + + public OptionalLong currentTransactionStartOffset() { + return currentTransactionStartOffset; + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c446be31e96fe..3307d1c1428ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -34,7 +34,9 @@ import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec; import org.apache.kafka.clients.admin.internals.AdminMetadataManager; import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; +import org.apache.kafka.clients.admin.internals.DescribeProducersRequestDriver; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; +import org.apache.kafka.clients.admin.internals.RequestDriver; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; @@ -718,20 +720,36 @@ abstract class Call { private final String callName; private final long deadlineMs; private final NodeProvider nodeProvider; - private int tries = 0; + protected int tries; private boolean aborted = false; private Node curNode = null; - private long nextAllowedTryMs = 0; - - Call(boolean internal, String callName, long deadlineMs, NodeProvider nodeProvider) { + private long nextAllowedTryMs; + + Call(boolean internal, + String callName, + long nextAllowedTryMs, + int tries, + long deadlineMs, + NodeProvider nodeProvider + ) { this.internal = internal; this.callName = callName; + this.nextAllowedTryMs = nextAllowedTryMs; + this.tries = tries; this.deadlineMs = deadlineMs; this.nodeProvider = nodeProvider; } + Call(boolean internal, String callName, long deadlineMs, NodeProvider nodeProvider) { + this(internal, callName, 0, 0, deadlineMs, nodeProvider); + } + Call(String callName, long deadlineMs, NodeProvider nodeProvider) { - this(false, callName, deadlineMs, nodeProvider); + this(false, callName, 0, 0, deadlineMs, nodeProvider); + } + + Call(String callName, long nextAllowedTryMs, int tries, long deadlineMs, NodeProvider nodeProvider) { + this(false, callName, nextAllowedTryMs, tries, deadlineMs, nodeProvider); } protected Node curNode() { @@ -811,8 +829,7 @@ private void failWithTimeout(long now, Throwable cause) { * * @return The AbstractRequest builder. */ - @SuppressWarnings("rawtypes") - abstract AbstractRequest.Builder createRequest(int timeoutMs); + abstract AbstractRequest.Builder createRequest(int timeoutMs); /** * Process the call response. @@ -4256,7 +4273,20 @@ private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getS private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte[] password, byte[] salt, int iterations) throws NoSuchAlgorithmException, InvalidKeyException { return new ScramFormatter(org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(publicScramMechanism.mechanismName())) - .hi(password, salt, iterations); + .hi(password, salt, iterations); + } + + @Override + public DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options) { + if (partitions.isEmpty()) { + return new DescribeProducersResult(Collections.emptyMap()); + } + long currentTimeMs = time.milliseconds(); + long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + partitions, options, deadlineMs, retryBackoffMs); + maybeSendRequests(currentTimeMs, driver); + return new DescribeProducersResult(driver.futures()); } /** @@ -4270,4 +4300,38 @@ static Throwable getSubLevelError(Map subLevelErrors, K subKey, S return subLevelErrors.get(subKey).exception(); } } + + private void maybeSendRequests(long currentTimeMs, RequestDriver driver) { + for (RequestDriver.RequestSpec spec : driver.poll()) { + runnable.call(newCall(driver, spec), currentTimeMs); + } + } + + private Call newCall(RequestDriver driver, RequestDriver.RequestSpec spec) { + NodeProvider nodeProvider = spec.scope.destinationBrokerId().isPresent() ? + new ConstantNodeIdProvider(spec.scope.destinationBrokerId().getAsInt()) : + new LeastLoadedNodeProvider(); + + return new Call("", spec.nextAllowedTryMs, spec.tries, spec.deadlineMs, nodeProvider) { + @Override + AbstractRequest.Builder createRequest(int timeoutMs) { + return spec.request; + } + + @Override + void handleResponse(AbstractResponse response) { + long currentTimeMs = time.milliseconds(); + driver.onResponse(currentTimeMs, spec, response); + maybeSendRequests(currentTimeMs, driver); + } + + @Override + void handleFailure(Throwable throwable) { + long currentTimeMs = time.milliseconds(); + driver.onFailure(currentTimeMs, spec, throwable); + maybeSendRequests(currentTimeMs, driver); + } + }; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java new file mode 100644 index 0000000000000..e0a4c3c4b8b29 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.requests.FindCoordinatorRequest; + +import java.util.Objects; + +public class CoordinatorKey { + public final String key; + public final FindCoordinatorRequest.CoordinatorType type; + + public CoordinatorKey(String key, FindCoordinatorRequest.CoordinatorType type) { + this.key = key; + this.type = type; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CoordinatorKey that = (CoordinatorKey) o; + return Objects.equals(key, that.key) && + type == that.type; + } + + @Override + public int hashCode() { + return Objects.hash(key, type); + } + + @Override + public String toString() { + return "CoordinatorKey(" + + "key='" + key + '\'' + + ", type=" + type + + ')'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java new file mode 100644 index 0000000000000..892ddf4fda6d2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; +import org.apache.kafka.common.message.FindCoordinatorRequestData; +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.FindCoordinatorResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Objects; +import java.util.Set; + +public abstract class CoordinatorRequestDriver extends RequestDriver { + private static final Logger log = LoggerFactory.getLogger(CoordinatorRequestDriver.class); + + public CoordinatorRequestDriver( + Collection keys, + long deadlineMs, + long retryBackoffMs + ) { + super(keys, deadlineMs, retryBackoffMs); + } + + @Override + RequestScope 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); + } + + @Override + FindCoordinatorRequest.Builder buildLookupRequest(Set keys) { + CoordinatorKey key = requireSingleton(keys); + return new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKey(key.key) + .setKeyType(key.type.id()) + ); + } + + @Override + void handleLookupResponse(Set keys, AbstractResponse abstractResponse) { + CoordinatorKey key = requireSingleton(keys); + FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; + Errors error = response.error(); + + switch (error) { + case NONE: + super.map(key, response.data().nodeId()); + break; + + case NOT_COORDINATOR: + case COORDINATOR_NOT_AVAILABLE: + case COORDINATOR_LOAD_IN_PROGRESS: + log.debug("FindCoordinator request for key {} returned topic-level error {}. Will retry", + key, error); + break; + + case GROUP_AUTHORIZATION_FAILED: + completeExceptionally(key, new GroupAuthorizationException("FindCoordinator request for groupId " + + "`" + key + "` failed due to authorization failure", key.key)); + break; + + case TRANSACTIONAL_ID_AUTHORIZATION_FAILED: + completeExceptionally(key, new TransactionalIdAuthorizationException("FindCoordinator request for " + + "transactionalId `" + key + "` failed due to authorization failure")); + break; + + default: + completeExceptionally(key, error.exception("FindCoordinator request for key " + + "`" + key + "` failed due to an unexpected error")); + } + } + + private CoordinatorKey requireSingleton(Set keys) { + if (keys.size() != 1) { + throw new IllegalArgumentException("Unexpected lookup key set"); + } + return keys.iterator().next(); + } + + private static class LookupRequestScope implements RequestScope { + final CoordinatorKey key; + + private LookupRequestScope(CoordinatorKey key) { + this.key = key; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + LookupRequestScope that = (LookupRequestScope) o; + return Objects.equals(key, that.key); + } + + @Override + public int hashCode() { + return Objects.hash(key); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java new file mode 100644 index 0000000000000..47cd99b5da8d2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.DescribeProducersOptions; +import org.apache.kafka.clients.admin.DescribeProducersResult; +import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; +import org.apache.kafka.clients.admin.DescribeProducersResult.ProducerState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.DescribeProducersRequest; +import org.apache.kafka.common.requests.DescribeProducersResponse; +import org.apache.kafka.common.utils.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.OptionalLong; +import java.util.Set; +import java.util.stream.Collectors; + +public class DescribeProducersRequestDriver extends MetadataRequestDriver { + private static final Logger log = LoggerFactory.getLogger(DescribeProducersRequestDriver.class); + + private final DescribeProducersOptions options; + + public DescribeProducersRequestDriver( + Collection topicPartitions, + DescribeProducersOptions options, + long deadlineMs, + long retryBackoffMs + ) { + super(topicPartitions, deadlineMs, retryBackoffMs); + this.options = options; + + // If the request options indicate a specific target broker, then we directly + // map the topic partitions to avoid the unneeded `Metadata` lookup. + if (options.brokerId().isPresent()) { + int destinationBrokerId = options.brokerId().getAsInt(); + for (TopicPartition topicPartition : topicPartitions) { + super.map(topicPartition, destinationBrokerId); + } + } + } + + @Override + DescribeProducersRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + DescribeProducersRequestData request = new DescribeProducersRequestData(); + DescribeProducersRequest.Builder builder = new DescribeProducersRequest.Builder(request); + + CollectionUtils.groupPartitionsByTopic( + topicPartitions, + builder::addTopic, + (topicRequest, partitionId) -> topicRequest.partitionIndexes().add(partitionId) + ); + + return builder; + } + + private void handlePartitionError( + TopicPartition topicPartition, + Errors error + ) { + switch (error) { + case NOT_LEADER_OR_FOLLOWER: + if (options.brokerId().isPresent()) { + // Typically these errors are retriable, but if the user specified the brokerId + // explicitly, then they are fatal + super.completeExceptionally(topicPartition, error.exception("Failed to describe active producers " + + "for partition " + topicPartition + " on brokerId " + options.brokerId().getAsInt())); + } else { + // Otherwise, we unmap the partition so that we can find the new leader + super.unmap(topicPartition); + } + break; + + case UNKNOWN_TOPIC_OR_PARTITION: + log.debug("Received unknown topic/partition error when trying to describe active producers " + + "for partition " + topicPartition + ". Will retry later."); + break; + + case TOPIC_AUTHORIZATION_FAILED: + super.completeExceptionally(topicPartition, new TopicAuthorizationException("Failed to describe " + + "active producers for partition " + topicPartition + " due to authorization failure on topic" + + " `" + topicPartition.topic() + "`", Collections.singleton(topicPartition.topic()))); + break; + + default: + super.completeExceptionally(topicPartition, error.exception("Failed to describe active " + + "producers for partition " + topicPartition + " due to unexpected error")); + break; + + } + } + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + DescribeProducersResponse response = (DescribeProducersResponse) abstractResponse; + + for (DescribeProducersResponseData.TopicResponse topicResponse : response.data().topics()) { + for (DescribeProducersResponseData.PartitionResponse partitionResponse : topicResponse.partitions()) { + TopicPartition topicPartition = new TopicPartition( + topicResponse.name(), partitionResponse.partitionIndex()); + + Errors error = Errors.forCode(partitionResponse.errorCode()); + if (error != Errors.NONE) { + handlePartitionError(topicPartition, error); + continue; + } + + List activeProducers = partitionResponse.activeProducers().stream() + .map(activeProducer -> { + OptionalLong currentTransactionFirstOffset = + activeProducer.currentTxnStartTimestamp() < 0 ? + OptionalLong.empty() : + OptionalLong.of(activeProducer.currentTxnStartTimestamp()); + return new ProducerState( + activeProducer.producerId(), + activeProducer.producerEpoch(), + activeProducer.lastSequence(), + activeProducer.lastTimestamp(), + currentTransactionFirstOffset + ); + }).collect(Collectors.toList()); + + super.complete(topicPartition, new DescribeProducersResult.PartitionProducerState(activeProducers)); + } + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java new file mode 100644 index 0000000000000..79e02da6a1a5e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +public abstract class MetadataRequestDriver extends RequestDriver { + private static final Logger log = LoggerFactory.getLogger(MetadataRequestDriver.class); + private static final RequestScope REQUEST_CONTEXT = new RequestScope() { + }; + + public MetadataRequestDriver( + Collection futures, + long deadlineMs, + long retryBackoffMs + ) { + super(futures, deadlineMs, retryBackoffMs); + } + + @Override + RequestScope lookupScope(TopicPartition key) { + // Metadata requests can group topic partitions arbitrarily, so they can all share + // the same request context + return REQUEST_CONTEXT; + } + + @Override + MetadataRequest.Builder buildLookupRequest(Set partitions) { + List topics = new ArrayList<>(); + for (TopicPartition partition : partitions) { + topics.add(partition.topic()); + } + return new MetadataRequest.Builder(topics, false); + } + + private void handleTopicError( + String topic, + Errors topicError, + Set requestPartitions + ) { + switch (topicError) { + case UNKNOWN_TOPIC_OR_PARTITION: + case LEADER_NOT_AVAILABLE: + log.debug("Metadata request for topic {} returned topic-level error {}. Will retry", + topic, topicError); + break; + + + case TOPIC_AUTHORIZATION_FAILED: + failAllPartitionsForTopic(topic, requestPartitions, tp -> new TopicAuthorizationException( + "Failed to fetch metadata for partition " + tp + " due to topic authorization failure", + Collections.singleton(topic))); + break; + + case INVALID_TOPIC_EXCEPTION: + failAllPartitionsForTopic(topic, requestPartitions, tp -> new InvalidTopicException( + "Failed to fetch metadata for partition " + tp + " due to invalid topic `" + topic + "`")); + break; + + default: + failAllPartitionsForTopic(topic, requestPartitions, tp -> new InvalidTopicException( + "Failed to fetch metadata for partition " + tp + " due to unexpected error for topic `" + topic + "`")); + } + } + + private void failAllPartitionsForTopic( + String topic, + Set partitions, + Function exceptionGenerator + ) { + partitions.stream().filter(tp -> tp.topic().equals(topic)).forEach(tp -> { + super.completeExceptionally(tp, exceptionGenerator.apply(tp)); + }); + } + + private void handlePartitionError( + TopicPartition topicPartition, + Errors partitionError + ) { + switch (partitionError) { + case NOT_LEADER_OR_FOLLOWER: + case REPLICA_NOT_AVAILABLE: + case LEADER_NOT_AVAILABLE: + case BROKER_NOT_AVAILABLE: + case KAFKA_STORAGE_ERROR: + log.debug("Metadata request for partition {} returned partition-level error {}. Will retry", + topicPartition, partitionError); + break; + + default: + super.completeExceptionally(topicPartition, partitionError.exception( + "Unexpected error during metadata lookup for " + topicPartition)); + } + } + + @Override + void handleLookupResponse(Set requestPartitions, AbstractResponse abstractResponse) { + MetadataResponse response = (MetadataResponse) abstractResponse; + for (MetadataResponse.TopicMetadata topicMetadata : response.topicMetadata()) { + Errors topicError = topicMetadata.error(); + if (topicError != Errors.NONE) { + handleTopicError(topicMetadata.topic(), topicError, requestPartitions); + continue; + } + + for (MetadataResponse.PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata()) { + TopicPartition topicPartition = partitionMetadata.topicPartition; + Errors partitionError = partitionMetadata.error; + + if (!super.contains(topicPartition)) { + // The `Metadata` response may include partitions that we are not interested in + continue; + } + + if (partitionError != Errors.NONE) { + handlePartitionError(topicPartition, partitionError); + continue; + } + + Optional leaderIdOpt = partitionMetadata.leaderId; + if (leaderIdOpt.isPresent()) { + super.map(topicPartition, leaderIdOpt.get()); + } else { + log.debug("Metadata request for {} returned no error, but the leader is unknown. Will retry", + topicPartition); + } + } + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java new file mode 100644 index 0000000000000..c2914504d9c1e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -0,0 +1,446 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.function.Function; + +/** + * The `KafkaAdminClient`'s internal `Call` primitive is not a good fit for multi-stage + * request workflows such as we see with the group coordinator APIs or any request which + * needs to be sent to a partition leader. Typically these APIs have two concrete stages: + * + * 1. Lookup: Find the broker that can fulfill the request (e.g. partition leader or group + * coordinator) + * 2. Fulfillment: Send the request to the broker found in the first step + * + * This is complicated by the fact that `Admin` APIs are typically batched, which + * means the Lookup stage may result in a set of brokers. For example, take a `ListOffsets` + * request for a set of topic partitions. In the Lookup stage, we will find the partition + * leaders for this set of partitions; in the Fulfillment stage, we will group together + * partition according to the IDs of the discovered leaders. + * + * Additionally, the flow between these two stages is bi-directional. We may find after + * sending a `ListOffsets` request to an expected leader that there was a leader change. + * This would result in a topic partition being sent back to the Lookup stage. + * + * Managing this complexity by chaining together `Call` implementations is challenging, + * so instead we use this class to do the bookkeeping. It handles both the batching + * aspect as well as the transitions between the Lookup and Fulfillment stages. + * + * Note that the interpretation of the AdminClient `retries` configuration becomes + * ambiguous for this kind of pipeline. We could treat it as an overall limit on the + * number of requests that can be sent, but that is not very useful because each pipeline + * has a minimum number of requests that need to be sent in order to satisfy the request. + * Instead, we treat this number of tries independently at each stage so that each + * stage has at least one opportunity to complete. So if a user sets `retries=1`, then + * the full pipeline can still complete as long as there are no request failures. + * + * @param The key type, which is also the granularity of the request routing (e.g. + * this could be `TopicPartition` in the case of requests intended for a partition + * leader or the `GroupId` in the case of consumer group requests intended for + * the group coordinator) + * @param The fulfillment type for each key (e.g. this could be consumer group state + * when the key type is a consumer `GroupId`) + */ +public abstract class RequestDriver { + private static final Logger log = LoggerFactory.getLogger(RequestDriver.class); + + private final long retryBackoffMs; + private final long deadlineMs; + private final Map> futures; + + private final BiMultimap lookupMap = new BiMultimap<>(); + private final BiMultimap fulfillmentMap = new BiMultimap<>(); + private final Map requestStates = new HashMap<>(); + + public RequestDriver( + Collection keys, + long deadlineMs, + long retryBackoffMs + ) { + this.futures = Utils.initializeMap(keys, KafkaFutureImpl::new); + this.deadlineMs = deadlineMs; + this.retryBackoffMs = retryBackoffMs; + initializeLookupKeys(); + } + + private void initializeLookupKeys() { + for (K key : futures.keySet()) { + lookupMap.put(lookupScope(key), key); + } + } + + /** + * Check whether the . This is useful when a response contains more partitions + * than are strictly needed. For example, a `Metadata` response always includes + * all partitions for each requested topic, even if we are only interested in + * a subset of them. + */ + boolean contains(K key) { + return futures.containsKey(key); + } + + /** + * Associate a key with a brokerId. This is called after a response in the Lookup + * stage reveals the mapping (e.g. when the `FindCoordinator` tells us the the + * group coordinator for a specific consumer group). + */ + void map(K key, Integer brokerId) { + lookupMap.remove(key); + fulfillmentMap.put(new BrokerScope(brokerId), key); + } + + /** + * Disassociate a key from the currently mapped brokerId. This will send the key + * back to the Lookup stage, which will allow us to attempt lookup again. + */ + void unmap(K key) { + fulfillmentMap.remove(key); + lookupMap.put(lookupScope(key), key); + } + + private void clear(K key) { + lookupMap.remove(key); + fulfillmentMap.remove(key); + } + + /** + * Complete the future associated with the given key exceptionally. After is called, + * the key will be taken out of both the Lookup and Fulfillment stages so that request + * are not retried. + */ + void completeExceptionally(K key, Throwable t) { + KafkaFutureImpl future = futures.get(key); + if (future == null) { + log.warn("Attempt to complete future for {}, which was not requested", key); + } else { + clear(key); + future.completeExceptionally(t); + } + } + + /** + * Complete the future associated with the given key. After is called, the key will + * be taken out of both the Lookup and Fulfillment stages so that request are not retried. + */ + void complete(K key, V value) { + KafkaFutureImpl future = futures.get(key); + if (future == null) { + log.warn("Attempt to complete future for {}, which was not requested", key); + } else { + clear(key); + future.complete(value); + } + } + + /** + * 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)}). + * + * @return A list of requests that need to be sent + */ + public List poll() { + List requests = new ArrayList<>(); + collectLookupRequests(requests); + collectFulfillmentRequests(requests); + return requests; + } + + /** + * Get a map of the futures that are awaiting completion. + */ + public Map> futures() { + return futures; + } + + /** + * Callback that is invoked when a `Call` returns a response successfully. + */ + public void onResponse( + long currentTimeMs, + RequestSpec spec, + AbstractResponse response + ) { + clearInflightRequest(currentTimeMs, spec); + if (spec.scope instanceof RequestDriver.BrokerScope) { + handleFulfillmentResponse(spec.keys, response); + } else { + handleLookupResponse(spec.keys, response); + } + } + + /** + * Callback that is invoked when a `Call` is failed. + */ + public void onFailure( + long currentTimeMs, + RequestSpec spec, + Throwable t + ) { + clearInflightRequest(currentTimeMs, spec); + spec.keys.forEach(key -> completeExceptionally(key, t)); + } + + /** + * The Lookup stage is complicated by the need to accommodate different batching mechanics. + * Specifically, a `Metadata` request supports arbitrary batching of topic partitions, but + * a `FindCoordinator` request only supports lookup of a single key. See the implementations + * in {@link MetadataRequestDriver#lookupScope(TopicPartition)} and + * {@link CoordinatorRequestDriver#lookupScope(CoordinatorKey)}. + */ + abstract RequestScope lookupScope(K key); + + /** + * Build the lookup request for a set of keys. The grouping of the keys is controlled + * through {@link #lookupScope(Object)}. In other words, each set of keys that map + * to the same request scope object will be sent to this method. + */ + abstract AbstractRequest.Builder buildLookupRequest(Set keys); + + /** + * Callback that is invoked when a Lookup request returns successfully. The handler + * should parse the response, check for errors, and updating mappings with + * {@link #map(Object, Integer)} and {@link #unmap(Object)} as needed. + */ + abstract void handleLookupResponse(Set keys, AbstractResponse response); + + /** + * Build the fulfillment request. The set of keys are derived during the Lookup stage + * as the set of keys which all map to the same destination broker. + */ + abstract AbstractRequest.Builder buildFulfillmentRequest(Set keys); + + /** + * Callback that is invoked when a Fulfillment request returns successfully. + * The handler should parse the response, check for errors, update mappings as + * needed, and complete any futures which can be completed. + */ + abstract void handleFulfillmentResponse(Set keys, AbstractResponse response); + + private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { + RequestState requestState = requestStates.get(spec.scope); + if (requestState != null) { + requestState.clearInflight(currentTimeMs); + } + } + + private void collectRequests( + List requests, + BiMultimap multimap, + Function, AbstractRequest.Builder> buildRequest + ) { + for (Map.Entry> entry : multimap.entrySet()) { + T scope = entry.getKey(); + + Set keys = entry.getValue(); + if (keys.isEmpty()) { + continue; + } + + RequestState requestState = requestStates.computeIfAbsent(scope, c -> new RequestState()); + if (requestState.hasInflight()) { + continue; + } + + AbstractRequest.Builder request = buildRequest.apply(keys); + RequestSpec spec = new RequestSpec( + scope, + new HashSet<>(keys), // copy to avoid exposing mutable state + request, + requestState.nextAllowedRetryMs, + deadlineMs, + requestState.tries + ); + + requestState.setInflight(spec); + requests.add(spec); + } + } + + private void collectLookupRequests(List requests) { + collectRequests( + requests, + lookupMap, + this::buildLookupRequest + ); + } + + private void collectFulfillmentRequests(List requests) { + collectRequests( + requests, + fulfillmentMap, + this::buildFulfillmentRequest + ); + } + + /** + * This is a helper class which helps us to map requests that need to be sent to + * the to the internal `Call` implementation that is used internally in + * {@link org.apache.kafka.clients.admin.KafkaAdminClient}. + */ + public class RequestSpec { + public final RequestScope scope; + public final Set keys; + public final AbstractRequest.Builder request; + public final long nextAllowedTryMs; + public final long deadlineMs; + public final int tries; + + public RequestSpec( + RequestScope scope, + Set keys, + AbstractRequest.Builder request, + long nextAllowedTryMs, + long deadlineMs, + int tries + ) { + this.scope = scope; + this.keys = keys; + this.request = request; + this.nextAllowedTryMs = nextAllowedTryMs; + this.deadlineMs = deadlineMs; + this.tries = tries; + } + } + + /** + * Helper class used to track the request state within each request scope. + * This class enforces a maximum number of inflight request and keeps track + * of backoff/retry state. + */ + private class RequestState { + private Optional inflightRequest = Optional.empty(); + private int tries = 0; + private long nextAllowedRetryMs = 0; + + boolean hasInflight() { + return inflightRequest.isPresent(); + } + + public void clearInflight(long currentTimeMs) { + this.inflightRequest = Optional.empty(); + this.nextAllowedRetryMs = currentTimeMs + retryBackoffMs; + } + + public void setInflight(RequestSpec spec) { + this.inflightRequest = Optional.of(spec); + this.tries++; + } + } + + /** + * Interface which is used to identify the scope of a request in either stage + * of the pipeline. This is primarily used to support backoff/retry mechanics. + */ + public interface RequestScope { + default OptionalInt destinationBrokerId() { + return OptionalInt.empty(); + } + } + + /** + * Completion of the Lookup stage results in a destination broker to send the + * fulfillment request to. Each destination broker in the Fulfillment stage + * gets its own request scope. + */ + private static class BrokerScope implements RequestScope { + public final int destinationBrokerId; + + private BrokerScope(int destinationBrokerId) { + this.destinationBrokerId = destinationBrokerId; + } + + @Override + public OptionalInt destinationBrokerId() { + return OptionalInt.of(destinationBrokerId); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + BrokerScope that = (BrokerScope) o; + return destinationBrokerId == that.destinationBrokerId; + } + + @Override + public int hashCode() { + return Objects.hash(destinationBrokerId); + } + } + + /** + * Helper class which maintains a bi-directional mapping from a key to a set of values. + * Each value can map to one and only one key, but many values can be associated with + * a single key. + * + * @param The key type + * @param The value type + */ + private static class BiMultimap { + private final Map reverseMap = new HashMap<>(); + private final Map> map = new HashMap<>(); + + Set get(K key) { + return map.get(key); + } + + void put(K key, V value) { + remove(value); + reverseMap.put(value, key); + map.computeIfAbsent(key, k -> new HashSet<>()).add(value); + } + + void remove(V value) { + K key = reverseMap.get(value); + if (key != null) { + Set set = map.get(key); + if (set != null) { + set.remove(value); + if (set.isEmpty()) { + map.remove(key); + } + } + } + } + + Set>> entrySet() { + return map.entrySet(); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index b23cfef3585b3..4a0ff185c04bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -16,8 +16,14 @@ */ package org.apache.kafka.common.protocol; +import org.apache.kafka.common.message.AddOffsetsToTxnRequestData; +import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.AddPartitionsToTxnRequestData; import org.apache.kafka.common.message.AddPartitionsToTxnResponseData; +import org.apache.kafka.common.message.AlterClientQuotasRequestData; +import org.apache.kafka.common.message.AlterClientQuotasResponseData; +import org.apache.kafka.common.message.AlterConfigsRequestData; +import org.apache.kafka.common.message.AlterConfigsResponseData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData; @@ -25,14 +31,8 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData; import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; import org.apache.kafka.common.message.ApiMessageType; -import org.apache.kafka.common.message.AddOffsetsToTxnRequestData; -import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.ApiVersionsRequestData; import org.apache.kafka.common.message.ApiVersionsResponseData; -import org.apache.kafka.common.message.AlterClientQuotasRequestData; -import org.apache.kafka.common.message.AlterClientQuotasResponseData; -import org.apache.kafka.common.message.AlterConfigsRequestData; -import org.apache.kafka.common.message.AlterConfigsResponseData; import org.apache.kafka.common.message.ControlledShutdownRequestData; import org.apache.kafka.common.message.ControlledShutdownResponseData; import org.apache.kafka.common.message.CreateAclsRequestData; @@ -63,6 +63,8 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData; import org.apache.kafka.common.message.DescribeLogDirsRequestData; import org.apache.kafka.common.message.DescribeLogDirsResponseData; +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.DescribeProducersResponseData; import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; @@ -109,10 +111,10 @@ import org.apache.kafka.common.message.StopReplicaResponseData; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupResponseData; -import org.apache.kafka.common.message.UpdateMetadataRequestData; -import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.message.TxnOffsetCommitRequestData; import org.apache.kafka.common.message.TxnOffsetCommitResponseData; +import org.apache.kafka.common.message.UpdateMetadataRequestData; +import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.message.WriteTxnMarkersRequestData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.types.Schema; @@ -225,7 +227,9 @@ public Struct parseResponse(short version, ByteBuffer buffer) { DESCRIBE_USER_SCRAM_CREDENTIALS(50, "DescribeUserScramCredentials", DescribeUserScramCredentialsRequestData.SCHEMAS, DescribeUserScramCredentialsResponseData.SCHEMAS), ALTER_USER_SCRAM_CREDENTIALS(51, "AlterUserScramCredentials", AlterUserScramCredentialsRequestData.SCHEMAS, - AlterUserScramCredentialsResponseData.SCHEMAS); + AlterUserScramCredentialsResponseData.SCHEMAS), + DESCRIBE_PRODUCERS(52, "DescribeProducers", DescribeProducersRequestData.SCHEMAS, + DescribeProducersResponseData.SCHEMAS); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 676a4adcc0485..37fe61595caf6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -248,6 +248,8 @@ public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Str return new DescribeUserScramCredentialsRequest(struct, apiVersion); case ALTER_USER_SCRAM_CREDENTIALS: return new AlterUserScramCredentialsRequest(struct, apiVersion); + case DESCRIBE_PRODUCERS: + return new DescribeProducersRequest(struct, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 7fd28b81cbd9c..39ae579470102 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -191,6 +191,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor return new DescribeUserScramCredentialsResponse(struct, version); case ALTER_USER_SCRAM_CREDENTIALS: return new AlterUserScramCredentialsResponse(struct, version); + case DESCRIBE_PRODUCERS: + return new DescribeProducersResponse(struct, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index 8f003acad5b97..f3f72182651f0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -27,7 +27,7 @@ public class DescribeGroupsRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { - private final DescribeGroupsRequestData data; + public final DescribeGroupsRequestData data; public Builder(DescribeGroupsRequestData data) { super(ApiKeys.DESCRIBE_GROUPS); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java new file mode 100644 index 0000000000000..563fda788fc07 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java @@ -0,0 +1,103 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.DescribeProducersRequestData.TopicRequest; +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.message.DescribeProducersResponseData.PartitionResponse; +import org.apache.kafka.common.message.DescribeProducersResponseData.TopicResponse; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class DescribeProducersRequest extends AbstractRequest { + public static class Builder extends AbstractRequest.Builder { + public final DescribeProducersRequestData data; + + public Builder(DescribeProducersRequestData data) { + super(ApiKeys.DESCRIBE_PRODUCERS); + this.data = data; + } + + public DescribeProducersRequestData.TopicRequest addTopic(String topic) { + DescribeProducersRequestData.TopicRequest topicRequest = + new DescribeProducersRequestData.TopicRequest().setName(topic); + data.topics().add(topicRequest); + return topicRequest; + } + + @Override + public DescribeProducersRequest build(short version) { + return new DescribeProducersRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final DescribeProducersRequestData data; + private final short version; + + private DescribeProducersRequest(DescribeProducersRequestData data, short version) { + super(ApiKeys.DESCRIBE_PRODUCERS, version); + this.data = data; + this.version = version; + } + + public DescribeProducersRequest(Struct struct, short version) { + super(ApiKeys.DESCRIBE_PRODUCERS, version); + this.data = new DescribeProducersRequestData(struct, version); + this.version = version; + } + + public DescribeProducersRequestData data() { + return data; + } + + @Override + protected Struct toStruct() { + return data.toStruct(version); + } + + @Override + public DescribeProducersResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + DescribeProducersResponseData response = new DescribeProducersResponseData(); + for (TopicRequest topicRequest : data.topics()) { + TopicResponse topicResponse = new TopicResponse() + .setName(topicRequest.name()); + for (int partitionId : topicRequest.partitionIndexes()) { + topicResponse.partitions().add( + new PartitionResponse() + .setPartitionIndex(partitionId) + .setErrorCode(error.code()) + ); + } + } + return new DescribeProducersResponse(response); + } + + public static DescribeProducersRequest parse(ByteBuffer buffer, short version) { + return new DescribeProducersRequest(ApiKeys.DESCRIBE_PRODUCERS.parseRequest(version, buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java new file mode 100644 index 0000000000000..2f043b7b15779 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java @@ -0,0 +1,66 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.message.DescribeProducersResponseData.PartitionResponse; +import org.apache.kafka.common.message.DescribeProducersResponseData.TopicResponse; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public class DescribeProducersResponse extends AbstractResponse { + private final DescribeProducersResponseData data; + + public DescribeProducersResponse(Struct struct, short version) { + this(new DescribeProducersResponseData(struct, version)); + } + + public DescribeProducersResponse(DescribeProducersResponseData data) { + this.data = data; + } + + public DescribeProducersResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + Map errorCounts = new HashMap<>(); + for (TopicResponse topicResponse : data.topics()) { + for (PartitionResponse partitionResponse : topicResponse.partitions()) { + updateErrorCounts(errorCounts, Errors.forCode(partitionResponse.errorCode())); + } + } + return errorCounts; + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + public static DescribeProducersResponse parse(ByteBuffer buffer, short version) { + return new DescribeProducersResponse( + ApiKeys.DESCRIBE_PRODUCERS.responseSchema(version).read(buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 8aefc6766f31c..8243f98310109 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -60,7 +60,7 @@ public class MetadataResponse extends AbstractResponse { public static final int NO_LEADER_ID = -1; public static final int AUTHORIZED_OPERATIONS_OMITTED = Integer.MIN_VALUE; - private final MetadataResponseData data; + public final MetadataResponseData data; private volatile Holder holder; private final boolean hasReliableLeaderEpochs; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java index f3a7dbd52bf08..cd348a2e70cbe 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; import java.util.Collection; +import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.stream.Collectors; public final class CollectionUtils { @@ -57,19 +59,40 @@ public static Map> groupPartitionDataByTopic(Map> groupPartitionsByTopic(Collection partitions) { - Map> partitionsByTopic = new HashMap<>(); + return groupPartitionsByTopic( + partitions, + topic -> new ArrayList<>(), + List::add + ); + } + + /** + * Group a collection of partitions by topic + * + * @param partitions + * @param buildGroup + * @param addToGroup + * @param + * @return The map used to group the partitions + */ + public static Map groupPartitionsByTopic( + Collection partitions, + Function buildGroup, + BiConsumer addToGroup + ) { + Map dataByTopic = new HashMap<>(); for (TopicPartition tp : partitions) { String topic = tp.topic(); - List topicData = partitionsByTopic.computeIfAbsent(topic, t -> new ArrayList<>()); - topicData.add(tp.partition()); + T topicData = dataByTopic.computeIfAbsent(topic, buildGroup); + addToGroup.accept(topicData, tp.partition()); } - return partitionsByTopic; + return dataByTopic; } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index d89d57ac9e355..119181a4fa101 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -1271,4 +1271,10 @@ public static Map propsToMap(Properties properties) { } return map; } + + public static Map initializeMap(Collection keys, Supplier valueSupplier) { + Map res = new HashMap<>(); + keys.forEach(key -> res.put(key, valueSupplier.get())); + return res; + } } diff --git a/clients/src/main/resources/common/message/DescribeProducersRequest.json b/clients/src/main/resources/common/message/DescribeProducersRequest.json new file mode 100644 index 0000000000000..dab6eff5f1bb8 --- /dev/null +++ b/clients/src/main/resources/common/message/DescribeProducersRequest.json @@ -0,0 +1,31 @@ +// 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. + +{ + "apiKey": 52, + "type": "request", + "name": "DescribeProducersRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "Topics", "type": "[]TopicRequest", "versions": "0+", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+", + "about": "The indexes of the partitions to list producers for." } + ]} + ]} + ] +} diff --git a/clients/src/main/resources/common/message/DescribeProducersResponse.json b/clients/src/main/resources/common/message/DescribeProducersResponse.json new file mode 100644 index 0000000000000..97081f904e749 --- /dev/null +++ b/clients/src/main/resources/common/message/DescribeProducersResponse.json @@ -0,0 +1,45 @@ +// 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. + +{ + "apiKey": 52, + "type": "response", + "name": "DescribeProducersResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "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": "Topics", "type": "[]TopicResponse", "versions": "0+", + "about": "Each topic in the response.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name" }, + { "name": "Partitions", "type": "[]PartitionResponse", "versions": "0+", + "about": "Each partition in the response.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The partition error code, or 0 if there was no error." }, + { "name": "ActiveProducers", "type": "[]ProducerState", "versions": "0+", "fields": [ + { "name": "ProducerId", "type": "int64", "versions": "0+" }, + { "name": "ProducerEpoch", "type": "int32", "versions": "0+" }, + { "name": "LastSequence", "type": "int32", "versions": "0+", "default": "-1" }, + { "name": "LastTimestamp", "type": "int64", "versions": "0+", "default": "-1" }, + { "name": "CurrentTxnStartTimestamp", "type": "int64", "versions": "0+", "default": "-1" } + ]} + ]} + ]} + ] +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 03a50c8e9ffc8..111b58b18aea6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -818,12 +818,17 @@ else if (spec instanceof OffsetSpec.EarliestSpec) @Override public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) { - throw new UnsupportedOperationException("Not implement yet"); + throw new UnsupportedOperationException("Not implemented yet"); } @Override public AlterClientQuotasResult alterClientQuotas(Collection entries, AlterClientQuotasOptions options) { - throw new UnsupportedOperationException("Not implement yet"); + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminRequestUtil.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminRequestUtil.java new file mode 100644 index 0000000000000..0840989461af2 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminRequestUtil.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.util.Map; + +public class AdminRequestUtil { + + private AdminRequestUtil() {} + + public static MetadataResponseData metadataResponse( + Map partitionResponses + ) { + MetadataResponseData response = new MetadataResponseData(); + + Map> partitionResponsesByTopic = + CollectionUtils.groupPartitionDataByTopic(partitionResponses); + + for (Map.Entry> entry : partitionResponsesByTopic.entrySet()) { + String topic = entry.getKey(); + Map partitionResponseMap = entry.getValue(); + + MetadataResponseTopic topicResponse = new MetadataResponseTopic(); + response.topics().add(topicResponse.setName(topic)); + + for (Map.Entry partitionEntry : partitionResponseMap.entrySet()) { + Integer partitionId = partitionEntry.getKey(); + MetadataResponsePartition partitionResponse = partitionEntry.getValue(); + topicResponse.partitions().add(partitionResponse.setPartitionIndex(partitionId)); + } + } + + return response; + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java new file mode 100644 index 0000000000000..ef67aa03ef665 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.message.DescribeGroupsRequestData; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.DescribeGroupsRequest; +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.MockTime; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.OptionalInt; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Collections.singletonList; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class CoordinatorRequestDriverTest { + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testLookupGrouping() { + CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); + CoordinatorKey group2 = new CoordinatorKey("bar", CoordinatorType.GROUP); + Set groupIds = mkSet(group1, group2); + + TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + List.RequestSpec> requests = driver.poll(); + assertEquals(2, requests.size()); + + // While the FindCoordinator requests are inflight, we will not send any more + assertEquals(0, driver.poll().size()); + + RequestDriver.RequestSpec spec1 = requests.stream() + .filter(spec -> spec.keys.contains(group1)) + .findFirst() + .get(); + + assertEquals(mkSet(group1), spec1.keys); + assertEquals(OptionalInt.empty(), spec1.scope.destinationBrokerId()); + assertEquals(deadlineMs, spec1.deadlineMs); + assertEquals(0, spec1.tries); + assertEquals(0, spec1.nextAllowedTryMs); + assertTrue(spec1.request instanceof FindCoordinatorRequest.Builder); + FindCoordinatorRequest.Builder findCoordinatorRequest1 = (FindCoordinatorRequest.Builder) spec1.request; + assertEquals(group1.key, findCoordinatorRequest1.data().key()); + assertEquals(group1.type.id(), findCoordinatorRequest1.data().keyType()); + + RequestDriver.RequestSpec spec2 = requests.stream() + .filter(spec -> spec.keys.contains(group2)) + .findFirst() + .get(); + + assertEquals(mkSet(group2), spec2.keys); + assertEquals(OptionalInt.empty(), spec2.scope.destinationBrokerId()); + assertEquals(deadlineMs, spec2.deadlineMs); + assertEquals(0, spec2.tries); + assertEquals(0, spec2.nextAllowedTryMs); + assertTrue(spec2.request instanceof FindCoordinatorRequest.Builder); + FindCoordinatorRequest.Builder findCoordinatorRequest2 = (FindCoordinatorRequest.Builder) spec2.request; + assertEquals(group2.key, findCoordinatorRequest2.data().key()); + assertEquals(group2.type.id(), findCoordinatorRequest2.data().keyType()); + } + + @Test + public void testSuccessfulLeaderDiscovery() { + CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); + CoordinatorKey group2 = new CoordinatorKey("bar", CoordinatorType.GROUP); + Set groupIds = mkSet(group1, group2); + + TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(2, lookupRequests.size()); + + RequestDriver.RequestSpec lookupSpec1 = lookupRequests.stream() + .filter(spec -> spec.keys.contains(group1)) + .findFirst() + .get(); + + driver.onResponse(time.milliseconds(), lookupSpec1, new FindCoordinatorResponse(new FindCoordinatorResponseData() + .setErrorCode(Errors.NONE.code()) + .setHost("localhost") + .setPort(9092) + .setNodeId(1) + )); + + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + RequestDriver.RequestSpec requestSpec1 = requests1.get(0); + assertEquals(mkSet(group1), requestSpec1.keys); + assertEquals(OptionalInt.of(1), requestSpec1.scope.destinationBrokerId()); + assertEquals(0, requestSpec1.tries); + assertEquals(deadlineMs, requestSpec1.deadlineMs); + assertEquals(0, requestSpec1.nextAllowedTryMs); + assertTrue(requestSpec1.request instanceof DescribeGroupsRequest.Builder); + DescribeGroupsRequest.Builder request = (DescribeGroupsRequest.Builder) requestSpec1.request; + assertEquals(singletonList(group1.key), request.data.groups()); + + RequestDriver.RequestSpec lookupSpec2 = lookupRequests.stream() + .filter(spec -> spec.keys.contains(group2)) + .findFirst() + .get(); + + driver.onResponse(time.milliseconds(), lookupSpec2, new FindCoordinatorResponse(new FindCoordinatorResponseData() + .setErrorCode(Errors.NONE.code()) + .setHost("localhost") + .setPort(9093) + .setNodeId(2) + )); + + List.RequestSpec> requests2 = driver.poll(); + assertEquals(1, requests2.size()); + RequestDriver.RequestSpec requestSpec2 = requests2.get(0); + assertEquals(mkSet(group2), requestSpec2.keys); + assertEquals(OptionalInt.of(2), requestSpec2.scope.destinationBrokerId()); + assertEquals(0, requestSpec2.tries); + assertEquals(deadlineMs, requestSpec2.deadlineMs); + assertEquals(0, requestSpec2.nextAllowedTryMs); + assertTrue(requestSpec2.request instanceof DescribeGroupsRequest.Builder); + DescribeGroupsRequest.Builder request2 = (DescribeGroupsRequest.Builder) requestSpec2.request; + assertEquals(singletonList(group2.key), request2.data.groups()); + } + + @Test + public void testRetriableFindCoordinatorError() { + CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); + Set groupIds = mkSet(group1); + + TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + + RequestDriver.RequestSpec lookupSpec = lookupRequests1.get(0); + driver.onResponse(time.milliseconds(), lookupSpec, new FindCoordinatorResponse(new FindCoordinatorResponseData() + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + )); + + List.RequestSpec> lookupRequests2 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + + RequestDriver.RequestSpec retryLookupSpec = lookupRequests2.get(0); + assertEquals(1, retryLookupSpec.tries); + assertEquals(time.milliseconds() + retryBackoffMs, retryLookupSpec.nextAllowedTryMs); + assertEquals(deadlineMs, retryLookupSpec.deadlineMs); + assertEquals(groupIds, retryLookupSpec.keys); + assertEquals(OptionalInt.empty(), retryLookupSpec.scope.destinationBrokerId()); + } + + @Test + public void testFatalFindCoordinatorError() { + CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); + Set groupIds = mkSet(group1); + + TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + + RequestDriver.RequestSpec lookupSpec = lookupRequests1.get(0); + driver.onResponse(time.milliseconds(), lookupSpec, new FindCoordinatorResponse(new FindCoordinatorResponseData() + .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code()) + )); + + assertEquals(Collections.emptyList(), driver.poll()); + GroupAuthorizationException groupAuthorizationException = assertFutureThrows( + driver.futures().get(group1), GroupAuthorizationException.class); + assertEquals(group1.key, groupAuthorizationException.groupId()); + } + + private final class TestCoordinatorRequestDriver extends CoordinatorRequestDriver { + + public TestCoordinatorRequestDriver(Set groupIds) { + super(groupIds, deadlineMs, retryBackoffMs); + } + + @Override + AbstractRequest.Builder buildFulfillmentRequest(Set coordinatorKeys) { + return new DescribeGroupsRequest.Builder(new DescribeGroupsRequestData() + .setGroups(coordinatorKeys.stream() + .map(coordinatorKey -> coordinatorKey.key) + .collect(Collectors.toList()))); + } + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse response) { + throw new UnsupportedOperationException(); + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java new file mode 100644 index 0000000000000..300f1fc998d6c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.DescribeProducersOptions; +import org.apache.kafka.clients.admin.DescribeProducersResult; +import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.DescribeProducersRequestData.TopicRequest; +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.message.DescribeProducersResponseData.PartitionResponse; +import org.apache.kafka.common.message.DescribeProducersResponseData.ProducerState; +import org.apache.kafka.common.message.DescribeProducersResponseData.TopicResponse; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.DescribeProducersRequest; +import org.apache.kafka.common.requests.DescribeProducersResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.MockTime; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.OptionalInt; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.clients.admin.internals.AdminRequestUtil.metadataResponse; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class DescribeProducersRequestDriverTest { + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testSuccessfulResponseWithoutProvidedBrokerId() throws Exception { + TopicPartition tp = new TopicPartition("foo", 5); + int leaderId = 1; + DescribeProducersOptions options = new DescribeProducersOptions(); + + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + singleton(tp), + options, + deadlineMs, + retryBackoffMs + ); + + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(1, lookupRequests.size()); + + RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + assertEquals(OptionalInt.empty(), lookupSpec.scope.destinationBrokerId()); + assertEquals(0, lookupSpec.tries); + assertEquals(0, lookupSpec.nextAllowedTryMs); + assertEquals(deadlineMs, lookupSpec.deadlineMs); + assertEquals(singleton(tp), lookupSpec.keys); + + assertTrue(lookupSpec.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder lookupRequest = (MetadataRequest.Builder) lookupSpec.request; + assertEquals(singletonList(tp.topic()), lookupRequest.topics()); + + driver.onResponse(time.milliseconds(), lookupSpec, new MetadataResponse(metadataResponse( + singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(leaderId) + .setLeaderEpoch(15) + .setReplicaNodes(asList(1, 2, 3)) + .setIsrNodes(asList(1, 2, 3))) + ))); + + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + RequestDriver.RequestSpec spec = requests.get(0); + assertEquals(OptionalInt.of(leaderId), spec.scope.destinationBrokerId()); + assertEquals(singleton(tp), spec.keys); + assertEquals(0, spec.tries); + assertEquals(0, spec.nextAllowedTryMs); + assertEquals(deadlineMs, spec.deadlineMs); + assertSuccessfulFulfillment(driver, tp, spec); + } + + @Test + public void testRetryLookupAfterNotLeaderErrorWithoutProvidedBrokerId() { + TopicPartition tp = new TopicPartition("foo", 5); + int leaderId = 1; + DescribeProducersOptions options = new DescribeProducersOptions(); + + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + singleton(tp), + options, + deadlineMs, + retryBackoffMs + ); + + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + assertEquals(singleton(tp), lookupSpec1.keys); + assertTrue(lookupSpec1.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder lookupRequest1 = (MetadataRequest.Builder) lookupSpec1.request; + assertEquals(singletonList(tp.topic()), lookupRequest1.topics()); + + driver.onResponse(time.milliseconds(), lookupSpec1, new MetadataResponse(metadataResponse( + singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(leaderId) + .setLeaderEpoch(15) + .setReplicaNodes(asList(1, 2, 3)) + .setIsrNodes(asList(1, 2, 3))) + ))); + + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + // A `NOT_LEADER_OR_FOLLOWER` error should cause a retry of the `Metadata` request + RequestDriver.RequestSpec spec = requests.get(0); + driver.onResponse(time.milliseconds(), spec, describeProducersResponse( + singletonMap(tp, new PartitionResponse() + .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code())) + )); + + List.RequestSpec> lookupRequests2 = driver.poll(); + assertEquals(1, lookupRequests2.size()); + + RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); + assertEquals(OptionalInt.empty(), lookupSpec2.scope.destinationBrokerId()); + assertEquals(singleton(tp), lookupSpec2.keys); + assertEquals(1, lookupSpec2.tries); + assertEquals(time.milliseconds() + retryBackoffMs, lookupSpec2.nextAllowedTryMs); + assertEquals(deadlineMs, lookupSpec2.deadlineMs); + + assertTrue(lookupSpec2.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder lookupRequest2 = (MetadataRequest.Builder) lookupSpec2.request; + assertEquals(singletonList(tp.topic()), lookupRequest2.topics()); + } + + @Test + public void testSuccessfulResponseWithProvidedBrokerId() throws Exception { + TopicPartition tp = new TopicPartition("foo", 5); + int brokerId = 1; + DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); + + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + singleton(tp), + options, + deadlineMs, + retryBackoffMs + ); + + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + // Note there should be no `Metadata` lookup since we specified the target brokerId directly + RequestDriver.RequestSpec spec = requests.get(0); + assertEquals(singleton(tp), spec.keys); + assertEquals(OptionalInt.of(brokerId), spec.scope.destinationBrokerId()); + assertEquals(0, spec.tries); + assertEquals(0, spec.nextAllowedTryMs); + assertEquals(deadlineMs, spec.deadlineMs); + + assertTrue(spec.request instanceof DescribeProducersRequest.Builder); + DescribeProducersRequest.Builder request = (DescribeProducersRequest.Builder) spec.request; + assertEquals(1, request.data.topics().size()); + TopicRequest topicRequest = request.data.topics().get(0); + assertEquals(tp.topic(), topicRequest.name()); + assertEquals(singletonList(tp.partition()), topicRequest.partitionIndexes()); + assertSuccessfulFulfillment(driver, tp, spec); + } + + @Test + public void testNotLeaderErrorWithProvidedBrokerId() { + TopicPartition tp = new TopicPartition("foo", 0); + int brokerId = 1; + DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); + + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + singleton(tp), + options, + deadlineMs, + retryBackoffMs + ); + + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + // Note there should be no `Metadata` lookup since we specified the target brokerId directly + RequestDriver.RequestSpec spec = requests.get(0); + assertEquals(singleton(tp), spec.keys); + assertEquals(OptionalInt.of(brokerId), spec.scope.destinationBrokerId()); + assertTrue(spec.request instanceof DescribeProducersRequest.Builder); + + driver.onResponse(time.milliseconds(), spec, describeProducersResponse(singletonMap(tp, + new PartitionResponse().setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) + ))); + + assertEquals(Collections.emptyList(), driver.poll()); + assertFutureThrows(driver.futures().get(tp), NotLeaderOrFollowerException.class); + } + + @Test + public void testFatalErrorWithoutProvidedBrokerId() { + TopicPartition tp = new TopicPartition("foo", 5); + int leaderId = 1; + DescribeProducersOptions options = new DescribeProducersOptions(); + + DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + singleton(tp), + options, + deadlineMs, + retryBackoffMs + ); + + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + assertEquals(singleton(tp), lookupSpec1.keys); + assertTrue(lookupSpec1.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder lookupRequest1 = (MetadataRequest.Builder) lookupSpec1.request; + assertEquals(singletonList(tp.topic()), lookupRequest1.topics()); + + driver.onResponse(time.milliseconds(), lookupSpec1, new MetadataResponse(metadataResponse( + singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(leaderId) + .setLeaderEpoch(15) + .setReplicaNodes(asList(1, 2, 3)) + .setIsrNodes(asList(1, 2, 3))) + ))); + + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + RequestDriver.RequestSpec spec = requests.get(0); + driver.onResponse(time.milliseconds(), spec, describeProducersResponse( + singletonMap(tp, new PartitionResponse() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())) + )); + + assertEquals(Collections.emptyList(), driver.poll()); + assertFutureThrows(driver.futures().get(tp), UnknownServerException.class); + } + + private void assertSuccessfulFulfillment( + DescribeProducersRequestDriver driver, + TopicPartition topicPartition, + RequestDriver.RequestSpec describeProducerSpec + ) throws Exception { + List activeProducers = sampleProducerState(); + driver.onResponse(time.milliseconds(), describeProducerSpec, describeProducersResponse( + singletonMap(topicPartition, new PartitionResponse() + .setErrorCode(Errors.NONE.code()) + .setActiveProducers(activeProducers) + ))); + + assertEquals(Collections.emptyList(), driver.poll()); + KafkaFutureImpl future = driver.futures().get(topicPartition); + assertTrue(future.isDone()); + PartitionProducerState partitionProducerState = future.get(); + assertEquals(2, partitionProducerState.activeProducers().size()); + assertMatchingProducers(activeProducers, partitionProducerState.activeProducers()); + } + + private List sampleProducerState() { + return asList( + new ProducerState() + .setProducerId(12345L) + .setProducerEpoch(15) + .setLastSequence(75) + .setLastTimestamp(time.milliseconds()) + .setCurrentTxnStartTimestamp(-1L), + new ProducerState() + .setProducerId(98765L) + .setProducerEpoch(30) + .setLastSequence(150) + .setLastTimestamp(time.milliseconds()) + .setCurrentTxnStartTimestamp(time.milliseconds()) + ); + } + + private void assertMatchingProducers( + List expected, + List actual + ) { + assertEquals(expected.size(), actual.size()); + + Map expectedByProducerId = expected.stream().collect(Collectors.toMap( + ProducerState::producerId, + Function.identity() + )); + + for (DescribeProducersResult.ProducerState actualProducerState : actual) { + ProducerState expectedProducerState = expectedByProducerId.get(actualProducerState.producerId()); + assertNotNull(expectedProducerState); + assertEquals(expectedProducerState.producerEpoch(), actualProducerState.producerEpoch()); + assertEquals(expectedProducerState.lastSequence(), actualProducerState.lastSequence()); + assertEquals(expectedProducerState.lastTimestamp(), actualProducerState.lastTimestamp()); + assertEquals(expectedProducerState.currentTxnStartTimestamp(), + actualProducerState.currentTransactionStartOffset().orElse(-1L)); + } + } + + private DescribeProducersResponse describeProducersResponse( + Map partitionResponses + ) { + DescribeProducersResponseData response = new DescribeProducersResponseData(); + Map> partitionResponsesByTopic = + CollectionUtils.groupPartitionDataByTopic(partitionResponses); + + for (Map.Entry> topicEntry : partitionResponsesByTopic.entrySet()) { + String topic = topicEntry.getKey(); + Map topicPartitionResponses = topicEntry.getValue(); + + TopicResponse topicResponse = new TopicResponse().setName(topic); + response.topics().add(topicResponse); + + for (Map.Entry partitionEntry : topicPartitionResponses.entrySet()) { + Integer partitionId = partitionEntry.getKey(); + PartitionResponse partitionResponse = partitionEntry.getValue(); + topicResponse.partitions().add(partitionResponse.setPartitionIndex(partitionId)); + } + } + + return new DescribeProducersResponse(response); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java new file mode 100644 index 0000000000000..7cbf60e908500 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -0,0 +1,306 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.DescribeProducersRequest; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.OptionalInt; +import java.util.Set; + +import static java.util.Arrays.asList; +import static org.apache.kafka.clients.admin.internals.AdminRequestUtil.metadataResponse; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class MetadataRequestDriverTest { + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testLookupGrouping() { + Set topicPartitions = mkSet( + new TopicPartition("foo", 0), + new TopicPartition("foo", 2), + new TopicPartition("bar", 1)); + + TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + // While a Metadata request is inflight, we will not send another + assertEquals(0, driver.poll().size()); + + RequestDriver.RequestSpec spec = requests.get(0); + assertEquals(topicPartitions, spec.keys); + assertEquals(0, spec.tries); + assertEquals(deadlineMs, spec.deadlineMs); + assertEquals(0, spec.nextAllowedTryMs); + assertTrue(spec.request instanceof MetadataRequest.Builder); + + MetadataRequest.Builder metadataRequest = (MetadataRequest.Builder) spec.request; + assertEquals(mkSet("foo", "bar"), new HashSet<>(metadataRequest.topics())); + } + + @Test + public void testSuccessfulLeaderDiscovery() { + TopicPartition tp0 = new TopicPartition("foo", 0); + TopicPartition tp1 = new TopicPartition("foo", 1); + TopicPartition tp2 = new TopicPartition("foo", 2); + + // Request includes 2 of 3 partitions for the topic + Set topicPartitions = mkSet(tp0, tp2); + + TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + + RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( + mkEntry(tp0, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(1) + .setLeaderEpoch(15) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))), + mkEntry(tp1, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(2) + .setLeaderEpoch(37) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))), + mkEntry(tp2, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(3) + .setLeaderEpoch(99) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))) + )))); + + // We should have two fulfillment requests now sent to the leaders of partitions 0 and 2 + List.RequestSpec> requests2 = driver.poll(); + assertEquals(2, requests2.size()); + + RequestDriver.RequestSpec spec0 = requests2.stream() + .filter(spec -> spec.keys.contains(tp0)) + .findFirst() + .get(); + assertEquals(mkSet(tp0), spec0.keys); + assertEquals(OptionalInt.of(1), spec0.scope.destinationBrokerId()); + + RequestDriver.RequestSpec spec1 = requests2.stream() + .filter(spec -> spec.keys.contains(tp2)) + .findFirst() + .get(); + assertEquals(mkSet(tp2), spec1.keys); + assertEquals(OptionalInt.of(3), spec1.scope.destinationBrokerId()); + } + + private RequestDriver.RequestSpec findRequest(TopicPartition tp) { + + } + + @Test + public void testRetryLeaderDiscovery() { + TopicPartition tp0 = new TopicPartition("foo", 0); + TopicPartition tp1 = new TopicPartition("foo", 1); + TopicPartition tp2 = new TopicPartition("bar", 0); + + // Request includes 2 of 3 partitions for the topic + Set topicPartitions = mkSet(tp0, tp2); + + TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + + RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( + mkEntry(tp0, new MetadataResponsePartition() + .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code())), + mkEntry(tp1, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(2) + .setLeaderEpoch(37) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))), + mkEntry(tp2, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(3) + .setLeaderEpoch(99) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))) + )))); + + List.RequestSpec> requests2 = driver.poll(); + assertEquals(2, requests2.size()); + + // We should retry the Metadata request with only one of the two topics + RequestDriver.RequestSpec spec0 = requests2.stream() + .filter(spec -> spec.keys.contains(tp0)) + .findFirst() + .get(); + assertEquals(mkSet(tp0), spec0.keys); + assertEquals(OptionalInt.empty(), spec0.scope.destinationBrokerId()); + assertTrue(spec0.request instanceof MetadataRequest.Builder); + assertEquals(1, spec0.tries); + assertEquals(time.milliseconds() + retryBackoffMs, spec0.nextAllowedTryMs); + MetadataRequest.Builder retryMetadataRequest = (MetadataRequest.Builder) spec0.request; + assertEquals(mkSet("foo"), new HashSet<>(retryMetadataRequest.topics())); + + RequestDriver.RequestSpec spec1 = requests2.stream() + .filter(spec -> spec.keys.contains(tp2)) + .findFirst() + .get(); + assertEquals(mkSet(tp2), spec1.keys); + assertEquals(OptionalInt.of(3), spec1.scope.destinationBrokerId()); + } + + @Test + public void testFatalTopicError() { + TopicPartition tp0 = new TopicPartition("foo", 0); + TopicPartition tp2 = new TopicPartition("bar", 0); + Set topicPartitions = mkSet(tp0, tp2); + + TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + + RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + MetadataResponseData metadataResponseData = metadataResponse(Utils.mkMap( + mkEntry(tp2, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(3) + .setLeaderEpoch(99) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))) + )); + + metadataResponseData.topics().add(new MetadataResponseTopic() + .setName("foo") + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code()) + ); + + driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponseData)); + + List.RequestSpec> requests2 = driver.poll(); + assertEquals(1, requests2.size()); + + // The lookup for "foo" should fail and not be retried + TopicAuthorizationException topicAuthorizationException = TestUtils.assertFutureThrows( + driver.futures().get(tp0), TopicAuthorizationException.class); + assertEquals(mkSet("foo"), topicAuthorizationException.unauthorizedTopics()); + + // However, the lookup for the other other topic should proceed + RequestDriver.RequestSpec spec = requests2.get(0); + assertEquals(mkSet(tp2), spec.keys); + assertEquals(OptionalInt.of(3), spec.scope.destinationBrokerId()); + } + + @Test + public void testFatalPartitionError() { + TopicPartition tp0 = new TopicPartition("foo", 0); + TopicPartition tp1 = new TopicPartition("foo", 1); + TopicPartition tp2 = new TopicPartition("bar", 0); + + // Request includes 2 of 3 partitions for the topic + Set topicPartitions = mkSet(tp0, tp2); + + TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + + RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + + // Any unexpected partition error will cause the partition to fail + driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( + mkEntry(tp0, new MetadataResponsePartition() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())), + mkEntry(tp1, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(2) + .setLeaderEpoch(37) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))), + mkEntry(tp2, new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(3) + .setLeaderEpoch(99) + .setIsrNodes(asList(1, 2, 3)) + .setReplicaNodes(asList(1, 2, 3))) + )))); + + List.RequestSpec> requests2 = driver.poll(); + assertEquals(1, requests2.size()); + + // The lookup for "foo-0" should fail and not be retried + TestUtils.assertFutureThrows(driver.futures().get(tp0), UnknownServerException.class); + + // However, the lookup for the other other partition should proceed + RequestDriver.RequestSpec spec = requests2.get(0); + assertEquals(mkSet(tp2), spec.keys); + assertEquals(OptionalInt.of(3), spec.scope.destinationBrokerId()); + } + + private final class TestMetadataRequestDriver extends MetadataRequestDriver { + + public TestMetadataRequestDriver(Collection futures) { + super(futures, deadlineMs, retryBackoffMs); + } + + @Override + AbstractRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + DescribeProducersRequestData request = new DescribeProducersRequestData(); + DescribeProducersRequest.Builder builder = new DescribeProducersRequest.Builder(request); + + CollectionUtils.groupPartitionsByTopic( + topicPartitions, + builder::addTopic, + (topicRequest, partitionId) -> topicRequest.partitionIndexes().add(partitionId) + ); + + return builder; + } + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse response) { + throw new UnsupportedOperationException(); + } + } + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 4af683d7f64d2..ce7e68e92776e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -87,6 +87,8 @@ import org.apache.kafka.common.message.DescribeGroupsRequestData; import org.apache.kafka.common.message.DescribeGroupsResponseData; import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.DescribeProducersResponseData; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.EndTxnRequestData; @@ -491,6 +493,9 @@ public void testSerialization() throws Exception { checkRequest(createAlterClientQuotasRequest(), true); checkErrorResponse(createAlterClientQuotasRequest(), unknownServerException, true); checkResponse(createAlterClientQuotasResponse(), 0, true); + checkRequest(createDescribeProducersRequest(), true); + checkErrorResponse(createDescribeProducersRequest(), unknownServerException, true); + checkResponse(createDescribeProducersResponse(), 0, true); } @Test @@ -2388,4 +2393,36 @@ private AlterClientQuotasResponse createAlterClientQuotasResponse() { ClientQuotaEntity entity = new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, "user")); return new AlterClientQuotasResponse(Collections.singletonMap(entity, ApiError.NONE), 0); } + + private DescribeProducersRequest createDescribeProducersRequest() { + DescribeProducersRequestData data = new DescribeProducersRequestData(); + DescribeProducersRequestData.TopicRequest topicRequest = new DescribeProducersRequestData.TopicRequest(); + topicRequest.partitionIndexes().add(0); + topicRequest.partitionIndexes().add(1); + data.topics().add(topicRequest); + return new DescribeProducersRequest.Builder(data).build(ApiKeys.DESCRIBE_PRODUCERS.latestVersion()); + } + + private DescribeProducersResponse createDescribeProducersResponse() { + DescribeProducersResponseData data = new DescribeProducersResponseData(); + DescribeProducersResponseData.TopicResponse topicResponse = new DescribeProducersResponseData.TopicResponse(); + topicResponse.partitions().add(new DescribeProducersResponseData.PartitionResponse() + .setErrorCode(Errors.NONE.code()) + .setPartitionIndex(0) + .setActiveProducers(Arrays.asList( + new DescribeProducersResponseData.ProducerState() + .setProducerId(1234L) + .setProducerEpoch(15) + .setLastTimestamp(13490218304L) + .setCurrentTxnStartTimestamp(13490218304L), + new DescribeProducersResponseData.ProducerState() + .setProducerId(9876L) + .setProducerEpoch(32) + .setLastTimestamp(13490218399L) + )) + ); + data.topics().add(topicResponse); + return new DescribeProducersResponse(data); + } + } diff --git a/core/src/main/scala/kafka/admin/TransactionCommand.scala b/core/src/main/scala/kafka/admin/TransactionCommand.scala new file mode 100644 index 0000000000000..12c921e7efa7a --- /dev/null +++ b/core/src/main/scala/kafka/admin/TransactionCommand.scala @@ -0,0 +1,179 @@ +/* + * 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 kafka.admin + +import java.util.Properties + +import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Logging} +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeProducersOptions} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.utils.Utils + +import scala.jdk.CollectionConverters._ + +object TransactionCommand extends Logging { + + private def createAdmin(commandOptions: TransactionCommandOptions): Admin = { + val props = Option(commandOptions.options.valueOf(commandOptions.adminClientConfig)).map { config => + Utils.loadProps(config) + }.getOrElse(new Properties()) + + props.setProperty( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, + commandOptions.options.valueOf(commandOptions.bootstrapServer) + ) + + Admin.create(props) + } + + private def prettyPrintTable( + headers: Array[String], + rows: Iterable[Array[String]] + ): Unit = { + val columnLengths = headers.map(_.length) + for (row <- rows) { + row.indices.foreach { i => + columnLengths(i) = math.max(columnLengths(i), row(i).length) + } + } + + def printColumn(str: String, len: Int): Unit = { + val padLength = len - str.length + print(str + (" " * padLength)) + } + + def printRow(row: Array[String]): Unit = { + columnLengths.indices.foreach { i => + val columnLength = columnLengths(i) + val columnValue = row(i) + printColumn(columnValue, columnLength) + print('\t') + } + println() + } + + printRow(headers) + rows.foreach(printRow) + } + + private def describeProducers( + admin: Admin, + brokerId: Option[Int], + topicPartition: TopicPartition + ): Unit = { + val options = new DescribeProducersOptions() + brokerId.foreach(options.setBrokerId) + + val result = admin.describeProducers(Seq(topicPartition).asJava, options) + .partitionResult(topicPartition).get() + + val headers = Array( + "ProducerId", "ProducerEpoch", "LastSequence", "LastTimestamp", "CurrentTransactionStartOffset" + ) + + val rows = result.activeProducers().asScala.map { activeProducer => + val currentTransactionStartOffsetColumnValue = if (activeProducer.currentTransactionStartOffset.isPresent) { + activeProducer.currentTransactionStartOffset.toString + } else { + "None" + } + + Array(activeProducer.producerId.toString, + activeProducer.producerEpoch.toString, + activeProducer.lastSequence.toString, + activeProducer.lastTimestamp.toString, + currentTransactionStartOffsetColumnValue + ) + } + + prettyPrintTable(headers, rows) + } + + def main(args: Array[String]): Unit = { + val commandOptions = new TransactionCommandOptions(args) + CommandLineUtils.printHelpAndExitIfNeeded( + commandOptions, + "This tool attempts to elect a new leader for a set of topic partitions. The type of elections supported are preferred replicas and unclean replicas." + ) + + CommandLineUtils.printHelpAndExitIfNeeded(commandOptions, "This tool is used to analyze transaction " + + "state and recover from hanging transactions") + + val admin = createAdmin(commandOptions) + val brokerId = Option(commandOptions.options.valueOf(commandOptions.brokerId)).map(Int.unbox) + val topicPartitionOpt = ( + Option(commandOptions.options.valueOf(commandOptions.topic)), + Option(commandOptions.options.valueOf(commandOptions.partition)) + ) match { + case (Some(topic), Some(partition)) => Some(new TopicPartition(topic, partition)) + case _ => None + } + + topicPartitionOpt.foreach { topicPartition => + describeProducers(admin, brokerId, topicPartition) + } + } +} + +private final class TransactionCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { + val bootstrapServer = parser + .accepts("bootstrap-server", + "(REQUIRED) A hostname and port for the broker to connect to, in the form host:port. " + + "Multiple comma separated URLs can be given.") + .withRequiredArg + .describedAs("host:port") + .ofType(classOf[String]) + val adminClientConfig = parser + .accepts("admin.config", + "Configuration properties files to pass to the admin client") + .withRequiredArg + .describedAs("config file") + .ofType(classOf[String]) + + val topic = parser + .accepts("topic", + "Name of topic for which to perform an election.") + .withRequiredArg + .describedAs("topic name") + .ofType(classOf[String]) + + val partition = parser + .accepts("partition", + "Partition id. REQUIRED if --topic is specified.") + .withRequiredArg + .describedAs("partition id") + .ofType(classOf[Integer]) + + val brokerId = parser + .accepts("broker-id", + "Used with --topic and --partition to indicate a specific broker to verify") + .withRequiredArg + .describedAs("broker id") + .ofType(classOf[Integer]) + + val describeProducersOptions = parser + .accepts("describe-producers", + "Used to describe active transactional/idempotent producers " + + "writing to a specific topic partition (you must specify --topic and --partition)") + + options = parser.parse(args: _*) + + if (Seq(describeProducersOptions).count(options.has) != 1) { + CommandLineUtils.printUsageAndDie(parser, + "Command must include exactly one action: --describe-producers") + } +} diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 1508690fdb1b3..0ae25ce3cb11c 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -31,6 +31,7 @@ import kafka.utils._ import kafka.zk.{AdminZkClient, KafkaZkClient} import kafka.zookeeper.ZooKeeperClientException import org.apache.kafka.common.errors._ +import org.apache.kafka.common.message.DescribeProducersResponseData import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.protocol.Errors @@ -368,6 +369,23 @@ class Partition(val topicPartition: TopicPartition, log.filter(_ => isLeader) } + def activeProducerState: DescribeProducersResponseData.PartitionResponse = { + val producerState = new DescribeProducersResponseData.PartitionResponse() + .setPartitionIndex(topicPartition.partition()) + + log.map(_.activeProducers) match { + case Some(producers) => + producerState + .setErrorCode(Errors.NONE.code) + .setActiveProducers(producers.asJava) + case None => + producerState + .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code) + } + + producerState + } + /** * Returns true if this node is currently leader for the Partition. */ diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 819957f60a2a4..d8553ed4547be 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -36,6 +36,7 @@ import kafka.server.epoch.LeaderEpochFileCache import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, LogDirFailureChannel, LogOffsetMetadata, OffsetAndEpoch} import kafka.utils._ import org.apache.kafka.common.errors._ +import org.apache.kafka.common.message.DescribeProducersResponseData import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record._ @@ -942,6 +943,19 @@ class Log(@volatile private var _dir: File, completedTxns.foreach(producerStateManager.completeTxn) } + def activeProducers: Seq[DescribeProducersResponseData.ProducerState] = { + lock synchronized { + producerStateManager.activeProducers.map { case (producerId, state) => + new DescribeProducersResponseData.ProducerState() + .setProducerId(producerId) + .setProducerEpoch(state.producerEpoch) + .setLastSequence(state.lastSeq) + .setLastTimestamp(state.lastTimestamp) + .setCurrentTxnStartTimestamp(state.currentTxnFirstOffset.getOrElse(-1L)) + } + }.toSeq + } + private[log] def activeProducersWithLastSequence: Map[Long, Int] = lock synchronized { producerStateManager.activeProducers.map { case (producerId, producerIdEntry) => (producerId, producerIdEntry.lastSeq) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 3b0cf8039c43a..10b94a59f220b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -51,7 +51,7 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -77,7 +77,6 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time, Utils} import org.apache.kafka.common.{Node, TopicPartition} -import org.apache.kafka.common.message.MetadataResponseData import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} import org.apache.kafka.server.authorizer._ @@ -181,6 +180,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.ALTER_CLIENT_QUOTAS => handleAlterClientQuotasRequest(request) case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => handleDescribeUserScramCredentialsRequest(request) case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => handleAlterUserScramCredentialsRequest(request) + case ApiKeys.DESCRIBE_PRODUCERS => handleDescribeProducersRequest(request) } } catch { case e: FatalExitError => throw e @@ -3054,6 +3054,44 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def handleDescribeProducersRequest(request: RequestChannel.Request): Unit = { + val describeProducersRequest = request.body[DescribeProducersRequest] + + def partitionError(topicPartition: TopicPartition, error: Errors): DescribeProducersResponseData.PartitionResponse = { + new DescribeProducersResponseData.PartitionResponse() + .setPartitionIndex(topicPartition.partition) + .setErrorCode(error.code) + } + + val response = new DescribeProducersResponseData() + describeProducersRequest.data.topics.forEach { topicRequest => + val topicResponse = new DescribeProducersResponseData.TopicResponse() + .setName(topicRequest.name) + val topicError = if (!authorize(request.context, DESCRIBE, TOPIC, topicRequest.name)) + Some(Errors.TOPIC_AUTHORIZATION_FAILED) + else if (!metadataCache.contains(topicRequest.name)) + Some(Errors.UNKNOWN_TOPIC_OR_PARTITION) + else + None + + topicRequest.partitionIndexes.forEach { partitionId => + val topicPartition = new TopicPartition(topicRequest.name, partitionId) + val partitionResponse = topicError match { + case Some(error) => partitionError(topicPartition, error) + case None => replicaManager.activeProducerState(topicPartition) + } + topicResponse.partitions.add(partitionResponse) + } + + if (!topicResponse.partitions.isEmpty) { + response.topics.add(topicResponse) + } + } + + sendResponseMaybeThrottle(request, requestThrottleMs => + new DescribeProducersResponse(response.setThrottleTimeMs(requestThrottleMs))) + } + // private package for testing private[server] def authorize(requestContext: RequestContext, operation: AclOperation, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b84d814c530f2..41980b17387f0 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult -import org.apache.kafka.common.message.{DescribeLogDirsResponseData, FetchResponseData, LeaderAndIsrResponseData} +import org.apache.kafka.common.message.{DescribeLogDirsResponseData, DescribeProducersResponseData, FetchResponseData, LeaderAndIsrResponseData} import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState import org.apache.kafka.common.metrics.Metrics @@ -1893,4 +1893,14 @@ class ReplicaManager(val config: KafkaConfig, controller.electLeaders(partitions, electionType, electionCallback) } + + def activeProducerState(requestPartition: TopicPartition): DescribeProducersResponseData.PartitionResponse = { + getPartitionOrError(requestPartition) match { + case Left(error) => new DescribeProducersResponseData.PartitionResponse() + .setPartitionIndex(requestPartition.partition) + .setErrorCode(error.code) + case Right(partition) => partition.activeProducerState + } + } + } From c4af9dbb7a5a8432a8a21f17bfddfc1debaac38b Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 3 Sep 2020 12:02:10 -0700 Subject: [PATCH 02/17] Add support for the `DescribeTransactions` API --- .../org/apache/kafka/clients/admin/Admin.java | 6 + .../admin/DescribeTransactionsOptions.java | 31 ++ .../admin/DescribeTransactionsResult.java | 118 +++++ .../kafka/clients/admin/KafkaAdminClient.java | 14 + .../admin/internals/CoordinatorKey.java | 12 +- .../internals/CoordinatorRequestDriver.java | 4 +- .../DescribeTransactionsRequestDriver.java | 142 ++++++ .../TransactionalIdNotFoundException.java | 24 + .../apache/kafka/common/protocol/ApiKeys.java | 9 +- .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../requests/DescribeTransactionsRequest.java | 89 ++++ .../DescribeTransactionsResponse.java | 64 +++ .../kafka/common/utils/CollectionUtils.java | 4 - .../message/DescribeTransactionsRequest.json | 25 + .../message/DescribeTransactionsResponse.json | 39 ++ .../kafka/clients/admin/MockAdminClient.java | 19 +- .../CoordinatorRequestDriverTest.java | 12 +- ...DescribeTransactionsRequestDriverTest.java | 459 ++++++++++++++++++ .../internals/MetadataRequestDriverTest.java | 4 - .../common/requests/RequestResponseTest.java | 45 +- .../kafka/admin/TransactionCommand.scala | 121 ++++- .../transaction/TransactionCoordinator.scala | 45 +- .../main/scala/kafka/server/KafkaApis.scala | 24 +- .../scala/kafka/utils/CommandLineUtils.scala | 5 + 26 files changed, 1271 insertions(+), 52 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdNotFoundException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java create mode 100644 clients/src/main/resources/common/message/DescribeTransactionsRequest.json create mode 100644 clients/src/main/resources/common/message/DescribeTransactionsResponse.json create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index da17a17c30527..21b04fcd044fc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1221,6 +1221,12 @@ default DescribeProducersResult describeProducers(Collection par DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options); + default DescribeTransactionsResult describeTransactions(Collection transactionalIds) { + return describeTransactions(transactionalIds, new DescribeTransactionsOptions()); + } + + DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options); + /** * Describe all SASL/SCRAM credentials. * diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java new file mode 100644 index 0000000000000..3759b49f8abca --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; + +/** + * Options for {@link Admin#describeTransactions(Collection)}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class DescribeTransactionsOptions extends AbstractOptions { + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java new file mode 100644 index 0000000000000..b790a62e25a1c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.clients.admin.internals.CoordinatorKey; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.requests.FindCoordinatorRequest; + +import java.util.HashMap; +import java.util.Map; +import java.util.OptionalLong; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +@InterfaceStability.Evolving +public class DescribeTransactionsResult { + private final Map> futures; + + DescribeTransactionsResult(Map> futures) { + this.futures = futures; + } + + public KafkaFuture transactionalIdResult(String transactionalId) { + CoordinatorKey key = buildKey(transactionalId); + KafkaFuture future = futures.get(key); + if (future == null) { + throw new IllegalArgumentException("TransactionalId " + + "`" + transactionalId + "` was not included in the request"); + } + return future; + } + + private CoordinatorKey buildKey(String transactionalId) { + return new CoordinatorKey(transactionalId, FindCoordinatorRequest.CoordinatorType.TRANSACTION); + } + + public KafkaFuture> all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])) + .thenApply(nil -> { + Map results = new HashMap<>(futures.size()); + for (Map.Entry> entry : futures.entrySet()) { + try { + results.put(entry.getKey().idValue, entry.getValue().get()); + } catch (InterruptedException | ExecutionException e) { + // This should be unreachable, because allOf ensured that all the futures completed successfully. + throw new RuntimeException(e); + } + } + return results; + }); + } + + public static class TransactionState { + private final String state; + private final long producerId; + private final int producerEpoch; + private final long transactionTimeoutMs; + private final OptionalLong transactionStartTimeMs; + private final Set topicPartitions; + + public TransactionState( + String state, + long producerId, + int producerEpoch, + long transactionTimeoutMs, + OptionalLong transactionStartTimeMs, + Set topicPartitions + ) { + this.state = state; + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.transactionTimeoutMs = transactionTimeoutMs; + this.transactionStartTimeMs = transactionStartTimeMs; + this.topicPartitions = topicPartitions; + } + + public String state() { + return state; + } + + public long producerId() { + return producerId; + } + + public int producerEpoch() { + return producerEpoch; + } + + public long transactionTimeoutMs() { + return transactionTimeoutMs; + } + + public OptionalLong transactionStartTimeMs() { + return transactionStartTimeMs; + } + + public Set topicPartitions() { + return topicPartitions; + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 3307d1c1428ad..2bdf50a300014 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -35,6 +35,7 @@ import org.apache.kafka.clients.admin.internals.AdminMetadataManager; import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; import org.apache.kafka.clients.admin.internals.DescribeProducersRequestDriver; +import org.apache.kafka.clients.admin.internals.DescribeTransactionsRequestDriver; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; import org.apache.kafka.clients.admin.internals.RequestDriver; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; @@ -4289,6 +4290,19 @@ public DescribeProducersResult describeProducers(Collection part return new DescribeProducersResult(driver.futures()); } + @Override + public DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options) { + if (transactionalIds.isEmpty()) { + return new DescribeTransactionsResult(Collections.emptyMap()); + } + long currentTimeMs = time.milliseconds(); + long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + transactionalIds, deadlineMs, retryBackoffMs); + maybeSendRequests(currentTimeMs, driver); + return new DescribeTransactionsResult(driver.futures()); + } + /** * Get a sub level error when the request is in batch. If given key was not found, * return an {@link IllegalArgumentException}. diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java index e0a4c3c4b8b29..1b08c25506bc7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java @@ -21,11 +21,11 @@ import java.util.Objects; public class CoordinatorKey { - public final String key; + public final String idValue; public final FindCoordinatorRequest.CoordinatorType type; - public CoordinatorKey(String key, FindCoordinatorRequest.CoordinatorType type) { - this.key = key; + public CoordinatorKey(String idValue, FindCoordinatorRequest.CoordinatorType type) { + this.idValue = idValue; this.type = type; } @@ -34,19 +34,19 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; CoordinatorKey that = (CoordinatorKey) o; - return Objects.equals(key, that.key) && + return Objects.equals(idValue, that.idValue) && type == that.type; } @Override public int hashCode() { - return Objects.hash(key, type); + return Objects.hash(idValue, type); } @Override public String toString() { return "CoordinatorKey(" + - "key='" + key + '\'' + + "idValue='" + idValue + '\'' + ", type=" + type + ')'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java index 892ddf4fda6d2..a5bca35a0f96d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java @@ -53,7 +53,7 @@ FindCoordinatorRequest.Builder buildLookupRequest(Set keys) { CoordinatorKey key = requireSingleton(keys); return new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() - .setKey(key.key) + .setKey(key.idValue) .setKeyType(key.type.id()) ); } @@ -78,7 +78,7 @@ void handleLookupResponse(Set keys, AbstractResponse abstractRes case GROUP_AUTHORIZATION_FAILED: completeExceptionally(key, new GroupAuthorizationException("FindCoordinator request for groupId " + - "`" + key + "` failed due to authorization failure", key.key)); + "`" + key + "` failed due to authorization failure", key.idValue)); break; case TRANSACTIONAL_ID_AUTHORIZATION_FAILED: diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java new file mode 100644 index 0000000000000..0c7d2380e0154 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.DescribeTransactionsResult.TransactionState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; +import org.apache.kafka.common.errors.TransactionalIdNotFoundException; +import org.apache.kafka.common.message.DescribeTransactionsRequestData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.OptionalLong; +import java.util.Set; +import java.util.stream.Collectors; + +public class DescribeTransactionsRequestDriver extends CoordinatorRequestDriver { + private static final Logger log = LoggerFactory.getLogger(DescribeTransactionsRequestDriver.class); + + public DescribeTransactionsRequestDriver( + Collection transactionalIds, + long deadlineMs, + long retryBackoffMs + ) { + super(transactionalIds.stream() + .map(DescribeTransactionsRequestDriver::asCoordinatorKey) + .collect(Collectors.toSet()), deadlineMs, retryBackoffMs); + } + + @Override + AbstractRequest.Builder buildFulfillmentRequest(Set keys) { + DescribeTransactionsRequestData request = new DescribeTransactionsRequestData(); + List transactionalIds = keys.stream().map(key -> key.idValue).collect(Collectors.toList()); + request.setTransactionalIds(transactionalIds); + return new DescribeTransactionsRequest.Builder(request); + } + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + DescribeTransactionsResponse response = (DescribeTransactionsResponse) abstractResponse; + for (DescribeTransactionsResponseData.TransactionState transactionState : response.data().transactionStates()) { + CoordinatorKey transactionalIdKey = asCoordinatorKey(transactionState.transactionalId()); + Errors error = Errors.forCode(transactionState.errorCode()); + + if (error != Errors.NONE) { + handleError(transactionalIdKey, error); + continue; + } + + OptionalLong transactionStartTimeMs = transactionState.transactionStartTimeMs() < 0 ? + OptionalLong.empty() : + OptionalLong.of(transactionState.transactionStartTimeMs()); + + complete(transactionalIdKey, new TransactionState( + transactionState.transactionState(), + transactionState.producerId(), + transactionState.producerEpoch(), + transactionState.transactionTimeoutMs(), + transactionStartTimeMs, + collectTopicPartitions(transactionState) + )); + } + } + + private Set collectTopicPartitions( + DescribeTransactionsResponseData.TransactionState transactionState + ) { + Set res = new HashSet<>(); + for (DescribeTransactionsResponseData.TopicData topicData : transactionState.topicPartitions()) { + String topic = topicData.name(); + for (Integer partitionId : topicData.partitionIndexes()) { + res.add(new TopicPartition(topic, partitionId)); + } + } + return res; + } + + public static CoordinatorKey asCoordinatorKey(String transactionalId) { + return new CoordinatorKey(transactionalId, FindCoordinatorRequest.CoordinatorType.TRANSACTION); + } + + private void handleError(CoordinatorKey transactionalIdKey, Errors error) { + switch (error) { + case TRANSACTIONAL_ID_AUTHORIZATION_FAILED: + completeExceptionally(transactionalIdKey, new TransactionalIdAuthorizationException( + "DescribeTransactions request for transactionalId `" + transactionalIdKey.idValue + "` " + + "failed due to authorization failure")); + break; + + case TRANSACTIONAL_ID_NOT_FOUND: + completeExceptionally(transactionalIdKey, new TransactionalIdNotFoundException( + "DescribeTransactions request for transactionalId `" + transactionalIdKey.idValue + "` " + + "failed because the ID could not be found")); + break; + + case COORDINATOR_LOAD_IN_PROGRESS: + // If the coordinator is in the middle of loading, then we just need to retry + log.debug("Desc>ribeTransactions request for transactionalId `{}` failed because the " + + " coordinator is still in the process of loading state. Will retry", + transactionalIdKey.idValue); + break; + + case NOT_COORDINATOR: + case COORDINATOR_NOT_AVAILABLE: + // If the coordinator is unavailable or there was a coordinator change, then we unmap + // the key so that we retry the `FindCoordinator` request + super.unmap(transactionalIdKey); + log.debug("DescribeTransactions request for transactionalId `{}` returned error {}. Will retry", + transactionalIdKey.idValue, error); + break; + + default: + super.completeExceptionally(transactionalIdKey, error.exception("DescribeTransactions request for " + + "transactionalId `" + transactionalIdKey.idValue + "` failed due to unexpected error")); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdNotFoundException.java b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdNotFoundException.java new file mode 100644 index 0000000000000..240eaa33a8474 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdNotFoundException.java @@ -0,0 +1,24 @@ +/* + * 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.common.errors; + +public class TransactionalIdNotFoundException extends ApiException { + + public TransactionalIdNotFoundException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 4a0ff185c04bb..c12fbdeedcd52 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -65,8 +65,13 @@ import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeProducersRequestData; import org.apache.kafka.common.message.DescribeProducersResponseData; +<<<<<<< HEAD import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; +======= +import org.apache.kafka.common.message.DescribeTransactionsRequestData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +>>>>>>> Add support for the `DescribeTransactions` API import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.EndTxnRequestData; @@ -229,7 +234,9 @@ public Struct parseResponse(short version, ByteBuffer buffer) { ALTER_USER_SCRAM_CREDENTIALS(51, "AlterUserScramCredentials", AlterUserScramCredentialsRequestData.SCHEMAS, AlterUserScramCredentialsResponseData.SCHEMAS), DESCRIBE_PRODUCERS(52, "DescribeProducers", DescribeProducersRequestData.SCHEMAS, - DescribeProducersResponseData.SCHEMAS); + DescribeProducersResponseData.SCHEMAS), + DESCRIBE_TRANSACTIONS(53, "DescribeTransactions", DescribeTransactionsRequestData.SCHEMAS, + DescribeTransactionsResponseData.SCHEMAS); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index c44e899c40a80..eb52ce1703716 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.DelegationTokenNotFoundException; import org.apache.kafka.common.errors.DelegationTokenOwnerMismatchException; import org.apache.kafka.common.errors.FencedLeaderEpochException; +import org.apache.kafka.common.errors.TransactionalIdNotFoundException; import org.apache.kafka.common.internals.InvalidProducerEpochException; import org.apache.kafka.common.errors.ListenerNotFoundException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; @@ -331,7 +332,8 @@ public enum Errors { "which fences the current one.", ProducerFencedException::new), RESOURCE_NOT_FOUND(91, "A request illegally referred to a resource that does not exist.", ResourceNotFoundException::new), DUPLICATE_RESOURCE(92, "A request illegally referred to the same resource twice.", DuplicateResourceException::new), - UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new); + UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new), + TRANSACTIONAL_ID_NOT_FOUND(91, "The transactionalId could not be found", TransactionalIdNotFoundException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 37fe61595caf6..f5f7a042df751 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -250,6 +250,8 @@ public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Str return new AlterUserScramCredentialsRequest(struct, apiVersion); case DESCRIBE_PRODUCERS: return new DescribeProducersRequest(struct, apiVersion); + case DESCRIBE_TRANSACTIONS: + return new DescribeTransactionsRequest(struct, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 39ae579470102..fcad8fa1b26a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -193,6 +193,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor return new AlterUserScramCredentialsResponse(struct, version); case DESCRIBE_PRODUCERS: return new DescribeProducersResponse(struct, version); + case DESCRIBE_TRANSACTIONS: + return new DescribeTransactionsResponse(struct, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java new file mode 100644 index 0000000000000..71619ca750915 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java @@ -0,0 +1,89 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.DescribeTransactionsRequestData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class DescribeTransactionsRequest extends AbstractRequest { + public static class Builder extends AbstractRequest.Builder { + public final DescribeTransactionsRequestData data; + + public Builder(DescribeTransactionsRequestData data) { + super(ApiKeys.DESCRIBE_TRANSACTIONS); + this.data = data; + } + + @Override + public DescribeTransactionsRequest build(short version) { + return new DescribeTransactionsRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final DescribeTransactionsRequestData data; + private final short version; + + private DescribeTransactionsRequest(DescribeTransactionsRequestData data, short version) { + super(ApiKeys.DESCRIBE_TRANSACTIONS, version); + this.data = data; + this.version = version; + } + + public DescribeTransactionsRequest(Struct struct, short version) { + super(ApiKeys.DESCRIBE_TRANSACTIONS, version); + this.data = new DescribeTransactionsRequestData(struct, version); + this.version = version; + } + + public DescribeTransactionsRequestData data() { + return data; + } + + @Override + protected Struct toStruct() { + return data.toStruct(version); + } + + @Override + public DescribeTransactionsResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + DescribeTransactionsResponseData response = new DescribeTransactionsResponseData(); + for (String transactionalId : data.transactionalIds()) { + DescribeTransactionsResponseData.TransactionState transactionState = + new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + .setErrorCode(error.code()); + response.transactionStates().add(transactionState); + } + return new DescribeTransactionsResponse(response); + } + + public static DescribeTransactionsRequest parse(ByteBuffer buffer, short version) { + return new DescribeTransactionsRequest(ApiKeys.DESCRIBE_TRANSACTIONS.parseRequest(version, buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java new file mode 100644 index 0000000000000..c6f6f1991b956 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java @@ -0,0 +1,64 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData.TransactionState; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public class DescribeTransactionsResponse extends AbstractResponse { + private final DescribeTransactionsResponseData data; + + public DescribeTransactionsResponse(Struct struct, short version) { + this(new DescribeTransactionsResponseData(struct, version)); + } + + public DescribeTransactionsResponse(DescribeTransactionsResponseData data) { + this.data = data; + } + + public DescribeTransactionsResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + Map errorCounts = new HashMap<>(); + for (TransactionState transactionState : data.transactionStates()) { + Errors error = Errors.forCode(transactionState.errorCode()); + updateErrorCounts(errorCounts, error); + } + return errorCounts; + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + public static DescribeTransactionsResponse parse(ByteBuffer buffer, short version) { + return new DescribeTransactionsResponse( + ApiKeys.DESCRIBE_TRANSACTIONS.responseSchema(version).read(buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java index cd348a2e70cbe..d2690a38021b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -75,10 +75,6 @@ public static Map> groupPartitionsByTopic(Collection * @return The map used to group the partitions */ public static Map groupPartitionsByTopic( diff --git a/clients/src/main/resources/common/message/DescribeTransactionsRequest.json b/clients/src/main/resources/common/message/DescribeTransactionsRequest.json new file mode 100644 index 0000000000000..8cccc35a89559 --- /dev/null +++ b/clients/src/main/resources/common/message/DescribeTransactionsRequest.json @@ -0,0 +1,25 @@ +// 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. + +{ + "apiKey": 53, + "type": "request", + "name": "DescribeTransactionsRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "TransactionalIds", "type": "[]string", "versions": "0+" } + ] +} diff --git a/clients/src/main/resources/common/message/DescribeTransactionsResponse.json b/clients/src/main/resources/common/message/DescribeTransactionsResponse.json new file mode 100644 index 0000000000000..deb11d5357fe4 --- /dev/null +++ b/clients/src/main/resources/common/message/DescribeTransactionsResponse.json @@ -0,0 +1,39 @@ +// 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. + +{ + "apiKey": 53, + "type": "response", + "name": "DescribeTransactionsResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "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": "TransactionStates", "type": "[]TransactionState", "versions": "0+", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+" }, + { "name": "TransactionalId", "type": "string", "versions": "0+" }, + { "name": "TransactionState", "type": "string", "versions": "0+" }, + { "name": "TransactionTimeoutMs", "type": "int32", "versions": "0+" }, + { "name": "TransactionStartTimeMs", "type": "int64", "versions": "0+" }, + { "name": "ProducerId", "type": "int64", "versions": "0+" }, + { "name": "ProducerEpoch", "type": "int32", "versions": "0+" }, + { "name": "TopicPartitions", "type": "[]TopicData", "versions": "0+", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName" }, + { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+" } + ]} + ]} + ] +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 111b58b18aea6..86c2e31efa3a8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -17,30 +17,30 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.TopicPartitionReplica; import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; +import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaFilter; +import org.apache.kafka.common.requests.DescribeLogDirsResponse; import java.time.Duration; import java.util.ArrayList; @@ -827,17 +827,22 @@ public AlterClientQuotasResult alterClientQuotas(Collection partitions, DescribeProducersOptions options) { + public DescribeUserScramCredentialsResult describeUserScramCredentials(List users, DescribeUserScramCredentialsOptions options) { throw new UnsupportedOperationException("Not implemented yet"); } @Override - public DescribeUserScramCredentialsResult describeUserScramCredentials(List users, DescribeUserScramCredentialsOptions options) { + public AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options) { throw new UnsupportedOperationException("Not implemented yet"); } @Override - public AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options) { + public DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options) { throw new UnsupportedOperationException("Not implemented yet"); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java index ef67aa03ef665..4e6e88bf8048e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java @@ -71,7 +71,7 @@ public void testLookupGrouping() { assertEquals(0, spec1.nextAllowedTryMs); assertTrue(spec1.request instanceof FindCoordinatorRequest.Builder); FindCoordinatorRequest.Builder findCoordinatorRequest1 = (FindCoordinatorRequest.Builder) spec1.request; - assertEquals(group1.key, findCoordinatorRequest1.data().key()); + assertEquals(group1.idValue, findCoordinatorRequest1.data().key()); assertEquals(group1.type.id(), findCoordinatorRequest1.data().keyType()); RequestDriver.RequestSpec spec2 = requests.stream() @@ -86,7 +86,7 @@ public void testLookupGrouping() { assertEquals(0, spec2.nextAllowedTryMs); assertTrue(spec2.request instanceof FindCoordinatorRequest.Builder); FindCoordinatorRequest.Builder findCoordinatorRequest2 = (FindCoordinatorRequest.Builder) spec2.request; - assertEquals(group2.key, findCoordinatorRequest2.data().key()); + assertEquals(group2.idValue, findCoordinatorRequest2.data().key()); assertEquals(group2.type.id(), findCoordinatorRequest2.data().keyType()); } @@ -122,7 +122,7 @@ public void testSuccessfulLeaderDiscovery() { assertEquals(0, requestSpec1.nextAllowedTryMs); assertTrue(requestSpec1.request instanceof DescribeGroupsRequest.Builder); DescribeGroupsRequest.Builder request = (DescribeGroupsRequest.Builder) requestSpec1.request; - assertEquals(singletonList(group1.key), request.data.groups()); + assertEquals(singletonList(group1.idValue), request.data.groups()); RequestDriver.RequestSpec lookupSpec2 = lookupRequests.stream() .filter(spec -> spec.keys.contains(group2)) @@ -146,7 +146,7 @@ public void testSuccessfulLeaderDiscovery() { assertEquals(0, requestSpec2.nextAllowedTryMs); assertTrue(requestSpec2.request instanceof DescribeGroupsRequest.Builder); DescribeGroupsRequest.Builder request2 = (DescribeGroupsRequest.Builder) requestSpec2.request; - assertEquals(singletonList(group2.key), request2.data.groups()); + assertEquals(singletonList(group2.idValue), request2.data.groups()); } @Test @@ -191,7 +191,7 @@ public void testFatalFindCoordinatorError() { assertEquals(Collections.emptyList(), driver.poll()); GroupAuthorizationException groupAuthorizationException = assertFutureThrows( driver.futures().get(group1), GroupAuthorizationException.class); - assertEquals(group1.key, groupAuthorizationException.groupId()); + assertEquals(group1.idValue, groupAuthorizationException.groupId()); } private final class TestCoordinatorRequestDriver extends CoordinatorRequestDriver { @@ -204,7 +204,7 @@ public TestCoordinatorRequestDriver(Set groupIds) { AbstractRequest.Builder buildFulfillmentRequest(Set coordinatorKeys) { return new DescribeGroupsRequest.Builder(new DescribeGroupsRequestData() .setGroups(coordinatorKeys.stream() - .map(coordinatorKey -> coordinatorKey.key) + .map(coordinatorKey -> coordinatorKey.idValue) .collect(Collectors.toList()))); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java new file mode 100644 index 0000000000000..1e738879b0265 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java @@ -0,0 +1,459 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.DescribeTransactionsResult; +import org.apache.kafka.clients.admin.DescribeTransactionsResult.TransactionState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.protocol.Errors; +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.FindCoordinatorResponse; +import org.apache.kafka.common.utils.MockTime; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class DescribeTransactionsRequestDriverTest { + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testDescribeTransactions() throws Exception { + String transactionalId1 = "foo"; + String transactionalId2 = "bar"; + Set transactionalIds = mkSet(transactionalId1, transactionalId2); + + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + transactionalIds, + deadlineMs, + retryBackoffMs + ); + + // Send `FindCoordinator` requests + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(2, lookupRequests.size()); + + RequestDriver.RequestSpec lookupSpec1 = + findRequestWithKey(transactionalId1, lookupRequests); + assertRetryBackoff(0, lookupSpec1); + assertFindCoordinatorRequest(transactionalId1, lookupSpec1); + + RequestDriver.RequestSpec loookupSpec2 = + findRequestWithKey(transactionalId2, lookupRequests); + assertRetryBackoff(0, loookupSpec2); + assertFindCoordinatorRequest(transactionalId2, loookupSpec2); + + // Receive `FindCoordinator` responses + int coordinator1 = 1; + int coordinator2 = 3; + + driver.onResponse(time.milliseconds(), lookupSpec1, + findCoordinatorResponse(OptionalInt.of(coordinator1))); + driver.onResponse(time.milliseconds(), loookupSpec2, + findCoordinatorResponse(OptionalInt.of(coordinator2))); + + // Send `DescribeTransactions` requests + List.RequestSpec> requests = driver.poll(); + assertEquals(2, requests.size()); + + RequestDriver.RequestSpec requestSpec1 = + findRequestWithKey(transactionalId1, requests); + assertRetryBackoff(0, requestSpec1); + assertDescribeTransactionsRequest(singleton(transactionalId1), coordinator1, requestSpec1); + + RequestDriver.RequestSpec requestSpec2 = + findRequestWithKey(transactionalId2, requests); + assertRetryBackoff(0, requestSpec2); + assertDescribeTransactionsRequest(singleton(transactionalId2), coordinator2, requestSpec2); + + // Receive `DescribeTransactions` responses + DescribeTransactionsResponseData.TransactionState transactionState1 = + sampleTransactionState1(transactionalId1); + DescribeTransactionsResponseData.TransactionState transactionState2 = + sampleTransactionState2(transactionalId2); + + driver.onResponse(time.milliseconds(), requestSpec1, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData().setTransactionStates(singletonList(transactionState1)))); + driver.onResponse(time.milliseconds(), requestSpec2, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData().setTransactionStates(singletonList(transactionState2)))); + + // We are all done, so there should be no requests left to be sent + assertEquals(Collections.emptyList(), driver.poll()); + + KafkaFutureImpl future1 = driver.futures() + .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); + KafkaFutureImpl future2 = driver.futures() + .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); + + assertTrue(future1.isDone()); + assertMatchingTransactionState(transactionState1, future1.get()); + assertTrue(future2.isDone()); + assertMatchingTransactionState(transactionState2, future2.get()); + } + + @Test + public void testDescribeTransactionsBatching() throws Exception { + String transactionalId1 = "foo"; + String transactionalId2 = "bar"; + Set transactionalIds = mkSet(transactionalId1, transactionalId2); + + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + transactionalIds, + deadlineMs, + retryBackoffMs + ); + + // Send `FindCoordinator` requests + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(2, lookupRequests.size()); + + RequestDriver.RequestSpec lookupSpec1 = + findRequestWithKey(transactionalId1, lookupRequests); + assertRetryBackoff(0, lookupSpec1); + assertFindCoordinatorRequest(transactionalId1, lookupSpec1); + + RequestDriver.RequestSpec loookupSpec2 = + findRequestWithKey(transactionalId2, lookupRequests); + assertRetryBackoff(0, loookupSpec2); + assertFindCoordinatorRequest(transactionalId2, loookupSpec2); + + // Receive `FindCoordinator` responses + int coordinator = 1; + + // Since both transactionalIds map to the same coordinator, then there + // should only be a single `DescribeTransactions` request sent + driver.onResponse(time.milliseconds(), lookupSpec1, + findCoordinatorResponse(OptionalInt.of(coordinator))); + driver.onResponse(time.milliseconds(), loookupSpec2, + findCoordinatorResponse(OptionalInt.of(coordinator))); + + // Send `DescribeTransactions` request + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + RequestDriver.RequestSpec requestSpec = requests.get(0); + assertRetryBackoff(0, requestSpec); + assertDescribeTransactionsRequest(transactionalIds, coordinator, requestSpec); + + // Receive `DescribeTransactions` response + DescribeTransactionsResponseData.TransactionState transactionState1 = + sampleTransactionState1(transactionalId1); + DescribeTransactionsResponseData.TransactionState transactionState2 = + sampleTransactionState2(transactionalId2); + + driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData() + .setTransactionStates(asList(transactionState1, transactionState2)))); + + // We are all done, so there should be no requests left to be sent + assertEquals(Collections.emptyList(), driver.poll()); + + KafkaFutureImpl future1 = driver.futures() + .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); + KafkaFutureImpl future2 = driver.futures() + .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); + + assertTrue(future1.isDone()); + assertMatchingTransactionState(transactionState1, future1.get()); + assertTrue(future2.isDone()); + assertMatchingTransactionState(transactionState2, future2.get()); + } + + @Test + public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() { + String transactionalId = "foo"; + + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + singleton(transactionalId), + deadlineMs, + retryBackoffMs + ); + + // Send first `FindCoordinator` request + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(1, lookupRequests.size()); + RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + assertRetryBackoff(0, lookupSpec); + assertFindCoordinatorRequest(transactionalId, lookupSpec); + + int coordinator = 5; + driver.onResponse(time.milliseconds(), lookupSpec, + findCoordinatorResponse(OptionalInt.of(coordinator))); + + // Send `DescribeTransactions` request + List.RequestSpec> requests1 = driver.poll(); + assertEquals(1, requests1.size()); + RequestDriver.RequestSpec requestSpec1 = requests1.get(0); + assertRetryBackoff(0, requestSpec1); + assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec1); + + // Receive `DescribeTransactions` response + driver.onResponse(time.milliseconds(), requestSpec1, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData().setTransactionStates( + singletonList(new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()))) + )); + + // Send retry `DescribeTransactions` request + assertFalse(futureFor(driver, transactionalId).isDone()); + List.RequestSpec> requests2 = driver.poll(); + assertEquals(1, requests2.size()); + RequestDriver.RequestSpec requestSpec2 = requests2.get(0); + assertRetryBackoff(1, requestSpec2); + assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec2); + } + + @Test + public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { + String transactionalId = "foo"; + + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + singleton(transactionalId), + deadlineMs, + retryBackoffMs + ); + + // Send first `FindCoordinator` request + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + assertRetryBackoff(0, lookupSpec1); + assertFindCoordinatorRequest(transactionalId, lookupSpec1); + + int coordinator = 5; + driver.onResponse(time.milliseconds(), lookupSpec1, + findCoordinatorResponse(OptionalInt.of(coordinator))); + + // Send `DescribeTransactions` request + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + RequestDriver.RequestSpec requestSpec = requests.get(0); + assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); + + driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData().setTransactionStates( + singletonList(new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + .setErrorCode(Errors.NOT_COORDINATOR.code()))) + )); + + // Send second `FindCoordinator` request + assertFalse(futureFor(driver, transactionalId).isDone()); + List.RequestSpec> lookupRequests2 = driver.poll(); + assertEquals(1, lookupRequests2.size()); + RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); + assertRetryBackoff(1, lookupSpec2); + assertFindCoordinatorRequest(transactionalId, lookupSpec2); + } + + @Test + public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() { + String transactionalId = "foo"; + + DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + singleton(transactionalId), + deadlineMs, + retryBackoffMs + ); + + // Send first `FindCoordinator` request + List.RequestSpec> lookupRequests1 = driver.poll(); + assertEquals(1, lookupRequests1.size()); + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + assertRetryBackoff(0, lookupSpec1); + assertFindCoordinatorRequest(transactionalId, lookupSpec1); + + int coordinator = 5; + driver.onResponse(time.milliseconds(), lookupSpec1, + findCoordinatorResponse(OptionalInt.of(coordinator))); + + // Send `DescribeTransactions` request + List.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + RequestDriver.RequestSpec requestSpec = requests.get(0); + assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); + + driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( + new DescribeTransactionsResponseData().setTransactionStates( + singletonList(new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()))) + )); + + KafkaFutureImpl future = futureFor(driver, transactionalId); + assertTrue(future.isDone()); + assertFutureThrows(future, UnknownServerException.class); + } + + private KafkaFutureImpl futureFor( + DescribeTransactionsRequestDriver driver, + String transactionalId + ) { + CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); + return driver.futures().get(key); + } + + private RequestDriver.RequestSpec findRequestWithKey( + String transactionalId, + List.RequestSpec> requests + ) { + CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); + + Optional.RequestSpec> firstMatch = requests.stream() + .filter(spec -> spec.keys.contains(key)) + .findFirst(); + + assertTrue(firstMatch.isPresent()); + + return firstMatch.get(); + } + + private void assertDescribeTransactionsRequest( + Set expectedTransactionalIds, + int expectedCoordinatorId, + RequestDriver.RequestSpec spec + ) { + Set keys = expectedTransactionalIds.stream() + .map(DescribeTransactionsRequestDriver::asCoordinatorKey) + .collect(Collectors.toSet()); + assertEquals(keys, spec.keys); + assertEquals(OptionalInt.of(expectedCoordinatorId), spec.scope.destinationBrokerId()); + + assertTrue(spec.request instanceof DescribeTransactionsRequest.Builder); + DescribeTransactionsRequest.Builder request = (DescribeTransactionsRequest.Builder) spec.request; + assertEquals(expectedTransactionalIds, new HashSet<>(request.data.transactionalIds())); + } + + private void assertFindCoordinatorRequest( + String expectedTransactionalId, + RequestDriver.RequestSpec spec + ) { + CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(expectedTransactionalId); + assertEquals(singleton(key), spec.keys); + assertEquals(OptionalInt.empty(), spec.scope.destinationBrokerId()); + + assertTrue(spec.request instanceof FindCoordinatorRequest.Builder); + FindCoordinatorRequest.Builder request1 = (FindCoordinatorRequest.Builder) spec.request; + assertEquals(expectedTransactionalId, request1.data().key()); + assertEquals(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), request1.data().keyType()); + } + + private void assertRetryBackoff( + int expectedTries, + RequestDriver.RequestSpec spec + ) { + assertEquals(deadlineMs, spec.deadlineMs); + assertEquals(expectedTries, spec.tries); + if (expectedTries == 0) { + assertEquals(0, spec.nextAllowedTryMs); + } else { + assertEquals(time.milliseconds() + (expectedTries * retryBackoffMs), spec.nextAllowedTryMs); + } + } + + private FindCoordinatorResponse findCoordinatorResponse( + OptionalInt coordinatorId + ) { + return new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setErrorCode(Errors.NONE.code()) + .setNodeId(coordinatorId.orElse(-1)) + .setHost("localhost") + .setPort(9092 + coordinatorId.orElse(-1)) + ); + } + + private DescribeTransactionsResponseData.TransactionState sampleTransactionState1( + String transactionalId + ) { + return new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setTransactionState("Ongoing") + .setTransactionalId(transactionalId) + .setProducerId(12345L) + .setProducerEpoch(15) + .setTransactionStartTimeMs(1599151791L) + .setTransactionTimeoutMs(10000) + .setTopicPartitions(asList( + new DescribeTransactionsResponseData.TopicData() + .setName("foo") + .setPartitionIndexes(asList(1, 3, 5)), + new DescribeTransactionsResponseData.TopicData() + .setName("bar") + .setPartitionIndexes(asList(1, 3, 5)) + )); + } + + private DescribeTransactionsResponseData.TransactionState sampleTransactionState2( + String transactionalId + ) { + return new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setTransactionState("Empty") + .setTransactionalId(transactionalId) + .setProducerId(98765L) + .setProducerEpoch(30) + .setTransactionStartTimeMs(-1); + } + + private void assertMatchingTransactionState( + DescribeTransactionsResponseData.TransactionState expected, + DescribeTransactionsResult.TransactionState actual + ) { + assertEquals(expected.producerId(), actual.producerId()); + assertEquals(expected.producerEpoch(), actual.producerEpoch()); + assertEquals(expected.transactionTimeoutMs(), actual.transactionTimeoutMs()); + assertEquals(expected.transactionStartTimeMs(), actual.transactionStartTimeMs().orElse(-1)); + assertEquals(collectTransactionPartitions(expected), actual.topicPartitions()); + } + + private Set collectTransactionPartitions( + DescribeTransactionsResponseData.TransactionState transactionState + ) { + Set topicPartitions = new HashSet<>(); + for (DescribeTransactionsResponseData.TopicData topicData : transactionState.topicPartitions()) { + for (Integer partitionId : topicData.partitionIndexes()) { + topicPartitions.add(new TopicPartition(topicData.name(), partitionId)); + } + } + return topicPartitions; + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java index 7cbf60e908500..fb1be02915a19 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -132,10 +132,6 @@ public void testSuccessfulLeaderDiscovery() { assertEquals(OptionalInt.of(3), spec1.scope.destinationBrokerId()); } - private RequestDriver.RequestSpec findRequest(TopicPartition tp) { - - } - @Test public void testRetryLeaderDiscovery() { TopicPartition tp0 = new TopicPartition("foo", 0); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index ce7e68e92776e..37c9026e3f033 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -89,6 +89,8 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; import org.apache.kafka.common.message.DescribeProducersRequestData; import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.message.DescribeTransactionsRequestData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.EndTxnRequestData; @@ -496,6 +498,9 @@ public void testSerialization() throws Exception { checkRequest(createDescribeProducersRequest(), true); checkErrorResponse(createDescribeProducersRequest(), unknownServerException, true); checkResponse(createDescribeProducersResponse(), 0, true); + checkRequest(createDescribeTransactionsRequest(), true); + checkErrorResponse(createDescribeTransactionsRequest(), unknownServerException, true); + checkResponse(createDescribeTransactionsResponse(), 0, true); } @Test @@ -2400,7 +2405,7 @@ private DescribeProducersRequest createDescribeProducersRequest() { topicRequest.partitionIndexes().add(0); topicRequest.partitionIndexes().add(1); data.topics().add(topicRequest); - return new DescribeProducersRequest.Builder(data).build(ApiKeys.DESCRIBE_PRODUCERS.latestVersion()); + return new DescribeProducersRequest.Builder(data).build(); } private DescribeProducersResponse createDescribeProducersResponse() { @@ -2425,4 +2430,42 @@ private DescribeProducersResponse createDescribeProducersResponse() { return new DescribeProducersResponse(data); } + private DescribeTransactionsRequest createDescribeTransactionsRequest() { + DescribeTransactionsRequestData data = new DescribeTransactionsRequestData() + .setTransactionalIds(asList("t1", "t2", "t3")); + return new DescribeTransactionsRequest.Builder(data).build(); + } + + private DescribeTransactionsResponse createDescribeTransactionsResponse() { + DescribeTransactionsResponseData data = new DescribeTransactionsResponseData(); + data.setTransactionStates(asList( + new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setTransactionalId("t1") + .setProducerId(12345L) + .setProducerEpoch(15) + .setTransactionStartTimeMs(13490218304L) + .setTransactionState("Empty"), + new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setTransactionalId("t2") + .setProducerId(98765L) + .setProducerEpoch(30) + .setTransactionStartTimeMs(13490218304L) + .setTransactionState("Ongoing") + .setTopicPartitions(asList( + new DescribeTransactionsResponseData.TopicData() + .setName("foo") + .setPartitionIndexes(asList(1, 3, 5, 7)), + new DescribeTransactionsResponseData.TopicData() + .setName("bar") + .setPartitionIndexes(asList(1, 3, 5, 7)) + )), + new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NOT_COORDINATOR.code()) + .setTransactionalId("t3") + )); + return new DescribeTransactionsResponse(data); + } + } diff --git a/core/src/main/scala/kafka/admin/TransactionCommand.scala b/core/src/main/scala/kafka/admin/TransactionCommand.scala index 12c921e7efa7a..e4f6e30dc2826 100644 --- a/core/src/main/scala/kafka/admin/TransactionCommand.scala +++ b/core/src/main/scala/kafka/admin/TransactionCommand.scala @@ -17,7 +17,9 @@ package kafka.admin import java.util.Properties +import java.util.concurrent.ExecutionException +import kafka.utils.CommandLineUtils.printErrorMessageAndDie import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Logging} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeProducersOptions} import org.apache.kafka.common.TopicPartition @@ -32,9 +34,15 @@ object TransactionCommand extends Logging { Utils.loadProps(config) }.getOrElse(new Properties()) + + val bootstrapServersOpt = Option(commandOptions.options.valueOf(commandOptions.bootstrapServer)) + if (bootstrapServersOpt.isEmpty) { + printErrorMessageAndDie("Missing required argument --bootstrap-server") + } + props.setProperty( AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, - commandOptions.options.valueOf(commandOptions.bootstrapServer) + bootstrapServersOpt.get ) Admin.create(props) @@ -70,6 +78,50 @@ object TransactionCommand extends Logging { rows.foreach(printRow) } + private def describeTransactions( + admin: Admin, + transactionalId: String + ): Unit = { + val result = try { + admin.describeTransactions(Seq(transactionalId).asJava) + .transactionalIdResult(transactionalId) + .get() + } catch { + case e: ExecutionException => + val cause = e.getCause + debug(s"Failed to describe transaction state of transactional-id `$transactionalId`", cause) + printErrorMessageAndDie(s"Failed to describe transaction state of transactional-id `$transactionalId`: " + + s"${cause.getMessage}. Enable debug logging for additional detail.") + } + + // TODO: Do we want a way to return coordinator ID? + val headers = Array( + "ProducerId", + "ProducerEpoch", + "TransactionState", + "TransactionTimeoutMs", + "CurrentTransactionStartTimeMs", + "TopicPartitions" + ) + + val transactionStartTimeMsColumnValue = if (result.transactionStartTimeMs.isPresent) { + result.transactionStartTimeMs.getAsLong.toString + } else { + "None" + } + + val rows = Array( + result.producerId.toString, + result.producerEpoch.toString, + result.state, + result.transactionTimeoutMs.toString, + transactionStartTimeMsColumnValue, + Utils.join(result.topicPartitions, ",") + ) + + prettyPrintTable(headers, Seq(rows)) + } + private def describeProducers( admin: Admin, brokerId: Option[Int], @@ -78,11 +130,26 @@ object TransactionCommand extends Logging { val options = new DescribeProducersOptions() brokerId.foreach(options.setBrokerId) - val result = admin.describeProducers(Seq(topicPartition).asJava, options) - .partitionResult(topicPartition).get() + val result = try { + admin.describeProducers(Seq(topicPartition).asJava, options) + .partitionResult(topicPartition) + .get() + } catch { + case e: ExecutionException => + val cause = e.getCause + val brokerClause = brokerId.map(id => s"broker $id").getOrElse("leader") + debug(s"Failed to describe producers for partition $topicPartition on $brokerClause", cause) + printErrorMessageAndDie(s"Failed to describe producers for partition $topicPartition on $brokerClause: " + + s"${cause.getMessage}. Enable debug logging for additional detail.") + } + // TODO: Add coordinator epoch val headers = Array( - "ProducerId", "ProducerEpoch", "LastSequence", "LastTimestamp", "CurrentTransactionStartOffset" + "ProducerId", + "ProducerEpoch", + "LastSequence", + "LastTimestamp", + "CurrentTransactionStartOffset" ) val rows = result.activeProducers().asScala.map { activeProducer => @@ -105,16 +172,11 @@ object TransactionCommand extends Logging { def main(args: Array[String]): Unit = { val commandOptions = new TransactionCommandOptions(args) - CommandLineUtils.printHelpAndExitIfNeeded( - commandOptions, - "This tool attempts to elect a new leader for a set of topic partitions. The type of elections supported are preferred replicas and unclean replicas." - ) - - CommandLineUtils.printHelpAndExitIfNeeded(commandOptions, "This tool is used to analyze transaction " + - "state and recover from hanging transactions") + CommandLineUtils.printHelpAndExitIfNeeded(commandOptions, "This tool is " + + "used to analyze transaction state and recover from hanging transactions") val admin = createAdmin(commandOptions) - val brokerId = Option(commandOptions.options.valueOf(commandOptions.brokerId)).map(Int.unbox) + val brokerIdOpt = Option(commandOptions.options.valueOf(commandOptions.brokerId)).map(Int.unbox) val topicPartitionOpt = ( Option(commandOptions.options.valueOf(commandOptions.topic)), Option(commandOptions.options.valueOf(commandOptions.partition)) @@ -123,8 +185,22 @@ object TransactionCommand extends Logging { case _ => None } - topicPartitionOpt.foreach { topicPartition => - describeProducers(admin, brokerId, topicPartition) + if (commandOptions.options.has(commandOptions.describeProducersOption)) { + topicPartitionOpt match { + case Some(topicPartition) => + describeProducers(admin, brokerIdOpt, topicPartition) + case None => + printErrorMessageAndDie("The --describe-producers action requires both " + + "the --topic and --partition arguments") + } + } else if (commandOptions.options.has(commandOptions.describeOption)) { + Option(commandOptions.options.valueOf(commandOptions.transactionalId)) match { + case Some(transactionalId) => + describeTransactions(admin, transactionalId) + case None => + printErrorMessageAndDie("The --describe action requires the " + + "--transactional-id argument") + } } } } @@ -165,14 +241,25 @@ private final class TransactionCommandOptions(args: Array[String]) extends Comma .describedAs("broker id") .ofType(classOf[Integer]) - val describeProducersOptions = parser + val transactionalId = parser + .accepts("transactional-id") + .withRequiredArg + .describedAs("transactional id") + .ofType(classOf[String]) + + val describeOption = parser + .accepts("describe", + "Used to describe the transaction state of a specific transactional id " + + "(requires --transactional-id)") + + val describeProducersOption = parser .accepts("describe-producers", "Used to describe active transactional/idempotent producers " + - "writing to a specific topic partition (you must specify --topic and --partition)") + "writing to a specific topic partition (requires --topic and --partition)") options = parser.parse(args: _*) - if (Seq(describeProducersOptions).count(options.has) != 1) { + if (Seq(describeOption, describeProducersOption).count(options.has) != 1) { CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe-producers") } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index ebc0c2a1ad561..bed0aa8ddd5f6 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -24,11 +24,14 @@ import kafka.utils.{Logging, Scheduler} import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.DescribeTransactionsResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.TransactionResult -import org.apache.kafka.common.utils.{LogContext, ProducerIdAndEpoch, Time} +import org.apache.kafka.common.utils.{CollectionUtils, LogContext, ProducerIdAndEpoch, Time} + +import scala.jdk.CollectionConverters._ object TransactionCoordinator { @@ -255,6 +258,46 @@ class TransactionCoordinator(brokerId: Int, } } + def handleDescribeTransactions(transactionalId: String): DescribeTransactionsResponseData.TransactionState = { + val transactionState = new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + + if (transactionalId == null || transactionalId.isEmpty) { + transactionState.setErrorCode(Errors.INVALID_REQUEST.code) + } else { + txnManager.getTransactionState(transactionalId) match { + case Left(error) => + transactionState.setErrorCode(error.code) + case Right(None) => + transactionState.setErrorCode(Errors.TRANSACTIONAL_ID_NOT_FOUND.code) + case Right(Some(coordinatorEpochAndMetadata)) => + val txnMetadata = coordinatorEpochAndMetadata.transactionMetadata + txnMetadata.inLock { + + // TODO: This partition list is incomplete when in the PrepareXX states. Do we + // want the full set in the transaction, or should we just document that the + // returned partitions are only those which have not had markers written to yet. + // The latter could actually be more useful in practice. + + val partitionsByTopic = CollectionUtils.groupPartitionsByTopic(txnMetadata.topicPartitions.asJava) + partitionsByTopic.forEach { (topic, partitions) => + new DescribeTransactionsResponseData.TopicData() + .setName(topic) + .setPartitionIndexes(partitions) + } + + transactionState + .setErrorCode(Errors.NONE.code) + .setProducerId(txnMetadata.producerId) + .setProducerEpoch(txnMetadata.producerEpoch) + .setTransactionState(txnMetadata.state.toString) + .setTransactionTimeoutMs(txnMetadata.txnTimeoutMs) + .setTransactionStartTimeMs(txnMetadata.txnStartTimestamp) + } + } + } + } + def handleAddPartitionsToTransaction(transactionalId: String, producerId: Long, producerEpoch: Short, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 10b94a59f220b..8f638cccbbba7 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -51,7 +51,7 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, DescribeTransactionsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -181,6 +181,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => handleDescribeUserScramCredentialsRequest(request) case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => handleAlterUserScramCredentialsRequest(request) case ApiKeys.DESCRIBE_PRODUCERS => handleDescribeProducersRequest(request) + case ApiKeys.DESCRIBE_TRANSACTIONS => handleDescribeTransactionsRequest(request) } } catch { case e: FatalExitError => throw e @@ -3092,7 +3093,26 @@ class KafkaApis(val requestChannel: RequestChannel, new DescribeProducersResponse(response.setThrottleTimeMs(requestThrottleMs))) } - // private package for testing + def handleDescribeTransactionsRequest(request: RequestChannel.Request): Unit = { + val describeTransactionsRequest = request.body[DescribeTransactionsRequest] + val response = new DescribeTransactionsResponseData() + + describeTransactionsRequest.data.transactionalIds.forEach { transactionalId => + val transactionState = if (!authorize(request.context, DESCRIBE, TRANSACTIONAL_ID, transactionalId)) { + new DescribeTransactionsResponseData.TransactionState() + .setTransactionalId(transactionalId) + .setErrorCode(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.code) + } else { + txnCoordinator.handleDescribeTransactions(transactionalId) + } + response.transactionStates.add(transactionState) + } + + sendResponseMaybeThrottle(request, requestThrottleMs => + new DescribeTransactionsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } + + // private package for testing private[server] def authorize(requestContext: RequestContext, operation: AclOperation, resourceType: ResourceType, diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala index 80726ce06b599..b1a634aea21e4 100644 --- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala +++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala @@ -103,6 +103,11 @@ object CommandLineUtils extends Logging { Exit.exit(1, Some(message)) } + def printErrorMessageAndDie(message: String): Nothing = { + System.err.println(message) + Exit.exit(1, Some(message)) + } + def printVersionAndDie(): Nothing = { System.out.println(VersionInfo.getVersionString) Exit.exit(0) From 89a97058e93658cecae55e248246bde257626cf4 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 3 Sep 2020 16:57:24 -0700 Subject: [PATCH 03/17] Add support for `ListTransactions` API --- .../org/apache/kafka/clients/admin/Admin.java | 6 + .../kafka/clients/admin/KafkaAdminClient.java | 11 + .../admin/ListTransactionsOptions.java | 58 ++++ .../clients/admin/ListTransactionsResult.java | 161 +++++++++ .../kafka/clients/admin/TransactionState.java | 56 ++++ .../internals/AllBrokerRequestDriver.java | 167 ++++++++++ .../ListTransactionsRequestDriver.java | 99 ++++++ .../internals/MetadataRequestDriver.java | 4 +- .../admin/internals/RequestDriver.java | 4 + .../apache/kafka/common/protocol/ApiKeys.java | 13 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../requests/ListTransactionsRequest.java | 83 +++++ .../requests/ListTransactionsResponse.java | 60 ++++ .../common/requests/MetadataRequest.java | 2 +- .../message/ListTransactionsRequest.json | 30 ++ .../message/ListTransactionsResponse.json | 32 ++ .../admin/ListTransactionsResultTest.java | 114 +++++++ .../kafka/clients/admin/MockAdminClient.java | 5 + .../ListTransactionsRequestDriverTest.java | 307 ++++++++++++++++++ .../common/requests/RequestResponseTest.java | 30 +- .../kafka/admin/TransactionCommand.scala | 45 ++- .../transaction/TransactionCoordinator.scala | 21 +- .../transaction/TransactionLog.scala | 4 +- .../transaction/TransactionMetadata.scala | 34 +- .../transaction/TransactionStateManager.scala | 41 ++- .../main/scala/kafka/server/KafkaApis.scala | 27 +- 27 files changed, 1389 insertions(+), 29 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java create mode 100644 clients/src/main/resources/common/message/ListTransactionsRequest.json create mode 100644 clients/src/main/resources/common/message/ListTransactionsResponse.json create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 21b04fcd044fc..cd2caa61261fd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1227,6 +1227,12 @@ default DescribeTransactionsResult describeTransactions(Collection trans DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options); + default ListTransactionsResult listTransactions() { + return listTransactions(new ListTransactionsOptions()); + } + + ListTransactionsResult listTransactions(ListTransactionsOptions options); + /** * Describe all SASL/SCRAM credentials. * diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 2bdf50a300014..49a923885d83e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -36,6 +36,7 @@ import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; import org.apache.kafka.clients.admin.internals.DescribeProducersRequestDriver; import org.apache.kafka.clients.admin.internals.DescribeTransactionsRequestDriver; +import org.apache.kafka.clients.admin.internals.ListTransactionsRequestDriver; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; import org.apache.kafka.clients.admin.internals.RequestDriver; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; @@ -4303,6 +4304,15 @@ public DescribeTransactionsResult describeTransactions(Collection transa return new DescribeTransactionsResult(driver.futures()); } + @Override + public ListTransactionsResult listTransactions(ListTransactionsOptions options) { + long currentTimeMs = time.milliseconds(); + long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + maybeSendRequests(currentTimeMs, driver); + return new ListTransactionsResult(driver.lookupFuture()); + } + /** * Get a sub level error when the request is in batch. If given key was not found, * return an {@link IllegalArgumentException}. @@ -4326,6 +4336,7 @@ private Call newCall(RequestDriver driver, RequestDriver.Requ new ConstantNodeIdProvider(spec.scope.destinationBrokerId().getAsInt()) : new LeastLoadedNodeProvider(); + // FIXME: Add name to RequestSpec return new Call("", spec.nextAllowedTryMs, spec.tries, spec.deadlineMs, nodeProvider) { @Override AbstractRequest.Builder createRequest(int timeoutMs) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java new file mode 100644 index 0000000000000..8312a05a0df23 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Options for {@link Admin#listTransactions()}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class ListTransactionsOptions extends AbstractOptions { + private Set filteredStates = Collections.emptySet(); + private Set filteredProducerIds = Collections.emptySet(); + + public ListTransactionsOptions filterStates(Set states) { + this.filteredStates = new HashSet<>(states); + return this; + } + + public ListTransactionsOptions filterProducerIds(Set producerIdFilters) { + this.filteredProducerIds = new HashSet<>(producerIdFilters); + return this; + } + + /** + * Returns the list of States that are requested or empty if no states have been specified + */ + public Set filteredStates() { + return filteredStates; + } + + public Set filteredProducerIds() { + return filteredProducerIds; + } + + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java new file mode 100644 index 0000000000000..39d4aa590dd79 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * The result of the {@link Admin#listTransactions()} call. + *

+ * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class ListTransactionsResult { + private final KafkaFutureImpl>>> future; + + ListTransactionsResult(KafkaFutureImpl>>> future) { + this.future = future; + } + + public KafkaFuture> all() { + return allByBrokerId().thenApply(map -> { + List allListings = new ArrayList<>(); + for (Collection listings : map.values()) { + allListings.addAll(listings); + } + return allListings; + }); + } + + public KafkaFuture> brokerIds() { + return future.thenApply(map -> new HashSet<>(map.keySet())); + } + + public KafkaFuture>> allByBrokerId() { + KafkaFutureImpl>> allFuture = new KafkaFutureImpl<>(); + Map> allListingsMap = new HashMap<>(); + + future.whenComplete((map, topLevelException) -> { + if (topLevelException != null) { + allFuture.completeExceptionally(topLevelException); + return; + } + + Set remainingResponses = new HashSet<>(map.keySet()); + for (Map.Entry>> entry : map.entrySet()) { + Integer brokerId = entry.getKey(); + KafkaFutureImpl> future = entry.getValue(); + future.whenComplete((listings, brokerException) -> { + + if (brokerException != null) { + allFuture.completeExceptionally(brokerException); + } else if (!allFuture.isDone()) { + allListingsMap.put(brokerId, listings); + remainingResponses.remove(brokerId); + + if (remainingResponses.isEmpty()) { + allFuture.complete(allListingsMap); + } + } + }); + } + }); + + return allFuture; + } + + public KafkaFuture> byBrokerId(Integer brokerId) { + KafkaFutureImpl> resultFuture = new KafkaFutureImpl<>(); + future.whenComplete((map, exception) -> { + if (exception != null) { + resultFuture.completeExceptionally(exception); + } else { + KafkaFutureImpl> brokerFuture = map.get(brokerId); + if (brokerFuture == null) { + resultFuture.completeExceptionally(new KafkaException("ListTransactions result " + + "did not include listings from broker " + brokerId + ". The included listings " + + "were from brokers " + map.keySet())); + } else { + brokerFuture.whenComplete((listings, brokerException) -> { + if (brokerException != null) { + resultFuture.completeExceptionally(brokerException); + } else { + resultFuture.complete(listings); + } + }); + } + } + }); + return resultFuture; + } + + public static class TransactionListing { + private final String transactionalId; + private final long producerId; + private final TransactionState transactionState; + + public TransactionListing( + String transactionalId, + long producerId, + TransactionState transactionState + ) { + this.transactionalId = transactionalId; + this.producerId = producerId; + this.transactionState = transactionState; + } + + public String transactionalId() { + return transactionalId; + } + + public long producerId() { + return producerId; + } + + public TransactionState transactionState() { + return transactionState; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TransactionListing that = (TransactionListing) o; + return producerId == that.producerId && + Objects.equals(transactionalId, that.transactionalId) && + transactionState == that.transactionState; + } + + @Override + public int hashCode() { + return Objects.hash(transactionalId, producerId, transactionState); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java new file mode 100644 index 0000000000000..8923916fbb6b8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import java.util.HashMap; + +public enum TransactionState { + ONGOING("Ongoing"), + PREPARE_ABORT("PrepareAbort"), + PREPARE_COMMIT("PrepareCommit"), + COMPLETE_ABORT("CompleteAbort"), + COMPLETE_COMMIT("CompleteCommit"), + EMPTY("Empty"), + PREPARE_EPOCH_FENCE("PrepareEpochFence"), + UNKNOWN("Unknown"); // FIXME: I hate this + + private final static HashMap NAME_TO_ENUM; + + static { + NAME_TO_ENUM = new HashMap<>(); + for (TransactionState state : TransactionState.values()) { + NAME_TO_ENUM.put(state.name, state); + } + } + + private final String name; + + TransactionState(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + + public static TransactionState parse(String name) { + TransactionState state = NAME_TO_ENUM.get(name); + return state == null ? UNKNOWN : state; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java new file mode 100644 index 0000000000000..29bc0779741a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.MetadataRequestData; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.OptionalInt; +import java.util.Set; + +/** + * This class is used for use cases which require requests to be sent to all + * brokers in the cluster. + * + * @param + */ +public abstract class AllBrokerRequestDriver extends RequestDriver { + private static final Logger log = LoggerFactory.getLogger(AllBrokerRequestDriver.class); + + private static final BrokerKey ALL_BROKERS = new BrokerKey(OptionalInt.empty()); + private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { + }; + + private final KafkaFutureImpl>> lookupFuture; + + public AllBrokerRequestDriver(long deadlineMs, long retryBackoffMs) { + super(Utils.mkSet(ALL_BROKERS), deadlineMs, retryBackoffMs); + + this.lookupFuture = new KafkaFutureImpl<>(); + super.futures().get(ALL_BROKERS).whenComplete((nil, exception) -> { + if (exception != null) { + this.lookupFuture.completeExceptionally(exception); + } else { + this.lookupFuture.complete(collectBrokerFutures()); + } + }); + } + + public KafkaFutureImpl>> lookupFuture() { + return lookupFuture; + } + + @Override + RequestScope lookupScope(BrokerKey key) { + return SINGLE_REQUEST_SCOPE; + } + + @Override + AbstractRequest.Builder buildLookupRequest(Set keys) { + // Send empty `Metadata` request. We are only interested in the brokers from the response + return new MetadataRequest.Builder(new MetadataRequestData()); + } + + @Override + void handleLookupResponse(Set keys, AbstractResponse abstractResponse) { + MetadataResponse response = (MetadataResponse) abstractResponse; + MetadataResponseData.MetadataResponseBrokerCollection brokers = response.data.brokers(); + + if (brokers.isEmpty()) { + log.debug("Metadata response contained no brokers. Will backoff and retry"); + return; + } + + for (MetadataResponseData.MetadataResponseBroker broker : brokers) { + int brokerId = broker.nodeId(); + super.map(new BrokerKey(OptionalInt.of(brokerId)), brokerId); + } + + super.complete(ALL_BROKERS, null); + } + + private Map> collectBrokerFutures() { + Map> brokerFutures = new HashMap<>(); + for (Map.Entry> entry : super.futures().entrySet()) { + BrokerKey key = entry.getKey(); + KafkaFutureImpl future = entry.getValue(); + if (key.brokerId.isPresent()) { + brokerFutures.put(key.brokerId.getAsInt(), future); + } + } + return brokerFutures; + } + + abstract AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId); + + @Override + AbstractRequest.Builder buildFulfillmentRequest(Set keys) { + Integer brokerId = requireSingleBrokerKey(keys); + return buildFulfillmentRequest(brokerId); + } + + abstract void handleFulfillmentResponse(Integer brokerId, AbstractResponse response); + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse response) { + Integer brokerId = requireSingleBrokerKey(keys); + handleFulfillmentResponse(brokerId, response); + } + + void completeExceptionally(Integer brokerId, Throwable t) { + super.completeExceptionally(new BrokerKey(OptionalInt.of(brokerId)), t); + } + + void complete(Integer brokerId, V value) { + super.complete(new BrokerKey(OptionalInt.of(brokerId)), value); + } + + private Integer requireSingleBrokerKey(Set keys) { + if (keys.size() != 1) { + throw new IllegalArgumentException("Unexpected fulfillment mapping " + keys); + } + + BrokerKey key = keys.iterator().next(); + if (!key.brokerId.isPresent()) { + throw new IllegalArgumentException("Unexpected fulfillment key " + key); + } + + return key.brokerId.getAsInt(); + } + + public static class BrokerKey { + private final OptionalInt brokerId; + + public BrokerKey(OptionalInt brokerId) { + this.brokerId = brokerId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + BrokerKey that = (BrokerKey) o; + return Objects.equals(brokerId, that.brokerId); + } + + @Override + public int hashCode() { + return Objects.hash(brokerId); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java new file mode 100644 index 0000000000000..efb95efe0ecf4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.ListTransactionsOptions; +import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionState; +import org.apache.kafka.common.errors.CoordinatorNotAvailableException; +import org.apache.kafka.common.message.ListTransactionsRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.ListTransactionsRequest; +import org.apache.kafka.common.requests.ListTransactionsResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +public class ListTransactionsRequestDriver extends AllBrokerRequestDriver> { + private static final Logger log = LoggerFactory.getLogger(ListTransactionsRequestDriver.class); + + private final ListTransactionsOptions options; + + public ListTransactionsRequestDriver( + ListTransactionsOptions options, + long deadlineMs, + long retryBackoffMs + ) { + super(deadlineMs, retryBackoffMs); + this.options = options; + } + + @Override + AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId) { + ListTransactionsRequestData request = new ListTransactionsRequestData(); + request.setProducerIdFilter(new ArrayList<>(options.filteredProducerIds())); + request.setStatesFilter(options.filteredStates().stream() + .map(TransactionState::toString) + .collect(Collectors.toList())); + return new ListTransactionsRequest.Builder(request); + } + + @Override + void handleFulfillmentResponse(Integer brokerId, AbstractResponse abstractResponse) { + ListTransactionsResponse response = (ListTransactionsResponse) abstractResponse; + Errors error = Errors.forCode(response.data().errorCode()); + + if (error != Errors.NONE) { + handleError(brokerId, error); + return; + } + + List listings = response.data().transactionStates().stream() + .map(transactionState -> new TransactionListing( + transactionState.transactionalId(), + transactionState.producerId(), + TransactionState.parse(transactionState.transactionState()))) + .collect(Collectors.toList()); + + super.complete(brokerId, listings); + } + + private void handleError(int brokerId, Errors error) { + switch (error) { + case COORDINATOR_LOAD_IN_PROGRESS: + log.debug("ListTransactions request sent to broker {} failed because the " + + "coordinator is still loading state. Will try again after backing off", brokerId); + break; + + case COORDINATOR_NOT_AVAILABLE: + super.completeExceptionally(brokerId, new CoordinatorNotAvailableException("ListTransactions " + + "request sent to broker " + brokerId + " failed because the coordinator is shutting down")); + break; + + default: + super.completeExceptionally(brokerId, error.exception("ListTransactions request " + + "sent to broker " + brokerId + " failed with an unexpected exception")); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java index 79e02da6a1a5e..e41eec93cbc7e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java @@ -36,7 +36,7 @@ public abstract class MetadataRequestDriver extends RequestDriver { private static final Logger log = LoggerFactory.getLogger(MetadataRequestDriver.class); - private static final RequestScope REQUEST_CONTEXT = new RequestScope() { + private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { }; public MetadataRequestDriver( @@ -51,7 +51,7 @@ public MetadataRequestDriver( RequestScope lookupScope(TopicPartition key) { // Metadata requests can group topic partitions arbitrarily, so they can all share // the same request context - return REQUEST_CONTEXT; + return SINGLE_REQUEST_SCOPE; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java index c2914504d9c1e..c5ae1736c48f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -120,6 +120,10 @@ boolean contains(K key) { void map(K key, Integer brokerId) { lookupMap.remove(key); fulfillmentMap.put(new BrokerScope(brokerId), key); + + // To allow for derived keys, we create futures dynamically if they + // do not already exist in the future map + futures.computeIfAbsent(key, k -> new KafkaFutureImpl<>()); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index c12fbdeedcd52..6c6c35b852119 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -65,13 +65,10 @@ import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeProducersRequestData; import org.apache.kafka.common.message.DescribeProducersResponseData; -<<<<<<< HEAD -import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; -import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; -======= import org.apache.kafka.common.message.DescribeTransactionsRequestData; import org.apache.kafka.common.message.DescribeTransactionsResponseData; ->>>>>>> Add support for the `DescribeTransactions` API +import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; +import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.EndTxnRequestData; @@ -98,6 +95,8 @@ import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.ListTransactionsRequestData; +import org.apache.kafka.common.message.ListTransactionsResponseData; import org.apache.kafka.common.message.MetadataRequestData; import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; @@ -236,7 +235,9 @@ public Struct parseResponse(short version, ByteBuffer buffer) { DESCRIBE_PRODUCERS(52, "DescribeProducers", DescribeProducersRequestData.SCHEMAS, DescribeProducersResponseData.SCHEMAS), DESCRIBE_TRANSACTIONS(53, "DescribeTransactions", DescribeTransactionsRequestData.SCHEMAS, - DescribeTransactionsResponseData.SCHEMAS); + DescribeTransactionsResponseData.SCHEMAS), + LIST_TRANSACTIONS(54, "ListTransactions", ListTransactionsRequestData.SCHEMAS, + ListTransactionsResponseData.SCHEMAS); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index f5f7a042df751..0ad9a5ed0b629 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -252,6 +252,8 @@ public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Str return new DescribeProducersRequest(struct, apiVersion); case DESCRIBE_TRANSACTIONS: return new DescribeTransactionsRequest(struct, apiVersion); + case LIST_TRANSACTIONS: + return new ListTransactionsRequest(struct, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index fcad8fa1b26a4..2b430dd1a4813 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -195,6 +195,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor return new DescribeProducersResponse(struct, version); case DESCRIBE_TRANSACTIONS: return new DescribeTransactionsResponse(struct, version); + case LIST_TRANSACTIONS: + return new ListTransactionsResponse(struct, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java new file mode 100644 index 0000000000000..2e30c7260566d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java @@ -0,0 +1,83 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.ListTransactionsRequestData; +import org.apache.kafka.common.message.ListTransactionsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class ListTransactionsRequest extends AbstractRequest { + public static class Builder extends AbstractRequest.Builder { + public final ListTransactionsRequestData data; + + public Builder(ListTransactionsRequestData data) { + super(ApiKeys.LIST_TRANSACTIONS); + this.data = data; + } + + @Override + public ListTransactionsRequest build(short version) { + return new ListTransactionsRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ListTransactionsRequestData data; + private final short version; + + private ListTransactionsRequest(ListTransactionsRequestData data, short version) { + super(ApiKeys.LIST_TRANSACTIONS, version); + this.data = data; + this.version = version; + } + + public ListTransactionsRequest(Struct struct, short version) { + super(ApiKeys.LIST_TRANSACTIONS, version); + this.data = new ListTransactionsRequestData(struct, version); + this.version = version; + } + + public ListTransactionsRequestData data() { + return data; + } + + @Override + protected Struct toStruct() { + return data.toStruct(version); + } + + @Override + public ListTransactionsResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + ListTransactionsResponseData response = new ListTransactionsResponseData() + .setErrorCode(error.code()); + return new ListTransactionsResponse(response); + } + + public static ListTransactionsRequest parse(ByteBuffer buffer, short version) { + return new ListTransactionsRequest(ApiKeys.LIST_TRANSACTIONS.parseRequest(version, buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java new file mode 100644 index 0000000000000..022f2d8f87f2e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java @@ -0,0 +1,60 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.ListTransactionsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public class ListTransactionsResponse extends AbstractResponse { + private final ListTransactionsResponseData data; + + public ListTransactionsResponse(Struct struct, short version) { + this(new ListTransactionsResponseData(struct, version)); + } + + public ListTransactionsResponse(ListTransactionsResponseData data) { + this.data = data; + } + + public ListTransactionsResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + Map errorCounts = new HashMap<>(); + updateErrorCounts(errorCounts, Errors.forCode(data.errorCode())); + return errorCounts; + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + public static ListTransactionsResponse parse(ByteBuffer buffer, short version) { + return new ListTransactionsResponse( + ApiKeys.LIST_TRANSACTIONS.responseSchema(version).read(buffer), version); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index c2533e4f3779c..a0356c9e41a09 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -36,7 +36,7 @@ public static class Builder extends AbstractRequest.Builder { private static final MetadataRequestData ALL_TOPICS_REQUEST_DATA = new MetadataRequestData(). setTopics(null).setAllowAutoTopicCreation(true); - private final MetadataRequestData data; + public final MetadataRequestData data; public Builder(MetadataRequestData data) { super(ApiKeys.METADATA); diff --git a/clients/src/main/resources/common/message/ListTransactionsRequest.json b/clients/src/main/resources/common/message/ListTransactionsRequest.json new file mode 100644 index 0000000000000..6cd97484ea37a --- /dev/null +++ b/clients/src/main/resources/common/message/ListTransactionsRequest.json @@ -0,0 +1,30 @@ +// 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. + +{ + "apiKey": 54, + "type": "request", + "name": "ListTransactionsRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "StatesFilter", "type": "[]string", "versions": "0+", + "about": "The states of transactions we want to include in the response." + }, + { "name": "ProducerIdFilter", "type": "[]int64", "versions": "0+", + "about": "The producerIds of the transactions we want to include in the response" + } + ] +} diff --git a/clients/src/main/resources/common/message/ListTransactionsResponse.json b/clients/src/main/resources/common/message/ListTransactionsResponse.json new file mode 100644 index 0000000000000..76e46a65e103e --- /dev/null +++ b/clients/src/main/resources/common/message/ListTransactionsResponse.json @@ -0,0 +1,32 @@ +// 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. + +{ + "apiKey": 54, + "type": "response", + "name": "ListTransactionsResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "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": "TransactionStates", "type": "[]TransactionState", "versions": "0+", "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "0+" }, + { "name": "ProducerId", "type": "int64", "versions": "0+" }, + { "name": "TransactionState", "type": "string", "versions": "0+" } + ]} + ] +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java new file mode 100644 index 0000000000000..587111a4c839c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.utils.Utils; +import org.junit.Test; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; + +public class ListTransactionsResultTest { + private final KafkaFutureImpl>>> future = + new KafkaFutureImpl<>(); + private final ListTransactionsResult result = new ListTransactionsResult(future); + + @Test + public void testAllFuturesFailIfLookupFails() { + future.completeExceptionally(new KafkaException()); + assertFutureThrows(result.all(), KafkaException.class); + assertFutureThrows(result.allByBrokerId(), KafkaException.class); + assertFutureThrows(result.byBrokerId(0), KafkaException.class); + assertFutureThrows(result.brokerIds(), KafkaException.class); + } + + @Test + public void testAllFuturesSucceed() throws Exception { + KafkaFutureImpl> future1 = new KafkaFutureImpl<>(); + KafkaFutureImpl> future2 = new KafkaFutureImpl<>(); + + Map>> brokerFutures = new HashMap<>(); + brokerFutures.put(1, future1); + brokerFutures.put(2, future2); + + future.complete(brokerFutures); + + List broker1Listings = asList( + new TransactionListing("foo", 12345L, TransactionState.ONGOING), + new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT) + ); + future1.complete(broker1Listings); + + List broker2Listings = singletonList( + new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT) + ); + future2.complete(broker2Listings); + + assertEquals(Utils.mkSet(1, 2), result.brokerIds().get()); + assertEquals(broker1Listings, result.byBrokerId(1).get()); + assertEquals(broker2Listings, result.byBrokerId(2).get()); + assertEquals(broker1Listings, result.allByBrokerId().get().get(1)); + assertEquals(broker2Listings, result.allByBrokerId().get().get(2)); + + Set allExpected = new HashSet<>(); + allExpected.addAll(broker1Listings); + allExpected.addAll(broker2Listings); + + assertEquals(allExpected, new HashSet<>(result.all().get())); + } + + @Test + public void testPartialFailure() throws Exception { + KafkaFutureImpl> future1 = new KafkaFutureImpl<>(); + KafkaFutureImpl> future2 = new KafkaFutureImpl<>(); + + Map>> brokerFutures = new HashMap<>(); + brokerFutures.put(1, future1); + brokerFutures.put(2, future2); + + future.complete(brokerFutures); + + List broker1Listings = asList( + new TransactionListing("foo", 12345L, TransactionState.ONGOING), + new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT) + ); + future1.complete(broker1Listings); + future2.completeExceptionally(new KafkaException()); + + // Ensure that the future for broker 1 completes successfully + assertEquals(Utils.mkSet(1, 2), result.brokerIds().get()); + assertEquals(broker1Listings, result.byBrokerId(1).get()); + + // Everything else should fail + assertFutureThrows(result.all(), KafkaException.class); + assertFutureThrows(result.allByBrokerId(), KafkaException.class); + assertFutureThrows(result.byBrokerId(2), KafkaException.class); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 86c2e31efa3a8..60a6d75e47e6e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -846,6 +846,11 @@ public DescribeTransactionsResult describeTransactions(Collection transa throw new UnsupportedOperationException("Not implemented yet"); } + @Override + public ListTransactionsResult listTransactions(ListTransactionsOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + @Override synchronized public void close(Duration timeout) {} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java new file mode 100644 index 0000000000000..14e14d31967de --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.ListTransactionsOptions; +import org.apache.kafka.clients.admin.ListTransactionsResult; +import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionState; +import org.apache.kafka.clients.admin.internals.AllBrokerRequestDriver.BrokerKey; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.ListTransactionsResponseData; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ListTransactionsRequest; +import org.apache.kafka.common.requests.ListTransactionsResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.MockTime; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class ListTransactionsRequestDriverTest { + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testSuccessfulListTransactions() throws Exception { + ListTransactionsOptions options = new ListTransactionsOptions(); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + + KafkaFutureImpl>>> lookupFuture = + driver.lookupFuture(); + + // Send `Metadata` request to find brokerIds + RequestDriver>.RequestSpec lookupRequestSpec = + assertLookupRequest(driver); + + // Receive `Metadata` response + driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(asList( + brokerMetadata(0), + brokerMetadata(1) + ))); + + assertTrue(lookupFuture.isDone()); + + Map>> brokerFutures = lookupFuture.get(); + assertEquals(mkSet(0, 1), brokerFutures.keySet()); + assertTrue(brokerFutures.values().stream().noneMatch(KafkaFutureImpl::isDone)); + + // Send `ListTransactions` requests + List>.RequestSpec> requestSpecs = driver.poll(); + assertEquals(2, requestSpecs.size()); + RequestDriver>.RequestSpec requestBroker0 = + findBrokerRequest(requestSpecs, 0); + assertListTransactionsRequest(options, requestBroker0); + + RequestDriver>.RequestSpec requestBroker1 = + findBrokerRequest(requestSpecs, 1); + assertListTransactionsRequest(options, requestBroker1); + + // Receive `ListTransactions` responses + KafkaFutureImpl> broker0Future = brokerFutures.get(0); + ListTransactionsResponse broker0Response = sampleListTransactionsResponse1(); + driver.onResponse(time.milliseconds(), requestBroker0, broker0Response); + assertTrue(broker0Future.isDone()); + assertExpectedTransactions(broker0Response.data().transactionStates(), broker0Future.get()); + + KafkaFutureImpl> broker1Future = brokerFutures.get(1); + ListTransactionsResponse broker1Response = sampleListTransactionsResponse2(); + driver.onResponse(time.milliseconds(), requestBroker1, broker1Response); + assertTrue(broker1Future.isDone()); + assertExpectedTransactions(broker1Response.data().transactionStates(), broker1Future.get()); + } + + @Test + public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception { + ListTransactionsOptions options = new ListTransactionsOptions(); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + + KafkaFutureImpl>>> lookupFuture = + driver.lookupFuture(); + + // Send `Metadata` request to find brokerIds + RequestDriver>.RequestSpec lookupRequestSpec = + assertLookupRequest(driver); + + // Receive `Metadata` response + driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(singletonList( + brokerMetadata(0) + ))); + + assertTrue(lookupFuture.isDone()); + + Map>> brokerFutures = lookupFuture.get(); + assertEquals(mkSet(0), brokerFutures.keySet()); + KafkaFutureImpl> brokerFuture = brokerFutures.get(0); + assertFalse(brokerFuture.isDone()); + + // Send `ListTransactions` requests + List>.RequestSpec> requestSpecs = driver.poll(); + assertEquals(1, requestSpecs.size()); + RequestDriver>.RequestSpec request = + findBrokerRequest(requestSpecs, 0); + assertListTransactionsRequest(options, request); + + // Receive `ListTransactions` responses + ListTransactionsResponse broker0Response = new ListTransactionsResponse( + new ListTransactionsResponseData().setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code())); + driver.onResponse(time.milliseconds(), request, broker0Response); + assertFalse(brokerFuture.isDone()); + + // Now we expect `ListTransactions` to be retried + List>.RequestSpec> retrySpecs = driver.poll(); + assertEquals(1, retrySpecs.size()); + RequestDriver>.RequestSpec retryRequest = + findBrokerRequest(requestSpecs, 0); + assertListTransactionsRequest(options, retryRequest); + + driver.onResponse(time.milliseconds(), retryRequest, sampleListTransactionsResponse1()); + assertTrue(brokerFuture.isDone()); + } + + @Test + public void testFatalListTransactionsError() throws Exception { + ListTransactionsOptions options = new ListTransactionsOptions(); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + + KafkaFutureImpl>>> lookupFuture = + driver.lookupFuture(); + + // Send `Metadata` request to find brokerIds + RequestDriver>.RequestSpec lookupRequestSpec = + assertLookupRequest(driver); + + // Receive `Metadata` response + driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(singletonList( + brokerMetadata(0) + ))); + + assertTrue(lookupFuture.isDone()); + + Map>> brokerFutures = lookupFuture.get(); + assertEquals(mkSet(0), brokerFutures.keySet()); + KafkaFutureImpl> brokerFuture = brokerFutures.get(0); + assertFalse(brokerFuture.isDone()); + + // Send `ListTransactions` requests + List>.RequestSpec> requestSpecs = driver.poll(); + assertEquals(1, requestSpecs.size()); + RequestDriver>.RequestSpec request = + findBrokerRequest(requestSpecs, 0); + assertListTransactionsRequest(options, request); + + // Receive `ListTransactions` responses with an unexpected error + ListTransactionsResponse broker0Response = new ListTransactionsResponse( + new ListTransactionsResponseData().setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())); + driver.onResponse(time.milliseconds(), request, broker0Response); + assertTrue(brokerFuture.isDone()); + assertFutureThrows(brokerFuture, UnknownServerException.class); + } + + private ListTransactionsResponse sampleListTransactionsResponse1() { + return new ListTransactionsResponse( + new ListTransactionsResponseData().setTransactionStates(asList( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("foo") + .setProducerId(12345L) + .setTransactionState("Ongoing"), + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("bar") + .setProducerId(98765L) + .setTransactionState("PrepareAbort") + )) + ); + } + + private ListTransactionsResponse sampleListTransactionsResponse2() { + return new ListTransactionsResponse( + new ListTransactionsResponseData().setTransactionStates(singletonList( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("baz") + .setProducerId(13579L) + .setTransactionState("CompleteCommit") + )) + ); + } + + private void assertExpectedTransactions( + List expected, + Collection actual + ) { + assertEquals(expected.size(), actual.size()); + + Map expectedMap = expected.stream().collect(Collectors.toMap( + ListTransactionsResponseData.TransactionState::transactionalId, + Function.identity() + )); + + for (TransactionListing actualListing : actual) { + ListTransactionsResponseData.TransactionState expectedState = + expectedMap.get(actualListing.transactionalId()); + assertNotNull(expectedState); + assertExpectedTransactionState(expectedState, actualListing); + } + } + + private void assertExpectedTransactionState( + ListTransactionsResponseData.TransactionState expected, + ListTransactionsResult.TransactionListing actual + ) { + assertEquals(expected.transactionalId(), actual.transactionalId()); + assertEquals(expected.producerId(), actual.producerId()); + assertEquals(expected.transactionState(), actual.transactionState().toString()); + } + + private void assertListTransactionsRequest( + ListTransactionsOptions options, + RequestDriver>.RequestSpec spec + ) { + assertTrue(spec.request instanceof ListTransactionsRequest.Builder); + ListTransactionsRequest.Builder request = (ListTransactionsRequest.Builder) spec.request; + + assertEquals(options.filteredProducerIds(), new HashSet<>(request.data.producerIdFilter())); + + Set expectedFilteredStates = options.filteredStates().stream() + .map(TransactionState::toString) + .collect(Collectors.toSet()); + assertEquals(expectedFilteredStates, new HashSet<>(request.data.statesFilter())); + } + + private RequestDriver>.RequestSpec findBrokerRequest( + List>.RequestSpec> requests, + Integer brokerId + ) { + Optional>.RequestSpec> requestOpt = requests.stream() + .filter(spec -> spec.scope.destinationBrokerId().isPresent() + && spec.scope.destinationBrokerId().getAsInt() == brokerId) + .findFirst(); + assertTrue(requestOpt.isPresent()); + return requestOpt.get(); + } + + private RequestDriver>.RequestSpec assertLookupRequest( + ListTransactionsRequestDriver driver + ) { + List>.RequestSpec> requests = driver.poll(); + assertEquals(1, requests.size()); + + RequestDriver>.RequestSpec lookupRequestSpec = requests.get(0); + assertEquals(OptionalInt.empty(), lookupRequestSpec.scope.destinationBrokerId()); + assertTrue(lookupRequestSpec.request instanceof MetadataRequest.Builder); + + MetadataRequest.Builder metadataRequest = (MetadataRequest.Builder) lookupRequestSpec.request; + assertFalse(metadataRequest.isAllTopics()); + assertEquals(Collections.emptyList(), metadataRequest.data.topics()); + return lookupRequestSpec; + } + + private MetadataResponse metadataResponse(List brokers) { + MetadataResponseData response = new MetadataResponseData(); + brokers.forEach(response.brokers()::add); + return new MetadataResponse(response); + } + + private MetadataResponseData.MetadataResponseBroker brokerMetadata(int brokerId) { + return new MetadataResponseData.MetadataResponseBroker() + .setNodeId(brokerId) + .setHost("localhost") + .setPort(9092 + brokerId); + } + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 37c9026e3f033..8f01a460dca47 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -119,6 +119,8 @@ import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.ListTransactionsRequestData; +import org.apache.kafka.common.message.ListTransactionsResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.message.OffsetDeleteRequestData; @@ -501,6 +503,9 @@ public void testSerialization() throws Exception { checkRequest(createDescribeTransactionsRequest(), true); checkErrorResponse(createDescribeTransactionsRequest(), unknownServerException, true); checkResponse(createDescribeTransactionsResponse(), 0, true); + checkRequest(createListTransactionsRequest(), true); + checkErrorResponse(createListTransactionsRequest(), unknownServerException, true); + checkResponse(createListTransactionsResponse(), 0, true); } @Test @@ -2459,7 +2464,7 @@ private DescribeTransactionsResponse createDescribeTransactionsResponse() { .setPartitionIndexes(asList(1, 3, 5, 7)), new DescribeTransactionsResponseData.TopicData() .setName("bar") - .setPartitionIndexes(asList(1, 3, 5, 7)) + .setPartitionIndexes(asList(1, 3)) )), new DescribeTransactionsResponseData.TransactionState() .setErrorCode(Errors.NOT_COORDINATOR.code()) @@ -2468,4 +2473,27 @@ private DescribeTransactionsResponse createDescribeTransactionsResponse() { return new DescribeTransactionsResponse(data); } + private ListTransactionsRequest createListTransactionsRequest() { + return new ListTransactionsRequest.Builder(new ListTransactionsRequestData() + .setStatesFilter(singletonList("Ongoing")) + .setProducerIdFilter(asList(1L, 2L, 15L)) + ).build(); + } + + private ListTransactionsResponse createListTransactionsResponse() { + ListTransactionsResponseData response = new ListTransactionsResponseData(); + response.setErrorCode(Errors.NONE.code()); + response.setTransactionStates(Arrays.asList( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("foo") + .setProducerId(12345L) + .setTransactionState("Ongoing"), + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("bar") + .setProducerId(98765L) + .setTransactionState("PrepareAbort") + )); + return new ListTransactionsResponse(response); + } + } diff --git a/core/src/main/scala/kafka/admin/TransactionCommand.scala b/core/src/main/scala/kafka/admin/TransactionCommand.scala index e4f6e30dc2826..4061fdf2b7fac 100644 --- a/core/src/main/scala/kafka/admin/TransactionCommand.scala +++ b/core/src/main/scala/kafka/admin/TransactionCommand.scala @@ -78,6 +78,42 @@ object TransactionCommand extends Logging { rows.foreach(printRow) } + private def listTransactions( + admin: Admin + ): Unit = { + val result = try { + admin.listTransactions() + .allByBrokerId() + .get() + } catch { + case e: ExecutionException => + val cause = e.getCause + debug("Failed to list transactions due to exception", e) + printErrorMessageAndDie(s"Failed to list transactions: ${cause.getMessage}: " + + s"Enable debug logging for additional detail") + } + + val headers = Array( + "TransactionalId", + "Coordinator", + "ProducerId", + "TransactionState" + ) + + val rows = result.asScala.flatMap { case (brokerId, transactions) => + transactions.asScala.map { transaction => + Array( + transaction.transactionalId, + brokerId.toString, + transaction.producerId.toString, + transaction.transactionState.toString + ) + } + }.toSeq + + prettyPrintTable(headers, rows) + } + private def describeTransactions( admin: Admin, transactionalId: String @@ -201,6 +237,9 @@ object TransactionCommand extends Logging { printErrorMessageAndDie("The --describe action requires the " + "--transactional-id argument") } + } else if (commandOptions.options.has(commandOptions.listOption)) { + // FIXME: Implement brokerId filtering + listTransactions(admin) } } } @@ -252,6 +291,10 @@ private final class TransactionCommandOptions(args: Array[String]) extends Comma "Used to describe the transaction state of a specific transactional id " + "(requires --transactional-id)") + val listOption = parser + .accepts("list", + "Used to list transactions") + val describeProducersOption = parser .accepts("describe-producers", "Used to describe active transactional/idempotent producers " + @@ -259,7 +302,7 @@ private final class TransactionCommandOptions(args: Array[String]) extends Comma options = parser.parse(args: _*) - if (Seq(describeOption, describeProducersOption).count(options.has) != 1) { + if (Seq(listOption, describeOption, describeProducersOption).count(options.has) != 1) { CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe-producers") } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index bed0aa8ddd5f6..b72fc634d1899 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -24,7 +24,7 @@ import kafka.utils.{Logging, Scheduler} import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.message.DescribeTransactionsResponseData +import org.apache.kafka.common.message.{DescribeTransactionsResponseData, ListTransactionsResponseData} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch @@ -258,11 +258,26 @@ class TransactionCoordinator(brokerId: Int, } } - def handleDescribeTransactions(transactionalId: String): DescribeTransactionsResponseData.TransactionState = { + def handleListTransactions( + filteredProducerIds: Set[Long], + filteredStates: Set[String] + ): Either[Errors, List[ListTransactionsResponseData.TransactionState]] = { + if (!isActive.get()) { + Left(Errors.COORDINATOR_NOT_AVAILABLE) + } else { + txnManager.listTransactionStates(filteredProducerIds, filteredStates) + } + } + + def handleDescribeTransactions( + transactionalId: String + ): DescribeTransactionsResponseData.TransactionState = { val transactionState = new DescribeTransactionsResponseData.TransactionState() .setTransactionalId(transactionalId) - if (transactionalId == null || transactionalId.isEmpty) { + if (!isActive.get()) { + transactionState.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code) + } else if (transactionalId == null || transactionalId.isEmpty) { transactionState.setErrorCode(Errors.INVALID_REQUEST.code) } else { txnManager.getTransactionState(transactionalId) match { diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index 7fc1f3b50cc96..ca725930a891b 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -151,7 +151,7 @@ object TransactionLog { value.set(ProducerIdField, txnMetadata.producerId) value.set(ProducerEpochField, txnMetadata.producerEpoch) value.set(TxnTimeoutField, txnMetadata.txnTimeoutMs) - value.set(TxnStatusField, txnMetadata.txnState.byte) + value.set(TxnStatusField, txnMetadata.txnState.id) value.set(TxnEntryTimestampField, txnMetadata.txnLastUpdateTimestamp) value.set(TxnStartTimestampField, txnMetadata.txnStartTimestamp) @@ -218,7 +218,7 @@ object TransactionLog { val timeout = value.getInt(TxnTimeoutField) val stateByte = value.getByte(TxnStatusField) - val state = TransactionMetadata.byteToState(stateByte) + val state = TransactionMetadata.fromId(stateByte) val entryTimestamp = value.getLong(TxnEntryTimestampField) val startTimestamp = value.getLong(TxnStartTimestampField) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala index e059b04a37374..c2b10afcf3563 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.record.RecordBatch import scala.collection.{immutable, mutable} -private[transaction] sealed trait TransactionState { def byte: Byte } +private[transaction] sealed trait TransactionState { def id: Byte } /** * Transaction has not existed yet @@ -33,7 +33,7 @@ private[transaction] sealed trait TransactionState { def byte: Byte } * transition: received AddPartitionsToTxnRequest => Ongoing * received AddOffsetsToTxnRequest => Ongoing */ -private[transaction] case object Empty extends TransactionState { val byte: Byte = 0 } +private[transaction] case object Empty extends TransactionState { val id: Byte = 0 } /** * Transaction has started and ongoing @@ -43,46 +43,46 @@ private[transaction] case object Empty extends TransactionState { val byte: Byte * received AddPartitionsToTxnRequest => Ongoing * received AddOffsetsToTxnRequest => Ongoing */ -private[transaction] case object Ongoing extends TransactionState { val byte: Byte = 1 } +private[transaction] case object Ongoing extends TransactionState { val id: Byte = 1 } /** * Group is preparing to commit * * transition: received acks from all partitions => CompleteCommit */ -private[transaction] case object PrepareCommit extends TransactionState { val byte: Byte = 2} +private[transaction] case object PrepareCommit extends TransactionState { val id: Byte = 2} /** * Group is preparing to abort * * transition: received acks from all partitions => CompleteAbort */ -private[transaction] case object PrepareAbort extends TransactionState { val byte: Byte = 3 } +private[transaction] case object PrepareAbort extends TransactionState { val id: Byte = 3 } /** * Group has completed commit * * Will soon be removed from the ongoing transaction cache */ -private[transaction] case object CompleteCommit extends TransactionState { val byte: Byte = 4 } +private[transaction] case object CompleteCommit extends TransactionState { val id: Byte = 4 } /** * Group has completed abort * * Will soon be removed from the ongoing transaction cache */ -private[transaction] case object CompleteAbort extends TransactionState { val byte: Byte = 5 } +private[transaction] case object CompleteAbort extends TransactionState { val id: Byte = 5 } /** * TransactionalId has expired and is about to be removed from the transaction cache */ -private[transaction] case object Dead extends TransactionState { val byte: Byte = 6 } +private[transaction] case object Dead extends TransactionState { val id: Byte = 6 } /** * We are in the middle of bumping the epoch and fencing out older producers. */ -private[transaction] case object PrepareEpochFence extends TransactionState { val byte: Byte = 7} +private[transaction] case object PrepareEpochFence extends TransactionState { val id: Byte = 7} private[transaction] object TransactionMetadata { def apply(transactionalId: String, producerId: Long, producerEpoch: Short, txnTimeoutMs: Int, timestamp: Long) = @@ -99,7 +99,21 @@ private[transaction] object TransactionMetadata { new TransactionMetadata(transactionalId, producerId, lastProducerId, producerEpoch, lastProducerEpoch, txnTimeoutMs, state, collection.mutable.Set.empty[TopicPartition], timestamp, timestamp) - def byteToState(byte: Byte): TransactionState = { + def fromName(name: String): Option[TransactionState] = { + name match { + case "Empty" => Some(Empty) + case "Ongoing" => Some(Ongoing) + case "PrepareCommit" => Some(PrepareCommit) + case "PrepareAbort" => Some(PrepareAbort) + case "CompleteCommit" => Some(CompleteCommit) + case "CompleteAbort" => Some(CompleteAbort) + case "PrepareEpochFence" => Some(PrepareEpochFence) + case "Dead" => Some(Dead) + case _ => None + } + } + + def fromId(byte: Byte): TransactionState = { byte match { case 0 => Empty case 1 => Ongoing diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 923aed9776635..232e359078940 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -29,6 +29,7 @@ import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} import kafka.zk.KafkaZkClient import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.ListTransactionsResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.stats.{Avg, Max} import org.apache.kafka.common.protocol.Errors @@ -38,8 +39,8 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} -import scala.jdk.CollectionConverters._ import scala.collection.mutable +import scala.jdk.CollectionConverters._ object TransactionStateManager { @@ -223,6 +224,44 @@ class TransactionStateManager(brokerId: Int, throw new IllegalStateException(s"Unexpected empty transaction metadata returned while putting $txnMetadata"))) } + def listTransactionStates( + filteredProducerIds: Set[Long], + filteredStateNames: Set[String] + ): Either[Errors, List[ListTransactionsResponseData.TransactionState]] = { + inReadLock(stateLock) { + if (loadingPartitions.nonEmpty) { + Left(Errors.COORDINATOR_LOAD_IN_PROGRESS) + } else { + val filteredStates = filteredStateNames.flatMap(TransactionMetadata.fromName) + val states = mutable.ListBuffer.empty[ListTransactionsResponseData.TransactionState] + + def shouldInclude(txnMetadata: TransactionMetadata): Boolean = { + if (txnMetadata.state == Dead) { + false + } else { + (filteredProducerIds.isEmpty && filteredStates.isEmpty) || + filteredProducerIds.contains(txnMetadata.producerId) || + filteredStates.contains(txnMetadata.state) + } + } + + transactionMetadataCache.foreach { case (_, cache) => + cache.metadataPerTransactionalId.values.foreach { txnMetadata => + txnMetadata.inLock { + if (shouldInclude(txnMetadata)) { + states += new ListTransactionsResponseData.TransactionState() + .setTransactionalId(txnMetadata.transactionalId) + .setProducerId(txnMetadata.producerId) + .setTransactionState(txnMetadata.state.toString) + } + } + } + } + Right(states.toList) + } + } + } + /** * Get the transaction metadata associated with the given transactional id, or an error if * the coordinator does not own the transaction partition or is still loading it; if not found diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 8f638cccbbba7..cb3d476d45f24 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -51,7 +51,7 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, DescribeTransactionsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, DescribeTransactionsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, ListTransactionsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -182,6 +182,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => handleAlterUserScramCredentialsRequest(request) case ApiKeys.DESCRIBE_PRODUCERS => handleDescribeProducersRequest(request) case ApiKeys.DESCRIBE_TRANSACTIONS => handleDescribeTransactionsRequest(request) + case ApiKeys.LIST_TRANSACTIONS => handleListTransactionsRequest(request) } } catch { case e: FatalExitError => throw e @@ -3112,7 +3113,29 @@ class KafkaApis(val requestChannel: RequestChannel, new DescribeTransactionsResponse(response.setThrottleTimeMs(requestThrottleMs))) } - // private package for testing + def handleListTransactionsRequest(request: RequestChannel.Request): Unit = { + val listTransactionsRequest = request.body[ListTransactionsRequest] + val response = new ListTransactionsResponseData() + + val filteredProducerIds = listTransactionsRequest.data.producerIdFilter.asScala.map(Long.unbox).toSet + val filteredStates = listTransactionsRequest.data.statesFilter.asScala.toSet + + txnCoordinator.handleListTransactions(filteredProducerIds, filteredStates) match { + case Left(error) => + response.setErrorCode(error.code) + case Right(transactions) => + val authorizedTransactions = transactions.filter { state => + authorize(request.context, DESCRIBE, TRANSACTIONAL_ID, state.transactionalId) + } + response.setErrorCode(Errors.NONE.code) + .setTransactionStates(authorizedTransactions.asJava) + } + + sendResponseMaybeThrottle(request, requestThrottleMs => + new ListTransactionsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } + + // private package for testing private[server] def authorize(requestContext: RequestContext, operation: AclOperation, resourceType: ResourceType, From dc4d02dc83401fa770e7b1bac15fe8648e70b88a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Sat, 5 Sep 2020 17:32:40 -0700 Subject: [PATCH 04/17] Add Admin support for aborting transactions --- bin/kafka-transactions.sh | 2 +- .../admin/AbortTransactionOptions.java | 23 + .../clients/admin/AbortTransactionResult.java | 52 ++ .../clients/admin/AbortTransactionSpec.java | 85 +++ .../org/apache/kafka/clients/admin/Admin.java | 6 + .../admin/DescribeProducersOptions.java | 22 + .../admin/DescribeProducersResult.java | 43 +- .../admin/DescribeTransactionsResult.java | 1 + .../kafka/clients/admin/KafkaAdminClient.java | 25 +- .../AbortTransactionRequestDriver.java | 147 +++++ .../DescribeProducersRequestDriver.java | 7 + .../ListTransactionsRequestDriver.java | 1 + .../requests/WriteTxnMarkersRequest.java | 6 + .../message/DescribeProducersResponse.json | 1 + .../message/WriteTxnMarkersRequest.json | 4 +- .../kafka/clients/admin/MockAdminClient.java | 5 + .../AbortTransactionRequestDriverTest.java | 226 +++++++ .../kafka/connect/mirror/MirrorMaker.java | 2 +- .../kafka/admin/TransactionCommand.scala | 309 --------- .../unit/kafka/server/RequestQuotaTest.scala | 3 +- .../kafka/message/MessageGenerator.java | 3 +- gradle/dependencies.gradle | 2 +- .../kafka/tools/ClientCompatibilityTest.java | 3 +- .../tools/PrintVersionAndExitAction.java | 51 ++ .../kafka/tools/ProducerPerformance.java | 3 +- .../tools/TransactionalMessageCopier.java | 3 +- .../kafka/tools/TransactionsCommand.java | 620 ++++++++++++++++++ .../kafka/tools/VerifiableConsumer.java | 3 +- .../kafka/tools/VerifiableLog4jAppender.java | 3 +- .../kafka/tools/VerifiableProducer.java | 3 +- .../org/apache/kafka/trogdor/agent/Agent.java | 3 +- .../kafka/trogdor/agent/AgentClient.java | 3 +- .../trogdor/coordinator/Coordinator.java | 3 +- .../coordinator/CoordinatorClient.java | 3 +- .../kafka/tools/TransactionsCommandTest.java | 520 +++++++++++++++ 35 files changed, 1863 insertions(+), 333 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java delete mode 100644 core/src/main/scala/kafka/admin/TransactionCommand.scala create mode 100644 tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java create mode 100644 tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java create mode 100644 tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java diff --git a/bin/kafka-transactions.sh b/bin/kafka-transactions.sh index 0bb79c6fd8e31..936eea72d79cb 100755 --- a/bin/kafka-transactions.sh +++ b/bin/kafka-transactions.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -exec $(dirname $0)/kafka-run-class.sh kafka.admin.TransactionCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.TransactionCommand "$@" diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java new file mode 100644 index 0000000000000..ad5827ac6d0f5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +@InterfaceStability.Evolving +public class AbortTransactionOptions extends AbstractOptions { +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java new file mode 100644 index 0000000000000..ed7421b89a539 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; + +import java.util.Map; +import java.util.concurrent.ExecutionException; + +@InterfaceStability.Evolving +public class AbortTransactionResult { + private final Map> futures; + + AbortTransactionResult(Map> futures) { + this.futures = futures; + } + + public KafkaFuture all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])) + .thenApply(nil -> { + for (Map.Entry> entry : futures.entrySet()) { + try { + KafkaFutureImpl future = entry.getValue(); + future.get(); + } catch (InterruptedException | ExecutionException e) { + // This should be unreachable, because allOf ensured that all the futures completed successfully. + throw new KafkaException(e); + } + } + return null; + }); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java new file mode 100644 index 0000000000000..8e9ae0c7bd366 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Objects; + +@InterfaceStability.Evolving +public class AbortTransactionSpec { + private final TopicPartition topicPartition; + private final long producerId; + private final int producerEpoch; + private final int coordinatorEpoch; + + public AbortTransactionSpec( + TopicPartition topicPartition, + long producerId, + int producerEpoch, + int coordinatorEpoch + ) { + this.topicPartition = topicPartition; + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.coordinatorEpoch = coordinatorEpoch; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + + public long producerId() { + return producerId; + } + + public int producerEpoch() { + return producerEpoch; + } + + public int coordinatorEpoch() { + return coordinatorEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AbortTransactionSpec that = (AbortTransactionSpec) o; + return producerId == that.producerId && + producerEpoch == that.producerEpoch && + coordinatorEpoch == that.coordinatorEpoch && + Objects.equals(topicPartition, that.topicPartition); + } + + @Override + public int hashCode() { + return Objects.hash(topicPartition, producerId, producerEpoch, coordinatorEpoch); + } + + @Override + public String toString() { + return "AbortTransactionSpec{" + + "topicPartition=" + topicPartition + + ", producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", coordinatorEpoch=" + coordinatorEpoch + + '}'; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index cd2caa61261fd..9db7076292e55 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1233,6 +1233,12 @@ default ListTransactionsResult listTransactions() { ListTransactionsResult listTransactions(ListTransactionsOptions options); + default AbortTransactionResult abortTransaction(AbortTransactionSpec spec) { + return abortTransaction(spec, new AbortTransactionOptions()); + } + + AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options); + /** * Describe all SASL/SCRAM credentials. * diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java index 2480195e3c147..e69fca0201a66 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; import java.util.Collection; +import java.util.Objects; import java.util.OptionalInt; /** @@ -38,4 +39,25 @@ public DescribeProducersOptions setBrokerId(int brokerId) { public OptionalInt brokerId() { return brokerId; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DescribeProducersOptions that = (DescribeProducersOptions) o; + return Objects.equals(brokerId, that.brokerId); + } + + @Override + public int hashCode() { + return Objects.hash(brokerId); + } + + @Override + public String toString() { + return "DescribeProducersOptions(" + + "brokerId=" + brokerId + + ", timeoutMs=" + timeoutMs + + ')'; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java index 006909d78e142..4db42e685022a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; @@ -24,6 +25,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.OptionalInt; import java.util.OptionalLong; import java.util.concurrent.ExecutionException; @@ -54,7 +57,7 @@ public KafkaFuture> all() { results.put(entry.getKey(), entry.getValue().get()); } catch (InterruptedException | ExecutionException e) { // This should be unreachable, because allOf ensured that all the futures completed successfully. - throw new RuntimeException(e); + throw new KafkaException(e); } } return results; @@ -78,6 +81,7 @@ public static class ProducerState { private final int producerEpoch; private final int lastSequence; private final long lastTimestamp; + private final OptionalInt coordinatorEpoch; private final OptionalLong currentTransactionStartOffset; public ProducerState( @@ -85,12 +89,14 @@ public ProducerState( int producerEpoch, int lastSequence, long lastTimestamp, + OptionalInt coordinatorEpoch, OptionalLong currentTransactionStartOffset ) { this.producerId = producerId; this.producerEpoch = producerEpoch; this.lastSequence = lastSequence; this.lastTimestamp = lastTimestamp; + this.coordinatorEpoch = coordinatorEpoch; this.currentTransactionStartOffset = currentTransactionStartOffset; } @@ -113,6 +119,41 @@ public long lastTimestamp() { public OptionalLong currentTransactionStartOffset() { return currentTransactionStartOffset; } + + public OptionalInt coordinatorEpoch() { + return coordinatorEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ProducerState that = (ProducerState) o; + return producerId == that.producerId && + producerEpoch == that.producerEpoch && + lastSequence == that.lastSequence && + lastTimestamp == that.lastTimestamp && + Objects.equals(coordinatorEpoch, that.coordinatorEpoch) && + Objects.equals(currentTransactionStartOffset, that.currentTransactionStartOffset); + } + + @Override + public int hashCode() { + return Objects.hash(producerId, producerEpoch, lastSequence, lastTimestamp, + coordinatorEpoch, currentTransactionStartOffset); + } + + @Override + public String toString() { + return "ProducerState(" + + "producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", lastSequence=" + lastSequence + + ", lastTimestamp=" + lastTimestamp + + ", coordinatorEpoch=" + coordinatorEpoch + + ", currentTransactionStartOffset=" + currentTransactionStartOffset + + ')'; + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java index b790a62e25a1c..2d51f11c19a7a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java @@ -75,6 +75,7 @@ public static class TransactionState { private final OptionalLong transactionStartTimeMs; private final Set topicPartitions; + // TODO: We need to use `TransactionState` public TransactionState( String state, long producerId, diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 49a923885d83e..90ffe5d816833 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -39,6 +39,7 @@ import org.apache.kafka.clients.admin.internals.ListTransactionsRequestDriver; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; import org.apache.kafka.clients.admin.internals.RequestDriver; +import org.apache.kafka.clients.admin.internals.AbortTransactionRequestDriver; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; @@ -307,6 +308,7 @@ public class KafkaAdminClient extends AdminClient { static final String NETWORK_THREAD_PREFIX = "kafka-admin-client-thread"; private final Logger log; + private final LogContext logContext; /** * The default timeout to use for an operation. @@ -562,6 +564,7 @@ private KafkaAdminClient(AdminClientConfig config, LogContext logContext) { this.clientId = clientId; this.log = logContext.logger(KafkaAdminClient.class); + this.logContext = logContext; this.requestTimeoutMs = config.getInt(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG); this.defaultApiTimeoutMs = configureDefaultApiTimeoutMs(config); this.time = time; @@ -4287,7 +4290,7 @@ public DescribeProducersResult describeProducers(Collection part long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( partitions, options, deadlineMs, retryBackoffMs); - maybeSendRequests(currentTimeMs, driver); + maybeSendRequests(driver, currentTimeMs); return new DescribeProducersResult(driver.futures()); } @@ -4300,7 +4303,7 @@ public DescribeTransactionsResult describeTransactions(Collection transa long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( transactionalIds, deadlineMs, retryBackoffMs); - maybeSendRequests(currentTimeMs, driver); + maybeSendRequests(driver, currentTimeMs); return new DescribeTransactionsResult(driver.futures()); } @@ -4309,10 +4312,20 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); - maybeSendRequests(currentTimeMs, driver); + maybeSendRequests(driver, currentTimeMs); return new ListTransactionsResult(driver.lookupFuture()); } + @Override + public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { + long currentTimeMs = time.milliseconds(); + long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(spec, deadlineMs, + retryBackoffMs, logContext); + maybeSendRequests(driver, currentTimeMs); + return new AbortTransactionResult(driver.futures()); + } + /** * Get a sub level error when the request is in batch. If given key was not found, * return an {@link IllegalArgumentException}. @@ -4325,7 +4338,7 @@ static Throwable getSubLevelError(Map subLevelErrors, K subKey, S } } - private void maybeSendRequests(long currentTimeMs, RequestDriver driver) { + private void maybeSendRequests(RequestDriver driver, long currentTimeMs) { for (RequestDriver.RequestSpec spec : driver.poll()) { runnable.call(newCall(driver, spec), currentTimeMs); } @@ -4347,14 +4360,14 @@ AbstractRequest.Builder createRequest(int timeoutMs) { void handleResponse(AbstractResponse response) { long currentTimeMs = time.milliseconds(); driver.onResponse(currentTimeMs, spec, response); - maybeSendRequests(currentTimeMs, driver); + maybeSendRequests(driver, currentTimeMs); } @Override void handleFailure(Throwable throwable) { long currentTimeMs = time.milliseconds(); driver.onFailure(currentTimeMs, spec, throwable); - maybeSendRequests(currentTimeMs, driver); + maybeSendRequests(driver, currentTimeMs); } }; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java new file mode 100644 index 0000000000000..073edcad38986 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.InvalidTxnStateException; +import org.apache.kafka.common.errors.TransactionCoordinatorFencedException; +import org.apache.kafka.common.message.WriteTxnMarkersResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.List; +import java.util.Set; + +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; + +public class AbortTransactionRequestDriver extends MetadataRequestDriver { + private final Logger log; + private final AbortTransactionSpec abortSpec; + private final TopicPartition topicPartition; + + public AbortTransactionRequestDriver( + AbortTransactionSpec abortSpec, + long deadlineMs, + long retryBackoffMs, + LogContext logContext + ) { + super(singleton(abortSpec.topicPartition()), deadlineMs, retryBackoffMs); + this.abortSpec = abortSpec; + this.topicPartition = abortSpec.topicPartition(); + this.log = logContext.logger(AbortTransactionRequestDriver.class); + } + + @Override + WriteTxnMarkersRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + if (!topicPartitions.equals(singleton(topicPartition))) { + throw new IllegalArgumentException("Received unexpected topic partitions " + topicPartitions + + " (expected " + singleton(topicPartition) + ")"); + } + WriteTxnMarkersRequest.TxnMarkerEntry markerEntry = new WriteTxnMarkersRequest.TxnMarkerEntry( + abortSpec.producerId(), + (short) abortSpec.producerEpoch(), + abortSpec.coordinatorEpoch(), + TransactionResult.ABORT, + singletonList(topicPartition) + ); + return new WriteTxnMarkersRequest.Builder(singletonList(markerEntry)); + } + + @Override + void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + WriteTxnMarkersResponse response = (WriteTxnMarkersResponse) abstractResponse; + List markerResponses = response.data.markers(); + + if (markerResponses.size() != 1) { + super.completeExceptionally(topicPartition, new KafkaException("WriteTxnMarkers response " + + "included unexpected marker entries: " + markerResponses + "(expected to find exactly one " + + "entry with producerId " + abortSpec.producerId() + ")")); + return; + } + + WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = markerResponses.get(0); + List topicResponses = markerResponse.topics(); + + if (topicResponses.size() != 1) { + super.completeExceptionally(topicPartition, new KafkaException("WriteTxnMarkers response " + + "included unexpected topic entries: " + markerResponses + "(expected to find exactly one " + + "entry with topic partition " + topicPartition + ")")); + return; + } + + WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = topicResponses.get(0); + List partitionResponses = + topicResponse.partitions(); + + if (partitionResponses.size() != 1) { + super.completeExceptionally(topicPartition, new KafkaException("WriteTxnMarkers response " + + "included unexpected partition entries for topic " + topicPartition.topic() + + ": " + markerResponses + "(expected to find exactly one entry with partition " + + topicPartition.partition() + ")")); + return; + } + + WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = partitionResponses.get(0); + Errors error = Errors.forCode(partitionResponse.errorCode()); + + if (error != Errors.NONE) { + handleError(error); + } else { + super.complete(topicPartition, null); + } + } + + private void handleError(Errors error) { + switch (error) { + case CLUSTER_AUTHORIZATION_FAILED: + super.completeExceptionally(topicPartition, new ClusterAuthorizationException( + "WriteTxnMarkers request with " + abortSpec + " failed due to cluster " + + "authorization error.")); + break; + + case TRANSACTION_COORDINATOR_FENCED: + super.completeExceptionally(topicPartition, new TransactionCoordinatorFencedException( + "WriteTxnMarkers request with " + abortSpec + " failed since the provided " + + "coordinator epoch " + abortSpec.coordinatorEpoch() + " has been fenced " + + "by the active coordinator")); + break; + + case NOT_LEADER_OR_FOLLOWER: + case REPLICA_NOT_AVAILABLE: + case BROKER_NOT_AVAILABLE: + case UNKNOWN_TOPIC_OR_PARTITION: + log.debug("WriteTxnMarkers request with {} failed due to {}. Will retry after backing off", + topicPartition, error); + super.unmap(topicPartition); + break; + + default: + super.completeExceptionally(topicPartition, error.exception( + "WriteTxnMarkers request with " + abortSpec + " failed due to unexpected error")); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java index 47cd99b5da8d2..64de23dd55e28 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -35,6 +35,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -134,11 +135,17 @@ void handleFulfillmentResponse(Set keys, AbstractResponse abstra activeProducer.currentTxnStartTimestamp() < 0 ? OptionalLong.empty() : OptionalLong.of(activeProducer.currentTxnStartTimestamp()); + OptionalInt coordinatorEpoch = + activeProducer.coordinatorEpoch() < 0 ? + OptionalInt.empty() : + OptionalInt.of(activeProducer.coordinatorEpoch()); + return new ProducerState( activeProducer.producerId(), activeProducer.producerEpoch(), activeProducer.lastSequence(), activeProducer.lastTimestamp(), + coordinatorEpoch, currentTransactionFirstOffset ); }).collect(Collectors.toList()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java index efb95efe0ecf4..43b0a5d99b9df 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java @@ -35,6 +35,7 @@ import java.util.stream.Collectors; public class ListTransactionsRequestDriver extends AllBrokerRequestDriver> { + // TODO: Use `LogContext` private static final Logger log = LoggerFactory.getLogger(ListTransactionsRequestDriver.class); private final ListTransactionsOptions options; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java index c27263384a5ee..9adc88a683021 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java @@ -105,6 +105,11 @@ public static class Builder extends AbstractRequest.Builder markers) { super(ApiKeys.WRITE_TXN_MARKERS); List dataMarkers = new ArrayList<>(); @@ -130,6 +135,7 @@ public Builder(final List markers) { @Override public WriteTxnMarkersRequest build(short version) { + // TODO: We need to rewrite the request for differences between version 0 and 1 return new WriteTxnMarkersRequest(data, version); } } diff --git a/clients/src/main/resources/common/message/DescribeProducersResponse.json b/clients/src/main/resources/common/message/DescribeProducersResponse.json index 97081f904e749..6590dc4004962 100644 --- a/clients/src/main/resources/common/message/DescribeProducersResponse.json +++ b/clients/src/main/resources/common/message/DescribeProducersResponse.json @@ -37,6 +37,7 @@ { "name": "ProducerEpoch", "type": "int32", "versions": "0+" }, { "name": "LastSequence", "type": "int32", "versions": "0+", "default": "-1" }, { "name": "LastTimestamp", "type": "int64", "versions": "0+", "default": "-1" }, + { "name": "CoordinatorEpoch", "type": "int32", "versions": "0+" }, { "name": "CurrentTxnStartTimestamp", "type": "int64", "versions": "0+", "default": "-1" } ]} ]} diff --git a/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json b/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json index 845a4ca3735f1..62e0fd07f4d1c 100644 --- a/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json +++ b/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json @@ -32,10 +32,10 @@ "about": "Each topic that we want to write transaction marker(s) for.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, - { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+", + { "name": "PartitionIndexes", "type": "[]int32", "versions": "0", "about": "The indexes of the partitions to write transaction markers for." } ]}, - { "name": "CoordinatorEpoch", "type": "int32", "versions": "0+", + { "name": "CoordinatorEpoch", "type": "int32", "versions": "0+", "default": "-1", "ignorable": "false", "about": "Epoch associated with the transaction state partition hosted by this transaction coordinator" } ]} ] diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 60a6d75e47e6e..a670ace8f75e4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -851,6 +851,11 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) throw new UnsupportedOperationException("Not implemented yet"); } + @Override + public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + @Override synchronized public void close(Duration timeout) {} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java new file mode 100644 index 0000000000000..017b9d8458a60 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin.internals; + +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.message.WriteTxnMarkersRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.OptionalInt; + +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +public class AbortTransactionRequestDriverTest { + private final LogContext logContext = new LogContext(); + private final MockTime time = new MockTime(); + private final long deadlineMs = time.milliseconds() + 10000; + private final long retryBackoffMs = 100; + + @Test + public void testSuccessfulAbortTransaction() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 4321); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + deadlineMs, retryBackoffMs, logContext); + int expectedLeaderId = 5; + assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); + assertWriteTxnMarkers(driver, abortTransactionSpec, Errors.NONE, expectedLeaderId, 0); + assertCompletedFuture(driver, abortTransactionSpec, Errors.NONE); + } + + @Test + public void testFatalTransactionCoordinatorFencedError() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 4321); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + deadlineMs, retryBackoffMs, logContext); + int expectedLeaderId = 5; + assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); + assertWriteTxnMarkers(driver, abortTransactionSpec, Errors.TRANSACTION_COORDINATOR_FENCED, expectedLeaderId, 0); + assertCompletedFuture(driver, abortTransactionSpec, Errors.TRANSACTION_COORDINATOR_FENCED); + } + + @Test + public void testFatalClusterAuthorizationError() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 4321); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + deadlineMs, retryBackoffMs, logContext); + int expectedLeaderId = 5; + assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); + assertWriteTxnMarkers(driver, abortTransactionSpec, Errors.CLUSTER_AUTHORIZATION_FAILED, expectedLeaderId, 0); + assertCompletedFuture(driver, abortTransactionSpec, Errors.CLUSTER_AUTHORIZATION_FAILED); + } + + @Test + public void testRetryLookupAfterNotLeaderError() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 4321); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + deadlineMs, retryBackoffMs, logContext); + + int initialLeaderId = 5; + assertMetadataLookup(driver, abortTransactionSpec, initialLeaderId, 0); + assertWriteTxnMarkers(driver, abortTransactionSpec, Errors.NOT_LEADER_OR_FOLLOWER, initialLeaderId, 0); + + int updatedLeaderId = 3; + assertMetadataLookup(driver, abortTransactionSpec, updatedLeaderId, 1); + assertWriteTxnMarkers(driver, abortTransactionSpec, Errors.NONE, updatedLeaderId, 0); + assertCompletedFuture(driver, abortTransactionSpec, Errors.NONE); + } + + private void assertMetadataLookup( + AbortTransactionRequestDriver driver, + AbortTransactionSpec abortTransactionSpec, + int expectedLeaderId, + int expectedTries + ) { + TopicPartition topicPartition = abortTransactionSpec.topicPartition(); + RequestDriver.RequestSpec lookupSpec = assertMetadataRequest( + driver, topicPartition, expectedTries); + driver.onResponse(time.milliseconds(), lookupSpec, new MetadataResponse( + AdminRequestUtil.metadataResponse(singletonMap(topicPartition, + new MetadataResponseData.MetadataResponsePartition() + .setLeaderId(expectedLeaderId) + .setReplicaNodes(singletonList(expectedLeaderId)) + .setLeaderEpoch(15) + .setIsrNodes(singletonList(expectedLeaderId)) + )))); + } + + private void assertWriteTxnMarkers( + AbortTransactionRequestDriver driver, + AbortTransactionSpec abortTransactionSpec, + Errors error, + int expectedLeaderId, + int expectedTries + ) { + RequestDriver.RequestSpec requestSpec = assertWriteTxnMarkersRequest( + driver, abortTransactionSpec, expectedLeaderId, expectedTries); + driver.onResponse(time.milliseconds(), requestSpec, + writeTxnMarkersResponse(abortTransactionSpec, error)); + } + + private void assertCompletedFuture( + AbortTransactionRequestDriver driver, + AbortTransactionSpec abortTransactionSpec, + Errors error + ) throws Exception { + assertEquals(Collections.emptyList(), driver.poll()); + KafkaFutureImpl future = driver.futures().get(abortTransactionSpec.topicPartition()); + assertTrue(future.isDone()); + if (error == Errors.NONE) { + assertNull(future.get()); + } else { + assertFutureThrows(future, error.exception().getClass()); + } + } + + private WriteTxnMarkersResponse writeTxnMarkersResponse( + AbortTransactionSpec abortSpec, + Errors error + ) { + Map partitionErrors = singletonMap(abortSpec.topicPartition(), error); + return new WriteTxnMarkersResponse(singletonMap(abortSpec.producerId(), partitionErrors)); + } + + private RequestDriver.RequestSpec assertWriteTxnMarkersRequest( + AbortTransactionRequestDriver driver, + AbortTransactionSpec abortSpec, + int expectedLeaderId, + int expectedTries + ) { + List.RequestSpec> requestSpecs = driver.poll(); + assertEquals(1, requestSpecs.size()); + + RequestDriver.RequestSpec requestSpec = requestSpecs.get(0); + assertExpectedBackoffAndDeadline(requestSpec, expectedTries); + assertEquals(OptionalInt.of(expectedLeaderId), requestSpec.scope.destinationBrokerId()); + + assertTrue(requestSpec.request instanceof WriteTxnMarkersRequest.Builder); + WriteTxnMarkersRequest.Builder request = (WriteTxnMarkersRequest.Builder) requestSpec.request; + + assertEquals(1, request.data.markers().size()); + WriteTxnMarkersRequestData.WritableTxnMarker requestTxnMarker = request.data.markers().get(0); + assertEquals(TransactionResult.ABORT.id, requestTxnMarker.transactionResult()); + assertEquals(abortSpec.producerId(), requestTxnMarker.producerId()); + assertEquals(abortSpec.producerEpoch(), requestTxnMarker.producerEpoch()); + assertEquals(abortSpec.coordinatorEpoch(), requestTxnMarker.coordinatorEpoch()); + + assertEquals(1, requestTxnMarker.topics().size()); + WriteTxnMarkersRequestData.WritableTxnMarkerTopic requestTopic = requestTxnMarker.topics().get(0); + assertEquals(abortSpec.topicPartition().topic(), requestTopic.name()); + assertEquals(singletonList(abortSpec.topicPartition().partition()), requestTopic.partitionIndexes()); + + return requestSpec; + } + + private RequestDriver.RequestSpec assertMetadataRequest( + AbortTransactionRequestDriver driver, + TopicPartition topicPartition, + int expectedTries + ) { + List.RequestSpec> lookupRequests = driver.poll(); + assertEquals(1, lookupRequests.size()); + + RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + assertExpectedBackoffAndDeadline(lookupSpec, expectedTries); + assertEquals(OptionalInt.empty(), lookupSpec.scope.destinationBrokerId()); + + assertTrue(lookupSpec.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder request = (MetadataRequest.Builder) lookupSpec.request; + assertEquals(singletonList(topicPartition.topic()), request.topics()); + return lookupSpec; + } + + private void assertExpectedBackoffAndDeadline( + RequestDriver.RequestSpec requestSpec, + int expectedTries + ) { + assertEquals(expectedTries, requestSpec.tries); + assertEquals(deadlineMs, requestSpec.deadlineMs); + if (expectedTries == 0) { + assertEquals(0, requestSpec.nextAllowedTryMs); + } else { + assertEquals(time.milliseconds() + (expectedTries * retryBackoffMs), requestSpec.nextAllowedTryMs); + } + } + +} \ No newline at end of file diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java index fa73f8de3a979..552a3916819b7 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java @@ -266,7 +266,7 @@ public void run() { } public static void main(String[] args) { - ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-mirror-maker"); + ArgumentParser parser = ArgumentParsers.newFor("connect-mirror-maker").build(); parser.description("MirrorMaker 2.0 driver"); parser.addArgument("config").type(Arguments.fileType().verifyCanRead()) .metavar("mm2.properties").required(true) diff --git a/core/src/main/scala/kafka/admin/TransactionCommand.scala b/core/src/main/scala/kafka/admin/TransactionCommand.scala deleted file mode 100644 index 4061fdf2b7fac..0000000000000 --- a/core/src/main/scala/kafka/admin/TransactionCommand.scala +++ /dev/null @@ -1,309 +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 kafka.admin - -import java.util.Properties -import java.util.concurrent.ExecutionException - -import kafka.utils.CommandLineUtils.printErrorMessageAndDie -import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Logging} -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeProducersOptions} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Utils - -import scala.jdk.CollectionConverters._ - -object TransactionCommand extends Logging { - - private def createAdmin(commandOptions: TransactionCommandOptions): Admin = { - val props = Option(commandOptions.options.valueOf(commandOptions.adminClientConfig)).map { config => - Utils.loadProps(config) - }.getOrElse(new Properties()) - - - val bootstrapServersOpt = Option(commandOptions.options.valueOf(commandOptions.bootstrapServer)) - if (bootstrapServersOpt.isEmpty) { - printErrorMessageAndDie("Missing required argument --bootstrap-server") - } - - props.setProperty( - AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, - bootstrapServersOpt.get - ) - - Admin.create(props) - } - - private def prettyPrintTable( - headers: Array[String], - rows: Iterable[Array[String]] - ): Unit = { - val columnLengths = headers.map(_.length) - for (row <- rows) { - row.indices.foreach { i => - columnLengths(i) = math.max(columnLengths(i), row(i).length) - } - } - - def printColumn(str: String, len: Int): Unit = { - val padLength = len - str.length - print(str + (" " * padLength)) - } - - def printRow(row: Array[String]): Unit = { - columnLengths.indices.foreach { i => - val columnLength = columnLengths(i) - val columnValue = row(i) - printColumn(columnValue, columnLength) - print('\t') - } - println() - } - - printRow(headers) - rows.foreach(printRow) - } - - private def listTransactions( - admin: Admin - ): Unit = { - val result = try { - admin.listTransactions() - .allByBrokerId() - .get() - } catch { - case e: ExecutionException => - val cause = e.getCause - debug("Failed to list transactions due to exception", e) - printErrorMessageAndDie(s"Failed to list transactions: ${cause.getMessage}: " + - s"Enable debug logging for additional detail") - } - - val headers = Array( - "TransactionalId", - "Coordinator", - "ProducerId", - "TransactionState" - ) - - val rows = result.asScala.flatMap { case (brokerId, transactions) => - transactions.asScala.map { transaction => - Array( - transaction.transactionalId, - brokerId.toString, - transaction.producerId.toString, - transaction.transactionState.toString - ) - } - }.toSeq - - prettyPrintTable(headers, rows) - } - - private def describeTransactions( - admin: Admin, - transactionalId: String - ): Unit = { - val result = try { - admin.describeTransactions(Seq(transactionalId).asJava) - .transactionalIdResult(transactionalId) - .get() - } catch { - case e: ExecutionException => - val cause = e.getCause - debug(s"Failed to describe transaction state of transactional-id `$transactionalId`", cause) - printErrorMessageAndDie(s"Failed to describe transaction state of transactional-id `$transactionalId`: " + - s"${cause.getMessage}. Enable debug logging for additional detail.") - } - - // TODO: Do we want a way to return coordinator ID? - val headers = Array( - "ProducerId", - "ProducerEpoch", - "TransactionState", - "TransactionTimeoutMs", - "CurrentTransactionStartTimeMs", - "TopicPartitions" - ) - - val transactionStartTimeMsColumnValue = if (result.transactionStartTimeMs.isPresent) { - result.transactionStartTimeMs.getAsLong.toString - } else { - "None" - } - - val rows = Array( - result.producerId.toString, - result.producerEpoch.toString, - result.state, - result.transactionTimeoutMs.toString, - transactionStartTimeMsColumnValue, - Utils.join(result.topicPartitions, ",") - ) - - prettyPrintTable(headers, Seq(rows)) - } - - private def describeProducers( - admin: Admin, - brokerId: Option[Int], - topicPartition: TopicPartition - ): Unit = { - val options = new DescribeProducersOptions() - brokerId.foreach(options.setBrokerId) - - val result = try { - admin.describeProducers(Seq(topicPartition).asJava, options) - .partitionResult(topicPartition) - .get() - } catch { - case e: ExecutionException => - val cause = e.getCause - val brokerClause = brokerId.map(id => s"broker $id").getOrElse("leader") - debug(s"Failed to describe producers for partition $topicPartition on $brokerClause", cause) - printErrorMessageAndDie(s"Failed to describe producers for partition $topicPartition on $brokerClause: " + - s"${cause.getMessage}. Enable debug logging for additional detail.") - } - - // TODO: Add coordinator epoch - val headers = Array( - "ProducerId", - "ProducerEpoch", - "LastSequence", - "LastTimestamp", - "CurrentTransactionStartOffset" - ) - - val rows = result.activeProducers().asScala.map { activeProducer => - val currentTransactionStartOffsetColumnValue = if (activeProducer.currentTransactionStartOffset.isPresent) { - activeProducer.currentTransactionStartOffset.toString - } else { - "None" - } - - Array(activeProducer.producerId.toString, - activeProducer.producerEpoch.toString, - activeProducer.lastSequence.toString, - activeProducer.lastTimestamp.toString, - currentTransactionStartOffsetColumnValue - ) - } - - prettyPrintTable(headers, rows) - } - - def main(args: Array[String]): Unit = { - val commandOptions = new TransactionCommandOptions(args) - CommandLineUtils.printHelpAndExitIfNeeded(commandOptions, "This tool is " + - "used to analyze transaction state and recover from hanging transactions") - - val admin = createAdmin(commandOptions) - val brokerIdOpt = Option(commandOptions.options.valueOf(commandOptions.brokerId)).map(Int.unbox) - val topicPartitionOpt = ( - Option(commandOptions.options.valueOf(commandOptions.topic)), - Option(commandOptions.options.valueOf(commandOptions.partition)) - ) match { - case (Some(topic), Some(partition)) => Some(new TopicPartition(topic, partition)) - case _ => None - } - - if (commandOptions.options.has(commandOptions.describeProducersOption)) { - topicPartitionOpt match { - case Some(topicPartition) => - describeProducers(admin, brokerIdOpt, topicPartition) - case None => - printErrorMessageAndDie("The --describe-producers action requires both " + - "the --topic and --partition arguments") - } - } else if (commandOptions.options.has(commandOptions.describeOption)) { - Option(commandOptions.options.valueOf(commandOptions.transactionalId)) match { - case Some(transactionalId) => - describeTransactions(admin, transactionalId) - case None => - printErrorMessageAndDie("The --describe action requires the " + - "--transactional-id argument") - } - } else if (commandOptions.options.has(commandOptions.listOption)) { - // FIXME: Implement brokerId filtering - listTransactions(admin) - } - } -} - -private final class TransactionCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - val bootstrapServer = parser - .accepts("bootstrap-server", - "(REQUIRED) A hostname and port for the broker to connect to, in the form host:port. " + - "Multiple comma separated URLs can be given.") - .withRequiredArg - .describedAs("host:port") - .ofType(classOf[String]) - val adminClientConfig = parser - .accepts("admin.config", - "Configuration properties files to pass to the admin client") - .withRequiredArg - .describedAs("config file") - .ofType(classOf[String]) - - val topic = parser - .accepts("topic", - "Name of topic for which to perform an election.") - .withRequiredArg - .describedAs("topic name") - .ofType(classOf[String]) - - val partition = parser - .accepts("partition", - "Partition id. REQUIRED if --topic is specified.") - .withRequiredArg - .describedAs("partition id") - .ofType(classOf[Integer]) - - val brokerId = parser - .accepts("broker-id", - "Used with --topic and --partition to indicate a specific broker to verify") - .withRequiredArg - .describedAs("broker id") - .ofType(classOf[Integer]) - - val transactionalId = parser - .accepts("transactional-id") - .withRequiredArg - .describedAs("transactional id") - .ofType(classOf[String]) - - val describeOption = parser - .accepts("describe", - "Used to describe the transaction state of a specific transactional id " + - "(requires --transactional-id)") - - val listOption = parser - .accepts("list", - "Used to list transactions") - - val describeProducersOption = parser - .accepts("describe-producers", - "Used to describe active transactional/idempotent producers " + - "writing to a specific topic partition (requires --topic and --partition)") - - options = parser.parse(args: _*) - - if (Seq(listOption, describeOption, describeProducersOption).count(options.has) != 1) { - CommandLineUtils.printUsageAndDie(parser, - "Command must include exactly one action: --describe-producers") - } -} diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index a95313659a0ce..3958f7613b11b 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -39,6 +39,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.quota.ClientQuotaFilter import org.apache.kafka.common.record._ +import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.{PatternType, ResourceType => AdminResourceType} import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder, SecurityProtocol} @@ -417,7 +418,7 @@ class RequestQuotaTest extends BaseRequestTest { ) case ApiKeys.WRITE_TXN_MARKERS => - new WriteTxnMarkersRequest.Builder(List.empty.asJava) + new WriteTxnMarkersRequest.Builder(List.empty[TxnMarkerEntry].asJava) case ApiKeys.TXN_OFFSET_COMMIT => new TxnOffsetCommitRequest.Builder( diff --git a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java index 030be39c713b0..3b74b6e045e5f 100644 --- a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java @@ -316,7 +316,8 @@ static int sizeOfUnsignedVarint(int value) { public static void main(String[] args) throws Exception { ArgumentParser parser = ArgumentParsers - .newArgumentParser("message-generator") + .newFor("message-generator") + .build() .defaultHelp(true) .description("The Kafka message generator"); parser.addArgument("--package", "-p") diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 1f2e0f28c2a7a..459395ceed212 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -57,7 +57,7 @@ versions += [ activation: "1.1.1", apacheda: "1.0.2", apacheds: "2.0.0-M24", - argparse4j: "0.7.0", + argparse4j: "0.8.1", bcpkix: "1.66", checkstyle: "8.20", commonsCli: "1.4", diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java index a5d6c7a835dbb..81ea75eb76c84 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -104,7 +104,8 @@ static class TestConfig { public static void main(String[] args) throws Exception { ArgumentParser parser = ArgumentParsers - .newArgumentParser("client-compatibility-test") + .newFor("client-compatibility-test") + .build() .defaultHelp(true) .description("This tool is used to verify client compatibility guarantees."); parser.addArgument("--topic") diff --git a/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java b/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java new file mode 100644 index 0000000000000..b52dfd454db61 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java @@ -0,0 +1,51 @@ +/* + * 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.tools; + +import net.sourceforge.argparse4j.inf.Argument; +import net.sourceforge.argparse4j.inf.ArgumentAction; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import org.apache.kafka.common.utils.AppInfoParser; + +import java.util.Map; + +class PrintVersionAndExitAction implements ArgumentAction { + + @Override + public void run( + ArgumentParser parser, + Argument arg, + Map attrs, + String flag, + Object value + ) { + String version = AppInfoParser.getVersion(); + String commitId = AppInfoParser.getCommitId(); + System.out.println(version + " (Commit:" + commitId + ")"); + System.exit(0); + } + + @Override + public void onAttach(Argument arg) { + + } + + @Override + public boolean consumeArgument() { + return false; + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index b76ce94ddb3b5..3eb02a7865daa 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -189,7 +189,8 @@ record = new ProducerRecord<>(topicName, payload); /** Get the command-line argument parser. */ private static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("producer-performance") + .newFor("producer-performance") + .build() .defaultHelp(true) .description("This tool is used to verify the producer performance."); diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java index cbdfa4df1a27f..3ff8d4d365101 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java @@ -64,7 +64,8 @@ public class TransactionalMessageCopier { /** Get the command-line argument parser. */ private static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("transactional-message-copier") + .newFor("transactional-message-copier") + .build() .defaultHelp(true) .description("This tool copies messages transactionally from an input partition to an output topic, " + "committing the consumed offsets along with the output messages"); diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java new file mode 100644 index 0000000000000..9c6d7f79daeea --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java @@ -0,0 +1,620 @@ +/* + * 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.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import net.sourceforge.argparse4j.inf.Subparser; +import net.sourceforge.argparse4j.inf.Subparsers; +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.DescribeProducersOptions; +import org.apache.kafka.clients.admin.DescribeProducersResult; +import org.apache.kafka.clients.admin.DescribeTransactionsResult; +import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static net.sourceforge.argparse4j.impl.Arguments.store; + +public abstract class TransactionsCommand { + private static final Logger log = LoggerFactory.getLogger(TransactionsCommand.class); + + protected final Time time; + + protected TransactionsCommand(Time time) { + this.time = time; + } + + /** + * Get the name of this command (e.g. `describe-producers`). + */ + abstract String name(); + + /** + * Specify the arguments needed for this command. + */ + abstract void addSubparser(Subparsers subparsers); + + /** + * Execute the command. + */ + abstract void execute(Admin admin, Namespace ns, PrintStream out) throws Exception; + + + static class AbortTransactionCommand extends TransactionsCommand { + + AbortTransactionCommand(Time time) { + super(time); + } + + @Override + String name() { + return "abort"; + } + + @Override + void addSubparser(Subparsers subparsers) { + Subparser subparser = subparsers.addParser(name()) + .help("abort a hanging transaction (requires administrative privileges)"); + + subparser.addArgument("--topic") + .help("topic name") + .action(store()) + .type(String.class) + .required(true); + + subparser.addArgument("--partition") + .help("partition number") + .action(store()) + .type(Integer.class) + .required(true); + + ArgumentGroup newBrokerArgumentGroup = subparser.addArgumentGroup("new brokers") + .description("For newer brokers, you must provide the start offset of the transaction " + + "to be aborted"); + + newBrokerArgumentGroup.addArgument("--start-offset") + .help("start offset of the transaction to abort") + .action(store()) + .type(Long.class); + + ArgumentGroup olderBrokerArgumentGroup = subparser.addArgumentGroup("older brokers") + .description("For older brokers, you must provide all of these arguments"); + + olderBrokerArgumentGroup.addArgument("--producer-id") + .help("producer id") + .action(store()) + .type(Long.class); + + olderBrokerArgumentGroup.addArgument("--producer-epoch") + .help("producer epoch") + .action(store()) + .type(Integer.class); + + olderBrokerArgumentGroup.addArgument("--coordinator-epoch") + .help("coordinator epoch") + .action(store()) + .type(Integer.class); + } + + private AbortTransactionSpec buildAbortSpec( + Admin admin, + TopicPartition topicPartition, + long startOffset + ) throws Exception { + final DescribeProducersResult.PartitionProducerState result; + try { + result = admin.describeProducers(singleton(topicPartition)) + .partitionResult(topicPartition) + .get(); + } catch (ExecutionException e) { + printErrorAndExit("Failed to validate producer state for partition " + + topicPartition, e.getCause()); + return null; + } + + Optional foundProducerState = result.activeProducers().stream() + .filter(producerState -> { + OptionalLong txnStartOffsetOpt = producerState.currentTransactionStartOffset(); + return txnStartOffsetOpt.isPresent() && txnStartOffsetOpt.getAsLong() == startOffset; + }) + .findFirst(); + + if (!foundProducerState.isPresent()) { + printErrorAndExit("Could not find any open transactions starting at offset " + + startOffset + " on partition " + topicPartition); + return null; + } + + DescribeProducersResult.ProducerState producerState = foundProducerState.get(); + return new AbortTransactionSpec( + topicPartition, + producerState.producerId(), + producerState.producerEpoch(), + producerState.coordinatorEpoch().orElse(0) + ); + } + + private void abortTransaction( + Admin admin, + AbortTransactionSpec abortSpec + ) throws Exception { + try { + admin.abortTransaction(abortSpec).all().get(); + } catch (ExecutionException e) { + TransactionsCommand.printErrorAndExit("Failed to abort transaction " + abortSpec, e.getCause()); + } + } + + @Override + void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { + String topicName = ns.getString("topic"); + Integer partitionId = ns.getInt("partition"); + TopicPartition topicPartition = new TopicPartition(topicName, partitionId); + + Long startOffset = ns.getLong("start_offset"); + Long producerId = ns.getLong("producer_id"); + + if (startOffset == null && producerId == null) { + printErrorAndExit("The transaction to abort must be identified either with " + + "--start-offset (for newer brokers) or with " + + "--producer-id, --producer-epoch, and --coordinator-epoch (for older brokers)"); + return; + } + + final AbortTransactionSpec abortSpec; + if (startOffset == null) { + Integer producerEpoch = ns.getInt("producer_epoch"); + if (producerEpoch == null) { + printErrorAndExit("Missing required argument --producer-epoch"); + return; + } + + Integer coordinatorEpoch = ns.getInt("coordinator_epoch"); + if (coordinatorEpoch == null) { + printErrorAndExit("Missing required argument --coordinator-epoch"); + return; + } + + // If a transaction was started by a new producerId and became hanging + // before the initial commit/abort, then the coordinator epoch will be -1 + // as seen in the `DescribeProducers` output. In this case, we conservatively + // use a coordinator epoch of 0, which is less than or equal to any possible + // leader epoch. + if (coordinatorEpoch < 0) { + coordinatorEpoch = 0; + } + + abortSpec = new AbortTransactionSpec( + topicPartition, + producerId, + producerEpoch, + coordinatorEpoch + ); + } else { + abortSpec = buildAbortSpec(admin, topicPartition, startOffset); + } + + abortTransaction(admin, abortSpec); + } + } + + static class DescribeProducersCommand extends TransactionsCommand { + + DescribeProducersCommand(Time time) { + super(time); + } + + @Override + public String name() { + return "describe-producers"; + } + + @Override + public void addSubparser(Subparsers subparsers) { + Subparser subparser = subparsers.addParser(name()) + .help("describe the states of active producers for a topic partition"); + + subparser.addArgument("--broker-id") + .help("optional broker id to describe the producer state on a specific replica") + .action(store()) + .type(Integer.class) + .required(false); + + subparser.addArgument("--topic") + .help("topic name") + .action(store()) + .type(String.class) + .required(true); + + subparser.addArgument("--partition") + .help("partition number") + .action(store()) + .type(Integer.class) + .required(true); + } + + @Override + public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { + DescribeProducersOptions options = new DescribeProducersOptions(); + Optional.ofNullable(ns.getInt("broker_id")).ifPresent(options::setBrokerId); + + String topicName = ns.getString("topic"); + Integer partitionId = ns.getInt("partition"); + TopicPartition topicPartition = new TopicPartition(topicName, partitionId); + + final DescribeProducersResult.PartitionProducerState result; + + try { + result = admin.describeProducers(singleton(topicPartition), options) + .partitionResult(topicPartition) + .get(); + } catch (ExecutionException e) { + String brokerClause = options.brokerId().isPresent() ? + "broker " + options.brokerId().getAsInt() : + "leader"; + printErrorAndExit("Failed to describe producers for partition " + + topicPartition + " on " + brokerClause, e.getCause()); + return; + } + + String[] headers = new String[]{ + "ProducerId", + "ProducerEpoch", + "CoordinatorEpoch", + "LastSequence", + "LastTimestamp", + "CurrentTransactionStartOffset" + }; + + List rows = result.activeProducers().stream().map(producerState -> { + String currentTransactionStartOffsetColumnValue = + producerState.currentTransactionStartOffset().isPresent() ? + String.valueOf(producerState.currentTransactionStartOffset().getAsLong()) : + "None"; + + return new String[] { + String.valueOf(producerState.producerId()), + String.valueOf(producerState.producerEpoch()), + String.valueOf(producerState.coordinatorEpoch().orElse(-1)), + String.valueOf(producerState.lastSequence()), + String.valueOf(producerState.lastTimestamp()), + currentTransactionStartOffsetColumnValue + }; + }).collect(Collectors.toList()); + + prettyPrintTable(headers, rows, out); + } + } + + static class DescribeTransactionsCommand extends TransactionsCommand { + + DescribeTransactionsCommand(Time time) { + super(time); + } + + @Override + public String name() { + return "describe"; + } + + @Override + public void addSubparser(Subparsers subparsers) { + Subparser subparser = subparsers.addParser(name()) + .description("Describe the state of an active transactional-id.") + .help("describe the state of an active transactional-id"); + + subparser.addArgument("--transactional-id") + .help("transactional id") + .action(store()) + .type(String.class) + .required(true); + } + + @Override + public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { + String transactionalId = ns.getString("transactional_id"); + + final DescribeTransactionsResult.TransactionState result; + try { + result = admin.describeTransactions(singleton(transactionalId)) + .transactionalIdResult(transactionalId) + .get(); + } catch (ExecutionException e) { + printErrorAndExit("Failed to describe transaction state of " + + "transactional-id `" + transactionalId + "`", e.getCause()); + return; + } + + // TODO: Do we want a way to return coordinator ID? + String[] headers = new String[]{ + "TransactionalId", + "ProducerId", + "ProducerEpoch", + "TransactionState", + "TransactionTimeoutMs", + "CurrentTransactionStartTimeMs", + "TransactionDurationMs", + "TopicPartitions" + }; + + final String transactionDurationMsColumnValue; + final String transactionStartTimeMsColumnValue; + + if (result.transactionStartTimeMs().isPresent()) { + long transactionStartTimeMs = result.transactionStartTimeMs().getAsLong(); + transactionStartTimeMsColumnValue = String.valueOf(transactionStartTimeMs); + transactionDurationMsColumnValue = String.valueOf(time.milliseconds() - transactionStartTimeMs); + } else { + transactionStartTimeMsColumnValue = "None"; + transactionDurationMsColumnValue = "None"; + } + + String[] row = new String[]{ + transactionalId, + String.valueOf(result.producerId()), + String.valueOf(result.producerEpoch()), + result.state(), + String.valueOf(result.transactionTimeoutMs()), + transactionStartTimeMsColumnValue, + transactionDurationMsColumnValue, + Utils.join(result.topicPartitions(), ",") + }; + + prettyPrintTable(headers, singletonList(row), out); + } + } + + static class ListTransactionsCommand extends TransactionsCommand { + + ListTransactionsCommand(Time time) { + super(time); + } + + @Override + public String name() { + return "list"; + } + + @Override + public void addSubparser(Subparsers subparsers) { + subparsers.addParser(name()) + .help("list transactions"); + } + + @Override + public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception{ + final Map> result; + + try { + result = admin.listTransactions() + .allByBrokerId() + .get(); + } catch (ExecutionException e) { + printErrorAndExit("Failed to list transactions", e.getCause()); + return; + } + + String[] headers = new String[] { + "TransactionalId", + "Coordinator", + "ProducerId", + "TransactionState" + }; + + List rows = new ArrayList<>(); + for (Map.Entry> brokerListingsEntry : result.entrySet()) { + String coordinatorIdString = brokerListingsEntry.getKey().toString(); + Collection listings = brokerListingsEntry.getValue(); + + for (TransactionListing listing : listings) { + rows.add(new String[] { + listing.transactionalId(), + coordinatorIdString, + String.valueOf(listing.producerId()), + listing.transactionState().toString() + }); + } + } + + prettyPrintTable(headers, rows, out); + } + } + + private static void appendColumnValue( + StringBuilder rowBuilder, + String value, + int length + ) { + int padLength = length - value.length(); + rowBuilder.append(value); + for (int i = 0; i < padLength; i++) + rowBuilder.append(' '); + } + + private static void printRow( + List columnLengths, + String[] row, + PrintStream out + ) { + StringBuilder rowBuilder = new StringBuilder(); + for (int i = 0; i < row.length; i++) { + Integer columnLength = columnLengths.get(i); + String columnValue = row[i]; + appendColumnValue(rowBuilder, columnValue, columnLength); + rowBuilder.append('\t'); + } + out.println(rowBuilder.toString()); + } + + private static void prettyPrintTable( + String[] headers, + List rows, + PrintStream out + ) { + List columnLengths = Arrays.stream(headers) + .map(String::length) + .collect(Collectors.toList()); + + for (String[] row : rows) { + for (int i = 0; i < headers.length; i++) { + columnLengths.set(i, Math.max(columnLengths.get(i), row[i].length())); + } + } + + printRow(columnLengths, headers, out); + rows.forEach(row -> printRow(columnLengths, row, out)); + } + + private static void printErrorAndExit(String message, Throwable t) { + log.debug(message, t); + + String exitMessage = message + ": " + t.getMessage() + "." + + " Enable debug logging for additional detail."; + + printErrorAndExit(exitMessage); + } + + private static void printErrorAndExit(String message) { + System.err.println(message); + Exit.exit(1, message); + } + + private static Admin buildAdminClient(Namespace ns) { + final Properties properties; + + String configFile = ns.getString("command_config"); + if (configFile == null) { + properties = new Properties(); + } else { + try { + properties = Utils.loadProps(configFile); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + String bootstrapServers = ns.getString("bootstrap_server"); + properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + return Admin.create(properties); + } + + static ArgumentParser buildBaseParser() { + ArgumentParser parser = ArgumentParsers.newFor("kafka-transactions.sh").build(); + + parser.description("This tool is used to analyze the transactional state of producers in the cluster. " + + "It can be used to detect and recover from hanging transactions."); + + parser.addArgument("-v", "--version") + .action(new PrintVersionAndExitAction()) + .help("show the version of this Kafka distribution and exit"); + + parser.addArgument("--command-config") + .help("property file containing configs to be passed to admin client") + .action(store()) + .type(String.class) + .metavar("FILE") + .required(false); + + parser.addArgument("--bootstrap-server") + .help("hostname and port for the broker to connect to, in the form `host:port` " + + "(multiple comma-separated entries can be given)") + .action(store()) + .type(String.class) + .metavar("host:port") + .required(true); + + return parser; + } + + static void execute( + String[] args, + Function adminSupplier, + PrintStream out, + Time time + ) throws Exception { + List commands = Arrays.asList( + new ListTransactionsCommand(time), + new DescribeTransactionsCommand(time), + new DescribeProducersCommand(time), + new AbortTransactionCommand(time) + ); + + ArgumentParser parser = buildBaseParser(); + Subparsers subparsers = parser.addSubparsers() + .dest("command") + .title("commands") + .metavar("COMMAND"); + commands.forEach(command -> command.addSubparser(subparsers)); + + final Namespace ns; + + try { + ns = parser.parseArgs(args); + } catch (ArgumentParserException e) { + parser.handleError(e); + Exit.exit(1); + return; + } + + Admin admin = adminSupplier.apply(ns); + String commandName = ns.getString("command"); + + Optional commandOpt = commands.stream() + .filter(cmd -> cmd.name().equals(commandName)) + .findFirst(); + + if (!commandOpt.isPresent()) { + throw new IllegalArgumentException("Unexpected command " + commandName); + } + + TransactionsCommand command = commandOpt.get(); + command.execute(admin, ns, out); + Exit.exit(0); + } + + public static void main(String[] args) throws Exception { + execute(args, TransactionsCommand::buildAdminClient, System.out, Time.SYSTEM); + } + +} diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java index 5be76f000622f..56734c8bf092d 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java @@ -501,7 +501,8 @@ public long maxOffset() { private static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("verifiable-consumer") + .newFor("verifiable-consumer") + .build() .defaultHelp(true) .description("This tool consumes messages from a specific topic and emits consumer events (e.g. group rebalances, received messages, and offsets committed) as JSON objects to STDOUT."); MutuallyExclusiveGroup connectionGroup = parser.addMutuallyExclusiveGroup("Connection Group") diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java index 534c21b682f90..0ddd068bf47de 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java @@ -55,7 +55,8 @@ public class VerifiableLog4jAppender { /** Get the command-line argument parser. */ private static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("verifiable-log4j-appender") + .newFor("verifiable-log4j-appender") + .build() .defaultHelp(true) .description("This tool produces increasing integers to the specified topic using KafkaLog4jAppender."); diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 3862255e223e4..07ef87d9d223b 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -109,7 +109,8 @@ public VerifiableProducer(KafkaProducer producer, String topic, /** Get the command-line argument parser. */ private static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("verifiable-producer") + .newFor("verifiable-producer") + .build() .defaultHelp(true) .description("This tool produces increasing integers to the specified topic and prints JSON metadata to stdout on each \"send\" request, making externally visible which messages have been acked and which have not."); diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java index 1674b2d92aaa5..aeec2f3adf8b4 100644 --- a/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java +++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java @@ -203,7 +203,8 @@ boolean exec(TaskSpec spec, PrintStream out) throws Exception { public static void main(String[] args) throws Exception { ArgumentParser parser = ArgumentParsers - .newArgumentParser("trogdor-agent") + .newFor("trogdor-agent") + .build() .defaultHelp(true) .description("The Trogdor fault injection agent"); parser.addArgument("--agent.config", "-c") diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java index 56f53f3ba192b..57145b75f5ea1 100644 --- a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java +++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java @@ -199,7 +199,8 @@ private static void addWorkerIdArgument(ArgumentParser parser, String help) { public static void main(String[] args) throws Exception { ArgumentParser rootParser = ArgumentParsers - .newArgumentParser("trogdor-agent-client") + .newFor("trogdor-agent-client") + .build() .defaultHelp(true) .description("The Trogdor agent client."); Subparsers subParsers = rootParser.addSubparsers(). diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java index 47f80e5c7a15e..9414e86432937 100644 --- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java +++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java @@ -131,7 +131,8 @@ public void waitForShutdown() throws Exception { public static void main(String[] args) throws Exception { ArgumentParser parser = ArgumentParsers - .newArgumentParser("trogdor-coordinator") + .newFor("trogdor-coordinator") + .build() .defaultHelp(true) .description("The Trogdor fault injection coordinator"); parser.addArgument("--coordinator.config", "-c") diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java index ba40e7bc0d6bb..0f5af7d165e0f 100644 --- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java +++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java @@ -222,7 +222,8 @@ private static void addJsonArgument(ArgumentParser parser) { public static void main(String[] args) throws Exception { ArgumentParser rootParser = ArgumentParsers - .newArgumentParser("trogdor-coordinator-client") + .newFor("trogdor-coordinator-client") + .build() .description("The Trogdor coordinator client."); Subparsers subParsers = rootParser.addSubparsers(). dest("command"); diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java new file mode 100644 index 0000000000000..fc8b9eb654b4c --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java @@ -0,0 +1,520 @@ +/* + * 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.tools; + +import org.apache.kafka.clients.admin.AbortTransactionResult; +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DescribeProducersOptions; +import org.apache.kafka.clients.admin.DescribeProducersResult; +import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; +import org.apache.kafka.clients.admin.DescribeProducersResult.ProducerState; +import org.apache.kafka.clients.admin.DescribeTransactionsResult; +import org.apache.kafka.clients.admin.ListTransactionsResult; +import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionState; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.Set; + +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TransactionsCommandTest { + + private final MockExitProcedure exitProcedure = new MockExitProcedure(); + private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + private final PrintStream out = new PrintStream(outputStream); + private final MockTime time = new MockTime(); + Admin admin = Mockito.mock(Admin.class); + + @Before + public void setupExitProcedure() { + Exit.setExitProcedure(exitProcedure); + } + + @After + public void resetExitProcedure() { + Exit.resetExitProcedure(); + } + + @Test + public void testDescribeProducersTopicRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "describe-producers", + "--partition", + "0" + }); + } + + @Test + public void testDescribeProducersPartitionRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "describe-producers", + "--topic", + "foo" + }); + } + + @Test + public void testDescribeProducersLeader() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 5); + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "describe-producers", + "--topic", + topicPartition.topic(), + "--partition", + String.valueOf(topicPartition.partition()) + }; + + testDescribeProducers(topicPartition, args, new DescribeProducersOptions()); + } + + @Test + public void testDescribeProducersSpecificReplica() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 5); + int brokerId = 5; + + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "describe-producers", + "--topic", + topicPartition.topic(), + "--partition", + String.valueOf(topicPartition.partition()), + "--broker-id", + String.valueOf(brokerId) + }; + + testDescribeProducers(topicPartition, args, new DescribeProducersOptions().setBrokerId(brokerId)); + } + + private void testDescribeProducers( + TopicPartition topicPartition, + String[] args, + DescribeProducersOptions expectedOptions + ) throws Exception { + DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class); + KafkaFuture describeFuture = KafkaFutureImpl.completedFuture( + new PartitionProducerState(asList( + new ProducerState(12345L, 15, 1300, 1599509565L, + OptionalInt.of(20), OptionalLong.of(990)), + new ProducerState(98765L, 30, 2300, 1599509599L, + OptionalInt.empty(), OptionalLong.empty()) + ))); + + + Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture); + Mockito.when(admin.describeProducers(singleton(topicPartition), expectedOptions)).thenReturn(describeResult); + + execute(args); + assertNormalExit(); + + List> table = readOutputAsTable(); + assertEquals(3, table.size()); + + List expectedHeaders = asList("ProducerId", "ProducerEpoch", "CoordinatorEpoch", + "LastSequence", "LastTimestamp", "CurrentTransactionStartOffset"); + assertEquals(expectedHeaders, table.get(0)); + + Set> expectedRows = Utils.mkSet( + asList("12345", "15", "20", "1300", "1599509565", "990"), + asList("98765", "30", "-1", "2300", "1599509599", "None") + ); + assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size()))); + } + + @Test + public void testListTransactions() throws Exception { + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "list" + }; + + ListTransactionsResult listResult = Mockito.mock(ListTransactionsResult.class); + + Map> transactions = new HashMap<>(); + transactions.put(0, asList( + new TransactionListing("foo", 12345L, TransactionState.ONGOING), + new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT) + )); + transactions.put(1, singletonList( + new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT) + )); + + KafkaFuture>> listTransactionsFuture = + KafkaFutureImpl.completedFuture(transactions); + + Mockito.when(admin.listTransactions()).thenReturn(listResult); + Mockito.when(listResult.allByBrokerId()).thenReturn(listTransactionsFuture); + + execute(args); + assertNormalExit(); + + List> table = readOutputAsTable(); + assertEquals(4, table.size()); + + // Assert expected headers + List expectedHeaders = asList("TransactionalId", "Coordinator", "ProducerId", "TransactionState"); + assertEquals(expectedHeaders, table.get(0)); + + Set> expectedRows = Utils.mkSet( + asList("foo", "0", "12345", "Ongoing"), + asList("bar", "0", "98765", "PrepareAbort"), + asList("baz", "1", "13579", "CompleteCommit") + ); + assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size()))); + } + + @Test + public void testDescribeTransactionsTransactionalIdRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "describe" + }); + } + + @Test + public void testDescribeTransaction() throws Exception { + String transactionalId = "foo"; + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "describe", + "--transactional-id", + transactionalId + }; + + DescribeTransactionsResult describeResult = Mockito.mock(DescribeTransactionsResult.class); + + long transactionStartTime = time.milliseconds(); + KafkaFuture describeFuture = KafkaFutureImpl.completedFuture( + new DescribeTransactionsResult.TransactionState( + "Ongoing", + 12345L, + 15, + 10000, + OptionalLong.of(transactionStartTime), + singleton(new TopicPartition("bar", 0)) + )); + + Mockito.when(describeResult.transactionalIdResult(transactionalId)).thenReturn(describeFuture); + Mockito.when(admin.describeTransactions(singleton(transactionalId))).thenReturn(describeResult); + + time.sleep(5000); + + execute(args); + assertNormalExit(); + + List> table = readOutputAsTable(); + assertEquals(2, table.size()); + + List expectedHeaders = asList( + "TransactionalId", + "ProducerId", + "ProducerEpoch", + "TransactionState", + "TransactionTimeoutMs", + "CurrentTransactionStartTimeMs", + "TransactionDurationMs", + "TopicPartitions" + ); + assertEquals(expectedHeaders, table.get(0)); + + List expectedRow = asList( + transactionalId, + "12345", + "15", + "Ongoing", + "10000", + String.valueOf(transactionStartTime), + "5000", + "bar-0" + ); + assertEquals(expectedRow, table.get(1)); + } + + @Test + public void testDescribeTransactionsStartOffsetOrProducerIdRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + "foo", + "--partition", + "0" + }); + } + + @Test + public void testDescribeTransactionsTopicRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "abort", + "--partition", + "0", + "--start-offset", + "9990" + }); + } + + @Test + public void testDescribeTransactionsPartitionRequired() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + "foo", + "--start-offset", + "9990" + }); + } + + @Test + public void testDescribeTransactionsProducerEpochRequiredWithProducerId() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + "foo", + "--partition", + "0", + "--producer-id", + "12345" + }); + } + + @Test + public void testDescribeTransactionsCoordinatorEpochRequiredWithProducerId() throws Exception { + assertCommandFailure(new String[]{ + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + "foo", + "--partition", + "0", + "--producer-id", + "12345", + "--producer-epoch", + "15" + }); + } + + @Test + public void testNewBrokerAbortTransaction() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 5); + long startOffset = 9173; + long producerId = 12345L; + int producerEpoch = 15; + int coordinatorEpoch = 76; + + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + topicPartition.topic(), + "--partition", + String.valueOf(topicPartition.partition()), + "--start-offset", + String.valueOf(startOffset) + }; + + DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class); + KafkaFuture describeFuture = KafkaFutureImpl.completedFuture( + new PartitionProducerState(singletonList( + new ProducerState(producerId, producerEpoch, 1300, 1599509565L, + OptionalInt.of(coordinatorEpoch), OptionalLong.of(startOffset)) + ))); + + AbortTransactionResult abortTransactionResult = Mockito.mock(AbortTransactionResult.class); + KafkaFuture abortFuture = KafkaFutureImpl.completedFuture(null); + AbortTransactionSpec expectedAbortSpec = new AbortTransactionSpec( + topicPartition, producerId, producerEpoch, coordinatorEpoch); + + Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture); + Mockito.when(admin.describeProducers(singleton(topicPartition))).thenReturn(describeResult); + + Mockito.when(abortTransactionResult.all()).thenReturn(abortFuture); + Mockito.when(admin.abortTransaction(expectedAbortSpec)).thenReturn(abortTransactionResult); + + execute(args); + assertNormalExit(); + } + + @Test + public void testOldBrokerAbortTransaction() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 5); + long producerId = 12345L; + int producerEpoch = 15; + int coordinatorEpoch = 29; + + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + topicPartition.topic(), + "--partition", + String.valueOf(topicPartition.partition()), + "--producer-id", + String.valueOf(producerId), + "--producer-epoch", + String.valueOf(producerEpoch), + "--coordinator-epoch", + String.valueOf(coordinatorEpoch) + }; + + AbortTransactionResult abortTransactionResult = Mockito.mock(AbortTransactionResult.class); + KafkaFuture abortFuture = KafkaFutureImpl.completedFuture(null); + AbortTransactionSpec expectedAbortSpec = new AbortTransactionSpec( + topicPartition, producerId, producerEpoch, coordinatorEpoch); + + Mockito.when(abortTransactionResult.all()).thenReturn(abortFuture); + Mockito.when(admin.abortTransaction(expectedAbortSpec)).thenReturn(abortTransactionResult); + + execute(args); + assertNormalExit(); + } + + @Test + public void testOldBrokerAbortTransactionWithUnknownCoordinatorEpoch() throws Exception { + TopicPartition topicPartition = new TopicPartition("foo", 5); + long producerId = 12345L; + int producerEpoch = 15; + + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "abort", + "--topic", + topicPartition.topic(), + "--partition", + String.valueOf(topicPartition.partition()), + "--producer-id", + String.valueOf(producerId), + "--producer-epoch", + String.valueOf(producerEpoch), + "--coordinator-epoch", + "-1" + }; + + AbortTransactionResult abortTransactionResult = Mockito.mock(AbortTransactionResult.class); + KafkaFuture abortFuture = KafkaFutureImpl.completedFuture(null); + AbortTransactionSpec expectedAbortSpec = new AbortTransactionSpec( + topicPartition, producerId, producerEpoch, 0); + + Mockito.when(abortTransactionResult.all()).thenReturn(abortFuture); + Mockito.when(admin.abortTransaction(expectedAbortSpec)).thenReturn(abortTransactionResult); + + execute(args); + assertNormalExit(); + } + + private void execute(String[] args) throws Exception { + TransactionsCommand.execute(args, ns -> admin, out, time); + } + + private List> readOutputAsTable() throws IOException { + List> table = new ArrayList<>(); + ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + + while (true) { + List row = readRow(reader); + if (row == null) { + break; + } + table.add(row); + } + return table; + } + + private List readRow(BufferedReader reader) throws IOException { + String line = reader.readLine(); + if (line == null) { + return null; + } else { + return asList(line.split("\\s+")); + } + } + + private void assertNormalExit() { + assertTrue(exitProcedure.hasExited); + assertEquals(0, exitProcedure.statusCode); + } + + private void assertCommandFailure(String[] args) throws Exception { + execute(args); + assertTrue(exitProcedure.hasExited); + assertEquals(1, exitProcedure.statusCode); + } + + private static class MockExitProcedure implements Exit.Procedure { + private boolean hasExited = false; + private int statusCode; + + @Override + public void execute(int statusCode, String message) { + if (!this.hasExited) { + this.hasExited = true; + this.statusCode = statusCode; + } + } + } + +} \ No newline at end of file From 36600851247706bead3f61ad5b6c9eb6e1deedae Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 7 Sep 2020 16:13:11 -0700 Subject: [PATCH 05/17] Move classes to top-level and add missing fields --- .../admin/DescribeTransactionsResult.java | 66 ++--------- .../clients/admin/ListTransactionsResult.java | 44 ------- .../clients/admin/TransactionDescription.java | 111 ++++++++++++++++++ .../clients/admin/TransactionListing.java | 62 ++++++++++ .../kafka/clients/admin/TransactionState.java | 2 +- .../AbortTransactionRequestDriver.java | 5 +- .../internals/AllBrokerRequestDriver.java | 19 +-- .../DescribeProducersRequestDriver.java | 4 +- .../DescribeTransactionsRequestDriver.java | 14 ++- .../ListTransactionsRequestDriver.java | 2 +- .../admin/internals/RequestDriver.java | 17 +-- .../admin/ListTransactionsResultTest.java | 1 - .../CoordinatorRequestDriverTest.java | 4 +- ...DescribeTransactionsRequestDriverTest.java | 93 +++++++-------- .../ListTransactionsRequestDriverTest.java | 5 +- .../internals/MetadataRequestDriverTest.java | 4 +- .../kafka/tools/TransactionsCommand.java | 15 +-- .../kafka/tools/TransactionsCommandTest.java | 20 ++-- 18 files changed, 279 insertions(+), 209 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java index 2d51f11c19a7a..28cefe90e095f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java @@ -18,28 +18,25 @@ import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.requests.FindCoordinatorRequest; import java.util.HashMap; import java.util.Map; -import java.util.OptionalLong; -import java.util.Set; import java.util.concurrent.ExecutionException; @InterfaceStability.Evolving public class DescribeTransactionsResult { - private final Map> futures; + private final Map> futures; - DescribeTransactionsResult(Map> futures) { + DescribeTransactionsResult(Map> futures) { this.futures = futures; } - public KafkaFuture transactionalIdResult(String transactionalId) { + public KafkaFuture transactionalIdResult(String transactionalId) { CoordinatorKey key = buildKey(transactionalId); - KafkaFuture future = futures.get(key); + KafkaFuture future = futures.get(key); if (future == null) { throw new IllegalArgumentException("TransactionalId " + "`" + transactionalId + "` was not included in the request"); @@ -51,11 +48,11 @@ private CoordinatorKey buildKey(String transactionalId) { return new CoordinatorKey(transactionalId, FindCoordinatorRequest.CoordinatorType.TRANSACTION); } - public KafkaFuture> all() { + public KafkaFuture> all() { return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])) .thenApply(nil -> { - Map results = new HashMap<>(futures.size()); - for (Map.Entry> entry : futures.entrySet()) { + Map results = new HashMap<>(futures.size()); + for (Map.Entry> entry : futures.entrySet()) { try { results.put(entry.getKey().idValue, entry.getValue().get()); } catch (InterruptedException | ExecutionException e) { @@ -67,53 +64,4 @@ public KafkaFuture> all() { }); } - public static class TransactionState { - private final String state; - private final long producerId; - private final int producerEpoch; - private final long transactionTimeoutMs; - private final OptionalLong transactionStartTimeMs; - private final Set topicPartitions; - - // TODO: We need to use `TransactionState` - public TransactionState( - String state, - long producerId, - int producerEpoch, - long transactionTimeoutMs, - OptionalLong transactionStartTimeMs, - Set topicPartitions - ) { - this.state = state; - this.producerId = producerId; - this.producerEpoch = producerEpoch; - this.transactionTimeoutMs = transactionTimeoutMs; - this.transactionStartTimeMs = transactionStartTimeMs; - this.topicPartitions = topicPartitions; - } - - public String state() { - return state; - } - - public long producerId() { - return producerId; - } - - public int producerEpoch() { - return producerEpoch; - } - - public long transactionTimeoutMs() { - return transactionTimeoutMs; - } - - public OptionalLong transactionStartTimeMs() { - return transactionStartTimeMs; - } - - public Set topicPartitions() { - return topicPartitions; - } - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java index 39d4aa590dd79..53144df97cf62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsResult.java @@ -27,7 +27,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; /** @@ -115,47 +114,4 @@ public KafkaFuture> byBrokerId(Integer brokerId) return resultFuture; } - public static class TransactionListing { - private final String transactionalId; - private final long producerId; - private final TransactionState transactionState; - - public TransactionListing( - String transactionalId, - long producerId, - TransactionState transactionState - ) { - this.transactionalId = transactionalId; - this.producerId = producerId; - this.transactionState = transactionState; - } - - public String transactionalId() { - return transactionalId; - } - - public long producerId() { - return producerId; - } - - public TransactionState transactionState() { - return transactionState; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - TransactionListing that = (TransactionListing) o; - return producerId == that.producerId && - Objects.equals(transactionalId, that.transactionalId) && - transactionState == that.transactionState; - } - - @Override - public int hashCode() { - return Objects.hash(transactionalId, producerId, transactionState); - } - } - } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java new file mode 100644 index 0000000000000..9139f5d344422 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Objects; +import java.util.OptionalLong; +import java.util.Set; + +public class TransactionDescription { + private final int coordinatorId; + private final TransactionState state; + private final long producerId; + private final int producerEpoch; + private final long transactionTimeoutMs; + private final OptionalLong transactionStartTimeMs; + private final Set topicPartitions; + + public TransactionDescription( + int coordinatorId, + TransactionState state, + long producerId, + int producerEpoch, + long transactionTimeoutMs, + OptionalLong transactionStartTimeMs, + Set topicPartitions + ) { + this.coordinatorId = coordinatorId; + this.state = state; + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.transactionTimeoutMs = transactionTimeoutMs; + this.transactionStartTimeMs = transactionStartTimeMs; + this.topicPartitions = topicPartitions; + } + + public int coordinatorId() { + return coordinatorId; + } + + public TransactionState state() { + return state; + } + + public long producerId() { + return producerId; + } + + public int producerEpoch() { + return producerEpoch; + } + + public long transactionTimeoutMs() { + return transactionTimeoutMs; + } + + public OptionalLong transactionStartTimeMs() { + return transactionStartTimeMs; + } + + public Set topicPartitions() { + return topicPartitions; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TransactionDescription that = (TransactionDescription) o; + return coordinatorId == that.coordinatorId && + producerId == that.producerId && + producerEpoch == that.producerEpoch && + transactionTimeoutMs == that.transactionTimeoutMs && + state == that.state && + Objects.equals(transactionStartTimeMs, that.transactionStartTimeMs) && + Objects.equals(topicPartitions, that.topicPartitions); + } + + @Override + public int hashCode() { + return Objects.hash(coordinatorId, state, producerId, producerEpoch, transactionTimeoutMs, transactionStartTimeMs, topicPartitions); + } + + @Override + public String toString() { + return "TransactionDescription(" + + "coordinatorId=" + coordinatorId + + ", state=" + state + + ", producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", transactionTimeoutMs=" + transactionTimeoutMs + + ", transactionStartTimeMs=" + transactionStartTimeMs + + ", topicPartitions=" + topicPartitions + + ')'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java new file mode 100644 index 0000000000000..160a3e41f4ef1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import java.util.Objects; + +public class TransactionListing { + private final String transactionalId; + private final long producerId; + private final TransactionState transactionState; + + public TransactionListing( + String transactionalId, + long producerId, + TransactionState transactionState + ) { + this.transactionalId = transactionalId; + this.producerId = producerId; + this.transactionState = transactionState; + } + + public String transactionalId() { + return transactionalId; + } + + public long producerId() { + return producerId; + } + + public TransactionState transactionState() { + return transactionState; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TransactionListing that = (TransactionListing) o; + return producerId == that.producerId && + Objects.equals(transactionalId, that.transactionalId) && + transactionState == that.transactionState; + } + + @Override + public int hashCode() { + return Objects.hash(transactionalId, producerId, transactionState); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java index 8923916fbb6b8..40f94829e1d0d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java @@ -26,7 +26,7 @@ public enum TransactionState { COMPLETE_COMMIT("CompleteCommit"), EMPTY("Empty"), PREPARE_EPOCH_FENCE("PrepareEpochFence"), - UNKNOWN("Unknown"); // FIXME: I hate this + UNKNOWN("Unknown"); private final static HashMap NAME_TO_ENUM; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java index 073edcad38986..53bdc4fe8c327 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ClusterAuthorizationException; -import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.TransactionCoordinatorFencedException; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.Errors; @@ -55,7 +54,7 @@ public AbortTransactionRequestDriver( } @Override - WriteTxnMarkersRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + WriteTxnMarkersRequest.Builder buildFulfillmentRequest(Integer brokerId, Set topicPartitions) { if (!topicPartitions.equals(singleton(topicPartition))) { throw new IllegalArgumentException("Received unexpected topic partitions " + topicPartitions + " (expected " + singleton(topicPartition) + ")"); @@ -71,7 +70,7 @@ WriteTxnMarkersRequest.Builder buildFulfillmentRequest(Set topic } @Override - void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse abstractResponse) { WriteTxnMarkersResponse response = (WriteTxnMarkersResponse) abstractResponse; List markerResponses = response.data.markers(); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java index 29bc0779741a9..60caec8322711 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java @@ -109,16 +109,14 @@ private Map> collectBrokerFutures() { abstract AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId); @Override - AbstractRequest.Builder buildFulfillmentRequest(Set keys) { - Integer brokerId = requireSingleBrokerKey(keys); + AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set keys) { return buildFulfillmentRequest(brokerId); } abstract void handleFulfillmentResponse(Integer brokerId, AbstractResponse response); @Override - void handleFulfillmentResponse(Set keys, AbstractResponse response) { - Integer brokerId = requireSingleBrokerKey(keys); + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse response) { handleFulfillmentResponse(brokerId, response); } @@ -130,19 +128,6 @@ void complete(Integer brokerId, V value) { super.complete(new BrokerKey(OptionalInt.of(brokerId)), value); } - private Integer requireSingleBrokerKey(Set keys) { - if (keys.size() != 1) { - throw new IllegalArgumentException("Unexpected fulfillment mapping " + keys); - } - - BrokerKey key = keys.iterator().next(); - if (!key.brokerId.isPresent()) { - throw new IllegalArgumentException("Unexpected fulfillment key " + key); - } - - return key.brokerId.getAsInt(); - } - public static class BrokerKey { private final OptionalInt brokerId; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java index 64de23dd55e28..6f7f139cd2c62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -65,7 +65,7 @@ public DescribeProducersRequestDriver( } @Override - DescribeProducersRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + DescribeProducersRequest.Builder buildFulfillmentRequest(Integer brokerId, Set topicPartitions) { DescribeProducersRequestData request = new DescribeProducersRequestData(); DescribeProducersRequest.Builder builder = new DescribeProducersRequest.Builder(request); @@ -115,7 +115,7 @@ private void handlePartitionError( } @Override - void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse abstractResponse) { DescribeProducersResponse response = (DescribeProducersResponse) abstractResponse; for (DescribeProducersResponseData.TopicResponse topicResponse : response.data().topics()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java index 0c7d2380e0154..6859c64c523ba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java @@ -16,7 +16,8 @@ */ package org.apache.kafka.clients.admin.internals; -import org.apache.kafka.clients.admin.DescribeTransactionsResult.TransactionState; +import org.apache.kafka.clients.admin.TransactionDescription; +import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.errors.TransactionalIdNotFoundException; @@ -38,7 +39,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class DescribeTransactionsRequestDriver extends CoordinatorRequestDriver { +public class DescribeTransactionsRequestDriver extends CoordinatorRequestDriver { private static final Logger log = LoggerFactory.getLogger(DescribeTransactionsRequestDriver.class); public DescribeTransactionsRequestDriver( @@ -52,7 +53,7 @@ public DescribeTransactionsRequestDriver( } @Override - AbstractRequest.Builder buildFulfillmentRequest(Set keys) { + AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set keys) { DescribeTransactionsRequestData request = new DescribeTransactionsRequestData(); List transactionalIds = keys.stream().map(key -> key.idValue).collect(Collectors.toList()); request.setTransactionalIds(transactionalIds); @@ -60,7 +61,7 @@ AbstractRequest.Builder buildFulfillmentRequest(Set keys) { } @Override - void handleFulfillmentResponse(Set keys, AbstractResponse abstractResponse) { + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse abstractResponse) { DescribeTransactionsResponse response = (DescribeTransactionsResponse) abstractResponse; for (DescribeTransactionsResponseData.TransactionState transactionState : response.data().transactionStates()) { CoordinatorKey transactionalIdKey = asCoordinatorKey(transactionState.transactionalId()); @@ -75,8 +76,9 @@ void handleFulfillmentResponse(Set keys, AbstractResponse abstra OptionalLong.empty() : OptionalLong.of(transactionState.transactionStartTimeMs()); - complete(transactionalIdKey, new TransactionState( - transactionState.transactionState(), + complete(transactionalIdKey, new TransactionDescription( + brokerId, + TransactionState.parse(transactionState.transactionState()), transactionState.producerId(), transactionState.producerEpoch(), transactionState.transactionTimeoutMs(), diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java index 43b0a5d99b9df..5fe05ba807d68 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.ListTransactionsOptions; -import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.common.errors.CoordinatorNotAvailableException; import org.apache.kafka.common.message.ListTransactionsRequestData; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java index c5ae1736c48f3..56381c1ebfa71 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -34,7 +34,7 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.Set; -import java.util.function.Function; +import java.util.function.BiFunction; /** * The `KafkaAdminClient`'s internal `Call` primitive is not a good fit for multi-stage @@ -201,7 +201,8 @@ public void onResponse( ) { clearInflightRequest(currentTimeMs, spec); if (spec.scope instanceof RequestDriver.BrokerScope) { - handleFulfillmentResponse(spec.keys, response); + int brokerId = ((BrokerScope) spec.scope).destinationBrokerId; + handleFulfillmentResponse(brokerId, spec.keys, response); } else { handleLookupResponse(spec.keys, response); } @@ -246,14 +247,14 @@ public void onFailure( * Build the fulfillment request. The set of keys are derived during the Lookup stage * as the set of keys which all map to the same destination broker. */ - abstract AbstractRequest.Builder buildFulfillmentRequest(Set keys); + abstract AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set keys); /** * Callback that is invoked when a Fulfillment request returns successfully. * The handler should parse the response, check for errors, update mappings as * needed, and complete any futures which can be completed. */ - abstract void handleFulfillmentResponse(Set keys, AbstractResponse response); + abstract void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse response); private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { RequestState requestState = requestStates.get(spec.scope); @@ -265,7 +266,7 @@ private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { private void collectRequests( List requests, BiMultimap multimap, - Function, AbstractRequest.Builder> buildRequest + BiFunction, T, AbstractRequest.Builder> buildRequest ) { for (Map.Entry> entry : multimap.entrySet()) { T scope = entry.getKey(); @@ -280,7 +281,7 @@ private void collectRequests( continue; } - AbstractRequest.Builder request = buildRequest.apply(keys); + AbstractRequest.Builder request = buildRequest.apply(keys, scope); RequestSpec spec = new RequestSpec( scope, new HashSet<>(keys), // copy to avoid exposing mutable state @@ -299,7 +300,7 @@ private void collectLookupRequests(List requests) { collectRequests( requests, lookupMap, - this::buildLookupRequest + (keys, scope) -> buildLookupRequest(keys) ); } @@ -307,7 +308,7 @@ private void collectFulfillmentRequests(List requests) { collectRequests( requests, fulfillmentMap, - this::buildFulfillmentRequest + (keys, scope) -> buildFulfillmentRequest(scope.destinationBrokerId, keys) ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java index 587111a4c839c..83372db29dc1f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/ListTransactionsResultTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.admin; -import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.utils.Utils; diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java index 4e6e88bf8048e..5e4054336c492 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java @@ -201,7 +201,7 @@ public TestCoordinatorRequestDriver(Set groupIds) { } @Override - AbstractRequest.Builder buildFulfillmentRequest(Set coordinatorKeys) { + AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set coordinatorKeys) { return new DescribeGroupsRequest.Builder(new DescribeGroupsRequestData() .setGroups(coordinatorKeys.stream() .map(coordinatorKey -> coordinatorKey.idValue) @@ -209,7 +209,7 @@ AbstractRequest.Builder buildFulfillmentRequest(Set coordinat } @Override - void handleFulfillmentResponse(Set keys, AbstractResponse response) { + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse response) { throw new UnsupportedOperationException(); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java index 1e738879b0265..dd200afbf479e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java @@ -16,8 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; -import org.apache.kafka.clients.admin.DescribeTransactionsResult; -import org.apache.kafka.clients.admin.DescribeTransactionsResult.TransactionState; +import org.apache.kafka.clients.admin.TransactionDescription; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -66,15 +65,15 @@ public void testDescribeTransactions() throws Exception { ); // Send `FindCoordinator` requests - List.RequestSpec> lookupRequests = driver.poll(); + List.RequestSpec> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec1 = + RequestDriver.RequestSpec lookupSpec1 = findRequestWithKey(transactionalId1, lookupRequests); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId1, lookupSpec1); - RequestDriver.RequestSpec loookupSpec2 = + RequestDriver.RequestSpec loookupSpec2 = findRequestWithKey(transactionalId2, lookupRequests); assertRetryBackoff(0, loookupSpec2); assertFindCoordinatorRequest(transactionalId2, loookupSpec2); @@ -89,15 +88,15 @@ public void testDescribeTransactions() throws Exception { findCoordinatorResponse(OptionalInt.of(coordinator2))); // Send `DescribeTransactions` requests - List.RequestSpec> requests = driver.poll(); + List.RequestSpec> requests = driver.poll(); assertEquals(2, requests.size()); - RequestDriver.RequestSpec requestSpec1 = + RequestDriver.RequestSpec requestSpec1 = findRequestWithKey(transactionalId1, requests); assertRetryBackoff(0, requestSpec1); assertDescribeTransactionsRequest(singleton(transactionalId1), coordinator1, requestSpec1); - RequestDriver.RequestSpec requestSpec2 = + RequestDriver.RequestSpec requestSpec2 = findRequestWithKey(transactionalId2, requests); assertRetryBackoff(0, requestSpec2); assertDescribeTransactionsRequest(singleton(transactionalId2), coordinator2, requestSpec2); @@ -116,15 +115,15 @@ public void testDescribeTransactions() throws Exception { // We are all done, so there should be no requests left to be sent assertEquals(Collections.emptyList(), driver.poll()); - KafkaFutureImpl future1 = driver.futures() + KafkaFutureImpl future1 = driver.futures() .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); - KafkaFutureImpl future2 = driver.futures() + KafkaFutureImpl future2 = driver.futures() .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); assertTrue(future1.isDone()); - assertMatchingTransactionState(transactionState1, future1.get()); + assertMatchingTransactionState(coordinator1, transactionState1, future1.get()); assertTrue(future2.isDone()); - assertMatchingTransactionState(transactionState2, future2.get()); + assertMatchingTransactionState(coordinator2, transactionState2, future2.get()); } @Test @@ -140,15 +139,15 @@ public void testDescribeTransactionsBatching() throws Exception { ); // Send `FindCoordinator` requests - List.RequestSpec> lookupRequests = driver.poll(); + List.RequestSpec> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec1 = + RequestDriver.RequestSpec lookupSpec1 = findRequestWithKey(transactionalId1, lookupRequests); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId1, lookupSpec1); - RequestDriver.RequestSpec loookupSpec2 = + RequestDriver.RequestSpec loookupSpec2 = findRequestWithKey(transactionalId2, lookupRequests); assertRetryBackoff(0, loookupSpec2); assertFindCoordinatorRequest(transactionalId2, loookupSpec2); @@ -164,10 +163,10 @@ public void testDescribeTransactionsBatching() throws Exception { findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List.RequestSpec> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestDriver.RequestSpec requestSpec = requests.get(0); assertRetryBackoff(0, requestSpec); assertDescribeTransactionsRequest(transactionalIds, coordinator, requestSpec); @@ -184,15 +183,15 @@ public void testDescribeTransactionsBatching() throws Exception { // We are all done, so there should be no requests left to be sent assertEquals(Collections.emptyList(), driver.poll()); - KafkaFutureImpl future1 = driver.futures() + KafkaFutureImpl future1 = driver.futures() .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); - KafkaFutureImpl future2 = driver.futures() + KafkaFutureImpl future2 = driver.futures() .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); assertTrue(future1.isDone()); - assertMatchingTransactionState(transactionState1, future1.get()); + assertMatchingTransactionState(coordinator, transactionState1, future1.get()); assertTrue(future2.isDone()); - assertMatchingTransactionState(transactionState2, future2.get()); + assertMatchingTransactionState(coordinator, transactionState2, future2.get()); } @Test @@ -206,9 +205,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests = driver.poll(); + List.RequestSpec> lookupRequests = driver.poll(); assertEquals(1, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); assertRetryBackoff(0, lookupSpec); assertFindCoordinatorRequest(transactionalId, lookupSpec); @@ -217,9 +216,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests1 = driver.poll(); + List.RequestSpec> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec requestSpec1 = requests1.get(0); + RequestDriver.RequestSpec requestSpec1 = requests1.get(0); assertRetryBackoff(0, requestSpec1); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec1); @@ -233,9 +232,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() // Send retry `DescribeTransactions` request assertFalse(futureFor(driver, transactionalId).isDone()); - List.RequestSpec> requests2 = driver.poll(); + List.RequestSpec> requests2 = driver.poll(); assertEquals(1, requests2.size()); - RequestDriver.RequestSpec requestSpec2 = requests2.get(0); + RequestDriver.RequestSpec requestSpec2 = requests2.get(0); assertRetryBackoff(1, requestSpec2); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec2); } @@ -251,9 +250,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests1 = driver.poll(); + List.RequestSpec> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId, lookupSpec1); @@ -262,9 +261,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List.RequestSpec> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestDriver.RequestSpec requestSpec = requests.get(0); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( @@ -276,9 +275,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { // Send second `FindCoordinator` request assertFalse(futureFor(driver, transactionalId).isDone()); - List.RequestSpec> lookupRequests2 = driver.poll(); + List.RequestSpec> lookupRequests2 = driver.poll(); assertEquals(1, lookupRequests2.size()); - RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); + RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); assertRetryBackoff(1, lookupSpec2); assertFindCoordinatorRequest(transactionalId, lookupSpec2); } @@ -294,9 +293,9 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests1 = driver.poll(); + List.RequestSpec> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId, lookupSpec1); @@ -305,9 +304,9 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List.RequestSpec> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestDriver.RequestSpec requestSpec = requests.get(0); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( @@ -317,12 +316,12 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()))) )); - KafkaFutureImpl future = futureFor(driver, transactionalId); + KafkaFutureImpl future = futureFor(driver, transactionalId); assertTrue(future.isDone()); assertFutureThrows(future, UnknownServerException.class); } - private KafkaFutureImpl futureFor( + private KafkaFutureImpl futureFor( DescribeTransactionsRequestDriver driver, String transactionalId ) { @@ -330,13 +329,13 @@ private KafkaFutureImpl futureFor( return driver.futures().get(key); } - private RequestDriver.RequestSpec findRequestWithKey( + private RequestDriver.RequestSpec findRequestWithKey( String transactionalId, - List.RequestSpec> requests + List.RequestSpec> requests ) { CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); - Optional.RequestSpec> firstMatch = requests.stream() + Optional.RequestSpec> firstMatch = requests.stream() .filter(spec -> spec.keys.contains(key)) .findFirst(); @@ -348,7 +347,7 @@ private RequestDriver.RequestSpec findRequestW private void assertDescribeTransactionsRequest( Set expectedTransactionalIds, int expectedCoordinatorId, - RequestDriver.RequestSpec spec + RequestDriver.RequestSpec spec ) { Set keys = expectedTransactionalIds.stream() .map(DescribeTransactionsRequestDriver::asCoordinatorKey) @@ -363,7 +362,7 @@ private void assertDescribeTransactionsRequest( private void assertFindCoordinatorRequest( String expectedTransactionalId, - RequestDriver.RequestSpec spec + RequestDriver.RequestSpec spec ) { CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(expectedTransactionalId); assertEquals(singleton(key), spec.keys); @@ -377,7 +376,7 @@ private void assertFindCoordinatorRequest( private void assertRetryBackoff( int expectedTries, - RequestDriver.RequestSpec spec + RequestDriver.RequestSpec spec ) { assertEquals(deadlineMs, spec.deadlineMs); assertEquals(expectedTries, spec.tries); @@ -434,9 +433,11 @@ private DescribeTransactionsResponseData.TransactionState sampleTransactionState } private void assertMatchingTransactionState( + int expectedCoordinatorId, DescribeTransactionsResponseData.TransactionState expected, - DescribeTransactionsResult.TransactionState actual + TransactionDescription actual ) { + assertEquals(expectedCoordinatorId, actual.coordinatorId()); assertEquals(expected.producerId(), actual.producerId()); assertEquals(expected.producerEpoch(), actual.producerEpoch()); assertEquals(expected.transactionTimeoutMs(), actual.transactionTimeoutMs()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java index 14e14d31967de..a82060e1a186e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.ListTransactionsOptions; -import org.apache.kafka.clients.admin.ListTransactionsResult; -import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.clients.admin.internals.AllBrokerRequestDriver.BrokerKey; import org.apache.kafka.common.errors.UnknownServerException; @@ -241,7 +240,7 @@ private void assertExpectedTransactions( private void assertExpectedTransactionState( ListTransactionsResponseData.TransactionState expected, - ListTransactionsResult.TransactionListing actual + TransactionListing actual ) { assertEquals(expected.transactionalId(), actual.transactionalId()); assertEquals(expected.producerId(), actual.producerId()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java index fb1be02915a19..11731baf706ba 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -280,7 +280,7 @@ public TestMetadataRequestDriver(Collection futures) { } @Override - AbstractRequest.Builder buildFulfillmentRequest(Set topicPartitions) { + AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set topicPartitions) { DescribeProducersRequestData request = new DescribeProducersRequestData(); DescribeProducersRequest.Builder builder = new DescribeProducersRequest.Builder(request); @@ -294,7 +294,7 @@ AbstractRequest.Builder buildFulfillmentRequest(Set topicPart } @Override - void handleFulfillmentResponse(Set keys, AbstractResponse response) { + void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse response) { throw new UnsupportedOperationException(); } } diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java index 9c6d7f79daeea..52ea4addbc76f 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java @@ -28,8 +28,8 @@ import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult; -import org.apache.kafka.clients.admin.DescribeTransactionsResult; -import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionListing; +import org.apache.kafka.clients.admin.TransactionDescription; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Time; @@ -75,7 +75,7 @@ protected TransactionsCommand(Time time) { abstract void addSubparser(Subparsers subparsers); /** - * Execute the command. + * Execute the command logic. */ abstract void execute(Admin admin, Namespace ns, PrintStream out) throws Exception; @@ -300,7 +300,7 @@ public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception String[] headers = new String[]{ "ProducerId", "ProducerEpoch", - "CoordinatorEpoch", + "LatestCoordinatorEpoch", "LastSequence", "LastTimestamp", "CurrentTransactionStartOffset" @@ -354,7 +354,7 @@ public void addSubparser(Subparsers subparsers) { public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { String transactionalId = ns.getString("transactional_id"); - final DescribeTransactionsResult.TransactionState result; + final TransactionDescription result; try { result = admin.describeTransactions(singleton(transactionalId)) .transactionalIdResult(transactionalId) @@ -365,8 +365,8 @@ public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception return; } - // TODO: Do we want a way to return coordinator ID? String[] headers = new String[]{ + "CoordinatorId", "TransactionalId", "ProducerId", "ProducerEpoch", @@ -390,10 +390,11 @@ public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception } String[] row = new String[]{ + String.valueOf(result.coordinatorId()), transactionalId, String.valueOf(result.producerId()), String.valueOf(result.producerEpoch()), - result.state(), + result.state().toString(), String.valueOf(result.transactionTimeoutMs()), transactionStartTimeMsColumnValue, transactionDurationMsColumnValue, diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java index fc8b9eb654b4c..48d55636173fd 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java @@ -25,7 +25,8 @@ import org.apache.kafka.clients.admin.DescribeProducersResult.ProducerState; import org.apache.kafka.clients.admin.DescribeTransactionsResult; import org.apache.kafka.clients.admin.ListTransactionsResult; -import org.apache.kafka.clients.admin.ListTransactionsResult.TransactionListing; +import org.apache.kafka.clients.admin.TransactionDescription; +import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; @@ -66,7 +67,7 @@ public class TransactionsCommandTest { private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); private final PrintStream out = new PrintStream(outputStream); private final MockTime time = new MockTime(); - Admin admin = Mockito.mock(Admin.class); + private final Admin admin = Mockito.mock(Admin.class); @Before public void setupExitProcedure() { @@ -160,7 +161,7 @@ private void testDescribeProducers( List> table = readOutputAsTable(); assertEquals(3, table.size()); - List expectedHeaders = asList("ProducerId", "ProducerEpoch", "CoordinatorEpoch", + List expectedHeaders = asList("ProducerId", "ProducerEpoch", "LatestCoordinatorEpoch", "LastSequence", "LastTimestamp", "CurrentTransactionStartOffset"); assertEquals(expectedHeaders, table.get(0)); @@ -236,10 +237,13 @@ public void testDescribeTransaction() throws Exception { DescribeTransactionsResult describeResult = Mockito.mock(DescribeTransactionsResult.class); + int coordinatorId = 5; long transactionStartTime = time.milliseconds(); - KafkaFuture describeFuture = KafkaFutureImpl.completedFuture( - new DescribeTransactionsResult.TransactionState( - "Ongoing", + + KafkaFuture describeFuture = KafkaFutureImpl.completedFuture( + new TransactionDescription( + coordinatorId, + TransactionState.ONGOING, 12345L, 15, 10000, @@ -259,6 +263,7 @@ public void testDescribeTransaction() throws Exception { assertEquals(2, table.size()); List expectedHeaders = asList( + "CoordinatorId", "TransactionalId", "ProducerId", "ProducerEpoch", @@ -271,6 +276,7 @@ public void testDescribeTransaction() throws Exception { assertEquals(expectedHeaders, table.get(0)); List expectedRow = asList( + String.valueOf(coordinatorId), transactionalId, "12345", "15", @@ -517,4 +523,4 @@ public void execute(int statusCode, String message) { } } -} \ No newline at end of file +} From de09c6c57538d7ded4d0a41ff5c21ac479c4764b Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 7 Sep 2020 17:23:47 -0700 Subject: [PATCH 06/17] Some test cleanup --- .../kafka/clients/admin/KafkaAdminClient.java | 5 +- .../admin/internals/RequestDriver.java | 28 ++- .../AbortTransactionRequestDriverTest.java | 21 +- .../CoordinatorRequestDriverTest.java | 61 +++--- .../DescribeProducersRequestDriverTest.java | 193 ++++++++---------- ...DescribeTransactionsRequestDriverTest.java | 73 +++---- .../ListTransactionsRequestDriverTest.java | 189 +++++++++-------- .../internals/MetadataRequestDriverTest.java | 86 ++++---- 8 files changed, 318 insertions(+), 338 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 90ffe5d816833..9102c726a08c8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -40,6 +40,7 @@ import org.apache.kafka.clients.admin.internals.MetadataOperationContext; import org.apache.kafka.clients.admin.internals.RequestDriver; import org.apache.kafka.clients.admin.internals.AbortTransactionRequestDriver; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; @@ -4339,12 +4340,12 @@ static Throwable getSubLevelError(Map subLevelErrors, K subKey, S } private void maybeSendRequests(RequestDriver driver, long currentTimeMs) { - for (RequestDriver.RequestSpec spec : driver.poll()) { + for (RequestSpec spec : driver.poll()) { runnable.call(newCall(driver, spec), currentTimeMs); } } - private Call newCall(RequestDriver driver, RequestDriver.RequestSpec spec) { + private Call newCall(RequestDriver driver, RequestSpec spec) { NodeProvider nodeProvider = spec.scope.destinationBrokerId().isPresent() ? new ConstantNodeIdProvider(spec.scope.destinationBrokerId().getAsInt()) : new LeastLoadedNodeProvider(); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java index 56381c1ebfa71..884524c0fae59 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -177,8 +177,8 @@ void complete(K key, V value) { * * @return A list of requests that need to be sent */ - public List poll() { - List requests = new ArrayList<>(); + public List> poll() { + List> requests = new ArrayList<>(); collectLookupRequests(requests); collectFulfillmentRequests(requests); return requests; @@ -196,7 +196,7 @@ public Map> futures() { */ public void onResponse( long currentTimeMs, - RequestSpec spec, + RequestSpec spec, AbstractResponse response ) { clearInflightRequest(currentTimeMs, spec); @@ -213,7 +213,7 @@ public void onResponse( */ public void onFailure( long currentTimeMs, - RequestSpec spec, + RequestSpec spec, Throwable t ) { clearInflightRequest(currentTimeMs, spec); @@ -256,7 +256,7 @@ public void onFailure( */ abstract void handleFulfillmentResponse(Integer brokerId, Set keys, AbstractResponse response); - private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { + private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { RequestState requestState = requestStates.get(spec.scope); if (requestState != null) { requestState.clearInflight(currentTimeMs); @@ -264,7 +264,7 @@ private void clearInflightRequest(long currentTimeMs, RequestSpec spec) { } private void collectRequests( - List requests, + List> requests, BiMultimap multimap, BiFunction, T, AbstractRequest.Builder> buildRequest ) { @@ -282,7 +282,7 @@ private void collectRequests( } AbstractRequest.Builder request = buildRequest.apply(keys, scope); - RequestSpec spec = new RequestSpec( + RequestSpec spec = new RequestSpec<>( scope, new HashSet<>(keys), // copy to avoid exposing mutable state request, @@ -296,7 +296,7 @@ private void collectRequests( } } - private void collectLookupRequests(List requests) { + private void collectLookupRequests(List> requests) { collectRequests( requests, lookupMap, @@ -304,7 +304,7 @@ private void collectLookupRequests(List requests) { ); } - private void collectFulfillmentRequests(List requests) { + private void collectFulfillmentRequests(List> requests) { collectRequests( requests, fulfillmentMap, @@ -317,7 +317,7 @@ private void collectFulfillmentRequests(List requests) { * the to the internal `Call` implementation that is used internally in * {@link org.apache.kafka.clients.admin.KafkaAdminClient}. */ - public class RequestSpec { + public static class RequestSpec { public final RequestScope scope; public final Set keys; public final AbstractRequest.Builder request; @@ -348,7 +348,7 @@ public RequestSpec( * of backoff/retry state. */ private class RequestState { - private Optional inflightRequest = Optional.empty(); + private Optional> inflightRequest = Optional.empty(); private int tries = 0; private long nextAllowedRetryMs = 0; @@ -361,7 +361,7 @@ public void clearInflight(long currentTimeMs) { this.nextAllowedRetryMs = currentTimeMs + retryBackoffMs; } - public void setInflight(RequestSpec spec) { + public void setInflight(RequestSpec spec) { this.inflightRequest = Optional.of(spec); this.tries++; } @@ -420,10 +420,6 @@ private static class BiMultimap { private final Map reverseMap = new HashMap<>(); private final Map> map = new HashMap<>(); - Set get(K key) { - return map.get(key); - } - void put(K key, V value) { remove(value); reverseMap.put(value, key); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java index 017b9d8458a60..f5f0e52f2a329 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.message.MetadataResponseData; @@ -113,7 +114,7 @@ private void assertMetadataLookup( int expectedTries ) { TopicPartition topicPartition = abortTransactionSpec.topicPartition(); - RequestDriver.RequestSpec lookupSpec = assertMetadataRequest( + RequestSpec lookupSpec = assertMetadataRequest( driver, topicPartition, expectedTries); driver.onResponse(time.milliseconds(), lookupSpec, new MetadataResponse( AdminRequestUtil.metadataResponse(singletonMap(topicPartition, @@ -132,7 +133,7 @@ private void assertWriteTxnMarkers( int expectedLeaderId, int expectedTries ) { - RequestDriver.RequestSpec requestSpec = assertWriteTxnMarkersRequest( + RequestSpec requestSpec = assertWriteTxnMarkersRequest( driver, abortTransactionSpec, expectedLeaderId, expectedTries); driver.onResponse(time.milliseconds(), requestSpec, writeTxnMarkersResponse(abortTransactionSpec, error)); @@ -161,16 +162,16 @@ private WriteTxnMarkersResponse writeTxnMarkersResponse( return new WriteTxnMarkersResponse(singletonMap(abortSpec.producerId(), partitionErrors)); } - private RequestDriver.RequestSpec assertWriteTxnMarkersRequest( + private RequestSpec assertWriteTxnMarkersRequest( AbortTransactionRequestDriver driver, AbortTransactionSpec abortSpec, int expectedLeaderId, int expectedTries ) { - List.RequestSpec> requestSpecs = driver.poll(); + List> requestSpecs = driver.poll(); assertEquals(1, requestSpecs.size()); - RequestDriver.RequestSpec requestSpec = requestSpecs.get(0); + RequestSpec requestSpec = requestSpecs.get(0); assertExpectedBackoffAndDeadline(requestSpec, expectedTries); assertEquals(OptionalInt.of(expectedLeaderId), requestSpec.scope.destinationBrokerId()); @@ -192,15 +193,15 @@ private RequestDriver.RequestSpec assertWriteTxnMarkersReq return requestSpec; } - private RequestDriver.RequestSpec assertMetadataRequest( + private RequestSpec assertMetadataRequest( AbortTransactionRequestDriver driver, TopicPartition topicPartition, int expectedTries ) { - List.RequestSpec> lookupRequests = driver.poll(); + List> lookupRequests = driver.poll(); assertEquals(1, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + RequestSpec lookupSpec = lookupRequests.get(0); assertExpectedBackoffAndDeadline(lookupSpec, expectedTries); assertEquals(OptionalInt.empty(), lookupSpec.scope.destinationBrokerId()); @@ -211,7 +212,7 @@ private RequestDriver.RequestSpec assertMetadataRequest( } private void assertExpectedBackoffAndDeadline( - RequestDriver.RequestSpec requestSpec, + RequestSpec requestSpec, int expectedTries ) { assertEquals(expectedTries, requestSpec.tries); @@ -223,4 +224,4 @@ private void assertExpectedBackoffAndDeadline( } } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java index 5e4054336c492..316fd20faaa8a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.message.DescribeGroupsRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; @@ -31,6 +32,7 @@ import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.OptionalInt; import java.util.Set; import java.util.stream.Collectors; @@ -53,17 +55,13 @@ public void testLookupGrouping() { Set groupIds = mkSet(group1, group2); TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(2, requests.size()); // While the FindCoordinator requests are inflight, we will not send any more assertEquals(0, driver.poll().size()); - RequestDriver.RequestSpec spec1 = requests.stream() - .filter(spec -> spec.keys.contains(group1)) - .findFirst() - .get(); - + RequestSpec spec1 = lookupRequest(requests, group1); assertEquals(mkSet(group1), spec1.keys); assertEquals(OptionalInt.empty(), spec1.scope.destinationBrokerId()); assertEquals(deadlineMs, spec1.deadlineMs); @@ -74,11 +72,7 @@ public void testLookupGrouping() { assertEquals(group1.idValue, findCoordinatorRequest1.data().key()); assertEquals(group1.type.id(), findCoordinatorRequest1.data().keyType()); - RequestDriver.RequestSpec spec2 = requests.stream() - .filter(spec -> spec.keys.contains(group2)) - .findFirst() - .get(); - + RequestSpec spec2 = lookupRequest(requests, group2); assertEquals(mkSet(group2), spec2.keys); assertEquals(OptionalInt.empty(), spec2.scope.destinationBrokerId()); assertEquals(deadlineMs, spec2.deadlineMs); @@ -97,14 +91,10 @@ public void testSuccessfulLeaderDiscovery() { Set groupIds = mkSet(group1, group2); TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); - List.RequestSpec> lookupRequests = driver.poll(); + List> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests.stream() - .filter(spec -> spec.keys.contains(group1)) - .findFirst() - .get(); - + RequestSpec lookupSpec1 = lookupRequest(lookupRequests, group1); driver.onResponse(time.milliseconds(), lookupSpec1, new FindCoordinatorResponse(new FindCoordinatorResponseData() .setErrorCode(Errors.NONE.code()) .setHost("localhost") @@ -112,9 +102,9 @@ public void testSuccessfulLeaderDiscovery() { .setNodeId(1) )); - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec requestSpec1 = requests1.get(0); + RequestSpec requestSpec1 = requests1.get(0); assertEquals(mkSet(group1), requestSpec1.keys); assertEquals(OptionalInt.of(1), requestSpec1.scope.destinationBrokerId()); assertEquals(0, requestSpec1.tries); @@ -124,11 +114,7 @@ public void testSuccessfulLeaderDiscovery() { DescribeGroupsRequest.Builder request = (DescribeGroupsRequest.Builder) requestSpec1.request; assertEquals(singletonList(group1.idValue), request.data.groups()); - RequestDriver.RequestSpec lookupSpec2 = lookupRequests.stream() - .filter(spec -> spec.keys.contains(group2)) - .findFirst() - .get(); - + RequestSpec lookupSpec2 = lookupRequest(lookupRequests, group2); driver.onResponse(time.milliseconds(), lookupSpec2, new FindCoordinatorResponse(new FindCoordinatorResponseData() .setErrorCode(Errors.NONE.code()) .setHost("localhost") @@ -136,9 +122,9 @@ public void testSuccessfulLeaderDiscovery() { .setNodeId(2) )); - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(1, requests2.size()); - RequestDriver.RequestSpec requestSpec2 = requests2.get(0); + RequestSpec requestSpec2 = requests2.get(0); assertEquals(mkSet(group2), requestSpec2.keys); assertEquals(OptionalInt.of(2), requestSpec2.scope.destinationBrokerId()); assertEquals(0, requestSpec2.tries); @@ -155,18 +141,18 @@ public void testRetriableFindCoordinatorError() { Set groupIds = mkSet(group1); TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); - List.RequestSpec> lookupRequests1 = driver.poll(); + List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec = lookupRequests1.get(0); + RequestSpec lookupSpec = lookupRequests1.get(0); driver.onResponse(time.milliseconds(), lookupSpec, new FindCoordinatorResponse(new FindCoordinatorResponseData() .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) )); - List.RequestSpec> lookupRequests2 = driver.poll(); + List> lookupRequests2 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec retryLookupSpec = lookupRequests2.get(0); + RequestSpec retryLookupSpec = lookupRequests2.get(0); assertEquals(1, retryLookupSpec.tries); assertEquals(time.milliseconds() + retryBackoffMs, retryLookupSpec.nextAllowedTryMs); assertEquals(deadlineMs, retryLookupSpec.deadlineMs); @@ -180,10 +166,10 @@ public void testFatalFindCoordinatorError() { Set groupIds = mkSet(group1); TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); - List.RequestSpec> lookupRequests1 = driver.poll(); + List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec = lookupRequests1.get(0); + RequestSpec lookupSpec = lookupRequests1.get(0); driver.onResponse(time.milliseconds(), lookupSpec, new FindCoordinatorResponse(new FindCoordinatorResponseData() .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code()) )); @@ -194,6 +180,17 @@ public void testFatalFindCoordinatorError() { assertEquals(group1.idValue, groupAuthorizationException.groupId()); } + private RequestSpec lookupRequest( + List> requests, + CoordinatorKey key + ) { + Optional> foundRequestOpt = requests.stream() + .filter(spec -> spec.keys.contains(key)) + .findFirst(); + assertTrue(foundRequestOpt.isPresent()); + return foundRequestOpt.get(); + } + private final class TestCoordinatorRequestDriver extends CoordinatorRequestDriver { public TestCoordinatorRequestDriver(Set groupIds) { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java index 300f1fc998d6c..6f55496ae6e48 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownServerException; @@ -62,219 +63,152 @@ public class DescribeProducersRequestDriverTest { @Test public void testSuccessfulResponseWithoutProvidedBrokerId() throws Exception { - TopicPartition tp = new TopicPartition("foo", 5); + TopicPartition topicPartition = new TopicPartition("foo", 5); int leaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(tp), + singleton(topicPartition), options, deadlineMs, retryBackoffMs ); - List.RequestSpec> lookupRequests = driver.poll(); - assertEquals(1, lookupRequests.size()); - - RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); - assertEquals(OptionalInt.empty(), lookupSpec.scope.destinationBrokerId()); - assertEquals(0, lookupSpec.tries); - assertEquals(0, lookupSpec.nextAllowedTryMs); - assertEquals(deadlineMs, lookupSpec.deadlineMs); - assertEquals(singleton(tp), lookupSpec.keys); + assertMetadataLookup(driver, topicPartition, leaderId, 0); - assertTrue(lookupSpec.request instanceof MetadataRequest.Builder); - MetadataRequest.Builder lookupRequest = (MetadataRequest.Builder) lookupSpec.request; - assertEquals(singletonList(tp.topic()), lookupRequest.topics()); - - driver.onResponse(time.milliseconds(), lookupSpec, new MetadataResponse(metadataResponse( - singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() - .setErrorCode(Errors.NONE.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(15) - .setReplicaNodes(asList(1, 2, 3)) - .setIsrNodes(asList(1, 2, 3))) - ))); - - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec spec = requests.get(0); + RequestSpec spec = requests.get(0); assertEquals(OptionalInt.of(leaderId), spec.scope.destinationBrokerId()); - assertEquals(singleton(tp), spec.keys); - assertEquals(0, spec.tries); - assertEquals(0, spec.nextAllowedTryMs); - assertEquals(deadlineMs, spec.deadlineMs); - assertSuccessfulFulfillment(driver, tp, spec); + assertEquals(singleton(topicPartition), spec.keys); + assertExpectedBackoffAndDeadline(spec, 0); + assertSuccessfulFulfillment(driver, topicPartition, spec); } @Test public void testRetryLookupAfterNotLeaderErrorWithoutProvidedBrokerId() { - TopicPartition tp = new TopicPartition("foo", 5); - int leaderId = 1; + TopicPartition topicPartition = new TopicPartition("foo", 5); + int initialLeaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(tp), + singleton(topicPartition), options, deadlineMs, retryBackoffMs ); - List.RequestSpec> lookupRequests1 = driver.poll(); - assertEquals(1, lookupRequests1.size()); + assertMetadataLookup(driver, topicPartition, initialLeaderId, 0); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); - assertEquals(singleton(tp), lookupSpec1.keys); - assertTrue(lookupSpec1.request instanceof MetadataRequest.Builder); - MetadataRequest.Builder lookupRequest1 = (MetadataRequest.Builder) lookupSpec1.request; - assertEquals(singletonList(tp.topic()), lookupRequest1.topics()); - - driver.onResponse(time.milliseconds(), lookupSpec1, new MetadataResponse(metadataResponse( - singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() - .setErrorCode(Errors.NONE.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(15) - .setReplicaNodes(asList(1, 2, 3)) - .setIsrNodes(asList(1, 2, 3))) - ))); - - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); // A `NOT_LEADER_OR_FOLLOWER` error should cause a retry of the `Metadata` request - RequestDriver.RequestSpec spec = requests.get(0); + RequestSpec spec = requests.get(0); driver.onResponse(time.milliseconds(), spec, describeProducersResponse( - singletonMap(tp, new PartitionResponse() + singletonMap(topicPartition, new PartitionResponse() .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code())) )); - List.RequestSpec> lookupRequests2 = driver.poll(); - assertEquals(1, lookupRequests2.size()); - - RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); - assertEquals(OptionalInt.empty(), lookupSpec2.scope.destinationBrokerId()); - assertEquals(singleton(tp), lookupSpec2.keys); - assertEquals(1, lookupSpec2.tries); - assertEquals(time.milliseconds() + retryBackoffMs, lookupSpec2.nextAllowedTryMs); - assertEquals(deadlineMs, lookupSpec2.deadlineMs); - - assertTrue(lookupSpec2.request instanceof MetadataRequest.Builder); - MetadataRequest.Builder lookupRequest2 = (MetadataRequest.Builder) lookupSpec2.request; - assertEquals(singletonList(tp.topic()), lookupRequest2.topics()); + int updatedLeaderId = 2; + assertMetadataLookup(driver, topicPartition, updatedLeaderId, 1); } @Test public void testSuccessfulResponseWithProvidedBrokerId() throws Exception { - TopicPartition tp = new TopicPartition("foo", 5); + TopicPartition topicPartition = new TopicPartition("foo", 5); int brokerId = 1; DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(tp), + singleton(topicPartition), options, deadlineMs, retryBackoffMs ); - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); // Note there should be no `Metadata` lookup since we specified the target brokerId directly - RequestDriver.RequestSpec spec = requests.get(0); - assertEquals(singleton(tp), spec.keys); + RequestSpec spec = requests.get(0); + assertEquals(singleton(topicPartition), spec.keys); assertEquals(OptionalInt.of(brokerId), spec.scope.destinationBrokerId()); - assertEquals(0, spec.tries); - assertEquals(0, spec.nextAllowedTryMs); - assertEquals(deadlineMs, spec.deadlineMs); + assertExpectedBackoffAndDeadline(spec, 0); assertTrue(spec.request instanceof DescribeProducersRequest.Builder); DescribeProducersRequest.Builder request = (DescribeProducersRequest.Builder) spec.request; assertEquals(1, request.data.topics().size()); TopicRequest topicRequest = request.data.topics().get(0); - assertEquals(tp.topic(), topicRequest.name()); - assertEquals(singletonList(tp.partition()), topicRequest.partitionIndexes()); - assertSuccessfulFulfillment(driver, tp, spec); + assertEquals(topicPartition.topic(), topicRequest.name()); + assertEquals(singletonList(topicPartition.partition()), topicRequest.partitionIndexes()); + assertSuccessfulFulfillment(driver, topicPartition, spec); } @Test public void testNotLeaderErrorWithProvidedBrokerId() { - TopicPartition tp = new TopicPartition("foo", 0); + TopicPartition topicPartition = new TopicPartition("foo", 0); int brokerId = 1; DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(tp), + singleton(topicPartition), options, deadlineMs, retryBackoffMs ); - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); // Note there should be no `Metadata` lookup since we specified the target brokerId directly - RequestDriver.RequestSpec spec = requests.get(0); - assertEquals(singleton(tp), spec.keys); + RequestSpec spec = requests.get(0); + assertEquals(singleton(topicPartition), spec.keys); assertEquals(OptionalInt.of(brokerId), spec.scope.destinationBrokerId()); + assertExpectedBackoffAndDeadline(spec, 0); assertTrue(spec.request instanceof DescribeProducersRequest.Builder); - driver.onResponse(time.milliseconds(), spec, describeProducersResponse(singletonMap(tp, + driver.onResponse(time.milliseconds(), spec, describeProducersResponse(singletonMap(topicPartition, new PartitionResponse().setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) ))); assertEquals(Collections.emptyList(), driver.poll()); - assertFutureThrows(driver.futures().get(tp), NotLeaderOrFollowerException.class); + assertFutureThrows(driver.futures().get(topicPartition), NotLeaderOrFollowerException.class); } @Test public void testFatalErrorWithoutProvidedBrokerId() { - TopicPartition tp = new TopicPartition("foo", 5); + TopicPartition topicPartition = new TopicPartition("foo", 5); int leaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(tp), + singleton(topicPartition), options, deadlineMs, retryBackoffMs ); - List.RequestSpec> lookupRequests1 = driver.poll(); - assertEquals(1, lookupRequests1.size()); - - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); - assertEquals(singleton(tp), lookupSpec1.keys); - assertTrue(lookupSpec1.request instanceof MetadataRequest.Builder); - MetadataRequest.Builder lookupRequest1 = (MetadataRequest.Builder) lookupSpec1.request; - assertEquals(singletonList(tp.topic()), lookupRequest1.topics()); - - driver.onResponse(time.milliseconds(), lookupSpec1, new MetadataResponse(metadataResponse( - singletonMap(tp, new MetadataResponseData.MetadataResponsePartition() - .setErrorCode(Errors.NONE.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(15) - .setReplicaNodes(asList(1, 2, 3)) - .setIsrNodes(asList(1, 2, 3))) - ))); + assertMetadataLookup(driver, topicPartition, leaderId, 0); - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec spec = requests.get(0); + RequestSpec spec = requests.get(0); driver.onResponse(time.milliseconds(), spec, describeProducersResponse( - singletonMap(tp, new PartitionResponse() + singletonMap(topicPartition, new PartitionResponse() .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())) )); assertEquals(Collections.emptyList(), driver.poll()); - assertFutureThrows(driver.futures().get(tp), UnknownServerException.class); + assertFutureThrows(driver.futures().get(topicPartition), UnknownServerException.class); } private void assertSuccessfulFulfillment( DescribeProducersRequestDriver driver, TopicPartition topicPartition, - RequestDriver.RequestSpec describeProducerSpec + RequestSpec describeProducerSpec ) throws Exception { List activeProducers = sampleProducerState(); driver.onResponse(time.milliseconds(), describeProducerSpec, describeProducersResponse( @@ -291,6 +225,47 @@ private void assertSuccessfulFulfillment( assertMatchingProducers(activeProducers, partitionProducerState.activeProducers()); } + private void assertMetadataLookup( + DescribeProducersRequestDriver driver, + TopicPartition topicPartition, + int leaderId, + int expectedTries + ) { + List> lookupRequests = driver.poll(); + assertEquals(1, lookupRequests.size()); + + RequestSpec lookupSpec = lookupRequests.get(0); + assertEquals(OptionalInt.empty(), lookupSpec.scope.destinationBrokerId()); + assertEquals(singleton(topicPartition), lookupSpec.keys); + assertExpectedBackoffAndDeadline(lookupSpec, expectedTries); + + assertTrue(lookupSpec.request instanceof MetadataRequest.Builder); + MetadataRequest.Builder lookupRequest = (MetadataRequest.Builder) lookupSpec.request; + assertEquals(singletonList(topicPartition.topic()), lookupRequest.topics()); + + driver.onResponse(time.milliseconds(), lookupSpec, new MetadataResponse(metadataResponse( + singletonMap(topicPartition, new MetadataResponseData.MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setLeaderId(leaderId) + .setLeaderEpoch(15) + .setReplicaNodes(singletonList(leaderId)) + .setIsrNodes(singletonList(leaderId))) + ))); + } + + private void assertExpectedBackoffAndDeadline( + RequestSpec requestSpec, + int expectedTries + ) { + assertEquals(expectedTries, requestSpec.tries); + assertEquals(deadlineMs, requestSpec.deadlineMs); + if (expectedTries == 0) { + assertEquals(0, requestSpec.nextAllowedTryMs); + } else { + assertEquals(time.milliseconds() + (expectedTries * retryBackoffMs), requestSpec.nextAllowedTryMs); + } + } + private List sampleProducerState() { return asList( new ProducerState() diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java index dd200afbf479e..d052bc9a70cca 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.TransactionDescription; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -65,18 +66,18 @@ public void testDescribeTransactions() throws Exception { ); // Send `FindCoordinator` requests - List.RequestSpec> lookupRequests = driver.poll(); + List> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec1 = + RequestSpec lookupSpec1 = findRequestWithKey(transactionalId1, lookupRequests); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId1, lookupSpec1); - RequestDriver.RequestSpec loookupSpec2 = + RequestSpec lookupSpec2 = findRequestWithKey(transactionalId2, lookupRequests); - assertRetryBackoff(0, loookupSpec2); - assertFindCoordinatorRequest(transactionalId2, loookupSpec2); + assertRetryBackoff(0, lookupSpec2); + assertFindCoordinatorRequest(transactionalId2, lookupSpec2); // Receive `FindCoordinator` responses int coordinator1 = 1; @@ -84,19 +85,19 @@ public void testDescribeTransactions() throws Exception { driver.onResponse(time.milliseconds(), lookupSpec1, findCoordinatorResponse(OptionalInt.of(coordinator1))); - driver.onResponse(time.milliseconds(), loookupSpec2, + driver.onResponse(time.milliseconds(), lookupSpec2, findCoordinatorResponse(OptionalInt.of(coordinator2))); // Send `DescribeTransactions` requests - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(2, requests.size()); - RequestDriver.RequestSpec requestSpec1 = + RequestSpec requestSpec1 = findRequestWithKey(transactionalId1, requests); assertRetryBackoff(0, requestSpec1); assertDescribeTransactionsRequest(singleton(transactionalId1), coordinator1, requestSpec1); - RequestDriver.RequestSpec requestSpec2 = + RequestSpec requestSpec2 = findRequestWithKey(transactionalId2, requests); assertRetryBackoff(0, requestSpec2); assertDescribeTransactionsRequest(singleton(transactionalId2), coordinator2, requestSpec2); @@ -139,15 +140,15 @@ public void testDescribeTransactionsBatching() throws Exception { ); // Send `FindCoordinator` requests - List.RequestSpec> lookupRequests = driver.poll(); + List> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec1 = + RequestSpec lookupSpec1 = findRequestWithKey(transactionalId1, lookupRequests); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId1, lookupSpec1); - RequestDriver.RequestSpec loookupSpec2 = + RequestSpec loookupSpec2 = findRequestWithKey(transactionalId2, lookupRequests); assertRetryBackoff(0, loookupSpec2); assertFindCoordinatorRequest(transactionalId2, loookupSpec2); @@ -163,10 +164,10 @@ public void testDescribeTransactionsBatching() throws Exception { findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestSpec requestSpec = requests.get(0); assertRetryBackoff(0, requestSpec); assertDescribeTransactionsRequest(transactionalIds, coordinator, requestSpec); @@ -205,9 +206,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests = driver.poll(); + List> lookupRequests = driver.poll(); assertEquals(1, lookupRequests.size()); - RequestDriver.RequestSpec lookupSpec = lookupRequests.get(0); + RequestSpec lookupSpec = lookupRequests.get(0); assertRetryBackoff(0, lookupSpec); assertFindCoordinatorRequest(transactionalId, lookupSpec); @@ -216,9 +217,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec requestSpec1 = requests1.get(0); + RequestSpec requestSpec1 = requests1.get(0); assertRetryBackoff(0, requestSpec1); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec1); @@ -232,9 +233,9 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() // Send retry `DescribeTransactions` request assertFalse(futureFor(driver, transactionalId).isDone()); - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(1, requests2.size()); - RequestDriver.RequestSpec requestSpec2 = requests2.get(0); + RequestSpec requestSpec2 = requests2.get(0); assertRetryBackoff(1, requestSpec2); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec2); } @@ -250,9 +251,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests1 = driver.poll(); + List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + RequestSpec lookupSpec1 = lookupRequests1.get(0); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId, lookupSpec1); @@ -261,9 +262,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestSpec requestSpec = requests.get(0); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( @@ -275,9 +276,9 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { // Send second `FindCoordinator` request assertFalse(futureFor(driver, transactionalId).isDone()); - List.RequestSpec> lookupRequests2 = driver.poll(); + List> lookupRequests2 = driver.poll(); assertEquals(1, lookupRequests2.size()); - RequestDriver.RequestSpec lookupSpec2 = lookupRequests2.get(0); + RequestSpec lookupSpec2 = lookupRequests2.get(0); assertRetryBackoff(1, lookupSpec2); assertFindCoordinatorRequest(transactionalId, lookupSpec2); } @@ -293,9 +294,9 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() ); // Send first `FindCoordinator` request - List.RequestSpec> lookupRequests1 = driver.poll(); + List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); - RequestDriver.RequestSpec lookupSpec1 = lookupRequests1.get(0); + RequestSpec lookupSpec1 = lookupRequests1.get(0); assertRetryBackoff(0, lookupSpec1); assertFindCoordinatorRequest(transactionalId, lookupSpec1); @@ -304,9 +305,9 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() findCoordinatorResponse(OptionalInt.of(coordinator))); // Send `DescribeTransactions` request - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver.RequestSpec requestSpec = requests.get(0); + RequestSpec requestSpec = requests.get(0); assertDescribeTransactionsRequest(singleton(transactionalId), coordinator, requestSpec); driver.onResponse(time.milliseconds(), requestSpec, new DescribeTransactionsResponse( @@ -329,13 +330,13 @@ private KafkaFutureImpl futureFor( return driver.futures().get(key); } - private RequestDriver.RequestSpec findRequestWithKey( + private RequestSpec findRequestWithKey( String transactionalId, - List.RequestSpec> requests + List> requests ) { CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); - Optional.RequestSpec> firstMatch = requests.stream() + Optional> firstMatch = requests.stream() .filter(spec -> spec.keys.contains(key)) .findFirst(); @@ -347,7 +348,7 @@ private RequestDriver.RequestSpec findRe private void assertDescribeTransactionsRequest( Set expectedTransactionalIds, int expectedCoordinatorId, - RequestDriver.RequestSpec spec + RequestSpec spec ) { Set keys = expectedTransactionalIds.stream() .map(DescribeTransactionsRequestDriver::asCoordinatorKey) @@ -362,7 +363,7 @@ private void assertDescribeTransactionsRequest( private void assertFindCoordinatorRequest( String expectedTransactionalId, - RequestDriver.RequestSpec spec + RequestSpec spec ) { CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(expectedTransactionalId); assertEquals(singleton(key), spec.keys); @@ -376,7 +377,7 @@ private void assertFindCoordinatorRequest( private void assertRetryBackoff( int expectedTries, - RequestDriver.RequestSpec spec + RequestSpec spec ) { assertEquals(deadlineMs, spec.deadlineMs); assertEquals(expectedTries, spec.tries); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java index a82060e1a186e..3ade7eaa21d66 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionState; import org.apache.kafka.clients.admin.internals.AllBrokerRequestDriver.BrokerKey; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.message.ListTransactionsResponseData; @@ -44,6 +46,7 @@ import java.util.stream.Collectors; import static java.util.Arrays.asList; +import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.test.TestUtils.assertFutureThrows; @@ -58,38 +61,35 @@ public class ListTransactionsRequestDriverTest { private final long retryBackoffMs = 100; @Test - public void testSuccessfulListTransactions() throws Exception { + public void testFailedMetadataRequest() { ListTransactionsOptions options = new ListTransactionsOptions(); ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); KafkaFutureImpl>>> lookupFuture = driver.lookupFuture(); - // Send `Metadata` request to find brokerIds - RequestDriver>.RequestSpec lookupRequestSpec = - assertLookupRequest(driver); - - // Receive `Metadata` response - driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(asList( - brokerMetadata(0), - brokerMetadata(1) - ))); + RequestSpec lookupRequestSpec = assertLookupRequest(driver); + driver.onFailure(time.milliseconds(), lookupRequestSpec, new TimeoutException()); + assertFutureThrows(lookupFuture, TimeoutException.class); + } - assertTrue(lookupFuture.isDone()); + @Test + public void testMultiBrokerListTransactions() throws Exception { + ListTransactionsOptions options = new ListTransactionsOptions(); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); - Map>> brokerFutures = lookupFuture.get(); + Map>> brokerFutures = + assertMetadataLookup(driver, mkSet(0, 1)); assertEquals(mkSet(0, 1), brokerFutures.keySet()); assertTrue(brokerFutures.values().stream().noneMatch(KafkaFutureImpl::isDone)); // Send `ListTransactions` requests - List>.RequestSpec> requestSpecs = driver.poll(); + List> requestSpecs = driver.poll(); assertEquals(2, requestSpecs.size()); - RequestDriver>.RequestSpec requestBroker0 = - findBrokerRequest(requestSpecs, 0); + RequestSpec requestBroker0 = findBrokerRequest(requestSpecs, 0); assertListTransactionsRequest(options, requestBroker0); - RequestDriver>.RequestSpec requestBroker1 = - findBrokerRequest(requestSpecs, 1); + RequestSpec requestBroker1 = findBrokerRequest(requestSpecs, 1); assertListTransactionsRequest(options, requestBroker1); // Receive `ListTransactions` responses @@ -111,47 +111,24 @@ public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception ListTransactionsOptions options = new ListTransactionsOptions(); ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); - KafkaFutureImpl>>> lookupFuture = - driver.lookupFuture(); - - // Send `Metadata` request to find brokerIds - RequestDriver>.RequestSpec lookupRequestSpec = - assertLookupRequest(driver); - - // Receive `Metadata` response - driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(singletonList( - brokerMetadata(0) - ))); - - assertTrue(lookupFuture.isDone()); - - Map>> brokerFutures = lookupFuture.get(); - assertEquals(mkSet(0), brokerFutures.keySet()); - KafkaFutureImpl> brokerFuture = brokerFutures.get(0); + int brokerId = 0; + Map>> brokerFutures = + assertMetadataLookup(driver, singleton(brokerId)); + assertEquals(singleton(brokerId), brokerFutures.keySet()); + KafkaFutureImpl> brokerFuture = brokerFutures.get(brokerId); assertFalse(brokerFuture.isDone()); - // Send `ListTransactions` requests - List>.RequestSpec> requestSpecs = driver.poll(); - assertEquals(1, requestSpecs.size()); - RequestDriver>.RequestSpec request = - findBrokerRequest(requestSpecs, 0); - assertListTransactionsRequest(options, request); - - // Receive `ListTransactions` responses - ListTransactionsResponse broker0Response = new ListTransactionsResponse( - new ListTransactionsResponseData().setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code())); - driver.onResponse(time.milliseconds(), request, broker0Response); + // Send `ListTransactions` once and receive loading error + ListTransactionsResponse errorResponse = listTransactionsResponseWithError( + Errors.COORDINATOR_LOAD_IN_PROGRESS); + assertListTransactions(driver, options, errorResponse, brokerId); assertFalse(brokerFuture.isDone()); // Now we expect `ListTransactions` to be retried - List>.RequestSpec> retrySpecs = driver.poll(); - assertEquals(1, retrySpecs.size()); - RequestDriver>.RequestSpec retryRequest = - findBrokerRequest(requestSpecs, 0); - assertListTransactionsRequest(options, retryRequest); - - driver.onResponse(time.milliseconds(), retryRequest, sampleListTransactionsResponse1()); + ListTransactionsResponse response = sampleListTransactionsResponse1(); + assertListTransactions(driver, options, response, brokerId); assertTrue(brokerFuture.isDone()); + assertExpectedTransactions(response.data().transactionStates(), brokerFuture.get()); } @Test @@ -159,62 +136,84 @@ public void testFatalListTransactionsError() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + int brokerId = 0; + Map>> brokerFutures = + assertMetadataLookup(driver, singleton(brokerId)); + assertEquals(singleton(brokerId), brokerFutures.keySet()); + KafkaFutureImpl> brokerFuture = brokerFutures.get(brokerId); + assertFalse(brokerFuture.isDone()); + + ListTransactionsResponse errorResponse = listTransactionsResponseWithError(Errors.UNKNOWN_SERVER_ERROR); + assertListTransactions(driver, options, errorResponse, brokerId); + assertTrue(brokerFuture.isDone()); + assertFutureThrows(brokerFuture, UnknownServerException.class); + } + + private void assertListTransactions( + ListTransactionsRequestDriver driver, + ListTransactionsOptions options, + ListTransactionsResponse response, + int brokerId + ) { + List> requestSpecs = driver.poll(); + assertEquals(1, requestSpecs.size()); + RequestSpec request = findBrokerRequest(requestSpecs, brokerId); + assertListTransactionsRequest(options, request); + driver.onResponse(time.milliseconds(), request, response); + } + + private Map>> assertMetadataLookup( + ListTransactionsRequestDriver driver, + Set brokers + ) throws Exception { KafkaFutureImpl>>> lookupFuture = driver.lookupFuture(); // Send `Metadata` request to find brokerIds - RequestDriver>.RequestSpec lookupRequestSpec = - assertLookupRequest(driver); + RequestSpec lookupRequestSpec = assertLookupRequest(driver); // Receive `Metadata` response - driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse(singletonList( - brokerMetadata(0) - ))); + MetadataResponse metadataResponse = metadataResponse(brokers.stream() + .map(this::brokerMetadata) + .collect(Collectors.toList()) + ); + driver.onResponse(time.milliseconds(), lookupRequestSpec, metadataResponse); assertTrue(lookupFuture.isDone()); - Map>> brokerFutures = lookupFuture.get(); - assertEquals(mkSet(0), brokerFutures.keySet()); - KafkaFutureImpl> brokerFuture = brokerFutures.get(0); - assertFalse(brokerFuture.isDone()); - - // Send `ListTransactions` requests - List>.RequestSpec> requestSpecs = driver.poll(); - assertEquals(1, requestSpecs.size()); - RequestDriver>.RequestSpec request = - findBrokerRequest(requestSpecs, 0); - assertListTransactionsRequest(options, request); + return lookupFuture.get(); + } - // Receive `ListTransactions` responses with an unexpected error - ListTransactionsResponse broker0Response = new ListTransactionsResponse( - new ListTransactionsResponseData().setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())); - driver.onResponse(time.milliseconds(), request, broker0Response); - assertTrue(brokerFuture.isDone()); - assertFutureThrows(brokerFuture, UnknownServerException.class); + private ListTransactionsResponse listTransactionsResponseWithError(Errors error) { + return new ListTransactionsResponse(new ListTransactionsResponseData().setErrorCode(error.code())); } private ListTransactionsResponse sampleListTransactionsResponse1() { return new ListTransactionsResponse( - new ListTransactionsResponseData().setTransactionStates(asList( - new ListTransactionsResponseData.TransactionState() - .setTransactionalId("foo") - .setProducerId(12345L) - .setTransactionState("Ongoing"), - new ListTransactionsResponseData.TransactionState() - .setTransactionalId("bar") - .setProducerId(98765L) - .setTransactionState("PrepareAbort") + new ListTransactionsResponseData() + .setErrorCode(Errors.NONE.code()) + .setTransactionStates(asList( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("foo") + .setProducerId(12345L) + .setTransactionState("Ongoing"), + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("bar") + .setProducerId(98765L) + .setTransactionState("PrepareAbort") )) ); } private ListTransactionsResponse sampleListTransactionsResponse2() { return new ListTransactionsResponse( - new ListTransactionsResponseData().setTransactionStates(singletonList( - new ListTransactionsResponseData.TransactionState() - .setTransactionalId("baz") - .setProducerId(13579L) - .setTransactionState("CompleteCommit") + new ListTransactionsResponseData() + .setErrorCode(Errors.NONE.code()) + .setTransactionStates(singletonList( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("baz") + .setProducerId(13579L) + .setTransactionState("CompleteCommit") )) ); } @@ -249,7 +248,7 @@ private void assertExpectedTransactionState( private void assertListTransactionsRequest( ListTransactionsOptions options, - RequestDriver>.RequestSpec spec + RequestSpec spec ) { assertTrue(spec.request instanceof ListTransactionsRequest.Builder); ListTransactionsRequest.Builder request = (ListTransactionsRequest.Builder) spec.request; @@ -262,11 +261,11 @@ private void assertListTransactionsRequest( assertEquals(expectedFilteredStates, new HashSet<>(request.data.statesFilter())); } - private RequestDriver>.RequestSpec findBrokerRequest( - List>.RequestSpec> requests, + private RequestSpec findBrokerRequest( + List> requests, Integer brokerId ) { - Optional>.RequestSpec> requestOpt = requests.stream() + Optional> requestOpt = requests.stream() .filter(spec -> spec.scope.destinationBrokerId().isPresent() && spec.scope.destinationBrokerId().getAsInt() == brokerId) .findFirst(); @@ -274,13 +273,13 @@ private RequestDriver>.RequestSpec fin return requestOpt.get(); } - private RequestDriver>.RequestSpec assertLookupRequest( + private RequestSpec assertLookupRequest( ListTransactionsRequestDriver driver ) { - List>.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); - RequestDriver>.RequestSpec lookupRequestSpec = requests.get(0); + RequestSpec lookupRequestSpec = requests.get(0); assertEquals(OptionalInt.empty(), lookupRequestSpec.scope.destinationBrokerId()); assertTrue(lookupRequestSpec.request instanceof MetadataRequest.Builder); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java index 11731baf706ba..a227b53915920 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.UnknownServerException; @@ -38,6 +39,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.OptionalInt; import java.util.Set; @@ -61,17 +63,15 @@ public void testLookupGrouping() { new TopicPartition("bar", 1)); TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); - List.RequestSpec> requests = driver.poll(); + List> requests = driver.poll(); assertEquals(1, requests.size()); // While a Metadata request is inflight, we will not send another assertEquals(0, driver.poll().size()); - RequestDriver.RequestSpec spec = requests.get(0); + RequestSpec spec = requests.get(0); assertEquals(topicPartitions, spec.keys); - assertEquals(0, spec.tries); - assertEquals(deadlineMs, spec.deadlineMs); - assertEquals(0, spec.nextAllowedTryMs); + assertExpectedBackoffAndDeadline(spec, 0); assertTrue(spec.request instanceof MetadataRequest.Builder); MetadataRequest.Builder metadataRequest = (MetadataRequest.Builder) spec.request; @@ -88,10 +88,10 @@ public void testSuccessfulLeaderDiscovery() { Set topicPartitions = mkSet(tp0, tp2); TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + RequestSpec metadataSpec = requests1.iterator().next(); driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( mkEntry(tp0, new MetadataResponsePartition() .setErrorCode(Errors.NONE.code()) @@ -114,20 +114,14 @@ public void testSuccessfulLeaderDiscovery() { )))); // We should have two fulfillment requests now sent to the leaders of partitions 0 and 2 - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(2, requests2.size()); - RequestDriver.RequestSpec spec0 = requests2.stream() - .filter(spec -> spec.keys.contains(tp0)) - .findFirst() - .get(); + RequestSpec spec0 = lookupRequest(requests2, tp0); assertEquals(mkSet(tp0), spec0.keys); assertEquals(OptionalInt.of(1), spec0.scope.destinationBrokerId()); - RequestDriver.RequestSpec spec1 = requests2.stream() - .filter(spec -> spec.keys.contains(tp2)) - .findFirst() - .get(); + RequestSpec spec1 = lookupRequest(requests2, tp2); assertEquals(mkSet(tp2), spec1.keys); assertEquals(OptionalInt.of(3), spec1.scope.destinationBrokerId()); } @@ -142,10 +136,10 @@ public void testRetryLeaderDiscovery() { Set topicPartitions = mkSet(tp0, tp2); TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + RequestSpec metadataSpec = requests1.iterator().next(); driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( mkEntry(tp0, new MetadataResponsePartition() .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code())), @@ -163,26 +157,19 @@ public void testRetryLeaderDiscovery() { .setReplicaNodes(asList(1, 2, 3))) )))); - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(2, requests2.size()); // We should retry the Metadata request with only one of the two topics - RequestDriver.RequestSpec spec0 = requests2.stream() - .filter(spec -> spec.keys.contains(tp0)) - .findFirst() - .get(); + RequestSpec spec0 = lookupRequest(requests2, tp0); assertEquals(mkSet(tp0), spec0.keys); assertEquals(OptionalInt.empty(), spec0.scope.destinationBrokerId()); assertTrue(spec0.request instanceof MetadataRequest.Builder); - assertEquals(1, spec0.tries); - assertEquals(time.milliseconds() + retryBackoffMs, spec0.nextAllowedTryMs); + assertExpectedBackoffAndDeadline(spec0, 1); MetadataRequest.Builder retryMetadataRequest = (MetadataRequest.Builder) spec0.request; assertEquals(mkSet("foo"), new HashSet<>(retryMetadataRequest.topics())); - RequestDriver.RequestSpec spec1 = requests2.stream() - .filter(spec -> spec.keys.contains(tp2)) - .findFirst() - .get(); + RequestSpec spec1 = lookupRequest(requests2, tp2); assertEquals(mkSet(tp2), spec1.keys); assertEquals(OptionalInt.of(3), spec1.scope.destinationBrokerId()); } @@ -194,10 +181,10 @@ public void testFatalTopicError() { Set topicPartitions = mkSet(tp0, tp2); TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + RequestSpec metadataSpec = requests1.iterator().next(); MetadataResponseData metadataResponseData = metadataResponse(Utils.mkMap( mkEntry(tp2, new MetadataResponsePartition() .setErrorCode(Errors.NONE.code()) @@ -214,7 +201,7 @@ public void testFatalTopicError() { driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponseData)); - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(1, requests2.size()); // The lookup for "foo" should fail and not be retried @@ -223,7 +210,7 @@ public void testFatalTopicError() { assertEquals(mkSet("foo"), topicAuthorizationException.unauthorizedTopics()); // However, the lookup for the other other topic should proceed - RequestDriver.RequestSpec spec = requests2.get(0); + RequestSpec spec = requests2.get(0); assertEquals(mkSet(tp2), spec.keys); assertEquals(OptionalInt.of(3), spec.scope.destinationBrokerId()); } @@ -238,10 +225,10 @@ public void testFatalPartitionError() { Set topicPartitions = mkSet(tp0, tp2); TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); - List.RequestSpec> requests1 = driver.poll(); + List> requests1 = driver.poll(); assertEquals(1, requests1.size()); - RequestDriver.RequestSpec metadataSpec = requests1.iterator().next(); + RequestSpec metadataSpec = requests1.iterator().next(); // Any unexpected partition error will cause the partition to fail driver.onResponse(time.milliseconds(), metadataSpec, new MetadataResponse(metadataResponse(Utils.mkMap( @@ -261,18 +248,29 @@ public void testFatalPartitionError() { .setReplicaNodes(asList(1, 2, 3))) )))); - List.RequestSpec> requests2 = driver.poll(); + List> requests2 = driver.poll(); assertEquals(1, requests2.size()); // The lookup for "foo-0" should fail and not be retried TestUtils.assertFutureThrows(driver.futures().get(tp0), UnknownServerException.class); // However, the lookup for the other other partition should proceed - RequestDriver.RequestSpec spec = requests2.get(0); + RequestSpec spec = requests2.get(0); assertEquals(mkSet(tp2), spec.keys); assertEquals(OptionalInt.of(3), spec.scope.destinationBrokerId()); } + private RequestSpec lookupRequest( + List> requests, + TopicPartition key + ) { + Optional> foundRequestOpt = requests.stream() + .filter(spec -> spec.keys.contains(key)) + .findFirst(); + assertTrue(foundRequestOpt.isPresent()); + return foundRequestOpt.get(); + } + private final class TestMetadataRequestDriver extends MetadataRequestDriver { public TestMetadataRequestDriver(Collection futures) { @@ -299,4 +297,16 @@ void handleFulfillmentResponse(Integer brokerId, Set keys, Abstr } } -} \ No newline at end of file + private void assertExpectedBackoffAndDeadline( + RequestSpec requestSpec, + int expectedTries + ) { + assertEquals(expectedTries, requestSpec.tries); + assertEquals(deadlineMs, requestSpec.deadlineMs); + if (expectedTries == 0) { + assertEquals(0, requestSpec.nextAllowedTryMs); + } else { + assertEquals(time.milliseconds() + (expectedTries * retryBackoffMs), requestSpec.nextAllowedTryMs); + } + } +} From ee860970f3fd1e229272bc49010f87591336e4e8 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 7 Sep 2020 18:20:22 -0700 Subject: [PATCH 07/17] Add base `KafkaAdminClient` test cases --- .../admin/DescribeProducersResult.java | 83 ------ .../kafka/clients/admin/ProducerState.java | 101 +++++++ .../clients/admin/TransactionListing.java | 2 +- .../DescribeProducersRequestDriver.java | 2 +- .../clients/admin/KafkaAdminClientTest.java | 253 +++++++++++++++++- .../DescribeProducersRequestDriverTest.java | 5 +- .../ListTransactionsRequestDriverTest.java | 2 +- .../internals/MetadataRequestDriverTest.java | 2 +- .../kafka/tools/TransactionsCommand.java | 7 +- .../kafka/tools/TransactionsCommandTest.java | 2 +- 10 files changed, 351 insertions(+), 108 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/ProducerState.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java index 4db42e685022a..473191661f349 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java @@ -25,9 +25,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.OptionalInt; -import java.util.OptionalLong; import java.util.concurrent.ExecutionException; @InterfaceStability.Evolving @@ -76,84 +73,4 @@ public List activeProducers() { } } - public static class ProducerState { - private final long producerId; - private final int producerEpoch; - private final int lastSequence; - private final long lastTimestamp; - private final OptionalInt coordinatorEpoch; - private final OptionalLong currentTransactionStartOffset; - - public ProducerState( - long producerId, - int producerEpoch, - int lastSequence, - long lastTimestamp, - OptionalInt coordinatorEpoch, - OptionalLong currentTransactionStartOffset - ) { - this.producerId = producerId; - this.producerEpoch = producerEpoch; - this.lastSequence = lastSequence; - this.lastTimestamp = lastTimestamp; - this.coordinatorEpoch = coordinatorEpoch; - this.currentTransactionStartOffset = currentTransactionStartOffset; - } - - public long producerId() { - return producerId; - } - - public int producerEpoch() { - return producerEpoch; - } - - public int lastSequence() { - return lastSequence; - } - - public long lastTimestamp() { - return lastTimestamp; - } - - public OptionalLong currentTransactionStartOffset() { - return currentTransactionStartOffset; - } - - public OptionalInt coordinatorEpoch() { - return coordinatorEpoch; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ProducerState that = (ProducerState) o; - return producerId == that.producerId && - producerEpoch == that.producerEpoch && - lastSequence == that.lastSequence && - lastTimestamp == that.lastTimestamp && - Objects.equals(coordinatorEpoch, that.coordinatorEpoch) && - Objects.equals(currentTransactionStartOffset, that.currentTransactionStartOffset); - } - - @Override - public int hashCode() { - return Objects.hash(producerId, producerEpoch, lastSequence, lastTimestamp, - coordinatorEpoch, currentTransactionStartOffset); - } - - @Override - public String toString() { - return "ProducerState(" + - "producerId=" + producerId + - ", producerEpoch=" + producerEpoch + - ", lastSequence=" + lastSequence + - ", lastTimestamp=" + lastTimestamp + - ", coordinatorEpoch=" + coordinatorEpoch + - ", currentTransactionStartOffset=" + currentTransactionStartOffset + - ')'; - } - } - } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ProducerState.java b/clients/src/main/java/org/apache/kafka/clients/admin/ProducerState.java new file mode 100644 index 0000000000000..243edded6c138 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ProducerState.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import java.util.Objects; +import java.util.OptionalInt; +import java.util.OptionalLong; + +public class ProducerState { + private final long producerId; + private final int producerEpoch; + private final int lastSequence; + private final long lastTimestamp; + private final OptionalInt coordinatorEpoch; + private final OptionalLong currentTransactionStartOffset; + + public ProducerState( + long producerId, + int producerEpoch, + int lastSequence, + long lastTimestamp, + OptionalInt coordinatorEpoch, + OptionalLong currentTransactionStartOffset + ) { + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.lastSequence = lastSequence; + this.lastTimestamp = lastTimestamp; + this.coordinatorEpoch = coordinatorEpoch; + this.currentTransactionStartOffset = currentTransactionStartOffset; + } + + public long producerId() { + return producerId; + } + + public int producerEpoch() { + return producerEpoch; + } + + public int lastSequence() { + return lastSequence; + } + + public long lastTimestamp() { + return lastTimestamp; + } + + public OptionalLong currentTransactionStartOffset() { + return currentTransactionStartOffset; + } + + public OptionalInt coordinatorEpoch() { + return coordinatorEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ProducerState that = (ProducerState) o; + return producerId == that.producerId && + producerEpoch == that.producerEpoch && + lastSequence == that.lastSequence && + lastTimestamp == that.lastTimestamp && + Objects.equals(coordinatorEpoch, that.coordinatorEpoch) && + Objects.equals(currentTransactionStartOffset, that.currentTransactionStartOffset); + } + + @Override + public int hashCode() { + return Objects.hash(producerId, producerEpoch, lastSequence, lastTimestamp, + coordinatorEpoch, currentTransactionStartOffset); + } + + @Override + public String toString() { + return "ProducerState(" + + "producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", lastSequence=" + lastSequence + + ", lastTimestamp=" + lastTimestamp + + ", coordinatorEpoch=" + coordinatorEpoch + + ", currentTransactionStartOffset=" + currentTransactionStartOffset + + ')'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java index 160a3e41f4ef1..7384613e79b67 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionListing.java @@ -41,7 +41,7 @@ public long producerId() { return producerId; } - public TransactionState transactionState() { + public TransactionState state() { return transactionState; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java index 6f7f139cd2c62..65a8818abed1c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; -import org.apache.kafka.clients.admin.DescribeProducersResult.ProducerState; +import org.apache.kafka.clients.admin.ProducerState; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.message.DescribeProducersRequestData; diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index e3c1b5071b519..7659bb44a1e79 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -53,12 +53,12 @@ import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.LogDirNotFoundException; -import org.apache.kafka.common.errors.ThrottlingQuotaExceededException; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.SecurityDisabledException; +import org.apache.kafka.common.errors.ThrottlingQuotaExceededException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicDeletionDisabledException; import org.apache.kafka.common.errors.TopicExistsException; @@ -70,10 +70,13 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult; +<<<<<<< HEAD import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; +======= +import org.apache.kafka.common.message.CreateAclsResponseData; +>>>>>>> Add base `KafkaAdminClient` test cases import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; -import org.apache.kafka.common.message.CreateAclsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResultCollection; @@ -91,10 +94,16 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic; +<<<<<<< HEAD import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo; +======= +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.message.DescribeTransactionsResponseData; +>>>>>>> Add base `KafkaAdminClient` test cases import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; +import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; @@ -102,8 +111,10 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; -import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; +import org.apache.kafka.common.message.ListTransactionsResponseData; +import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; @@ -135,8 +146,16 @@ import org.apache.kafka.common.requests.DescribeConfigsResponse; import org.apache.kafka.common.requests.DescribeGroupsResponse; import org.apache.kafka.common.requests.DescribeLogDirsResponse; +<<<<<<< HEAD import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse; +======= +import org.apache.kafka.common.requests.DescribeProducersRequest; +import org.apache.kafka.common.requests.DescribeProducersResponse; +import org.apache.kafka.common.requests.DescribeTransactionsRequest; +import org.apache.kafka.common.requests.DescribeTransactionsResponse; +>>>>>>> Add base `KafkaAdminClient` test cases import org.apache.kafka.common.requests.ElectLeadersResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; import org.apache.kafka.common.requests.JoinGroupRequest; @@ -146,11 +165,15 @@ import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.requests.ListOffsetResponse.PartitionData; import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; +import org.apache.kafka.common.requests.ListTransactionsRequest; +import org.apache.kafka.common.requests.ListTransactionsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePatternFilter; @@ -177,6 +200,7 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -189,7 +213,9 @@ import static java.util.Arrays.asList; 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 java.util.Collections.singletonMap; import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse; import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse; import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingPartitionReassignment; @@ -4455,12 +4481,12 @@ public void testDescribeUserScramCredentials() throws Exception { DescribeUserScramCredentialsResponseData responseData = new DescribeUserScramCredentialsResponseData(); responseData.setResults(Arrays.asList( - new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult() - .setUser(user0Name) - .setCredentialInfos(Arrays.asList(user0CredentialInfo0, user0CredentialInfo1)), - new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult() - .setUser(user1Name) - .setCredentialInfos(Arrays.asList(user1CredentialInfo)))); + new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult() + .setUser(user0Name) + .setCredentialInfos(Arrays.asList(user0CredentialInfo0, user0CredentialInfo1)), + new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult() + .setUser(user1Name) + .setCredentialInfos(Arrays.asList(user1CredentialInfo)))); env.kafkaClient().prepareResponse(new DescribeUserScramCredentialsResponse(responseData)); @@ -4509,14 +4535,14 @@ public void testAlterUserScramCredentialsUnknownMechanism() throws Exception { AlterUserScramCredentialsResponseData responseData = new AlterUserScramCredentialsResponseData(); responseData.setResults(Arrays.asList( - new AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult().setUser(user2Name))); + new AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult().setUser(user2Name))); env.kafkaClient().prepareResponse(new AlterUserScramCredentialsResponse(responseData)); AlterUserScramCredentialsResult result = env.adminClient().alterUserScramCredentials(Arrays.asList( - new UserScramCredentialDeletion(user0Name, user0ScramMechanism0), - new UserScramCredentialUpsertion(user1Name, new ScramCredentialInfo(user1ScramMechanism0, 8192), "password"), - new UserScramCredentialUpsertion(user2Name, new ScramCredentialInfo(user2ScramMechanism0, 4096), "password"))); + new UserScramCredentialDeletion(user0Name, user0ScramMechanism0), + new UserScramCredentialUpsertion(user1Name, new ScramCredentialInfo(user1ScramMechanism0, 8192), "password"), + new UserScramCredentialUpsertion(user2Name, new ScramCredentialInfo(user2ScramMechanism0, 4096), "password"))); Map> resultData = result.values(); assertEquals(3, resultData.size()); Arrays.asList(user0Name, user1Name).stream().forEach(u -> { @@ -4576,6 +4602,205 @@ public void testAlterUserScramCredentials() throws Exception { } } + @Test + public void testListTransactions() throws Exception { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + MetadataResponseData.MetadataResponseBrokerCollection brokers = + new MetadataResponseData.MetadataResponseBrokerCollection(); + + env.cluster().nodes().forEach(node -> { + brokers.add(new MetadataResponseData.MetadataResponseBroker() + .setHost(node.host()) + .setNodeId(node.id()) + .setPort(node.port()) + .setRack(node.rack()) + ); + }); + + env.kafkaClient().prepareResponse( + request -> request instanceof MetadataRequest, + new MetadataResponse(new MetadataResponseData().setBrokers(brokers)) + ); + + List expected = Arrays.asList( + new TransactionListing("foo", 12345L, TransactionState.ONGOING), + new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT), + new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT) + ); + assertEquals(Utils.mkSet(0, 1, 2), env.cluster().nodes().stream().map(Node::id) + .collect(Collectors.toSet())); + + env.cluster().nodes().forEach(node -> { + ListTransactionsResponseData response = new ListTransactionsResponseData() + .setErrorCode(Errors.NONE.code()); + + TransactionListing listing = expected.get(node.id()); + response.transactionStates().add(new ListTransactionsResponseData.TransactionState() + .setTransactionalId(listing.transactionalId()) + .setProducerId(listing.producerId()) + .setTransactionState(listing.state().toString()) + ); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof ListTransactionsRequest, + new ListTransactionsResponse(response), + node + ); + }); + + ListTransactionsResult result = env.adminClient().listTransactions(); + assertEquals(new HashSet<>(expected), new HashSet<>(result.all().get())); + } + } + + @Test + public void testDescribeTransactions() throws Exception { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + String transactionalId = "foo"; + Node coordinator = env.cluster().nodes().iterator().next(); + TransactionDescription expected = new TransactionDescription( + coordinator.id(), TransactionState.COMPLETE_COMMIT, 12345L, + 15, 10000L, OptionalLong.empty(), emptySet()); + + + env.kafkaClient().prepareResponse( + request -> request instanceof FindCoordinatorRequest, + new FindCoordinatorResponse(new FindCoordinatorResponseData() + .setErrorCode(Errors.NONE.code()) + .setNodeId(coordinator.id()) + .setHost(coordinator.host()) + .setPort(coordinator.port())) + ); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof DescribeTransactionsRequest, + new DescribeTransactionsResponse(new DescribeTransactionsResponseData().setTransactionStates( + singletonList(new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setProducerEpoch(expected.producerEpoch()) + .setProducerId(expected.producerId()) + .setTransactionalId(transactionalId) + .setTransactionTimeoutMs(10000) + .setTransactionStartTimeMs(-1) + .setTransactionState(expected.state().toString()) + ) + )), + coordinator + ); + + DescribeTransactionsResult result = env.adminClient().describeTransactions(singleton(transactionalId)); + KafkaFuture future = result.transactionalIdResult(transactionalId); + assertEquals(expected, future.get()); + } + } + + @Test + public void testAbortTransaction() throws Exception { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 200); + Node leader = env.cluster().nodes().iterator().next(); + + expectMetadataRequest(env, topicPartition, leader); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof WriteTxnMarkersRequest, + new WriteTxnMarkersResponse(singletonMap(abortSpec.producerId(), singletonMap(topicPartition, Errors.NONE))), + leader + ); + + AbortTransactionResult result = env.adminClient().abortTransaction(abortSpec); + assertNull(result.all().get()); + } + } + + @Test + public void testDescribeProducers() throws Exception { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + TopicPartition topicPartition = new TopicPartition("foo", 0); + AbortTransactionSpec abortSpec = new AbortTransactionSpec( + topicPartition, 12345L, 15, 200); + Node leader = env.cluster().nodes().iterator().next(); + + expectMetadataRequest(env, topicPartition, leader); + + DescribeProducersResponseData response = new DescribeProducersResponseData(); + + DescribeProducersResponseData.TopicResponse topicResponse = + new DescribeProducersResponseData.TopicResponse() + .setName(topicPartition.topic()); + response.topics().add(topicResponse); + + DescribeProducersResponseData.PartitionResponse partitionResponse = + new DescribeProducersResponseData.PartitionResponse() + .setPartitionIndex(topicPartition.partition()) + .setErrorCode(Errors.NONE.code()); + topicResponse.partitions().add(partitionResponse); + + List expected = Arrays.asList( + new ProducerState(12345L, 15, 30, env.time().milliseconds(), + OptionalInt.of(99), OptionalLong.empty()), + new ProducerState(12345L, 15, 30, env.time().milliseconds(), + OptionalInt.empty(), OptionalLong.of(23423L)) + ); + + partitionResponse.setActiveProducers(expected.stream().map(producerState -> + new DescribeProducersResponseData.ProducerState() + .setProducerId(producerState.producerId()) + .setProducerEpoch(producerState.producerEpoch()) + .setCoordinatorEpoch(producerState.coordinatorEpoch().orElse(-1)) + .setLastSequence(producerState.lastSequence()) + .setLastTimestamp(producerState.lastTimestamp()) + .setCurrentTxnStartTimestamp(producerState.currentTransactionStartOffset().orElse(-1L)) + ).collect(Collectors.toList())); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof DescribeProducersRequest, + new DescribeProducersResponse(response), + leader + ); + + DescribeProducersResult result = env.adminClient().describeProducers(singleton(topicPartition)); + KafkaFuture partitionFuture = + result.partitionResult(topicPartition); + assertEquals(new HashSet<>(expected), new HashSet<>(partitionFuture.get().activeProducers())); + } + } + + private void expectMetadataRequest( + AdminClientUnitTestEnv env, + TopicPartition topicPartition, + Node leader + ) { + MetadataResponseData.MetadataResponseTopicCollection responseTopics = + new MetadataResponseData.MetadataResponseTopicCollection(); + + MetadataResponseTopic responseTopic = new MetadataResponseTopic() + .setName(topicPartition.topic()) + .setErrorCode(Errors.NONE.code()); + responseTopics.add(responseTopic); + + MetadataResponsePartition responsePartition = new MetadataResponsePartition() + .setErrorCode(Errors.NONE.code()) + .setPartitionIndex(topicPartition.partition()) + .setLeaderId(leader.id()) + .setReplicaNodes(singletonList(leader.id())) + .setIsrNodes(singletonList(leader.id())); + responseTopic.partitions().add(responsePartition); + + env.kafkaClient().prepareResponse( + request -> { + if (!(request instanceof MetadataRequest)) { + return false; + } + MetadataRequest metadataRequest = (MetadataRequest) request; + return metadataRequest.topics().equals(singletonList(topicPartition.topic())); + }, + new MetadataResponse(new MetadataResponseData().setTopics(responseTopics)) + ); + } + private void createAlterLogDirsResponse(AdminClientUnitTestEnv env, Node node, Errors error, int... partitions) { env.kafkaClient().prepareResponseFrom( prepareAlterLogDirsResponse(error, "topic", partitions), node); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java index 6f55496ae6e48..14aaddbb99882 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.DescribeProducersOptions; -import org.apache.kafka.clients.admin.DescribeProducersResult; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; @@ -285,7 +284,7 @@ private List sampleProducerState() { private void assertMatchingProducers( List expected, - List actual + List actual ) { assertEquals(expected.size(), actual.size()); @@ -294,7 +293,7 @@ private void assertMatchingProducers( Function.identity() )); - for (DescribeProducersResult.ProducerState actualProducerState : actual) { + for (org.apache.kafka.clients.admin.ProducerState actualProducerState : actual) { ProducerState expectedProducerState = expectedByProducerId.get(actualProducerState.producerId()); assertNotNull(expectedProducerState); assertEquals(expectedProducerState.producerEpoch(), actualProducerState.producerEpoch()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java index 3ade7eaa21d66..51ceb78047acb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java @@ -243,7 +243,7 @@ private void assertExpectedTransactionState( ) { assertEquals(expected.transactionalId(), actual.transactionalId()); assertEquals(expected.producerId(), actual.producerId()); - assertEquals(expected.transactionState(), actual.transactionState().toString()); + assertEquals(expected.transactionState(), actual.state().toString()); } private void assertListTransactionsRequest( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java index a227b53915920..9c80be989e55c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -164,8 +164,8 @@ public void testRetryLeaderDiscovery() { RequestSpec spec0 = lookupRequest(requests2, tp0); assertEquals(mkSet(tp0), spec0.keys); assertEquals(OptionalInt.empty(), spec0.scope.destinationBrokerId()); - assertTrue(spec0.request instanceof MetadataRequest.Builder); assertExpectedBackoffAndDeadline(spec0, 1); + assertTrue(spec0.request instanceof MetadataRequest.Builder); MetadataRequest.Builder retryMetadataRequest = (MetadataRequest.Builder) spec0.request; assertEquals(mkSet("foo"), new HashSet<>(retryMetadataRequest.topics())); diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java index 52ea4addbc76f..0c2d2b80cfc4d 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java @@ -28,6 +28,7 @@ import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult; +import org.apache.kafka.clients.admin.ProducerState; import org.apache.kafka.clients.admin.TransactionListing; import org.apache.kafka.clients.admin.TransactionDescription; import org.apache.kafka.common.TopicPartition; @@ -152,7 +153,7 @@ private AbortTransactionSpec buildAbortSpec( return null; } - Optional foundProducerState = result.activeProducers().stream() + Optional foundProducerState = result.activeProducers().stream() .filter(producerState -> { OptionalLong txnStartOffsetOpt = producerState.currentTransactionStartOffset(); return txnStartOffsetOpt.isPresent() && txnStartOffsetOpt.getAsLong() == startOffset; @@ -165,7 +166,7 @@ private AbortTransactionSpec buildAbortSpec( return null; } - DescribeProducersResult.ProducerState producerState = foundProducerState.get(); + ProducerState producerState = foundProducerState.get(); return new AbortTransactionSpec( topicPartition, producerState.producerId(), @@ -452,7 +453,7 @@ public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception listing.transactionalId(), coordinatorIdString, String.valueOf(listing.producerId()), - listing.transactionState().toString() + listing.state().toString() }); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java index 48d55636173fd..d85989ab65f33 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; -import org.apache.kafka.clients.admin.DescribeProducersResult.ProducerState; +import org.apache.kafka.clients.admin.ProducerState; import org.apache.kafka.clients.admin.DescribeTransactionsResult; import org.apache.kafka.clients.admin.ListTransactionsResult; import org.apache.kafka.clients.admin.TransactionDescription; From 8b61934d154f36bb4bb67938c972ba97d4eb252d Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 10:14:29 -0700 Subject: [PATCH 08/17] A few extra cleanups --- .../DescribeProducersRequestDriver.java | 4 +- .../requests/DescribeProducersRequest.java | 5 + .../requests/DescribeProducersResponse.java | 5 + .../requests/DescribeTransactionsRequest.java | 5 + .../DescribeTransactionsResponse.java | 5 + .../requests/ListTransactionsRequest.java | 5 + .../requests/ListTransactionsResponse.java | 5 + .../message/DescribeProducersResponse.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 2 +- .../DescribeProducersRequestDriverTest.java | 6 +- .../common/requests/RequestResponseTest.java | 16 +- core/src/main/scala/kafka/log/Log.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 172 ++++++++++++++++++ .../tools/PrintVersionAndExitAction.java | 3 +- .../kafka/tools/TransactionsCommand.java | 4 +- 16 files changed, 223 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java index 65a8818abed1c..fa27f274d32bd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -132,9 +132,9 @@ void handleFulfillmentResponse(Integer brokerId, Set keys, Abstr List activeProducers = partitionResponse.activeProducers().stream() .map(activeProducer -> { OptionalLong currentTransactionFirstOffset = - activeProducer.currentTxnStartTimestamp() < 0 ? + activeProducer.currentTxnStartOffset() < 0 ? OptionalLong.empty() : - OptionalLong.of(activeProducer.currentTxnStartTimestamp()); + OptionalLong.of(activeProducer.currentTxnStartOffset()); OptionalInt coordinatorEpoch = activeProducer.coordinatorEpoch() < 0 ? OptionalInt.empty() : diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java index 563fda788fc07..acc9a7e3c2400 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersRequest.java @@ -100,4 +100,9 @@ public static DescribeProducersRequest parse(ByteBuffer buffer, short version) { return new DescribeProducersRequest(ApiKeys.DESCRIBE_PRODUCERS.parseRequest(version, buffer), version); } + @Override + public String toString(boolean verbose) { + return data.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java index 2f043b7b15779..b80fdbc9ec8ea 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java @@ -63,4 +63,9 @@ public static DescribeProducersResponse parse(ByteBuffer buffer, short version) ApiKeys.DESCRIBE_PRODUCERS.responseSchema(version).read(buffer), version); } + @Override + public String toString() { + return data.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java index 71619ca750915..1bfee824d360b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsRequest.java @@ -86,4 +86,9 @@ public static DescribeTransactionsRequest parse(ByteBuffer buffer, short version return new DescribeTransactionsRequest(ApiKeys.DESCRIBE_TRANSACTIONS.parseRequest(version, buffer), version); } + @Override + public String toString(boolean verbose) { + return data.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java index c6f6f1991b956..bf40ac0ea7135 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java @@ -61,4 +61,9 @@ public static DescribeTransactionsResponse parse(ByteBuffer buffer, short versio ApiKeys.DESCRIBE_TRANSACTIONS.responseSchema(version).read(buffer), version); } + @Override + public String toString() { + return data.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java index 2e30c7260566d..576a77ff172da 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java @@ -80,4 +80,9 @@ public static ListTransactionsRequest parse(ByteBuffer buffer, short version) { return new ListTransactionsRequest(ApiKeys.LIST_TRANSACTIONS.parseRequest(version, buffer), version); } + @Override + public String toString(boolean verbose) { + return data.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java index 022f2d8f87f2e..7c13f52481659 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java @@ -57,4 +57,9 @@ public static ListTransactionsResponse parse(ByteBuffer buffer, short version) { ApiKeys.LIST_TRANSACTIONS.responseSchema(version).read(buffer), version); } + @Override + public String toString() { + return data.toString(); + } + } diff --git a/clients/src/main/resources/common/message/DescribeProducersResponse.json b/clients/src/main/resources/common/message/DescribeProducersResponse.json index 6590dc4004962..d9284166800f0 100644 --- a/clients/src/main/resources/common/message/DescribeProducersResponse.json +++ b/clients/src/main/resources/common/message/DescribeProducersResponse.json @@ -38,7 +38,7 @@ { "name": "LastSequence", "type": "int32", "versions": "0+", "default": "-1" }, { "name": "LastTimestamp", "type": "int64", "versions": "0+", "default": "-1" }, { "name": "CoordinatorEpoch", "type": "int32", "versions": "0+" }, - { "name": "CurrentTxnStartTimestamp", "type": "int64", "versions": "0+", "default": "-1" } + { "name": "CurrentTxnStartOffset", "type": "int64", "versions": "0+", "default": "-1" } ]} ]} ]} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 7659bb44a1e79..4334bf6bb9eea 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -4752,7 +4752,7 @@ public void testDescribeProducers() throws Exception { .setCoordinatorEpoch(producerState.coordinatorEpoch().orElse(-1)) .setLastSequence(producerState.lastSequence()) .setLastTimestamp(producerState.lastTimestamp()) - .setCurrentTxnStartTimestamp(producerState.currentTransactionStartOffset().orElse(-1L)) + .setCurrentTxnStartOffset(producerState.currentTransactionStartOffset().orElse(-1L)) ).collect(Collectors.toList())); env.kafkaClient().prepareResponseFrom( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java index 14aaddbb99882..39d793e310b4c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java @@ -272,13 +272,13 @@ private List sampleProducerState() { .setProducerEpoch(15) .setLastSequence(75) .setLastTimestamp(time.milliseconds()) - .setCurrentTxnStartTimestamp(-1L), + .setCurrentTxnStartOffset(-1L), new ProducerState() .setProducerId(98765L) .setProducerEpoch(30) .setLastSequence(150) .setLastTimestamp(time.milliseconds()) - .setCurrentTxnStartTimestamp(time.milliseconds()) + .setCurrentTxnStartOffset(5000) ); } @@ -299,7 +299,7 @@ private void assertMatchingProducers( assertEquals(expectedProducerState.producerEpoch(), actualProducerState.producerEpoch()); assertEquals(expectedProducerState.lastSequence(), actualProducerState.lastSequence()); assertEquals(expectedProducerState.lastTimestamp(), actualProducerState.lastTimestamp()); - assertEquals(expectedProducerState.currentTxnStartTimestamp(), + assertEquals(expectedProducerState.currentTxnStartOffset(), actualProducerState.currentTransactionStartOffset().orElse(-1L)); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 8f01a460dca47..656ecc49194c7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -2424,7 +2424,7 @@ private DescribeProducersResponse createDescribeProducersResponse() { .setProducerId(1234L) .setProducerEpoch(15) .setLastTimestamp(13490218304L) - .setCurrentTxnStartTimestamp(13490218304L), + .setCurrentTxnStartOffset(5000), new DescribeProducersResponseData.ProducerState() .setProducerId(9876L) .setProducerEpoch(32) @@ -2444,13 +2444,13 @@ private DescribeTransactionsRequest createDescribeTransactionsRequest() { private DescribeTransactionsResponse createDescribeTransactionsResponse() { DescribeTransactionsResponseData data = new DescribeTransactionsResponseData(); data.setTransactionStates(asList( - new DescribeTransactionsResponseData.TransactionState() - .setErrorCode(Errors.NONE.code()) - .setTransactionalId("t1") - .setProducerId(12345L) - .setProducerEpoch(15) - .setTransactionStartTimeMs(13490218304L) - .setTransactionState("Empty"), + new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code()) + .setTransactionalId("t1") + .setProducerId(12345L) + .setProducerEpoch(15) + .setTransactionStartTimeMs(13490218304L) + .setTransactionState("Empty"), new DescribeTransactionsResponseData.TransactionState() .setErrorCode(Errors.NONE.code()) .setTransactionalId("t2") diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index d8553ed4547be..8c394fe722ac3 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -951,7 +951,7 @@ class Log(@volatile private var _dir: File, .setProducerEpoch(state.producerEpoch) .setLastSequence(state.lastSeq) .setLastTimestamp(state.lastTimestamp) - .setCurrentTxnStartTimestamp(state.currentTxnFirstOffset.getOrElse(-1L)) + .setCurrentTxnStartOffset(state.currentTxnFirstOffset.getOrElse(-1L)) } }.toSeq } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index cb3d476d45f24..296193457f5ec 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3069,7 +3069,7 @@ class KafkaApis(val requestChannel: RequestChannel, describeProducersRequest.data.topics.forEach { topicRequest => val topicResponse = new DescribeProducersResponseData.TopicResponse() .setName(topicRequest.name) - val topicError = if (!authorize(request.context, DESCRIBE, TOPIC, topicRequest.name)) + val topicError = if (!authorize(request.context, READ, TOPIC, topicRequest.name)) Some(Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicRequest.name)) Some(Errors.UNKNOWN_TOPIC_OR_PARTITION) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index eec57a4295ff0..182631adeadee 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2349,4 +2349,176 @@ class KafkaApisTest { Errors.LOG_DIR_NOT_FOUND -> 1, Errors.INVALID_TOPIC_EXCEPTION -> 1).asJava, response.errorCounts) } + + @Test + def testListTransactionsAuthorization(): Unit = { + val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) + val data = new ListTransactionsRequestData() + val listTransactionsRequest = new ListTransactionsRequest.Builder(data).build() + val request = buildRequest(listTransactionsRequest) + val capturedResponse = expectNoThrottling() + + EasyMock.expect(txnCoordinator.handleListTransactions(Set.empty[Long], Set.empty[String])) + .andReturn(Right(List( + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("foo") + .setProducerId(12345L) + .setTransactionState("Ongoing"), + new ListTransactionsResponseData.TransactionState() + .setTransactionalId("bar") + .setProducerId(98765) + .setTransactionState("PrepareAbort") + ))) + + def buildExpectedActions(transactionalId: String): util.List[Action] = { + val pattern = new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL) + val action = new Action(AclOperation.DESCRIBE, pattern, 1, true, true) + Collections.singletonList(action) + } + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions("foo")))) + .andReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .once() + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions("bar")))) + .andReturn(Seq(AuthorizationResult.DENIED).asJava) + .once() + + EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, txnCoordinator, authorizer) + createKafkaApis(authorizer = Some(authorizer)).handleListTransactionsRequest(request) + + val response = readResponse(ApiKeys.LIST_TRANSACTIONS, listTransactionsRequest, capturedResponse) + .asInstanceOf[ListTransactionsResponse] + assertEquals(1, response.data.transactionStates.size()) + val transactionState = response.data.transactionStates.get(0) + assertEquals("foo", transactionState.transactionalId) + assertEquals(12345L, transactionState.producerId) + assertEquals("Ongoing", transactionState.transactionState) + } + + @Test + def testDescribeTransactionsAuthorization(): Unit = { + val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) + val data = new DescribeTransactionsRequestData() + .setTransactionalIds(List("foo", "bar").asJava) + val describeTransactionsRequest = new DescribeTransactionsRequest.Builder(data).build() + val request = buildRequest(describeTransactionsRequest) + val capturedResponse = expectNoThrottling() + + def buildExpectedActions(transactionalId: String): util.List[Action] = { + val pattern = new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL) + val action = new Action(AclOperation.DESCRIBE, pattern, 1, true, true) + Collections.singletonList(action) + } + + EasyMock.expect(txnCoordinator.handleDescribeTransactions("foo")) + .andReturn(new DescribeTransactionsResponseData.TransactionState() + .setErrorCode(Errors.NONE.code) + .setTransactionalId("foo") + .setProducerId(12345L) + .setProducerEpoch(15) + .setTransactionStartTimeMs(time.milliseconds()) + .setTransactionState("Ongoing") + .setTopicPartitions(List.empty.asJava) + .setTransactionTimeoutMs(10000)) + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions("foo")))) + .andReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .once() + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions("bar")))) + .andReturn(Seq(AuthorizationResult.DENIED).asJava) + .once() + + EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, txnCoordinator, authorizer) + createKafkaApis(authorizer = Some(authorizer)).handleDescribeTransactionsRequest(request) + + val response = readResponse(ApiKeys.DESCRIBE_TRANSACTIONS, describeTransactionsRequest, capturedResponse) + .asInstanceOf[DescribeTransactionsResponse] + assertEquals(2, response.data.transactionStates.size()) + + val fooState = response.data.transactionStates.asScala.find(_.transactionalId == "foo").get + assertEquals(Errors.NONE.code, fooState.errorCode) + assertEquals(12345L, fooState.producerId) + assertEquals(15, fooState.producerEpoch) + assertEquals(time.milliseconds(), fooState.transactionStartTimeMs) + assertEquals("Ongoing", fooState.transactionState) + assertEquals(10000, fooState.transactionTimeoutMs) + assertEquals(List.empty.asJava, fooState.topicPartitions) + + val barState = response.data.transactionStates.asScala.find(_.transactionalId == "bar").get + assertEquals(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.code, barState.errorCode) + } + + @Test + def testDescribeProducersAuthorization(): Unit = { + val tp1 = new TopicPartition("foo", 0) + val tp2 = new TopicPartition("bar", 3) + + setupBasicMetadataCache(tp1.topic, 4) // We will only access the first topic + val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) + val data = new DescribeProducersRequestData().setTopics(List( + new DescribeProducersRequestData.TopicRequest() + .setName(tp1.topic) + .setPartitionIndexes(List(Int.box(tp1.partition)).asJava), + new DescribeProducersRequestData.TopicRequest() + .setName(tp2.topic) + .setPartitionIndexes(List(Int.box(tp2.partition)).asJava) + ).asJava) + val describeProducersRequest = new DescribeProducersRequest.Builder(data).build() + val request = buildRequest(describeProducersRequest) + val capturedResponse = expectNoThrottling() + + def buildExpectedActions(topic: String): util.List[Action] = { + val pattern = new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL) + val action = new Action(AclOperation.READ, pattern, 1, true, true) + Collections.singletonList(action) + } + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions(tp1.topic)))) + .andReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .once() + + EasyMock.expect(authorizer.authorize(anyObject[RequestContext], EasyMock.eq(buildExpectedActions(tp2.topic)))) + .andReturn(Seq(AuthorizationResult.DENIED).asJava) + .once() + + EasyMock.expect(replicaManager.activeProducerState(tp1)) + .andReturn(new DescribeProducersResponseData.PartitionResponse() + .setErrorCode(Errors.NONE.code) + .setPartitionIndex(tp1.partition) + .setActiveProducers(List( + new DescribeProducersResponseData.ProducerState() + .setProducerId(12345L) + .setProducerEpoch(15) + .setLastSequence(100) + .setLastTimestamp(time.milliseconds()) + .setCurrentTxnStartOffset(-1) + .setCoordinatorEpoch(200) + ).asJava)) + + EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, txnCoordinator, authorizer) + createKafkaApis(authorizer = Some(authorizer)).handleDescribeProducersRequest(request) + + val response = readResponse(ApiKeys.DESCRIBE_PRODUCERS, describeProducersRequest, capturedResponse) + .asInstanceOf[DescribeProducersResponse] + assertEquals(2, response.data.topics.size()) + + val fooTopic = response.data.topics.asScala.find(_.name == tp1.topic).get + val fooPartition = fooTopic.partitions.asScala.find(_.partitionIndex == tp1.partition).get + assertEquals(Errors.NONE.code, fooPartition.errorCode) + assertEquals(1, fooPartition.activeProducers.size) + val fooProducer = fooPartition.activeProducers.get(0) + assertEquals(12345L, fooProducer.producerId) + assertEquals(15, fooProducer.producerEpoch) + assertEquals(100, fooProducer.lastSequence) + assertEquals(time.milliseconds(), fooProducer.lastTimestamp) + assertEquals(-1, fooProducer.currentTxnStartOffset) + assertEquals(200, fooProducer.coordinatorEpoch) + + val barTopic = response.data.topics.asScala.find(_.name == tp2.topic).get + val barPartition = barTopic.partitions.asScala.find(_.partitionIndex == tp2.partition).get + assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, barPartition.errorCode) + } + } diff --git a/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java b/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java index b52dfd454db61..6846c7fab5b3e 100644 --- a/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java +++ b/tools/src/main/java/org/apache/kafka/tools/PrintVersionAndExitAction.java @@ -20,6 +20,7 @@ import net.sourceforge.argparse4j.inf.ArgumentAction; import net.sourceforge.argparse4j.inf.ArgumentParser; import org.apache.kafka.common.utils.AppInfoParser; +import org.apache.kafka.common.utils.Exit; import java.util.Map; @@ -36,7 +37,7 @@ public void run( String version = AppInfoParser.getVersion(); String commitId = AppInfoParser.getCommitId(); System.out.println(version + " (Commit:" + commitId + ")"); - System.exit(0); + Exit.exit(0); } @Override diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java index 0c2d2b80cfc4d..0dc699b9b8a6c 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java @@ -180,7 +180,7 @@ private void abortTransaction( AbortTransactionSpec abortSpec ) throws Exception { try { - admin.abortTransaction(abortSpec).all().get(); + admin.abortTransaction(abortSpec).all().get(); } catch (ExecutionException e) { TransactionsCommand.printErrorAndExit("Failed to abort transaction " + abortSpec, e.getCause()); } @@ -424,7 +424,7 @@ public void addSubparser(Subparsers subparsers) { } @Override - public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception{ + public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { final Map> result; try { From 4040d29fa3a032a2b659c422a6226b6a94100cfa Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 10:34:29 -0700 Subject: [PATCH 09/17] Fix breakage after rebase --- .../org/apache/kafka/common/protocol/Errors.java | 2 +- .../kafka/clients/admin/KafkaAdminClientTest.java | 15 +++------------ 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index eb52ce1703716..fca67aa866d05 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -333,7 +333,7 @@ public enum Errors { RESOURCE_NOT_FOUND(91, "A request illegally referred to a resource that does not exist.", ResourceNotFoundException::new), DUPLICATE_RESOURCE(92, "A request illegally referred to the same resource twice.", DuplicateResourceException::new), UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new), - TRANSACTIONAL_ID_NOT_FOUND(91, "The transactionalId could not be found", TransactionalIdNotFoundException::new); + TRANSACTIONAL_ID_NOT_FOUND(94, "The transactionalId could not be found", TransactionalIdNotFoundException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 4334bf6bb9eea..93fe020bca0d8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -70,11 +70,8 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult; -<<<<<<< HEAD import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; -======= import org.apache.kafka.common.message.CreateAclsResponseData; ->>>>>>> Add base `KafkaAdminClient` test cases import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsResponseData; @@ -94,13 +91,10 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic; -<<<<<<< HEAD -import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; -import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo; -======= import org.apache.kafka.common.message.DescribeProducersResponseData; import org.apache.kafka.common.message.DescribeTransactionsResponseData; ->>>>>>> Add base `KafkaAdminClient` test cases +import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; +import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.FindCoordinatorResponseData; @@ -146,14 +140,11 @@ import org.apache.kafka.common.requests.DescribeConfigsResponse; import org.apache.kafka.common.requests.DescribeGroupsResponse; import org.apache.kafka.common.requests.DescribeLogDirsResponse; -<<<<<<< HEAD -import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse; -======= import org.apache.kafka.common.requests.DescribeProducersRequest; import org.apache.kafka.common.requests.DescribeProducersResponse; import org.apache.kafka.common.requests.DescribeTransactionsRequest; import org.apache.kafka.common.requests.DescribeTransactionsResponse; ->>>>>>> Add base `KafkaAdminClient` test cases +import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse; import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; From 422839e1a33a936424f2fc55596ae08f5e6e81b6 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 10:49:25 -0700 Subject: [PATCH 10/17] Add Admin documentation --- .../org/apache/kafka/clients/admin/Admin.java | 111 ++++++++++++++---- 1 file changed, 87 insertions(+), 24 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 9db7076292e55..38beecbe0ccf0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1215,30 +1215,6 @@ default AlterClientQuotasResult alterClientQuotas(Collection entries, AlterClientQuotasOptions options); - default DescribeProducersResult describeProducers(Collection partitions) { - return describeProducers(partitions, new DescribeProducersOptions()); - } - - DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options); - - default DescribeTransactionsResult describeTransactions(Collection transactionalIds) { - return describeTransactions(transactionalIds, new DescribeTransactionsOptions()); - } - - DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options); - - default ListTransactionsResult listTransactions() { - return listTransactions(new ListTransactionsOptions()); - } - - ListTransactionsResult listTransactions(ListTransactionsOptions options); - - default AbortTransactionResult abortTransaction(AbortTransactionSpec spec) { - return abortTransaction(spec, new AbortTransactionOptions()); - } - - AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options); - /** * Describe all SASL/SCRAM credentials. * @@ -1330,6 +1306,93 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options); + /** + * Describe producer state on a set of topic partitions. See + * {@link #describeProducers(Collection, DescribeProducersOptions)} for more details. + * + * @param partitions The set of partitions to query + * @return The result + */ + default DescribeProducersResult describeProducers(Collection partitions) { + return describeProducers(partitions, new DescribeProducersOptions()); + } + + /** + * Describe active producer state on a set of topic partitions. Unless a specific broker + * is requested through {@link DescribeProducersOptions#setBrokerId(int)}, this will + * query the partition leader to find the producer state. + * + * @param partitions The set of partitions to query + * @param options Options to control the method behavior + * @return The result + */ + DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options); + + /** + * Describe the state of a set of transactionalIds. See + * {@link #describeTransactions(Collection, DescribeTransactionsOptions)} for more details. + * + * @param transactionalIds The set of transactionalIds to query + * @return The result + */ + default DescribeTransactionsResult describeTransactions(Collection transactionalIds) { + return describeTransactions(transactionalIds, new DescribeTransactionsOptions()); + } + + /** + * Describe the state of a set of transactionalIds from the respective transaction coordinators, + * which are dynamically discovered. + * + * @param transactionalIds The set of transactionalIds to query + * @param options Options to control the method behavior + * @return The result + */ + DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options); + + /** + * List active transactions in the cluster. See + * {@link #listTransactions(ListTransactionsOptions)} for more details. + * + * @return The result + */ + default ListTransactionsResult listTransactions() { + return listTransactions(new ListTransactionsOptions()); + } + + /** + * List active transactions in the cluster. This will query all potential transaction + * coordinators in the cluster and collect the state of all transactionalIds. Users + * should typically attempt to reduce the size of the result set using + * {@link ListTransactionsOptions#filterProducerIds(Set)} or + * {@link ListTransactionsOptions#filterStates(Set)} + * + * @param options Options to control the method behavior (including filters) + * @return The result + */ + ListTransactionsResult listTransactions(ListTransactionsOptions options); + + /** + * Forcefully abort a transaction which is open on a topic partition. See + * {@link #abortTransaction(AbortTransactionSpec, AbortTransactionOptions)} for more details. + * + * @param spec The transaction specification including topic partition and producer details + * @return The result + */ + default AbortTransactionResult abortTransaction(AbortTransactionSpec spec) { + return abortTransaction(spec, new AbortTransactionOptions()); + } + + /** + * Forcefully abort a transaction which is open on a topic partition. This will + * send a `WriteTxnMarkers` request to the partition leader in order to abort the + * transaction. This requires administrative privileges. + * + * @param spec The transaction specification including topic partition and producer details + * @param options Options to control the method behavior (including filters) + * @return The result + */ + AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options); + /** * Get the metrics kept by the adminClient */ From 54dff73b01007e378c244e16346a4fdf5f26c841 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 10:55:58 -0700 Subject: [PATCH 11/17] Fix typo in shell script --- bin/kafka-transactions.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-transactions.sh b/bin/kafka-transactions.sh index 936eea72d79cb..6fb523385557f 100755 --- a/bin/kafka-transactions.sh +++ b/bin/kafka-transactions.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.TransactionCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.TransactionsCommand "$@" From 01e774cf67a039edd43f916741075a8c735f2582 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 12:45:02 -0700 Subject: [PATCH 12/17] Fix comment --- .../kafka/clients/admin/internals/RequestDriver.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java index 884524c0fae59..f34a276fecc43 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -103,10 +103,10 @@ private void initializeLookupKeys() { } /** - * Check whether the . This is useful when a response contains more partitions - * than are strictly needed. For example, a `Metadata` response always includes - * all partitions for each requested topic, even if we are only interested in - * a subset of them. + * Check whether a particular key has been requested. This is useful when a response + * contains more partitions than are strictly needed. For example, a `Metadata` + * response always includes all partitions for each requested topic, even if we are + * only interested in a subset of them. */ boolean contains(K key) { return futures.containsKey(key); From 036e8b40eefed0dba47b2e9cbb2d5712f4b66e47 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 14:00:11 -0700 Subject: [PATCH 13/17] Use log context consistently --- .../kafka/clients/admin/KafkaAdminClient.java | 11 ++++--- .../AbortTransactionRequestDriver.java | 2 +- .../internals/AllBrokerRequestDriver.java | 15 ++++++--- .../internals/CoordinatorRequestDriver.java | 10 +++--- .../DescribeProducersRequestDriver.java | 10 +++--- .../DescribeTransactionsRequestDriver.java | 15 ++++++--- .../ListTransactionsRequestDriver.java | 12 +++---- .../internals/MetadataRequestDriver.java | 11 ++++--- .../admin/internals/RequestDriver.java | 7 ++-- .../CoordinatorRequestDriverTest.java | 3 +- .../DescribeProducersRequestDriverTest.java | 32 ++++--------------- ...DescribeTransactionsRequestDriverTest.java | 27 ++++------------ .../ListTransactionsRequestDriverTest.java | 14 +++++--- .../internals/MetadataRequestDriverTest.java | 3 +- 14 files changed, 86 insertions(+), 86 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 9102c726a08c8..e1f970f7f6a4c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -4290,7 +4290,7 @@ public DescribeProducersResult describeProducers(Collection part long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - partitions, options, deadlineMs, retryBackoffMs); + partitions, options, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new DescribeProducersResult(driver.futures()); } @@ -4303,7 +4303,7 @@ public DescribeTransactionsResult describeTransactions(Collection transa long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - transactionalIds, deadlineMs, retryBackoffMs); + transactionalIds, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new DescribeTransactionsResult(driver.futures()); } @@ -4312,7 +4312,8 @@ public DescribeTransactionsResult describeTransactions(Collection transa public ListTransactionsResult listTransactions(ListTransactionsOptions options) { long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + options, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new ListTransactionsResult(driver.lookupFuture()); } @@ -4321,8 +4322,8 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(spec, deadlineMs, - retryBackoffMs, logContext); + AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver( + spec, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new AbortTransactionResult(driver.futures()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java index 53bdc4fe8c327..ded49b8fef12d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java @@ -47,7 +47,7 @@ public AbortTransactionRequestDriver( long retryBackoffMs, LogContext logContext ) { - super(singleton(abortSpec.topicPartition()), deadlineMs, retryBackoffMs); + super(singleton(abortSpec.topicPartition()), deadlineMs, retryBackoffMs, logContext); this.abortSpec = abortSpec; this.topicPartition = abortSpec.topicPartition(); this.log = logContext.logger(AbortTransactionRequestDriver.class); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java index 60caec8322711..7ce23d8359fec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java @@ -23,9 +23,9 @@ import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.Map; @@ -40,18 +40,23 @@ * @param */ public abstract class AllBrokerRequestDriver extends RequestDriver { - private static final Logger log = LoggerFactory.getLogger(AllBrokerRequestDriver.class); - private static final BrokerKey ALL_BROKERS = new BrokerKey(OptionalInt.empty()); private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { }; + private final Logger log; private final KafkaFutureImpl>> lookupFuture; - public AllBrokerRequestDriver(long deadlineMs, long retryBackoffMs) { - super(Utils.mkSet(ALL_BROKERS), deadlineMs, retryBackoffMs); + public AllBrokerRequestDriver( + long deadlineMs, + long retryBackoffMs, + LogContext logContext + ) { + super(Utils.mkSet(ALL_BROKERS), deadlineMs, retryBackoffMs, logContext); this.lookupFuture = new KafkaFutureImpl<>(); + this.log = logContext.logger(AllBrokerRequestDriver.class); + super.futures().get(ALL_BROKERS).whenComplete((nil, exception) -> { if (exception != null) { this.lookupFuture.completeExceptionally(exception); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java index a5bca35a0f96d..fa32dc008a4b6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java @@ -23,22 +23,24 @@ import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Objects; import java.util.Set; public abstract class CoordinatorRequestDriver extends RequestDriver { - private static final Logger log = LoggerFactory.getLogger(CoordinatorRequestDriver.class); + private final Logger log; public CoordinatorRequestDriver( Collection keys, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { - super(keys, deadlineMs, retryBackoffMs); + super(keys, deadlineMs, retryBackoffMs, logContext); + this.log = logContext.logger(CoordinatorRequestDriver.class); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java index fa27f274d32bd..9addd9606ac18 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.requests.DescribeProducersRequest; import org.apache.kafka.common.requests.DescribeProducersResponse; import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,18 +42,19 @@ import java.util.stream.Collectors; public class DescribeProducersRequestDriver extends MetadataRequestDriver { - private static final Logger log = LoggerFactory.getLogger(DescribeProducersRequestDriver.class); - + private final Logger log; private final DescribeProducersOptions options; public DescribeProducersRequestDriver( Collection topicPartitions, DescribeProducersOptions options, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { - super(topicPartitions, deadlineMs, retryBackoffMs); + super(topicPartitions, deadlineMs, retryBackoffMs, logContext); this.options = options; + this.log = logContext.logger(DescribeProducersRequestDriver.class); // If the request options indicate a specific target broker, then we directly // map the topic partitions to avoid the unneeded `Metadata` lookup. diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java index 6859c64c523ba..f36344e2ea613 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java @@ -29,6 +29,7 @@ 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.utils.LogContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,16 +41,22 @@ import java.util.stream.Collectors; public class DescribeTransactionsRequestDriver extends CoordinatorRequestDriver { - private static final Logger log = LoggerFactory.getLogger(DescribeTransactionsRequestDriver.class); + private final Logger log; public DescribeTransactionsRequestDriver( Collection transactionalIds, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { - super(transactionalIds.stream() + super(buildKeySet(transactionalIds), deadlineMs, retryBackoffMs, logContext); + this.log = logContext.logger(DescribeTransactionsRequestDriver.class); + } + + private static Set buildKeySet(Collection transactionalIds) { + return transactionalIds.stream() .map(DescribeTransactionsRequestDriver::asCoordinatorKey) - .collect(Collectors.toSet()), deadlineMs, retryBackoffMs); + .collect(Collectors.toSet()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java index 5fe05ba807d68..f58a9f9ebf105 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java @@ -26,8 +26,8 @@ import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.ListTransactionsRequest; import org.apache.kafka.common.requests.ListTransactionsResponse; +import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; @@ -35,18 +35,18 @@ import java.util.stream.Collectors; public class ListTransactionsRequestDriver extends AllBrokerRequestDriver> { - // TODO: Use `LogContext` - private static final Logger log = LoggerFactory.getLogger(ListTransactionsRequestDriver.class); - + private final Logger log; private final ListTransactionsOptions options; public ListTransactionsRequestDriver( ListTransactionsOptions options, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { - super(deadlineMs, retryBackoffMs); + super(deadlineMs, retryBackoffMs, logContext); this.options = options; + this.log = logContext.logger(ListTransactionsRequestDriver.class); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java index e41eec93cbc7e..9cf29b1989bd5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; @@ -35,16 +35,19 @@ import java.util.function.Function; public abstract class MetadataRequestDriver extends RequestDriver { - private static final Logger log = LoggerFactory.getLogger(MetadataRequestDriver.class); private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { }; + private final Logger log; + public MetadataRequestDriver( Collection futures, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { - super(futures, deadlineMs, retryBackoffMs); + super(futures, deadlineMs, retryBackoffMs, logContext); + this.log = logContext.logger(MetadataRequestDriver.class); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java index f34a276fecc43..de21c343f8458 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,8 +76,8 @@ * when the key type is a consumer `GroupId`) */ public abstract class RequestDriver { - private static final Logger log = LoggerFactory.getLogger(RequestDriver.class); + private final Logger log; private final long retryBackoffMs; private final long deadlineMs; private final Map> futures; @@ -88,11 +89,13 @@ public abstract class RequestDriver { public RequestDriver( Collection keys, long deadlineMs, - long retryBackoffMs + long retryBackoffMs, + LogContext logContext ) { this.futures = Utils.initializeMap(keys, KafkaFutureImpl::new); this.deadlineMs = deadlineMs; this.retryBackoffMs = retryBackoffMs; + this.log = logContext.logger(RequestDriver.class); initializeLookupKeys(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java index 316fd20faaa8a..67d7d813110f1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java @@ -27,6 +27,7 @@ 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.apache.kafka.common.utils.MockTime; import org.junit.Test; @@ -194,7 +195,7 @@ private RequestSpec lookupRequest( private final class TestCoordinatorRequestDriver extends CoordinatorRequestDriver { public TestCoordinatorRequestDriver(Set groupIds) { - super(groupIds, deadlineMs, retryBackoffMs); + super(groupIds, deadlineMs, retryBackoffMs, new LogContext()); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java index 39d793e310b4c..fdef00b275d67 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.junit.Test; @@ -56,6 +57,7 @@ import static org.junit.Assert.assertTrue; public class DescribeProducersRequestDriverTest { + private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; private final long retryBackoffMs = 100; @@ -67,11 +69,7 @@ public void testSuccessfulResponseWithoutProvidedBrokerId() throws Exception { DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(topicPartition), - options, - deadlineMs, - retryBackoffMs - ); + singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, leaderId, 0); @@ -92,11 +90,7 @@ public void testRetryLookupAfterNotLeaderErrorWithoutProvidedBrokerId() { DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(topicPartition), - options, - deadlineMs, - retryBackoffMs - ); + singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, initialLeaderId, 0); @@ -121,11 +115,7 @@ public void testSuccessfulResponseWithProvidedBrokerId() throws Exception { DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(topicPartition), - options, - deadlineMs, - retryBackoffMs - ); + singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); List> requests = driver.poll(); assertEquals(1, requests.size()); @@ -152,11 +142,7 @@ public void testNotLeaderErrorWithProvidedBrokerId() { DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(topicPartition), - options, - deadlineMs, - retryBackoffMs - ); + singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); List> requests = driver.poll(); assertEquals(1, requests.size()); @@ -183,11 +169,7 @@ public void testFatalErrorWithoutProvidedBrokerId() { DescribeProducersOptions options = new DescribeProducersOptions(); DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( - singleton(topicPartition), - options, - deadlineMs, - retryBackoffMs - ); + singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, leaderId, 0); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java index d052bc9a70cca..979cc2377ec56 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.requests.DescribeTransactionsResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.junit.Test; @@ -49,6 +50,7 @@ import static org.junit.Assert.assertTrue; public class DescribeTransactionsRequestDriverTest { + private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; private final long retryBackoffMs = 100; @@ -60,10 +62,7 @@ public void testDescribeTransactions() throws Exception { Set transactionalIds = mkSet(transactionalId1, transactionalId2); DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - transactionalIds, - deadlineMs, - retryBackoffMs - ); + transactionalIds, deadlineMs, retryBackoffMs, logContext); // Send `FindCoordinator` requests List> lookupRequests = driver.poll(); @@ -134,10 +133,7 @@ public void testDescribeTransactionsBatching() throws Exception { Set transactionalIds = mkSet(transactionalId1, transactionalId2); DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - transactionalIds, - deadlineMs, - retryBackoffMs - ); + transactionalIds, deadlineMs, retryBackoffMs, logContext); // Send `FindCoordinator` requests List> lookupRequests = driver.poll(); @@ -200,10 +196,7 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() String transactionalId = "foo"; DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - singleton(transactionalId), - deadlineMs, - retryBackoffMs - ); + singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request List> lookupRequests = driver.poll(); @@ -245,10 +238,7 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { String transactionalId = "foo"; DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - singleton(transactionalId), - deadlineMs, - retryBackoffMs - ); + singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request List> lookupRequests1 = driver.poll(); @@ -288,10 +278,7 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() String transactionalId = "foo"; DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( - singleton(transactionalId), - deadlineMs, - retryBackoffMs - ); + singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request List> lookupRequests1 = driver.poll(); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java index 51ceb78047acb..2fe7d0e600b11 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.common.requests.ListTransactionsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.junit.Test; @@ -56,6 +57,7 @@ import static org.junit.Assert.assertTrue; public class ListTransactionsRequestDriverTest { + private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; private final long retryBackoffMs = 100; @@ -63,7 +65,8 @@ public class ListTransactionsRequestDriverTest { @Test public void testFailedMetadataRequest() { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + options, deadlineMs, retryBackoffMs, logContext); KafkaFutureImpl>>> lookupFuture = driver.lookupFuture(); @@ -76,7 +79,8 @@ public void testFailedMetadataRequest() { @Test public void testMultiBrokerListTransactions() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + options, deadlineMs, retryBackoffMs, logContext); Map>> brokerFutures = assertMetadataLookup(driver, mkSet(0, 1)); @@ -109,7 +113,8 @@ public void testMultiBrokerListTransactions() throws Exception { @Test public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + options, deadlineMs, retryBackoffMs, logContext); int brokerId = 0; Map>> brokerFutures = @@ -134,7 +139,8 @@ public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception @Test public void testFatalListTransactionsError() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver(options, deadlineMs, retryBackoffMs); + ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + options, deadlineMs, retryBackoffMs, logContext); int brokerId = 0; Map>> brokerFutures = diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java index 9c80be989e55c..6c4b1f9167548 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; @@ -274,7 +275,7 @@ private RequestSpec lookupRequest( private final class TestMetadataRequestDriver extends MetadataRequestDriver { public TestMetadataRequestDriver(Collection futures) { - super(futures, deadlineMs, retryBackoffMs); + super(futures, deadlineMs, retryBackoffMs, new LogContext()); } @Override From d886040f7e61183ad2df8885d0023e626c7568c7 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 14:51:12 -0700 Subject: [PATCH 14/17] Use `ApiDriver` instead of `RequestDriver` --- .../kafka/clients/admin/KafkaAdminClient.java | 27 ++++++++-------- ...river.java => AbortTransactionDriver.java} | 11 +++++-- ...estDriver.java => AllBrokerApiDriver.java} | 8 ++--- .../{RequestDriver.java => ApiDriver.java} | 22 +++++++++---- ...tDriver.java => CoordinatorApiDriver.java} | 6 ++-- ...iver.java => DescribeProducersDriver.java} | 12 ++++--- ...r.java => DescribeTransactionsDriver.java} | 14 +++++--- ...river.java => ListTransactionsDriver.java} | 11 +++++-- ...ver.java => PartitionLeaderApiDriver.java} | 9 ++++-- ...t.java => AbortTransactionDriverTest.java} | 22 ++++++------- ...est.java => CoordinatorApiDriverTest.java} | 21 +++++++----- ....java => DescribeProducersDriverTest.java} | 18 +++++------ ...va => DescribeTransactionsDriverTest.java} | 32 +++++++++---------- ...t.java => ListTransactionsDriverTest.java} | 20 ++++++------ ...java => PartitionLeaderApiDriverTest.java} | 23 +++++++------ 15 files changed, 146 insertions(+), 110 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{AbortTransactionRequestDriver.java => AbortTransactionDriver.java} (96%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{AllBrokerRequestDriver.java => AllBrokerApiDriver.java} (95%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{RequestDriver.java => ApiDriver.java} (96%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{CoordinatorRequestDriver.java => CoordinatorApiDriver.java} (95%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{DescribeProducersRequestDriver.java => DescribeProducersDriver.java} (96%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{DescribeTransactionsRequestDriver.java => DescribeTransactionsDriver.java} (95%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{ListTransactionsRequestDriver.java => ListTransactionsDriver.java} (93%) rename clients/src/main/java/org/apache/kafka/clients/admin/internals/{MetadataRequestDriver.java => PartitionLeaderApiDriver.java} (95%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{AbortTransactionRequestDriverTest.java => AbortTransactionDriverTest.java} (92%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{CoordinatorRequestDriverTest.java => CoordinatorApiDriverTest.java} (93%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{DescribeProducersRequestDriverTest.java => DescribeProducersDriverTest.java} (95%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{DescribeTransactionsRequestDriverTest.java => DescribeTransactionsDriverTest.java} (93%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{ListTransactionsRequestDriverTest.java => ListTransactionsDriverTest.java} (95%) rename clients/src/test/java/org/apache/kafka/clients/admin/internals/{MetadataRequestDriverTest.java => PartitionLeaderApiDriverTest.java} (93%) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index e1f970f7f6a4c..74a2fedcc5643 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -32,15 +32,15 @@ import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec; +import org.apache.kafka.clients.admin.internals.AbortTransactionDriver; import org.apache.kafka.clients.admin.internals.AdminMetadataManager; +import org.apache.kafka.clients.admin.internals.ApiDriver; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext; -import org.apache.kafka.clients.admin.internals.DescribeProducersRequestDriver; -import org.apache.kafka.clients.admin.internals.DescribeTransactionsRequestDriver; -import org.apache.kafka.clients.admin.internals.ListTransactionsRequestDriver; +import org.apache.kafka.clients.admin.internals.DescribeProducersDriver; +import org.apache.kafka.clients.admin.internals.DescribeTransactionsDriver; +import org.apache.kafka.clients.admin.internals.ListTransactionsDriver; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; -import org.apache.kafka.clients.admin.internals.RequestDriver; -import org.apache.kafka.clients.admin.internals.AbortTransactionRequestDriver; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; @@ -4289,7 +4289,7 @@ public DescribeProducersResult describeProducers(Collection part } long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( partitions, options, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new DescribeProducersResult(driver.futures()); @@ -4302,7 +4302,7 @@ public DescribeTransactionsResult describeTransactions(Collection transa } long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( transactionalIds, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new DescribeTransactionsResult(driver.futures()); @@ -4312,7 +4312,7 @@ public DescribeTransactionsResult describeTransactions(Collection transa public ListTransactionsResult listTransactions(ListTransactionsOptions options) { long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + ListTransactionsDriver driver = new ListTransactionsDriver( options, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new ListTransactionsResult(driver.lookupFuture()); @@ -4322,7 +4322,7 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { long currentTimeMs = time.milliseconds(); long deadlineMs = calcDeadlineMs(currentTimeMs, options.timeoutMs); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver( + AbortTransactionDriver driver = new AbortTransactionDriver( spec, deadlineMs, retryBackoffMs, logContext); maybeSendRequests(driver, currentTimeMs); return new AbortTransactionResult(driver.futures()); @@ -4340,19 +4340,18 @@ static Throwable getSubLevelError(Map subLevelErrors, K subKey, S } } - private void maybeSendRequests(RequestDriver driver, long currentTimeMs) { + private void maybeSendRequests(ApiDriver driver, long currentTimeMs) { for (RequestSpec spec : driver.poll()) { runnable.call(newCall(driver, spec), currentTimeMs); } } - private Call newCall(RequestDriver driver, RequestSpec spec) { + private Call newCall(ApiDriver driver, RequestSpec spec) { NodeProvider nodeProvider = spec.scope.destinationBrokerId().isPresent() ? new ConstantNodeIdProvider(spec.scope.destinationBrokerId().getAsInt()) : new LeastLoadedNodeProvider(); - // FIXME: Add name to RequestSpec - return new Call("", spec.nextAllowedTryMs, spec.tries, spec.deadlineMs, nodeProvider) { + return new Call(spec.name, spec.nextAllowedTryMs, spec.tries, spec.deadlineMs, nodeProvider) { @Override AbstractRequest.Builder createRequest(int timeoutMs) { return spec.request; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriver.java similarity index 96% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriver.java index ded49b8fef12d..b10ac30534bf4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriver.java @@ -36,12 +36,12 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; -public class AbortTransactionRequestDriver extends MetadataRequestDriver { +public class AbortTransactionDriver extends PartitionLeaderApiDriver { private final Logger log; private final AbortTransactionSpec abortSpec; private final TopicPartition topicPartition; - public AbortTransactionRequestDriver( + public AbortTransactionDriver( AbortTransactionSpec abortSpec, long deadlineMs, long retryBackoffMs, @@ -50,7 +50,12 @@ public AbortTransactionRequestDriver( super(singleton(abortSpec.topicPartition()), deadlineMs, retryBackoffMs, logContext); this.abortSpec = abortSpec; this.topicPartition = abortSpec.topicPartition(); - this.log = logContext.logger(AbortTransactionRequestDriver.class); + this.log = logContext.logger(AbortTransactionDriver.class); + } + + @Override + String apiName() { + return "abortTransaction"; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerApiDriver.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerApiDriver.java index 7ce23d8359fec..364b40827fa65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AllBrokerApiDriver.java @@ -36,10 +36,8 @@ /** * This class is used for use cases which require requests to be sent to all * brokers in the cluster. - * - * @param */ -public abstract class AllBrokerRequestDriver extends RequestDriver { +public abstract class AllBrokerApiDriver extends ApiDriver { private static final BrokerKey ALL_BROKERS = new BrokerKey(OptionalInt.empty()); private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { }; @@ -47,7 +45,7 @@ public abstract class AllBrokerRequestDriver extends RequestDriver>> lookupFuture; - public AllBrokerRequestDriver( + public AllBrokerApiDriver( long deadlineMs, long retryBackoffMs, LogContext logContext @@ -55,7 +53,7 @@ public AllBrokerRequestDriver( super(Utils.mkSet(ALL_BROKERS), deadlineMs, retryBackoffMs, logContext); this.lookupFuture = new KafkaFutureImpl<>(); - this.log = logContext.logger(AllBrokerRequestDriver.class); + this.log = logContext.logger(AllBrokerApiDriver.class); super.futures().get(ALL_BROKERS).whenComplete((nil, exception) -> { if (exception != null) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ApiDriver.java similarity index 96% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/ApiDriver.java index de21c343f8458..5593254091dcf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/RequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ApiDriver.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; @@ -75,7 +74,7 @@ * @param The fulfillment type for each key (e.g. this could be consumer group state * when the key type is a consumer `GroupId`) */ -public abstract class RequestDriver { +public abstract class ApiDriver { private final Logger log; private final long retryBackoffMs; @@ -86,7 +85,7 @@ public abstract class RequestDriver { private final BiMultimap fulfillmentMap = new BiMultimap<>(); private final Map requestStates = new HashMap<>(); - public RequestDriver( + public ApiDriver( Collection keys, long deadlineMs, long retryBackoffMs, @@ -95,7 +94,7 @@ public RequestDriver( this.futures = Utils.initializeMap(keys, KafkaFutureImpl::new); this.deadlineMs = deadlineMs; this.retryBackoffMs = retryBackoffMs; - this.log = logContext.logger(RequestDriver.class); + this.log = logContext.logger(ApiDriver.class); initializeLookupKeys(); } @@ -203,7 +202,7 @@ public void onResponse( AbstractResponse response ) { clearInflightRequest(currentTimeMs, spec); - if (spec.scope instanceof RequestDriver.BrokerScope) { + if (spec.scope instanceof ApiDriver.BrokerScope) { int brokerId = ((BrokerScope) spec.scope).destinationBrokerId; handleFulfillmentResponse(brokerId, spec.keys, response); } else { @@ -223,12 +222,17 @@ public void onFailure( spec.keys.forEach(key -> completeExceptionally(key, t)); } + /** + * Get a user-friendly name for the API this class is implementing. + */ + abstract String apiName(); + /** * The Lookup stage is complicated by the need to accommodate different batching mechanics. * Specifically, a `Metadata` request supports arbitrary batching of topic partitions, but * a `FindCoordinator` request only supports lookup of a single key. See the implementations - * in {@link MetadataRequestDriver#lookupScope(TopicPartition)} and - * {@link CoordinatorRequestDriver#lookupScope(CoordinatorKey)}. + * in {@link PartitionLeaderApiDriver#lookupScope(TopicPartition)} and + * {@link CoordinatorApiDriver#lookupScope(CoordinatorKey)}. */ abstract RequestScope lookupScope(K key); @@ -286,6 +290,7 @@ private void collectRequests( AbstractRequest.Builder request = buildRequest.apply(keys, scope); RequestSpec spec = new RequestSpec<>( + this.apiName() + "(api=" + request.apiKey() + ")", scope, new HashSet<>(keys), // copy to avoid exposing mutable state request, @@ -321,6 +326,7 @@ private void collectFulfillmentRequests(List> requests) { * {@link org.apache.kafka.clients.admin.KafkaAdminClient}. */ public static class RequestSpec { + public final String name; public final RequestScope scope; public final Set keys; public final AbstractRequest.Builder request; @@ -329,6 +335,7 @@ public static class RequestSpec { public final int tries; public RequestSpec( + String name, RequestScope scope, Set keys, AbstractRequest.Builder request, @@ -336,6 +343,7 @@ public RequestSpec( long deadlineMs, int tries ) { + this.name = name; this.scope = scope; this.keys = keys; this.request = request; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriver.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriver.java index fa32dc008a4b6..db2494a28022d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriver.java @@ -30,17 +30,17 @@ import java.util.Objects; import java.util.Set; -public abstract class CoordinatorRequestDriver extends RequestDriver { +public abstract class CoordinatorApiDriver extends ApiDriver { private final Logger log; - public CoordinatorRequestDriver( + public CoordinatorApiDriver( Collection keys, long deadlineMs, long retryBackoffMs, LogContext logContext ) { super(keys, deadlineMs, retryBackoffMs, logContext); - this.log = logContext.logger(CoordinatorRequestDriver.class); + this.log = logContext.logger(CoordinatorApiDriver.class); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriver.java similarity index 96% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriver.java index 9addd9606ac18..ce8a7968e101e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriver.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Collections; @@ -41,11 +40,11 @@ import java.util.Set; import java.util.stream.Collectors; -public class DescribeProducersRequestDriver extends MetadataRequestDriver { +public class DescribeProducersDriver extends PartitionLeaderApiDriver { private final Logger log; private final DescribeProducersOptions options; - public DescribeProducersRequestDriver( + public DescribeProducersDriver( Collection topicPartitions, DescribeProducersOptions options, long deadlineMs, @@ -54,7 +53,7 @@ public DescribeProducersRequestDriver( ) { super(topicPartitions, deadlineMs, retryBackoffMs, logContext); this.options = options; - this.log = logContext.logger(DescribeProducersRequestDriver.class); + this.log = logContext.logger(DescribeProducersDriver.class); // If the request options indicate a specific target broker, then we directly // map the topic partitions to avoid the unneeded `Metadata` lookup. @@ -66,6 +65,11 @@ public DescribeProducersRequestDriver( } } + @Override + String apiName() { + return "describeProducers"; + } + @Override DescribeProducersRequest.Builder buildFulfillmentRequest(Integer brokerId, Set topicPartitions) { DescribeProducersRequestData request = new DescribeProducersRequestData(); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriver.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriver.java index f36344e2ea613..4dfa5c07476ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriver.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.HashSet; @@ -40,25 +39,30 @@ import java.util.Set; import java.util.stream.Collectors; -public class DescribeTransactionsRequestDriver extends CoordinatorRequestDriver { +public class DescribeTransactionsDriver extends CoordinatorApiDriver { private final Logger log; - public DescribeTransactionsRequestDriver( + public DescribeTransactionsDriver( Collection transactionalIds, long deadlineMs, long retryBackoffMs, LogContext logContext ) { super(buildKeySet(transactionalIds), deadlineMs, retryBackoffMs, logContext); - this.log = logContext.logger(DescribeTransactionsRequestDriver.class); + this.log = logContext.logger(DescribeTransactionsDriver.class); } private static Set buildKeySet(Collection transactionalIds) { return transactionalIds.stream() - .map(DescribeTransactionsRequestDriver::asCoordinatorKey) + .map(DescribeTransactionsDriver::asCoordinatorKey) .collect(Collectors.toSet()); } + @Override + String apiName() { + return "describeTransactions"; + } + @Override AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set keys) { DescribeTransactionsRequestData request = new DescribeTransactionsRequestData(); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriver.java similarity index 93% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriver.java index f58a9f9ebf105..3843853907d04 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriver.java @@ -34,11 +34,11 @@ import java.util.List; import java.util.stream.Collectors; -public class ListTransactionsRequestDriver extends AllBrokerRequestDriver> { +public class ListTransactionsDriver extends AllBrokerApiDriver> { private final Logger log; private final ListTransactionsOptions options; - public ListTransactionsRequestDriver( + public ListTransactionsDriver( ListTransactionsOptions options, long deadlineMs, long retryBackoffMs, @@ -46,7 +46,12 @@ public ListTransactionsRequestDriver( ) { super(deadlineMs, retryBackoffMs, logContext); this.options = options; - this.log = logContext.logger(ListTransactionsRequestDriver.class); + this.log = logContext.logger(ListTransactionsDriver.class); + } + + @Override + String apiName() { + return "listTransactions"; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriver.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java rename to clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriver.java index 9cf29b1989bd5..f92f83ac1e4a8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriver.java @@ -34,20 +34,23 @@ import java.util.Set; import java.util.function.Function; -public abstract class MetadataRequestDriver extends RequestDriver { +/** + * Base driver implementation for APIs which target partition leaders. + */ +public abstract class PartitionLeaderApiDriver extends ApiDriver { private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { }; private final Logger log; - public MetadataRequestDriver( + public PartitionLeaderApiDriver( Collection futures, long deadlineMs, long retryBackoffMs, LogContext logContext ) { super(futures, deadlineMs, retryBackoffMs, logContext); - this.log = logContext.logger(MetadataRequestDriver.class); + this.log = logContext.logger(PartitionLeaderApiDriver.class); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriverTest.java similarity index 92% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriverTest.java index f5f0e52f2a329..006b8248b6537 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionDriverTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.AbortTransactionSpec; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.message.MetadataResponseData; @@ -44,7 +44,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -public class AbortTransactionRequestDriverTest { +public class AbortTransactionDriverTest { private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; @@ -55,7 +55,7 @@ public void testSuccessfulAbortTransaction() throws Exception { TopicPartition topicPartition = new TopicPartition("foo", 0); AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( topicPartition, 12345L, 15, 4321); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + AbortTransactionDriver driver = new AbortTransactionDriver(abortTransactionSpec, deadlineMs, retryBackoffMs, logContext); int expectedLeaderId = 5; assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); @@ -68,7 +68,7 @@ public void testFatalTransactionCoordinatorFencedError() throws Exception { TopicPartition topicPartition = new TopicPartition("foo", 0); AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( topicPartition, 12345L, 15, 4321); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + AbortTransactionDriver driver = new AbortTransactionDriver(abortTransactionSpec, deadlineMs, retryBackoffMs, logContext); int expectedLeaderId = 5; assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); @@ -81,7 +81,7 @@ public void testFatalClusterAuthorizationError() throws Exception { TopicPartition topicPartition = new TopicPartition("foo", 0); AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( topicPartition, 12345L, 15, 4321); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + AbortTransactionDriver driver = new AbortTransactionDriver(abortTransactionSpec, deadlineMs, retryBackoffMs, logContext); int expectedLeaderId = 5; assertMetadataLookup(driver, abortTransactionSpec, expectedLeaderId, 0); @@ -94,7 +94,7 @@ public void testRetryLookupAfterNotLeaderError() throws Exception { TopicPartition topicPartition = new TopicPartition("foo", 0); AbortTransactionSpec abortTransactionSpec = new AbortTransactionSpec( topicPartition, 12345L, 15, 4321); - AbortTransactionRequestDriver driver = new AbortTransactionRequestDriver(abortTransactionSpec, + AbortTransactionDriver driver = new AbortTransactionDriver(abortTransactionSpec, deadlineMs, retryBackoffMs, logContext); int initialLeaderId = 5; @@ -108,7 +108,7 @@ public void testRetryLookupAfterNotLeaderError() throws Exception { } private void assertMetadataLookup( - AbortTransactionRequestDriver driver, + AbortTransactionDriver driver, AbortTransactionSpec abortTransactionSpec, int expectedLeaderId, int expectedTries @@ -127,7 +127,7 @@ private void assertMetadataLookup( } private void assertWriteTxnMarkers( - AbortTransactionRequestDriver driver, + AbortTransactionDriver driver, AbortTransactionSpec abortTransactionSpec, Errors error, int expectedLeaderId, @@ -140,7 +140,7 @@ private void assertWriteTxnMarkers( } private void assertCompletedFuture( - AbortTransactionRequestDriver driver, + AbortTransactionDriver driver, AbortTransactionSpec abortTransactionSpec, Errors error ) throws Exception { @@ -163,7 +163,7 @@ private WriteTxnMarkersResponse writeTxnMarkersResponse( } private RequestSpec assertWriteTxnMarkersRequest( - AbortTransactionRequestDriver driver, + AbortTransactionDriver driver, AbortTransactionSpec abortSpec, int expectedLeaderId, int expectedTries @@ -194,7 +194,7 @@ private RequestSpec assertWriteTxnMarkersRequest( } private RequestSpec assertMetadataRequest( - AbortTransactionRequestDriver driver, + AbortTransactionDriver driver, TopicPartition topicPartition, int expectedTries ) { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriverTest.java similarity index 93% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriverTest.java index 67d7d813110f1..b9b687246dbd1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorApiDriverTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.message.DescribeGroupsRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; @@ -44,7 +44,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class CoordinatorRequestDriverTest { +public class CoordinatorApiDriverTest { private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; private final long retryBackoffMs = 100; @@ -55,7 +55,7 @@ public void testLookupGrouping() { CoordinatorKey group2 = new CoordinatorKey("bar", CoordinatorType.GROUP); Set groupIds = mkSet(group1, group2); - TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + TestCoordinatorDriver driver = new TestCoordinatorDriver(groupIds); List> requests = driver.poll(); assertEquals(2, requests.size()); @@ -91,7 +91,7 @@ public void testSuccessfulLeaderDiscovery() { CoordinatorKey group2 = new CoordinatorKey("bar", CoordinatorType.GROUP); Set groupIds = mkSet(group1, group2); - TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + TestCoordinatorDriver driver = new TestCoordinatorDriver(groupIds); List> lookupRequests = driver.poll(); assertEquals(2, lookupRequests.size()); @@ -141,7 +141,7 @@ public void testRetriableFindCoordinatorError() { CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); Set groupIds = mkSet(group1); - TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + TestCoordinatorDriver driver = new TestCoordinatorDriver(groupIds); List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); @@ -166,7 +166,7 @@ public void testFatalFindCoordinatorError() { CoordinatorKey group1 = new CoordinatorKey("foo", CoordinatorType.GROUP); Set groupIds = mkSet(group1); - TestCoordinatorRequestDriver driver = new TestCoordinatorRequestDriver(groupIds); + TestCoordinatorDriver driver = new TestCoordinatorDriver(groupIds); List> lookupRequests1 = driver.poll(); assertEquals(1, lookupRequests1.size()); @@ -192,12 +192,17 @@ private RequestSpec lookupRequest( return foundRequestOpt.get(); } - private final class TestCoordinatorRequestDriver extends CoordinatorRequestDriver { + private final class TestCoordinatorDriver extends CoordinatorApiDriver { - public TestCoordinatorRequestDriver(Set groupIds) { + public TestCoordinatorDriver(Set groupIds) { super(groupIds, deadlineMs, retryBackoffMs, new LogContext()); } + @Override + String apiName() { + return "testCoordinatorApi"; + } + @Override AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set coordinatorKeys) { return new DescribeGroupsRequest.Builder(new DescribeGroupsRequestData() diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriverTest.java similarity index 95% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriverTest.java index fdef00b275d67..d38624da4ee66 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersDriverTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.clients.admin.DescribeProducersOptions; import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownServerException; @@ -56,7 +56,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -public class DescribeProducersRequestDriverTest { +public class DescribeProducersDriverTest { private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; @@ -68,7 +68,7 @@ public void testSuccessfulResponseWithoutProvidedBrokerId() throws Exception { int leaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, leaderId, 0); @@ -89,7 +89,7 @@ public void testRetryLookupAfterNotLeaderErrorWithoutProvidedBrokerId() { int initialLeaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, initialLeaderId, 0); @@ -114,7 +114,7 @@ public void testSuccessfulResponseWithProvidedBrokerId() throws Exception { int brokerId = 1; DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); List> requests = driver.poll(); @@ -141,7 +141,7 @@ public void testNotLeaderErrorWithProvidedBrokerId() { int brokerId = 1; DescribeProducersOptions options = new DescribeProducersOptions().setBrokerId(brokerId); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); List> requests = driver.poll(); @@ -168,7 +168,7 @@ public void testFatalErrorWithoutProvidedBrokerId() { int leaderId = 1; DescribeProducersOptions options = new DescribeProducersOptions(); - DescribeProducersRequestDriver driver = new DescribeProducersRequestDriver( + DescribeProducersDriver driver = new DescribeProducersDriver( singleton(topicPartition), options, deadlineMs, retryBackoffMs, logContext); assertMetadataLookup(driver, topicPartition, leaderId, 0); @@ -187,7 +187,7 @@ public void testFatalErrorWithoutProvidedBrokerId() { } private void assertSuccessfulFulfillment( - DescribeProducersRequestDriver driver, + DescribeProducersDriver driver, TopicPartition topicPartition, RequestSpec describeProducerSpec ) throws Exception { @@ -207,7 +207,7 @@ private void assertSuccessfulFulfillment( } private void assertMetadataLookup( - DescribeProducersRequestDriver driver, + DescribeProducersDriver driver, TopicPartition topicPartition, int leaderId, int expectedTries diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriverTest.java similarity index 93% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriverTest.java index 979cc2377ec56..0fac18823c12c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsDriverTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.clients.admin.TransactionDescription; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -49,7 +49,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -public class DescribeTransactionsRequestDriverTest { +public class DescribeTransactionsDriverTest { private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; @@ -61,7 +61,7 @@ public void testDescribeTransactions() throws Exception { String transactionalId2 = "bar"; Set transactionalIds = mkSet(transactionalId1, transactionalId2); - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( transactionalIds, deadlineMs, retryBackoffMs, logContext); // Send `FindCoordinator` requests @@ -116,9 +116,9 @@ public void testDescribeTransactions() throws Exception { assertEquals(Collections.emptyList(), driver.poll()); KafkaFutureImpl future1 = driver.futures() - .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); + .get(DescribeTransactionsDriver.asCoordinatorKey(transactionalId1)); KafkaFutureImpl future2 = driver.futures() - .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); + .get(DescribeTransactionsDriver.asCoordinatorKey(transactionalId2)); assertTrue(future1.isDone()); assertMatchingTransactionState(coordinator1, transactionState1, future1.get()); @@ -132,7 +132,7 @@ public void testDescribeTransactionsBatching() throws Exception { String transactionalId2 = "bar"; Set transactionalIds = mkSet(transactionalId1, transactionalId2); - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( transactionalIds, deadlineMs, retryBackoffMs, logContext); // Send `FindCoordinator` requests @@ -181,9 +181,9 @@ public void testDescribeTransactionsBatching() throws Exception { assertEquals(Collections.emptyList(), driver.poll()); KafkaFutureImpl future1 = driver.futures() - .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId1)); + .get(DescribeTransactionsDriver.asCoordinatorKey(transactionalId1)); KafkaFutureImpl future2 = driver.futures() - .get(DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId2)); + .get(DescribeTransactionsDriver.asCoordinatorKey(transactionalId2)); assertTrue(future1.isDone()); assertMatchingTransactionState(coordinator, transactionState1, future1.get()); @@ -195,7 +195,7 @@ public void testDescribeTransactionsBatching() throws Exception { public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() { String transactionalId = "foo"; - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request @@ -237,7 +237,7 @@ public void testShouldRetryDescribeTransactionsIfCoordinatorLoadingInProgress() public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { String transactionalId = "foo"; - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request @@ -277,7 +277,7 @@ public void testShouldRetryFindCoordinatorAfterNotCoordinatorError() { public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() { String transactionalId = "foo"; - DescribeTransactionsRequestDriver driver = new DescribeTransactionsRequestDriver( + DescribeTransactionsDriver driver = new DescribeTransactionsDriver( singleton(transactionalId), deadlineMs, retryBackoffMs, logContext); // Send first `FindCoordinator` request @@ -310,10 +310,10 @@ public void testShouldFailTransactionalIdAfterFatalErrorInDescribeTransactions() } private KafkaFutureImpl futureFor( - DescribeTransactionsRequestDriver driver, + DescribeTransactionsDriver driver, String transactionalId ) { - CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); + CoordinatorKey key = DescribeTransactionsDriver.asCoordinatorKey(transactionalId); return driver.futures().get(key); } @@ -321,7 +321,7 @@ private RequestSpec findRequestWithKey( String transactionalId, List> requests ) { - CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(transactionalId); + CoordinatorKey key = DescribeTransactionsDriver.asCoordinatorKey(transactionalId); Optional> firstMatch = requests.stream() .filter(spec -> spec.keys.contains(key)) @@ -338,7 +338,7 @@ private void assertDescribeTransactionsRequest( RequestSpec spec ) { Set keys = expectedTransactionalIds.stream() - .map(DescribeTransactionsRequestDriver::asCoordinatorKey) + .map(DescribeTransactionsDriver::asCoordinatorKey) .collect(Collectors.toSet()); assertEquals(keys, spec.keys); assertEquals(OptionalInt.of(expectedCoordinatorId), spec.scope.destinationBrokerId()); @@ -352,7 +352,7 @@ private void assertFindCoordinatorRequest( String expectedTransactionalId, RequestSpec spec ) { - CoordinatorKey key = DescribeTransactionsRequestDriver.asCoordinatorKey(expectedTransactionalId); + CoordinatorKey key = DescribeTransactionsDriver.asCoordinatorKey(expectedTransactionalId); assertEquals(singleton(key), spec.keys); assertEquals(OptionalInt.empty(), spec.scope.destinationBrokerId()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriverTest.java similarity index 95% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriverTest.java index 2fe7d0e600b11..47a593be17af8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsDriverTest.java @@ -19,8 +19,8 @@ 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.clients.admin.internals.AllBrokerRequestDriver.BrokerKey; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.AllBrokerApiDriver.BrokerKey; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.internals.KafkaFutureImpl; @@ -56,7 +56,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -public class ListTransactionsRequestDriverTest { +public class ListTransactionsDriverTest { private final LogContext logContext = new LogContext(); private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; @@ -65,7 +65,7 @@ public class ListTransactionsRequestDriverTest { @Test public void testFailedMetadataRequest() { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + ListTransactionsDriver driver = new ListTransactionsDriver( options, deadlineMs, retryBackoffMs, logContext); KafkaFutureImpl>>> lookupFuture = @@ -79,7 +79,7 @@ public void testFailedMetadataRequest() { @Test public void testMultiBrokerListTransactions() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + ListTransactionsDriver driver = new ListTransactionsDriver( options, deadlineMs, retryBackoffMs, logContext); Map>> brokerFutures = @@ -113,7 +113,7 @@ public void testMultiBrokerListTransactions() throws Exception { @Test public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + ListTransactionsDriver driver = new ListTransactionsDriver( options, deadlineMs, retryBackoffMs, logContext); int brokerId = 0; @@ -139,7 +139,7 @@ public void testRetryListTransactionsAfterCoordinatorLoading() throws Exception @Test public void testFatalListTransactionsError() throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); - ListTransactionsRequestDriver driver = new ListTransactionsRequestDriver( + ListTransactionsDriver driver = new ListTransactionsDriver( options, deadlineMs, retryBackoffMs, logContext); int brokerId = 0; @@ -156,7 +156,7 @@ public void testFatalListTransactionsError() throws Exception { } private void assertListTransactions( - ListTransactionsRequestDriver driver, + ListTransactionsDriver driver, ListTransactionsOptions options, ListTransactionsResponse response, int brokerId @@ -169,7 +169,7 @@ private void assertListTransactions( } private Map>> assertMetadataLookup( - ListTransactionsRequestDriver driver, + ListTransactionsDriver driver, Set brokers ) throws Exception { KafkaFutureImpl>>> lookupFuture = @@ -280,7 +280,7 @@ private RequestSpec findBrokerRequest( } private RequestSpec assertLookupRequest( - ListTransactionsRequestDriver driver + ListTransactionsDriver driver ) { List> requests = driver.poll(); assertEquals(1, requests.size()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriverTest.java similarity index 93% rename from clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java rename to clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriverTest.java index 6c4b1f9167548..7943d2095a632 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/MetadataRequestDriverTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderApiDriverTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; -import org.apache.kafka.clients.admin.internals.RequestDriver.RequestSpec; +import org.apache.kafka.clients.admin.internals.ApiDriver.RequestSpec; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.UnknownServerException; @@ -51,7 +51,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class MetadataRequestDriverTest { +public class PartitionLeaderApiDriverTest { private final MockTime time = new MockTime(); private final long deadlineMs = time.milliseconds() + 10000; private final long retryBackoffMs = 100; @@ -63,7 +63,7 @@ public void testLookupGrouping() { new TopicPartition("foo", 2), new TopicPartition("bar", 1)); - TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + TestPartitionLeaderApiDriver driver = new TestPartitionLeaderApiDriver(topicPartitions); List> requests = driver.poll(); assertEquals(1, requests.size()); @@ -88,7 +88,7 @@ public void testSuccessfulLeaderDiscovery() { // Request includes 2 of 3 partitions for the topic Set topicPartitions = mkSet(tp0, tp2); - TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + TestPartitionLeaderApiDriver driver = new TestPartitionLeaderApiDriver(topicPartitions); List> requests1 = driver.poll(); assertEquals(1, requests1.size()); @@ -136,7 +136,7 @@ public void testRetryLeaderDiscovery() { // Request includes 2 of 3 partitions for the topic Set topicPartitions = mkSet(tp0, tp2); - TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + TestPartitionLeaderApiDriver driver = new TestPartitionLeaderApiDriver(topicPartitions); List> requests1 = driver.poll(); assertEquals(1, requests1.size()); @@ -181,7 +181,7 @@ public void testFatalTopicError() { TopicPartition tp2 = new TopicPartition("bar", 0); Set topicPartitions = mkSet(tp0, tp2); - TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + TestPartitionLeaderApiDriver driver = new TestPartitionLeaderApiDriver(topicPartitions); List> requests1 = driver.poll(); assertEquals(1, requests1.size()); @@ -225,7 +225,7 @@ public void testFatalPartitionError() { // Request includes 2 of 3 partitions for the topic Set topicPartitions = mkSet(tp0, tp2); - TestMetadataRequestDriver driver = new TestMetadataRequestDriver(topicPartitions); + TestPartitionLeaderApiDriver driver = new TestPartitionLeaderApiDriver(topicPartitions); List> requests1 = driver.poll(); assertEquals(1, requests1.size()); @@ -272,12 +272,17 @@ private RequestSpec lookupRequest( return foundRequestOpt.get(); } - private final class TestMetadataRequestDriver extends MetadataRequestDriver { + private final class TestPartitionLeaderApiDriver extends PartitionLeaderApiDriver { - public TestMetadataRequestDriver(Collection futures) { + public TestPartitionLeaderApiDriver(Collection futures) { super(futures, deadlineMs, retryBackoffMs, new LogContext()); } + @Override + String apiName() { + return "testPartitionLeaderApi"; + } + @Override AbstractRequest.Builder buildFulfillmentRequest(Integer brokerId, Set topicPartitions) { DescribeProducersRequestData request = new DescribeProducersRequestData(); From c237d005256f9b92400f9038de3d2784bc734dd0 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 8 Sep 2020 15:08:40 -0700 Subject: [PATCH 15/17] Fix `RequestQuotaTest` failures --- .../common/requests/DescribeProducersResponse.java | 5 +++++ .../requests/DescribeTransactionsResponse.java | 5 +++++ .../common/requests/ListTransactionsResponse.java | 5 +++++ .../scala/unit/kafka/server/RequestQuotaTest.scala | 13 +++++++++++++ 4 files changed, 28 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java index b80fdbc9ec8ea..484ed48c7bc1f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java @@ -68,4 +68,9 @@ public String toString() { return data.toString(); } + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java index bf40ac0ea7135..fd72ba83745bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java @@ -66,4 +66,9 @@ public String toString() { return data.toString(); } + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java index 7c13f52481659..ca57f52b79eb9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java @@ -62,4 +62,9 @@ public String toString() { return data.toString(); } + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 3958f7613b11b..d8b7fef158747 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -561,6 +561,19 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()) + case ApiKeys.DESCRIBE_PRODUCERS => + new DescribeProducersRequest.Builder(new DescribeProducersRequestData() + .setTopics(List(new DescribeProducersRequestData.TopicRequest() + .setName("test-topic") + .setPartitionIndexes(List(1, 2, 3).map(Int.box).asJava)).asJava)) + + case ApiKeys.LIST_TRANSACTIONS => + new ListTransactionsRequest.Builder(new ListTransactionsRequestData()) + + case ApiKeys.DESCRIBE_TRANSACTIONS => + new DescribeTransactionsRequest.Builder(new DescribeTransactionsRequestData() + .setTransactionalIds(List("test-transactional-id").asJava)) + case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } From 7732a73a51bac1c5daafa564cc279c91ba786fd0 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 18 Sep 2020 16:11:27 -0700 Subject: [PATCH 16/17] Add needed dependencies for tools testing --- build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index fd2e3efc6f0e7..fe56521cf0b0a 100644 --- a/build.gradle +++ b/build.gradle @@ -1160,7 +1160,8 @@ project(':tools') { compile libs.jettyServlets testCompile project(':clients') - testCompile libs.junitJupiter + testCompile libs.junitJupiterApi + testCompile libs.junitVintageEngine testCompile project(':clients').sourceSets.test.output testCompile libs.mockitoInline // supports mocking static methods, final classes, etc. From e7250d5760ababafb2e6cf03cf82ac432d458792 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 18 Sep 2020 16:59:31 -0700 Subject: [PATCH 17/17] Add test cases for describing transactions --- .../transaction/TransactionCoordinator.scala | 3 +- .../TransactionCoordinatorTest.scala | 69 ++++++++++++++++--- 2 files changed, 63 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index b72fc634d1899..4575e9748dbfe 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -296,9 +296,10 @@ class TransactionCoordinator(brokerId: Int, val partitionsByTopic = CollectionUtils.groupPartitionsByTopic(txnMetadata.topicPartitions.asJava) partitionsByTopic.forEach { (topic, partitions) => - new DescribeTransactionsResponseData.TopicData() + val topicData = new DescribeTransactionsResponseData.TopicData() .setName(topic) .setPartitionIndexes(partitions) + transactionState.topicPartitions.add(topicData) } transactionState diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 64e18e4b7ebd9..400ecdc8834c7 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -24,9 +24,10 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} import org.easymock.{Capture, EasyMock} import org.junit.Assert._ -import org.junit.Test +import org.junit.{Before, Test} import scala.collection.mutable +import scala.jdk.CollectionConverters._ class TransactionCoordinatorTest { @@ -61,6 +62,14 @@ class TransactionCoordinatorTest { var result: InitProducerIdResult = _ var error: Errors = Errors.NONE + @Before + def setup(): Unit = { + EasyMock.expect(transactionMarkerChannelManager.start()) + EasyMock.replay(transactionMarkerChannelManager) + coordinator.startup(enableTransactionalIdExpiration = false) + EasyMock.reset(transactionMarkerChannelManager) + } + private def mockPidManager(): Unit = { EasyMock.expect(pidManager.generateProducerId()).andAnswer(() => { nextPid += 1 @@ -68,7 +77,7 @@ class TransactionCoordinatorTest { }).anyTimes() } - private def initPidGenericMocks(transactionalId: String): Unit = { + private def initPidGenericMocks(): Unit = { mockPidManager() EasyMock.expect(transactionManager.validateTransactionTimeoutMs(EasyMock.anyInt())) .andReturn(true) @@ -99,7 +108,7 @@ class TransactionCoordinatorTest { @Test def shouldInitPidWithEpochZeroForNewTransactionalId(): Unit = { - initPidGenericMocks(transactionalId) + initPidGenericMocks() EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId))) .andReturn(Right(None)) @@ -126,7 +135,7 @@ class TransactionCoordinatorTest { @Test def shouldGenerateNewProducerIdIfNoStateAndProducerIdAndEpochProvided(): Unit = { - initPidGenericMocks(transactionalId) + initPidGenericMocks() EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId))) .andReturn(Right(None)) @@ -154,7 +163,7 @@ class TransactionCoordinatorTest { @Test def shouldGenerateNewProducerIdIfEpochsExhausted(): Unit = { - initPidGenericMocks(transactionalId) + initPidGenericMocks() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, Empty, mutable.Set.empty, time.milliseconds(), time.milliseconds()) @@ -966,7 +975,6 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) @@ -1011,7 +1019,6 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) @@ -1049,7 +1056,6 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) @@ -1078,6 +1084,53 @@ class TransactionCoordinatorTest { EasyMock.verify(transactionManager) } + @Test + def testDescribeTransactionsWithEmptyTransactionalId(): Unit = { + val result = coordinator.handleDescribeTransactions("") + assertEquals("", result.transactionalId) + assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode)) + } + + @Test + def testDescribeTransactionsWhileCoordinatorLoading(): Unit = { + EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId))) + .andReturn(Left(Errors.COORDINATOR_LOAD_IN_PROGRESS)) + + EasyMock.replay(transactionManager) + + val result = coordinator.handleDescribeTransactions(transactionalId) + assertEquals(transactionalId, result.transactionalId) + assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, Errors.forCode(result.errorCode)) + + EasyMock.verify(transactionManager) + } + + @Test + def testDescribeTransactions(): Unit = { + val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, producerEpoch, + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds()) + + EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId))) + .andReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + + EasyMock.replay(transactionManager) + + val result = coordinator.handleDescribeTransactions(transactionalId) + assertEquals(Errors.NONE, Errors.forCode(result.errorCode)) + assertEquals(transactionalId, result.transactionalId) + assertEquals(producerId, result.producerId) + assertEquals(producerEpoch, result.producerEpoch) + assertEquals(txnTimeoutMs, result.transactionTimeoutMs) + assertEquals(time.milliseconds(), result.transactionStartTimeMs) + + val addedPartitions = result.topicPartitions.asScala.flatMap { topicData => + topicData.partitionIndexes.asScala.map(partition => new TopicPartition(topicData.name, partition)) + }.toSet + assertEquals(partitions, addedPartitions) + + EasyMock.verify(transactionManager) + } + private def validateRespondsWithConcurrentTransactionsOnInitPidWhenInPrepareState(state: TransactionState): Unit = { EasyMock.expect(transactionManager.validateTransactionTimeoutMs(EasyMock.anyInt())) .andReturn(true).anyTimes()