bbejeck commented on code in PR #19116:
URL: https://github.com/apache/kafka/pull/19116#discussion_r1982259039


##########
clients/src/main/java/org/apache/kafka/clients/admin/DescribeStreamsGroupsResult.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * The result of the {@link KafkaAdminClient#describeStreamsGroups(Collection, 
DescribeStreamsGroupsOptions)}} call.
+ * <p>
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeStreamsGroupsResult {
+
+    private final Map<String, KafkaFuture<StreamsGroupDescription>> futures;
+
+    public DescribeStreamsGroupsResult(final Map<String, 
KafkaFuture<StreamsGroupDescription>> futures) {
+        this.futures = Map.copyOf(futures);
+    }
+
+    /**
+     * Return a map from group id to futures which yield streams group 
descriptions.
+     */
+    public Map<String, KafkaFuture<StreamsGroupDescription>> describedGroups() 
{
+        return new HashMap<>(futures);
+    }
+
+    /**
+     * Return a future which yields all StreamsGroupDescription objects, if 
all the describes succeed.
+     */
+    public KafkaFuture<Map<String, StreamsGroupDescription>> all() {
+        return KafkaFuture.allOf(futures.values().toArray(new 
KafkaFuture<?>[0])).thenApply(
+            nil -> {
+                Map<String, StreamsGroupDescription> descriptions = new 
HashMap<>(futures.size());
+                futures.forEach((key, future) -> {
+                    try {
+                        descriptions.put(key, future.get());
+                    } catch (InterruptedException | ExecutionException e) {

Review Comment:
   Looking at `KafkaFuture.allOf` it seems that any exception thrown would get 
propagated back here, but I can't think of a better approach than throwing a 
`RuntimeException` here.



##########
clients/src/main/java/org/apache/kafka/clients/admin/Admin.java:
##########
@@ -1957,6 +1957,29 @@ default DeleteShareGroupsResult 
deleteShareGroups(Collection<String> groupIds) {
      */
     DeleteShareGroupsResult deleteShareGroups(Collection<String> groupIds, 
DeleteShareGroupsOptions options);
 
+    /**
+     * Describe some streams groups in the cluster.

Review Comment:
   nit: remove some



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeStreamsGroupsHandler.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.StreamsGroupDescription;
+import org.apache.kafka.clients.admin.StreamsGroupMemberAssignment;
+import org.apache.kafka.clients.admin.StreamsGroupMemberDescription;
+import org.apache.kafka.clients.admin.StreamsGroupSubtopologyDescription;
+import org.apache.kafka.common.GroupState;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.acl.AclOperation;
+import org.apache.kafka.common.message.StreamsGroupDescribeRequestData;
+import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.requests.StreamsGroupDescribeRequest;
+import org.apache.kafka.common.requests.StreamsGroupDescribeResponse;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.kafka.clients.admin.internals.AdminUtils.validAclOperations;
+
+public class DescribeStreamsGroupsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, StreamsGroupDescription> {
+
+    private final boolean includeAuthorizedOperations;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public DescribeStreamsGroupsHandler(
+          boolean includeAuthorizedOperations,
+          LogContext logContext) {
+        this.includeAuthorizedOperations = includeAuthorizedOperations;
+        this.log = logContext.logger(DescribeStreamsGroupsHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    private static Set<CoordinatorKey> buildKeySet(Collection<String> 
groupIds) {
+        return groupIds.stream()
+            .map(CoordinatorKey::byGroupId)
+            .collect(Collectors.toSet());
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
StreamsGroupDescription> newFuture(Collection<String> groupIds) {
+        return AdminApiFuture.forKeys(buildKeySet(groupIds));
+    }
+
+    @Override
+    public String apiName() {
+        return "describeStreamsGroups";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    @Override
+    public StreamsGroupDescribeRequest.Builder buildBatchedRequest(int 
coordinatorId, Set<CoordinatorKey> keys) {
+        List<String> groupIds = keys.stream().map(key -> {
+            if (key.type != CoordinatorType.GROUP) {
+                throw new IllegalArgumentException("Invalid group coordinator 
key " + key +
+                    " when building `DescribeStreamsGroups` request");
+            }
+            return key.idValue;
+        }).collect(Collectors.toList());
+        StreamsGroupDescribeRequestData data = new 
StreamsGroupDescribeRequestData()
+            .setGroupIds(groupIds)
+            .setIncludeAuthorizedOperations(includeAuthorizedOperations);
+        return new StreamsGroupDescribeRequest.Builder(data, true);
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, StreamsGroupDescription> handleResponse(
+            Node coordinator,
+            Set<CoordinatorKey> groupIds,
+            AbstractResponse abstractResponse) {
+        final StreamsGroupDescribeResponse response = 
(StreamsGroupDescribeResponse) abstractResponse;
+        final Map<CoordinatorKey, StreamsGroupDescription> completed = new 
HashMap<>();
+        final Map<CoordinatorKey, Throwable> failed = new HashMap<>();
+        final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
+
+        for (StreamsGroupDescribeResponseData.DescribedGroup describedGroup : 
response.data().groups()) {
+            CoordinatorKey groupIdKey = 
CoordinatorKey.byGroupId(describedGroup.groupId());
+            Errors error = Errors.forCode(describedGroup.errorCode());
+            if (error != Errors.NONE) {
+                handleError(groupIdKey, describedGroup, coordinator, error, 
describedGroup.errorMessage(), completed, failed, groupsToUnmap);
+                continue;
+            }
+            if (describedGroup.topology() == null) {
+                log.error("`DescribeStreamsGroups` response for group id {} is 
missing the topology information", groupIdKey.idValue);

Review Comment:
   are the backticks intentional?



##########
clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java:
##########
@@ -3840,6 +3841,17 @@ public ListShareGroupOffsetsResult 
listShareGroupOffsets(final Map<String, ListS
         return new ListShareGroupOffsetsResult(future.all());
     }
 
+    @Override
+    public DescribeStreamsGroupsResult describeStreamsGroups(final 
Collection<String> groupIds,
+                                                         final 
DescribeStreamsGroupsOptions options) {

Review Comment:
   nit: formatting of parameter



##########
clients/src/main/java/org/apache/kafka/clients/admin/StreamsGroupDescription.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.GroupState;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.acl.AclOperation;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * A detailed description of a single streams group in the cluster.
+ */
+@InterfaceStability.Evolving
+public class StreamsGroupDescription {
+
+    private final String groupId;
+    private final int groupEpoch;
+    private final int targetAssignmentEpoch;
+    private final int topologyEpoch;
+    private final Collection<StreamsGroupSubtopologyDescription> subtopologies;
+    private final Collection<StreamsGroupMemberDescription> members;
+    private final GroupState groupState;
+    private final Node coordinator;
+    private final Set<AclOperation> authorizedOperations;
+
+    public StreamsGroupDescription(
+            final String groupId,
+            final int groupEpoch,
+            final int targetAssignmentEpoch,
+            final int topologyEpoch,
+            final Collection<StreamsGroupSubtopologyDescription> subtopologies,
+            final Collection<StreamsGroupMemberDescription> members,
+            final GroupState groupState,
+            final Node coordinator,
+            final Set<AclOperation> authorizedOperations

Review Comment:
   Do we want to include the endpoint to partitions information in the 
description? I'm not sure.



##########
clients/src/main/java/org/apache/kafka/clients/admin/Admin.java:
##########
@@ -1957,6 +1957,29 @@ default DeleteShareGroupsResult 
deleteShareGroups(Collection<String> groupIds) {
      */
     DeleteShareGroupsResult deleteShareGroups(Collection<String> groupIds, 
DeleteShareGroupsOptions options);
 
+    /**
+     * Describe some streams groups in the cluster.
+     *
+     * @param groupIds The IDs of the groups to describe.
+     * @param options  The options to use when describing the groups.
+     * @return The DescribeStreamsGroupsResult.
+     */
+    DescribeStreamsGroupsResult describeStreamsGroups(Collection<String> 
groupIds,
+                                                      
DescribeStreamsGroupsOptions options);
+
+    /**
+     * Describe some streams groups in the cluster, with the default options.

Review Comment:
   same here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to