skaundinya15 commented on a change in pull request #10743:
URL: https://github.com/apache/kafka/pull/10743#discussion_r655138186



##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public class ListConsumerGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> {
+
+    private final CoordinatorKey groupId;
+    private final List<TopicPartition> partitions;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public ListConsumerGroupOffsetsHandler(
+        String groupId,
+        List<TopicPartition> partitions,
+        LogContext logContext
+    ) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.partitions = partitions;
+        this.log = logContext.logger(ListConsumerGroupOffsetsHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, OffsetAndMetadata>> newFuture(
+        String groupId
+    ) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    @Override
+    public String apiName() {
+        return "offsetFetch";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    @Override
+    public OffsetFetchRequest.Builder buildRequest(int coordinatorId, 
Set<CoordinatorKey> keys) {
+        // Set the flag to false as for admin client request,
+        // we don't need to wait for any pending offset state to clear.
+        return new OffsetFetchRequest.Builder(groupId.idValue, false, 
partitions, false);
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> 
handleResponse(
+        Node coordinator,
+        Set<CoordinatorKey> groupIds,
+        AbstractResponse abstractResponse
+    ) {
+        final OffsetFetchResponse response = (OffsetFetchResponse) 
abstractResponse;
+        Map<CoordinatorKey, Map<TopicPartition, OffsetAndMetadata>> completed 
= new HashMap<>();
+        Map<CoordinatorKey, Throwable> failed = new HashMap<>();
+        List<CoordinatorKey> unmapped = new ArrayList<>();
+
+        if (response.error() != Errors.NONE) {
+            handleError(groupId, response.error(), failed, unmapped);
+        } else {
+            final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = 
new HashMap<>();
+            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> 
entry :
+                response.responseData().entrySet()) {
+                final TopicPartition topicPartition = entry.getKey();
+                OffsetFetchResponse.PartitionData partitionData = 
entry.getValue();
+                final Errors error = partitionData.error;
+
+                if (error == Errors.NONE) {
+                    final long offset = partitionData.offset;
+                    final String metadata = partitionData.metadata;
+                    final Optional<Integer> leaderEpoch = 
partitionData.leaderEpoch;
+                    // Negative offset indicates that the group has no 
committed offset for this partition
+                    if (offset < 0) {
+                        groupOffsetsListing.put(topicPartition, null);
+                    } else {
+                        groupOffsetsListing.put(topicPartition, new 
OffsetAndMetadata(offset, leaderEpoch, metadata));
+                    }
+                } else {
+                    log.warn("Skipping return offset for {} due to error {}.", 
topicPartition, error);
+                }
+            }
+            completed.put(groupId, groupOffsetsListing);
+        }
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handleError(
+        CoordinatorKey groupId,
+        Errors error,
+        Map<CoordinatorKey,
+        Throwable> failed,
+        List<CoordinatorKey> unmapped
+    ) {
+        switch (error) {
+            case GROUP_AUTHORIZATION_FAILED:
+                log.error("Received authorization failure for group {} in 
`DeleteConsumerGroupOffsets` response", groupId,

Review comment:
       Seems like there's a copy/paste error here. I think we want 
`DeleteConsumerGroupOffsets` here to be `ListConsumerGroupOffsets`. Perhaps 
instead of writing the name for each one of these, we could just call 
`apiName()` instead?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandler.java
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
+import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.LeaveGroupRequest;
+import org.apache.kafka.common.requests.LeaveGroupResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public class RemoveMembersFromConsumerGroupHandler implements 
AdminApiHandler<CoordinatorKey, Map<MemberIdentity, Errors>> {
+
+    private final CoordinatorKey groupId;
+    private final List<MemberIdentity> members;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public RemoveMembersFromConsumerGroupHandler(
+        String groupId,
+        List<MemberIdentity> members,
+        LogContext logContext
+    ) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.members = members;
+        this.log = 
logContext.logger(RemoveMembersFromConsumerGroupHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "leaveGroup";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<MemberIdentity, Errors>> newFuture(
+        String groupId
+    ) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    @Override
+    public LeaveGroupRequest.Builder buildRequest(int coordinatorId, 
Set<CoordinatorKey> keys) {
+        return new LeaveGroupRequest.Builder(groupId.idValue, members);
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<MemberIdentity, Errors>> 
handleResponse(
+        Node coordinator,
+        Set<CoordinatorKey> groupIds,
+        AbstractResponse abstractResponse
+    ) {
+        final LeaveGroupResponse response = (LeaveGroupResponse) 
abstractResponse;
+        Map<CoordinatorKey, Map<MemberIdentity, Errors>> completed = new 
HashMap<>();
+        Map<CoordinatorKey, Throwable> failed = new HashMap<>();
+        List<CoordinatorKey> unmapped = new ArrayList<>();
+
+        final Errors error = Errors.forCode(response.data().errorCode());
+        if (error != Errors.NONE) {
+            handleError(groupId, error, failed, unmapped);
+        } else {
+            final Map<MemberIdentity, Errors> memberErrors = new HashMap<>();
+            for (MemberResponse memberResponse : response.memberResponses()) {
+                memberErrors.put(new MemberIdentity()
+                                     .setMemberId(memberResponse.memberId())
+                                     
.setGroupInstanceId(memberResponse.groupInstanceId()),
+                                 Errors.forCode(memberResponse.errorCode()));
+
+            }
+            completed.put(groupId, memberErrors);
+        }
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handleError(
+        CoordinatorKey groupId,
+        Errors error, Map<CoordinatorKey,
+        Throwable> failed,
+        List<CoordinatorKey> unmapped
+    ) {
+        switch (error) {
+            case GROUP_AUTHORIZATION_FAILED:
+                log.error("Received authorization failure for group {} in 
`DeleteConsumerGroupOffsets` response", groupId,

Review comment:
       Another place where I think we want this to be 
`RemoveMembersFromConsumerGroup` instead of `DeleteConsumerGroupOffsets`




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

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


Reply via email to