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 54d103b22551d..4b6fe4967caa9 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
@@ -17,14 +17,6 @@
package org.apache.kafka.clients.admin;
-import java.time.Duration;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.ElectionType;
import org.apache.kafka.common.KafkaFuture;
@@ -42,6 +34,14 @@
import org.apache.kafka.common.quota.ClientQuotaFilter;
import org.apache.kafka.common.requests.LeaveGroupResponse;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+
/**
* The administrative client for Kafka, which supports managing and inspecting topics, brokers, configurations and ACLs.
*
@@ -303,7 +303,33 @@ default DescribeTopicsResult describeTopics(Collection topicNames) {
* @param options The options to use when describing the topic.
* @return The DescribeTopicsResult.
*/
- DescribeTopicsResult describeTopics(Collection topicNames, DescribeTopicsOptions options);
+ default DescribeTopicsResult describeTopics(Collection topicNames, DescribeTopicsOptions options) {
+ return describeTopics(TopicCollection.ofTopicNames(topicNames), options);
+ }
+
+ /**
+ * This is a convenience method for {@link #describeTopics(TopicCollection, DescribeTopicsOptions)}
+ * with default options. See the overload for more details.
+ *
+ * When using topic IDs, this operation is supported by brokers with version 3.1.0 or higher.
+ *
+ * @param topics The topics to describe.
+ * @return The DescribeTopicsResult.
+ */
+ default DescribeTopicsResult describeTopics(TopicCollection topics) {
+ return describeTopics(topics, new DescribeTopicsOptions());
+ }
+
+ /**
+ * Describe some topics in the cluster.
+ *
+ * When using topic IDs, this operation is supported by brokers with version 3.1.0 or higher.
+ *
+ * @param topics The topics to describe.
+ * @param options The options to use when describing the topics.
+ * @return The DescribeTopicsResult.
+ */
+ DescribeTopicsResult describeTopics(TopicCollection topics, DescribeTopicsOptions options);
/**
* Get information about the nodes in the cluster, using the default options.
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsResult.java
index b0d23d97b4ce1..725b82a78dee8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsResult.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsResult.java
@@ -70,12 +70,12 @@ public Map> topicNameValues() {
return nameFutures;
}
- @Deprecated
/**
* @return a map from topic names to futures which can be used to check the status of
* individual deletions if the deleteTopics request used topic names. Otherwise return null.
* @deprecated Since 3.0 use {@link #topicNameValues} instead
*/
+ @Deprecated
public Map> values() {
return nameFutures;
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsResult.java
index 7753984a7bda7..41593c52984f0 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsResult.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTopicsResult.java
@@ -18,6 +18,8 @@
package org.apache.kafka.clients.admin;
import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
@@ -32,28 +34,105 @@
*/
@InterfaceStability.Evolving
public class DescribeTopicsResult {
- private final Map> futures;
+ private final Map> topicIdFutures;
+ private final Map> nameFutures;
+ @Deprecated
protected DescribeTopicsResult(Map> futures) {
- this.futures = futures;
+ this(null, futures);
+ }
+
+ // VisibleForTesting
+ protected DescribeTopicsResult(Map> topicIdFutures, Map> nameFutures) {
+ if (topicIdFutures != null && nameFutures != null)
+ throw new IllegalArgumentException("topicIdFutures and nameFutures cannot both be specified.");
+ if (topicIdFutures == null && nameFutures == null)
+ throw new IllegalArgumentException("topicIdFutures and nameFutures cannot both be null.");
+ this.topicIdFutures = topicIdFutures;
+ this.nameFutures = nameFutures;
+ }
+
+ static DescribeTopicsResult ofTopicIds(Map> topicIdFutures) {
+ return new DescribeTopicsResult(topicIdFutures, null);
+ }
+
+ static DescribeTopicsResult ofTopicNames(Map> nameFutures) {
+ return new DescribeTopicsResult(null, nameFutures);
+ }
+
+ /**
+ * Use when {@link Admin#describeTopics(TopicCollection, DescribeTopicsOptions)} used a TopicIdCollection
+ *
+ * @return a map from topic IDs to futures which can be used to check the status of
+ * individual topics if the request used topic IDs, otherwise return null.
+ */
+ public Map> topicIdValues() {
+ return topicIdFutures;
+ }
+
+ /**
+ * Use when {@link Admin#describeTopics(TopicCollection, DescribeTopicsOptions)} used a TopicNameCollection
+ *
+ * @return a map from topic names to futures which can be used to check the status of
+ * individual topics if the request used topic names, otherwise return null.
+ */
+ public Map> topicNameValues() {
+ return nameFutures;
}
/**
- * Return a map from topic names to futures which can be used to check the status of
- * individual topics.
+ * @return a map from topic names to futures which can be used to check the status of
+ * individual topics if the request used topic names, otherwise return null.
+ *
+ * @deprecated Since 3.1.0 use {@link #topicNameValues} instead
*/
+ @Deprecated
public Map> values() {
- return futures;
+ return nameFutures;
}
/**
- * Return a future which succeeds only if all the topic descriptions succeed.
+ * @return A future map from topic names to descriptions which can be used to check
+ * the status of individual description if the describe topic request used
+ * topic names, otherwise return null, this request succeeds only if all the
+ * topic descriptions succeed
+ *
+ * @deprecated Since 3.1.0 use {@link #allTopicNames()} instead
*/
+ @Deprecated
public KafkaFuture