apoorvmittal10 commented on code in PR #18819:
URL: https://github.com/apache/kafka/pull/18819#discussion_r1949027495
##########
docs/security.html:
##########
@@ -2271,6 +2271,12 @@ <h5 class="anchor-heading"><a
id="operations_resources_and_protocols" class="anc
<td>Group</td>
<td></td>
</tr>
+ <tr>
+ <td>ALTER_SHARE_GROUP_OFFSETS (90)</td>
Review Comment:
Shouldn't it be 91?
```suggestion
<td>ALTER_SHARE_GROUP_OFFSETS (91)</td>
```
##########
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:
The KIP mentions `GROUP_NOT_EMPTY`, probably we need to update KIP as per
the Errors.java.
cc: @AndrewJSchofield
##########
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) {
Review Comment:
KIP also defines `GROUP_AUTHORIZATION_FAILED` and `KAFKA_STORAGE_ERROR` do
we need to handle that as well here?
##########
server/src/main/java/org/apache/kafka/network/RequestConvertToJson.java:
##########
@@ -559,6 +563,8 @@ public static JsonNode request(AbstractRequest request) {
return
WriteShareGroupStateRequestDataJsonConverter.write(((WriteShareGroupStateRequest)
request).data(), request.version());
case WRITE_TXN_MARKERS:
return
WriteTxnMarkersRequestDataJsonConverter.write(((WriteTxnMarkersRequest)
request).data(), request.version());
+ case ALTER_SHARE_GROUP_OFFSETS:
+ return
AlterShareGroupOffsetsRequestDataJsonConverter.write(((AlterShareGroupOffsetsRequest)
request).data(), request.version());
Review Comment:
The case here are in sorted order, can we please maintain that.
--
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]