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



##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import 
org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition;
+import 
org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic;
+import 
org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition;
+import 
org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.requests.OffsetCommitResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public class AlterConsumerGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+    private final Map<TopicPartition, OffsetAndMetadata> offsets;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public AlterConsumerGroupOffsetsHandler(
+        String groupId,
+        Map<TopicPartition, OffsetAndMetadata> offsets,
+        LogContext logContext
+    ) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.offsets = offsets;
+        this.log = logContext.logger(AlterConsumerGroupOffsetsHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "offsetCommit";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(
+            String groupId
+    ) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    @Override
+    public OffsetCommitRequest.Builder buildRequest(int coordinatorId, 
Set<CoordinatorKey> keys) {
+        List<OffsetCommitRequestTopic> topics = new ArrayList<>();
+        Map<String, List<OffsetCommitRequestPartition>> offsetData = new 
HashMap<>();
+        for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : 
offsets.entrySet()) {
+            String topic = entry.getKey().topic();
+            OffsetAndMetadata oam = entry.getValue();
+            OffsetCommitRequestPartition partition = new 
OffsetCommitRequestPartition()
+                    .setCommittedOffset(oam.offset())
+                    .setCommittedLeaderEpoch(oam.leaderEpoch().orElse(-1))
+                    .setCommittedMetadata(oam.metadata())
+                    .setPartitionIndex(entry.getKey().partition());
+            offsetData.computeIfAbsent(topic, key -> new 
ArrayList<>()).add(partition);
+        }
+        for (Map.Entry<String, List<OffsetCommitRequestPartition>> entry : 
offsetData.entrySet()) {
+            OffsetCommitRequestTopic topic = new OffsetCommitRequestTopic()
+                    .setName(entry.getKey())
+                    .setPartitions(entry.getValue());
+            topics.add(topic);
+        }
+        OffsetCommitRequestData data = new OffsetCommitRequestData()
+            .setGroupId(groupId.idValue)
+            .setTopics(topics);
+        return new OffsetCommitRequest.Builder(data);
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(Node coordinator, Set<CoordinatorKey> groupIds,
+            AbstractResponse abstractResponse) {

Review comment:
       Should we adopt one of the other formatting style already used in the PR?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupDescription.java
##########
@@ -49,7 +49,7 @@ public ConsumerGroupDescription(String groupId,
         this(groupId, isSimpleConsumerGroup, members, partitionAssignor, 
state, coordinator, Collections.emptySet());
     }
 
-    ConsumerGroupDescription(String groupId,
+    public ConsumerGroupDescription(String groupId,

Review comment:
       Could we realign the arguments?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import 
org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition;
+import 
org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic;
+import 
org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition;
+import 
org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.requests.OffsetCommitResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public class AlterConsumerGroupOffsetsHandler implements 
AdminApiHandler<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+    private final Map<TopicPartition, OffsetAndMetadata> offsets;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public AlterConsumerGroupOffsetsHandler(
+        String groupId,
+        Map<TopicPartition, OffsetAndMetadata> offsets,
+        LogContext logContext
+    ) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.offsets = offsets;
+        this.log = logContext.logger(AlterConsumerGroupOffsetsHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "offsetCommit";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(
+            String groupId

Review comment:
       The indentation of the arguments is not consistent in few places. For 
instance here, it should be consistent with the constructor.

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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,

Review comment:
       Indentation is not consistent 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.

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


Reply via email to