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


##########
clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java:
##########
@@ -5762,6 +5764,347 @@ public void 
testDeleteStreamsGroupOffsetsFindCoordinatorNonRetriableErrors() thr
         }
     }
 
+    @Test
+    public void testDescribeStreamsGroups() throws Exception {
+        try (AdminClientUnitTestEnv env = new 
AdminClientUnitTestEnv(mockCluster(1, 0))) {
+            env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+
+            // Retriable FindCoordinatorResponse errors should be retried
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE,
  Node.noNode()));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS,
  Node.noNode()));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            StreamsGroupDescribeResponseData data = new 
StreamsGroupDescribeResponseData();
+
+            // Retriable errors should be retried
+            data.groups().add(new 
StreamsGroupDescribeResponseData.DescribedGroup()
+                .setGroupId(GROUP_ID)
+                .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()));
+            env.kafkaClient().prepareResponse(new 
StreamsGroupDescribeResponse(data));
+
+            /*
+             * We need to return two responses here, one with NOT_COORDINATOR 
error when calling describe streams group
+             * api using coordinator that has moved. This will retry whole 
operation. So we need to again respond with a
+             * FindCoordinatorResponse.
+             *
+             * And the same reason for COORDINATOR_NOT_AVAILABLE error response
+             */
+            data = new StreamsGroupDescribeResponseData();
+            data.groups().add(new 
StreamsGroupDescribeResponseData.DescribedGroup()
+                .setGroupId(GROUP_ID)
+                .setErrorCode(Errors.NOT_COORDINATOR.code()));
+            env.kafkaClient().prepareResponse(new 
StreamsGroupDescribeResponse(data));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            data = new StreamsGroupDescribeResponseData();
+            data.groups().add(new 
StreamsGroupDescribeResponseData.DescribedGroup()
+                .setGroupId(GROUP_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()));
+            env.kafkaClient().prepareResponse(new 
StreamsGroupDescribeResponse(data));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            data = makeFullStreamsGroupDescribeResponse();
+
+            env.kafkaClient().prepareResponse(new 
StreamsGroupDescribeResponse(data));
+
+            final DescribeStreamsGroupsResult result = 
env.adminClient().describeStreamsGroups(singletonList(GROUP_ID));
+            final StreamsGroupDescription groupDescription = 
result.describedGroups().get(GROUP_ID).get();
+
+            final String subtopologyId = "my_subtopology";
+            StreamsGroupMemberAssignment.TaskIds expectedActiveTasks1 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(0, 1, 2));
+            StreamsGroupMemberAssignment.TaskIds expectedStandbyTasks1 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(3, 4, 5));
+            StreamsGroupMemberAssignment.TaskIds expectedWarmupTasks1 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(6, 7, 8));
+            StreamsGroupMemberAssignment.TaskIds expectedActiveTasks2 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(3, 4, 5));
+            StreamsGroupMemberAssignment.TaskIds expectedStandbyTasks2 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(6, 7, 8));
+            StreamsGroupMemberAssignment.TaskIds expectedWarmupTasks2 =
+                new StreamsGroupMemberAssignment.TaskIds(subtopologyId, 
asList(0, 1, 2));
+            StreamsGroupMemberAssignment expectedMemberAssignment = new 
StreamsGroupMemberAssignment(
+                singletonList(expectedActiveTasks1),
+                singletonList(expectedStandbyTasks1),
+                singletonList(expectedWarmupTasks1)
+            );
+            StreamsGroupMemberAssignment expectedTargetAssignment = new 
StreamsGroupMemberAssignment(
+                singletonList(expectedActiveTasks2),
+                singletonList(expectedStandbyTasks2),
+                singletonList(expectedWarmupTasks2)
+            );
+            final String instanceId = "instance-id";
+            final String rackId = "rack-id";
+            StreamsGroupMemberDescription expectedMemberOne = new 
StreamsGroupMemberDescription(
+                "0",
+                1,
+                Optional.of(instanceId),
+                Optional.of(rackId),
+                "clientId0",
+                "clientHost",
+                0,
+                "processId",
+                Optional.of(new 
StreamsGroupMemberDescription.Endpoint("localhost", 8080)),
+                Collections.singletonMap("key", "value"),
+                Collections.singletonList(new 
StreamsGroupMemberDescription.TaskOffset(subtopologyId, 0, 0)),
+                Collections.singletonList(new 
StreamsGroupMemberDescription.TaskOffset(subtopologyId, 0, 1)),
+                expectedMemberAssignment,
+                expectedTargetAssignment,
+                true
+            );
+
+            StreamsGroupMemberDescription expectedMemberTwo = new 
StreamsGroupMemberDescription(
+                "1",
+                2,
+                Optional.empty(),
+                Optional.empty(),
+                "clientId1",
+                "clientHost",
+                1,
+                "processId2",
+                Optional.empty(),
+                Collections.emptyMap(),
+                Collections.emptyList(),
+                Collections.emptyList(),
+                new StreamsGroupMemberAssignment(Collections.emptyList(), 
Collections.emptyList(), Collections.emptyList()),
+                new StreamsGroupMemberAssignment(Collections.emptyList(), 
Collections.emptyList(), Collections.emptyList()),
+                false
+            );
+
+            StreamsGroupSubtopologyDescription expectedSubtopologyDescription 
= new StreamsGroupSubtopologyDescription(
+                subtopologyId,
+                Collections.singletonList("my_source_topic"),
+                Collections.singletonList("my_repartition_sink_topic"),
+                Collections.singletonMap(
+                    "my_changelog_topic",
+                    new StreamsGroupSubtopologyDescription.TopicInfo(
+                        0,
+                        (short) 3,
+                        Collections.singletonMap("key1", "value1")
+                    )
+                ),
+                Collections.singletonMap(
+                    "my_repartition_topic",
+                    new StreamsGroupSubtopologyDescription.TopicInfo(
+                        99,
+                        (short) 0,
+                        Collections.emptyMap()
+                    )
+                )
+            );
+
+            assertEquals(1, result.describedGroups().size());
+            assertEquals(GROUP_ID, groupDescription.groupId());
+            assertEquals(2, groupDescription.members().size());
+            Iterator<StreamsGroupMemberDescription> members = 
groupDescription.members().iterator();
+            assertEquals(expectedMemberOne, members.next());
+            assertEquals(expectedMemberTwo, members.next());
+            assertEquals(1, groupDescription.subtopologies().size());
+            assertEquals(expectedSubtopologyDescription, 
groupDescription.subtopologies().iterator().next());
+            assertEquals(2, groupDescription.groupEpoch());
+            assertEquals(1, groupDescription.targetAssignmentEpoch());
+
+        }
+    }
+
+    private static StreamsGroupDescribeResponseData 
makeFullStreamsGroupDescribeResponse() {

Review Comment:
   nit: maybe move helper methods below all the tests?



##########
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:
   no that's fine, I agree about being consistent.



##########
clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java:
##########
@@ -5762,6 +5764,347 @@ public void 
testDeleteStreamsGroupOffsetsFindCoordinatorNonRetriableErrors() thr
         }
     }
 
+    @Test
+    public void testDescribeStreamsGroups() throws Exception {
+        try (AdminClientUnitTestEnv env = new 
AdminClientUnitTestEnv(mockCluster(1, 0))) {
+            env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+
+            // Retriable FindCoordinatorResponse errors should be retried
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE,
  Node.noNode()));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS,
  Node.noNode()));
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            StreamsGroupDescribeResponseData data = new 
StreamsGroupDescribeResponseData();
+
+            // Retriable errors should be retried
+            data.groups().add(new 
StreamsGroupDescribeResponseData.DescribedGroup()
+                .setGroupId(GROUP_ID)
+                .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()));
+            env.kafkaClient().prepareResponse(new 
StreamsGroupDescribeResponse(data));
+
+            /*

Review Comment:
   nit: is the standard way we do multi-line comments? It's minor point so I'll 
leave it to you.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to