AndrewJSchofield commented on code in PR #17775:
URL: https://github.com/apache/kafka/pull/17775#discussion_r1904599414


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -244,6 +261,11 @@ public GroupCoordinatorService build() {
      */
     private final GroupConfigManager groupConfigManager;
 
+    /**
+     * The Persister to persist the state of GC
+     */
+    private final Persister persister;

Review Comment:
   I don't see why the persister is necessary here.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.ListShareGroupOffsetsSpec;
+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.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+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.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ListShareGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, Long>> {
+
+    private final Map<String, ListShareGroupOffsetsSpec> groupSpecs;
+    private final Logger log;
+    private final CoordinatorStrategy lookupStrategy;
+
+    public ListShareGroupOffsetsHandler(
+        Map<String, ListShareGroupOffsetsSpec> groupSpecs,
+        LogContext logContext
+    ) {
+        this.log = logContext.logger(ListShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+        this.groupSpecs = groupSpecs;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Long>> newFuture(Collection<String> groupIds) {
+        return AdminApiFuture.forKeys(coordinatorKeys(groupIds));
+    }
+
+    @Override
+    public String apiName() {
+        return "offsetFetch";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    private void validateKeys(Set<CoordinatorKey> groupIds) {
+        Set<CoordinatorKey> keys = coordinatorKeys(groupSpecs.keySet());
+        if (!keys.containsAll(groupIds)) {
+            throw new IllegalArgumentException("Received unexpected group ids 
" + groupIds +
+                " (expected one of " + keys + ")");
+        }
+    }
+
+    private static Set<CoordinatorKey> coordinatorKeys(Collection<String> 
groupIds) {
+        return groupIds.stream()
+            .map(CoordinatorKey::byGroupId)
+            .collect(Collectors.toSet());
+    }
+
+    public OffsetFetchRequest.Builder buildBatchedRequest(Set<CoordinatorKey> 
groupIds) {
+        // Create a map that only contains the consumer groups owned by the 
coordinator.
+        Map<String, List<TopicPartition>> coordinatorGroupIdToTopicPartitions 
= new HashMap<>(groupIds.size());
+        groupIds.forEach(g -> {
+            ListShareGroupOffsetsSpec spec = groupSpecs.get(g.idValue);
+            List<TopicPartition> partitions = spec.topicPartitions() != null ? 
new ArrayList<>(spec.topicPartitions()) : null;
+            coordinatorGroupIdToTopicPartitions.put(g.idValue, partitions);
+        });
+
+        return new 
OffsetFetchRequest.Builder(coordinatorGroupIdToTopicPartitions, false, false);
+    }
+
+    @Override
+    public Collection<RequestAndKeys<CoordinatorKey>> buildRequest(int 
brokerId, Set<CoordinatorKey> groupIds) {
+        validateKeys(groupIds);
+
+        // When the OffsetFetchRequest fails with 
NoBatchedOffsetFetchRequestException, we completely disable
+        // the batching end-to-end, including the FindCoordinatorRequest.
+        if (lookupStrategy.batch()) {
+            return Collections.singletonList(new 
RequestAndKeys<>(buildBatchedRequest(groupIds), groupIds));
+        } else {
+            return groupIds.stream().map(groupId -> {
+                Set<CoordinatorKey> keys = Collections.singleton(groupId);
+                return new RequestAndKeys<>(buildBatchedRequest(keys), keys);
+            }).collect(Collectors.toList());
+        }
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, Long>> handleResponse(
+        Node coordinator,
+        Set<CoordinatorKey> groupIds,
+        AbstractResponse abstractResponse
+    ) {
+        validateKeys(groupIds);
+
+        final OffsetFetchResponse response = (OffsetFetchResponse) 
abstractResponse;
+
+        Map<CoordinatorKey, Map<TopicPartition, Long>> completed = new 
HashMap<>();
+        Map<CoordinatorKey, Throwable> failed = new HashMap<>();
+        List<CoordinatorKey> unmapped = new ArrayList<>();
+        for (CoordinatorKey coordinatorKey : groupIds) {
+            String group = coordinatorKey.idValue;
+            if (response.groupHasError(group)) {
+                handleGroupError(CoordinatorKey.byGroupId(group), 
response.groupLevelError(group), failed, unmapped);
+            } else {
+                final Map<TopicPartition, Long> groupOffsetsListing = new 
HashMap<>();
+                Map<TopicPartition, OffsetFetchResponse.PartitionData> 
responseData = response.partitionDataMap(group);
+                for (Map.Entry<TopicPartition, 
OffsetFetchResponse.PartitionData> partitionEntry : responseData.entrySet()) {
+                    final TopicPartition topicPartition = 
partitionEntry.getKey();
+                    OffsetFetchResponse.PartitionData partitionData = 
partitionEntry.getValue();
+                    final Errors error = partitionData.error;
+
+                    if (error == Errors.NONE) {
+                        final long offset = partitionData.offset;
+                        // Negative offset indicates that the group has no 
committed offset for this partition
+                        if (offset < 0) {
+                            groupOffsetsListing.put(topicPartition, null);
+                        } else {
+                            groupOffsetsListing.put(topicPartition, offset);
+                        }
+                    } else {
+                        log.warn("Skipping return offset for {} due to error 
{}.", topicPartition, error);
+                    }
+                }
+                completed.put(CoordinatorKey.byGroupId(group), 
groupOffsetsListing);
+            }
+        }
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handleGroupError(
+        CoordinatorKey groupId,
+        Errors error,
+        Map<CoordinatorKey, Throwable> failed,
+        List<CoordinatorKey> groupsToUnmap
+    ) {
+        switch (error) {
+            case GROUP_AUTHORIZATION_FAILED:
+            case UNKNOWN_MEMBER_ID:
+            case STALE_MEMBER_EPOCH:
+                log.debug("`OffsetFetch` request for group id {} failed due to 
error {}", groupId.idValue, error);

Review Comment:
   The remnants of `OffsetFetch` are still evident here. I don't think you need 
handling for `UNKNOWN_MEMBER_ID` or `STALE_MEMBER_EPOCH`. You do need 
`GROUP_ID_NOT_FOUND`.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.ListShareGroupOffsetsSpec;
+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.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+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.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ListShareGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, Long>> {

Review Comment:
   Agreed. It seems to me that `extends AdminApiHandler.Batched<CoordinatorKey, 
Map<TopicPartition, Long>>` is better.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.ListShareGroupOffsetsSpec;
+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.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+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.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ListShareGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, Long>> {
+
+    private final Map<String, ListShareGroupOffsetsSpec> groupSpecs;
+    private final Logger log;
+    private final CoordinatorStrategy lookupStrategy;
+
+    public ListShareGroupOffsetsHandler(
+        Map<String, ListShareGroupOffsetsSpec> groupSpecs,
+        LogContext logContext
+    ) {
+        this.log = logContext.logger(ListShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+        this.groupSpecs = groupSpecs;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Long>> newFuture(Collection<String> groupIds) {
+        return AdminApiFuture.forKeys(coordinatorKeys(groupIds));
+    }
+
+    @Override
+    public String apiName() {
+        return "offsetFetch";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    private void validateKeys(Set<CoordinatorKey> groupIds) {
+        Set<CoordinatorKey> keys = coordinatorKeys(groupSpecs.keySet());
+        if (!keys.containsAll(groupIds)) {
+            throw new IllegalArgumentException("Received unexpected group ids 
" + groupIds +
+                " (expected one of " + keys + ")");
+        }
+    }
+
+    private static Set<CoordinatorKey> coordinatorKeys(Collection<String> 
groupIds) {
+        return groupIds.stream()
+            .map(CoordinatorKey::byGroupId)
+            .collect(Collectors.toSet());
+    }
+
+    public OffsetFetchRequest.Builder buildBatchedRequest(Set<CoordinatorKey> 
groupIds) {
+        // Create a map that only contains the consumer groups owned by the 
coordinator.
+        Map<String, List<TopicPartition>> coordinatorGroupIdToTopicPartitions 
= new HashMap<>(groupIds.size());
+        groupIds.forEach(g -> {
+            ListShareGroupOffsetsSpec spec = groupSpecs.get(g.idValue);
+            List<TopicPartition> partitions = spec.topicPartitions() != null ? 
new ArrayList<>(spec.topicPartitions()) : null;
+            coordinatorGroupIdToTopicPartitions.put(g.idValue, partitions);
+        });
+
+        return new 
OffsetFetchRequest.Builder(coordinatorGroupIdToTopicPartitions, false, false);
+    }
+
+    @Override
+    public Collection<RequestAndKeys<CoordinatorKey>> buildRequest(int 
brokerId, Set<CoordinatorKey> groupIds) {

Review Comment:
   I would model this on `DescribeShareGroupsHandler`. We are going to add a 
bunch more handlers in AK 4.1 to fill out the rest of the admin support for 
KIP-932. There should be a lot of similarity between these handlers.



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