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


##########
clients/src/main/java/org/apache/kafka/clients/admin/Admin.java:
##########
@@ -1780,6 +1780,33 @@ RemoveRaftVoterResult removeRaftVoter(
     DescribeShareGroupsResult describeShareGroups(Collection<String> groupIds,
                                                   DescribeShareGroupsOptions 
options);
 
+    /**

Review Comment:
   You have placed these definitions in between the two definitions for 
`describeShareGroups`. Please untangle them. I suggest these code after both of 
the `describeShareGroups` definitions.



##########
clients/src/main/java/org/apache/kafka/clients/admin/Admin.java:
##########
@@ -1780,6 +1780,33 @@ RemoveRaftVoterResult removeRaftVoter(
     DescribeShareGroupsResult describeShareGroups(Collection<String> groupIds,
                                                   DescribeShareGroupsOptions 
options);
 
+    /**
+     * Alters offsets for the specified group. In order to succeed, the group 
must be empty.
+     *
+     * <p>This operation is not transactional, so it may succeed for some 
partitions while fail for others.
+     *
+     * @param groupId The group for which to alter offsets.
+     * @param offsets A map of offsets by partition. Partitions not specified 
in the map are ignored.
+     * @param options The options to use when altering the offsets.
+     * @return The AlterShareGroupOffsetsResult.
+     */
+    AlterShareGroupOffsetsResult alterShareGroupOffsets(String groupId, 
Map<TopicPartition, Long> offsets, AlterShareGroupOffsetsOptions options);
+

Review Comment:
   nit: Extra blank line.



##########
clients/src/main/resources/common/message/AlterShareGroupOffsetsResponse.json:
##########
@@ -0,0 +1,37 @@
+
+{
+  "apiKey": 91,
+  "type": "response",
+  "name": "AlterShareGroupOffsetsResponse",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  // Supported errors:
+  // - GROUP_AUTHORIZATION_FAILED (version 0+)
+  // - NOT_COORDINATOR (version 0+)
+  // - COORDINATOR_NOT_AVAILABLE (version 0+)
+  // - COORDINATOR_LOAD_IN_PROGRESS (version 0+)
+  // - GROUP_ID_NOT_FOUND (version 0+)
+  // - GROUP_NOT_EMPTY (version 0+)

Review Comment:
   `NON_EMPTY_GROUP`



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.AlterShareGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.KafkaAdminClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsRequestData;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is the handler for {@link 
KafkaAdminClient#alterShareGroupOffsets(String, Map, 
AlterShareGroupOffsetsOptions)} call
+ */
+public class AlterShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+
+    private final Logger log;
+
+    private final Map<TopicPartition, Long> offsets;
+
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+
+    public AlterShareGroupOffsetsHandler(String groupId, Map<TopicPartition, 
Long> offsets, LogContext logContext) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.offsets = offsets;
+        this.log = logContext.logger(AlterShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+    }
+
+    @Override
+    AlterShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<CoordinatorKey> groupIds) {
+        Map<String, Map<TopicPartition, Long>> offsetsByTopic = new 
HashMap<>();
+        offsets.forEach((topicPartition, offset) -> {
+            Map<TopicPartition, Long> offsetsByPartition = 
offsetsByTopic.computeIfAbsent(topicPartition.topic(), v -> new HashMap<>());
+            offsetsByPartition.put(topicPartition, offset);
+        });
+
+        
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopicCollection 
requestTopics = new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopicCollection();
+        for (Map.Entry<String, Map<TopicPartition, Long>> topicEntry : 
offsetsByTopic.entrySet()) {
+            String topic = topicEntry.getKey();
+            Map<TopicPartition, Long> offsetsByPartition = 
topicEntry.getValue();
+
+            
List<AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition> 
partitionList = new ArrayList<>();
+            for (Map.Entry<TopicPartition, Long> partitionEntry : 
offsetsByPartition.entrySet()) {
+                TopicPartition tp = partitionEntry.getKey();
+                Long startOffset = partitionEntry.getValue();
+
+                partitionList.add(new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition()
+                        .setPartitionIndex(tp.partition())
+                        .setStartOffset(startOffset)
+                );
+            }
+            requestTopics.add(new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopic()
+                    .setTopicName(topic)
+                    .setPartitions(partitionList));
+        }
+
+        AlterShareGroupOffsetsRequestData data = new 
AlterShareGroupOffsetsRequestData()
+                .setGroupId(groupId.idValue)
+                .setTopics(requestTopics);
+
+        return new AlterShareGroupOffsetsRequest.Builder(data);
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(String groupId) {

Review Comment:
   I'd prefer to see this method above `buildBatchedRequest`. It's really a 
kind of constructor I feel so putting it nearer the top of the class seems more 
pleasing.



##########
clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java:
##########
@@ -9296,4 +9298,60 @@ public void 
testListShareGroupOffsetsWithErrorInOnePartition() throws Exception
             assertEquals(500, 
partitionToOffsetAndMetadata.get(myTopicPartition3));
         }
     }
+
+    @Test
+    public void testAlterShareGroupOffsets() throws Exception {
+        try (AdminClientUnitTestEnv env = new 
AdminClientUnitTestEnv(mockCluster(1, 0))) {
+            env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            AlterShareGroupOffsetsResponseData data = new 
AlterShareGroupOffsetsResponseData().setResponses(
+                    List.of(
+                        new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic().setTopicName("foo").setPartitions(List.of(new
 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(0),
 new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(1))),
+                        new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic().setTopicName("bar").setPartitions(List.of(new
 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(0)))
+                    )
+            );
+
+            TopicPartition fooTopicPartition0 = new TopicPartition("foo", 0);
+            TopicPartition fooTopicPartition1 = new TopicPartition("foo", 1);
+            TopicPartition barPartition0 = new TopicPartition("bar", 0);
+            TopicPartition zooTopicPartition0 = new TopicPartition("zoo", 0);
+
+            env.kafkaClient().prepareResponse(new 
AlterShareGroupOffsetsResponse(data));
+            final AlterShareGroupOffsetsResult result = 
env.adminClient().alterShareGroupOffsets(GROUP_ID, Map.of(fooTopicPartition0, 
1L, fooTopicPartition1, 2L, barPartition0, 1L));
+
+            assertNull(result.all().get());
+            assertNull(result.partitionResult(fooTopicPartition0).get());
+            assertNull(result.partitionResult(fooTopicPartition1).get());
+            assertNull(result.partitionResult(barPartition0).get());
+            
TestUtils.assertFutureThrows(result.partitionResult(zooTopicPartition0), 
IllegalArgumentException.class);
+        }
+    }
+
+    @Test
+    public void testAlterShareGroupOffsetsWithErrorInOnePartition() throws 
Exception {
+        try (AdminClientUnitTestEnv env = new 
AdminClientUnitTestEnv(mockCluster(1, 0))) {
+            env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            AlterShareGroupOffsetsResponseData data = new 
AlterShareGroupOffsetsResponseData().setResponses(
+                    List.of(
+                            new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic().setTopicName("foo").setPartitions(List.of(new
 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(0),
 new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(1).setErrorCode(Errors.NON_EMPTY_GROUP.code()).setErrorMessage("The
 group is not empty"))),

Review Comment:
   And too wide here.



##########
clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java:
##########
@@ -1389,6 +1389,16 @@ public synchronized DescribeShareGroupsResult 
describeShareGroups(Collection<Str
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
+    @Override
+    public AlterShareGroupOffsetsResult alterShareGroupOffsets(String groupId, 
Map<TopicPartition, Long> offsets, AlterShareGroupOffsetsOptions options) {
+        throw new UnsupportedOperationException("Not implemented yet");
+    }
+
+    @Override

Review Comment:
   This method is unnecessary because of the default implementation in the 
interface.



##########
clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java:
##########
@@ -9296,4 +9298,60 @@ public void 
testListShareGroupOffsetsWithErrorInOnePartition() throws Exception
             assertEquals(500, 
partitionToOffsetAndMetadata.get(myTopicPartition3));
         }
     }
+
+    @Test
+    public void testAlterShareGroupOffsets() throws Exception {
+        try (AdminClientUnitTestEnv env = new 
AdminClientUnitTestEnv(mockCluster(1, 0))) {
+            env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+            
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
+
+            AlterShareGroupOffsetsResponseData data = new 
AlterShareGroupOffsetsResponseData().setResponses(
+                    List.of(
+                        new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic().setTopicName("foo").setPartitions(List.of(new
 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(0),
 new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition().setPartitionIndex(1))),

Review Comment:
   These lines are getting far too wide.



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TestingMetricsInterceptingAdminClient.java:
##########
@@ -416,6 +418,16 @@ public DescribeShareGroupsResult describeShareGroups(final 
Collection<String> gr
         return adminDelegate.describeShareGroups(groupIds, options);
     }
 
+    @Override
+    public AlterShareGroupOffsetsResult alterShareGroupOffsets(final String 
groupId, final Map<TopicPartition, Long> offsets, final 
AlterShareGroupOffsetsOptions options) {
+        return adminDelegate.alterShareGroupOffsets(groupId, offsets, options);
+    }
+
+    @Override

Review Comment:
   This method should not be necessary.



##########
clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java:
##########
@@ -3721,6 +3739,17 @@ private DescribeShareGroupOffsetsResponse 
createDescribeShareGroupOffsetsRespons
         return new DescribeShareGroupOffsetsResponse(data);
     }
 
+    private AlterShareGroupOffsetsResponse 
createAlterShareGroupOffsetsResponse() {
+        AlterShareGroupOffsetsResponseData data = new 
AlterShareGroupOffsetsResponseData()
+                .setResponses(List.of(new 
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()

Review Comment:
   The indentation here is off. Many of these lines are logically nested.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.AlterShareGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.KafkaAdminClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsRequestData;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is the handler for {@link 
KafkaAdminClient#alterShareGroupOffsets(String, Map, 
AlterShareGroupOffsetsOptions)} call
+ */
+public class AlterShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+
+    private final Logger log;
+
+    private final Map<TopicPartition, Long> offsets;
+
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+
+    public AlterShareGroupOffsetsHandler(String groupId, Map<TopicPartition, 
Long> offsets, LogContext logContext) {
+        this.groupId = CoordinatorKey.byGroupId(groupId);
+        this.offsets = offsets;
+        this.log = logContext.logger(AlterShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new 
CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.GROUP, logContext);
+    }
+
+    @Override
+    AlterShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<CoordinatorKey> groupIds) {
+        Map<String, Map<TopicPartition, Long>> offsetsByTopic = new 
HashMap<>();
+        offsets.forEach((topicPartition, offset) -> {
+            Map<TopicPartition, Long> offsetsByPartition = 
offsetsByTopic.computeIfAbsent(topicPartition.topic(), v -> new HashMap<>());
+            offsetsByPartition.put(topicPartition, offset);
+        });
+
+        
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopicCollection 
requestTopics = new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopicCollection();
+        for (Map.Entry<String, Map<TopicPartition, Long>> topicEntry : 
offsetsByTopic.entrySet()) {
+            String topic = topicEntry.getKey();
+            Map<TopicPartition, Long> offsetsByPartition = 
topicEntry.getValue();
+
+            
List<AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition> 
partitionList = new ArrayList<>();
+            for (Map.Entry<TopicPartition, Long> partitionEntry : 
offsetsByPartition.entrySet()) {
+                TopicPartition tp = partitionEntry.getKey();
+                Long startOffset = partitionEntry.getValue();
+
+                partitionList.add(new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition()
+                        .setPartitionIndex(tp.partition())
+                        .setStartOffset(startOffset)
+                );
+            }
+            requestTopics.add(new 
AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopic()
+                    .setTopicName(topic)
+                    .setPartitions(partitionList));
+        }
+
+        AlterShareGroupOffsetsRequestData data = new 
AlterShareGroupOffsetsRequestData()
+                .setGroupId(groupId.idValue)
+                .setTopics(requestTopics);
+
+        return new AlterShareGroupOffsetsRequest.Builder(data);
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Errors>> newFuture(String groupId) {
+        return 
AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId)));
+    }
+
+    @Override
+    public String apiName() {
+        return "alterShareGroupOffsets";
+    }
+
+    @Override
+    public ApiResult<CoordinatorKey, Map<TopicPartition, Errors>> 
handleResponse(Node broker, Set<CoordinatorKey> keys, AbstractResponse 
abstractResponse) {
+        AlterShareGroupOffsetsResponse response = 
(AlterShareGroupOffsetsResponse) abstractResponse;
+        final Map<TopicPartition, Errors> partitionResults = new HashMap<>();
+        final Set<CoordinatorKey> groupsToUnmap = new HashSet<>();
+        final Set<CoordinatorKey> groupsToRetry = new HashSet<>();
+
+        for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic topic : 
response.data().responses()) {
+            for 
(AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition 
partition : topic.partitions()) {
+                TopicPartition topicPartition = new 
TopicPartition(topic.topicName(), partition.partitionIndex());
+                Errors error = Errors.forCode(partition.errorCode());
+
+                if (error != Errors.NONE) {
+                    handleError(
+                            groupId,
+                            topicPartition,
+                            error,
+                            partitionResults,
+                            groupsToUnmap,
+                            groupsToRetry
+                    );
+                } else {
+                    partitionResults.put(topicPartition, error);
+                }
+            }
+        }
+
+        if (groupsToUnmap.isEmpty() && groupsToRetry.isEmpty()) {
+            return ApiResult.completed(groupId, partitionResults);
+        } else {
+            return ApiResult.unmapped(new ArrayList<>(groupsToUnmap));
+        }
+    }
+
+    private void handleError(
+            CoordinatorKey groupId,
+            TopicPartition topicPartition,
+            Errors error,
+            Map<TopicPartition, Errors> partitionResults,
+            Set<CoordinatorKey> groupsToUnmap,
+            Set<CoordinatorKey> groupsToRetry
+    ) {
+        switch (error) {
+            case COORDINATOR_LOAD_IN_PROGRESS:
+            case REBALANCE_IN_PROGRESS:
+                log.debug("AlterShareGroupOffsets request for group id {} 
returned error {}. Will retry.",
+                        groupId.idValue, error);
+                groupsToRetry.add(groupId);
+                break;
+            case COORDINATOR_NOT_AVAILABLE:
+            case NOT_COORDINATOR:
+                log.debug("AlterShareGroupOffsets request for group id {} 
returned error {}. Will rediscover the coordinator and retry.",
+                        groupId.idValue, error);
+                groupsToUnmap.add(groupId);
+                break;
+            case GROUP_ID_NOT_FOUND:
+            case NON_EMPTY_GROUP:

Review Comment:
   Quite right. It should be `NON_EMPTY_GROUP`. I'll update the KIP :)



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.AlterShareGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.KafkaAdminClient;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsRequestData;
+import org.apache.kafka.common.message.AlterShareGroupOffsetsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.AlterShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is the handler for {@link 
KafkaAdminClient#alterShareGroupOffsets(String, Map, 
AlterShareGroupOffsetsOptions)} call
+ */
+public class AlterShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Errors>> {
+
+    private final CoordinatorKey groupId;
+
+    private final Logger log;
+
+    private final Map<TopicPartition, Long> offsets;
+
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;

Review Comment:
   I would prefer the suggestion.



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