apoorvmittal10 commented on code in PR #18671:
URL: https://github.com/apache/kafka/pull/18671#discussion_r1931151512


##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.message.DescribeShareGroupOffsetsRequestData;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.DescribeShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.DescribeShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+
+public class DescribeShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Long>> {
+
+    private final Map<String, ListShareGroupOffsetsSpec> groupSpecs;
+    private final Logger log;
+    private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
+
+    public DescribeShareGroupOffsetsHandler(
+        Map<String, ListShareGroupOffsetsSpec> groupSpecs,
+        LogContext logContext) {
+        this.groupSpecs = groupSpecs;
+        this.log = logContext.logger(DescribeShareGroupOffsetsHandler.class);
+        this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, 
logContext);
+    }
+
+    public static AdminApiFuture.SimpleAdminApiFuture<CoordinatorKey, 
Map<TopicPartition, Long>> newFuture(Collection<String> groupIds) {
+        return AdminApiFuture.forKeys(coordinatorKeys(groupIds));
+    }
+
+    @Override
+    public String apiName() {
+        return "describeShareGroupOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<CoordinatorKey> lookupStrategy() {
+        return lookupStrategy;
+    }
+
+    @Override
+    public DescribeShareGroupOffsetsRequest.Builder buildBatchedRequest(int 
coordinatorId, Set<CoordinatorKey> keys) {
+        List<String> groupIds = keys.stream().map(key -> {
+            if (key.type != FindCoordinatorRequest.CoordinatorType.GROUP) {
+                throw new IllegalArgumentException("Invalid group coordinator 
key " + key +
+                    " when building `DescribeShareGroupOffsets` request");
+            }
+            return key.idValue;
+        }).collect(Collectors.toList());
+        String groupId = groupIds.isEmpty() ? null : groupIds.get(0);
+        if (groupId == null) {
+            throw new IllegalArgumentException("GroupId is null");
+        }

Review Comment:
   nit: may be:
   ```
   if (groupIds.isEmpty() || groupIds.get(0) == null) {
       throw new IllegalArgumentException("Missing group id in request");
   }



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeShareGroupOffsetsHandler.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.message.DescribeShareGroupOffsetsRequestData;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.DescribeShareGroupOffsetsRequest;
+import org.apache.kafka.common.requests.DescribeShareGroupOffsetsResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+
+public class DescribeShareGroupOffsetsHandler extends 
AdminApiHandler.Batched<CoordinatorKey, Map<TopicPartition, Long>> {

Review Comment:
   Also can we please add javadoc to the class.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -935,6 +958,53 @@ public 
CompletableFuture<OffsetFetchResponseData.OffsetFetchResponseGroup> fetch
         }
     }
 
+    /**
+     * See {@link GroupCoordinator#describeShareGroupOffsets(RequestContext, 
ReadShareGroupStateSummaryRequestData)}.
+     */
+    @Override
+    public CompletableFuture<ReadShareGroupStateSummaryResponseData> 
describeShareGroupOffsets(
+        RequestContext context,
+        ReadShareGroupStateSummaryRequestData requestData
+    ) {
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                new ReadShareGroupStateSummaryResponseData()
+                    
.setResults(ReadShareGroupStateSummaryRequest.getErrorReadShareGroupStateSummary(
+                        requestData.topics(),
+                        Errors.COORDINATOR_NOT_AVAILABLE
+                    ))
+            );
+        }
+        CompletableFuture<ReadShareGroupStateSummaryResponseData> future = new 
CompletableFuture<>();
+        
persister.readSummary(ReadShareGroupStateSummaryParameters.from(requestData))
+            .whenComplete((result, error) -> {
+                if (error != null) {
+                    log.error("Failed to read summary of the share partition");
+                    future.completeExceptionally(error);
+                    return;
+                }
+                if (result == null || result.topicsData() == null || 
result.topicsData().isEmpty()) {
+                    log.error("Result is null for the read state summary");
+                    future.completeExceptionally(new 
IllegalStateException("Result is null for the read state summary"));
+                    return;
+                }

Review Comment:
   All scenarios not covered in unit tests.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -175,6 +187,8 @@ public GroupCoordinatorService build() {
                 throw new IllegalArgumentException("GroupCoordinatorMetrics 
must be set.");
             if (groupConfigManager == null)
                 throw new IllegalArgumentException("GroupConfigManager must be 
set.");
+            if (persister == null)
+                throw new IllegalArgumentException("Persister must be set.");

Review Comment:
   nit: Can we have following and then we won't require to suppress 
`NPathComplexity` as well?
   
   ```
   requireNonNull(config, new IllegalArgumentException("Config must be set."));
   requireNonNull(writer, new IllegalArgumentException("Writer must be set."));
   ...
   ...
   ...
   }
   
   private static void requireNonNull(Object obj, RuntimeException throwable) {
           if (obj == null) {
               throw throwable;
           }
       }



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -935,6 +958,53 @@ public 
CompletableFuture<OffsetFetchResponseData.OffsetFetchResponseGroup> fetch
         }
     }
 
+    /**
+     * See {@link GroupCoordinator#describeShareGroupOffsets(RequestContext, 
ReadShareGroupStateSummaryRequestData)}.
+     */
+    @Override
+    public CompletableFuture<ReadShareGroupStateSummaryResponseData> 
describeShareGroupOffsets(
+        RequestContext context,
+        ReadShareGroupStateSummaryRequestData requestData
+    ) {
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                new ReadShareGroupStateSummaryResponseData()
+                    
.setResults(ReadShareGroupStateSummaryRequest.getErrorReadShareGroupStateSummary(
+                        requestData.topics(),
+                        Errors.COORDINATOR_NOT_AVAILABLE
+                    ))
+            );
+        }
+        CompletableFuture<ReadShareGroupStateSummaryResponseData> future = new 
CompletableFuture<>();
+        
persister.readSummary(ReadShareGroupStateSummaryParameters.from(requestData))
+            .whenComplete((result, error) -> {
+                if (error != null) {
+                    log.error("Failed to read summary of the share partition");
+                    future.completeExceptionally(error);
+                    return;
+                }
+                if (result == null || result.topicsData() == null || 
result.topicsData().isEmpty()) {

Review Comment:
   Query: is empty topicsData is also an invalid response?



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -10403,6 +10407,181 @@ class KafkaApisTest extends Logging {
     })
   }
 
+  @Test
+  def testDescribeShareGroupOffsetsReturnsUnsupportedVersion(): Unit = {
+    val describeShareGroupOffsetsRequest = new 
DescribeShareGroupOffsetsRequestData().setGroupId("group").setTopics(
+      List(new 
DescribeShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(List(1).map(Int.box).asJava)).asJava

Review Comment:
   Can it be written as below without conversions?
   
   ```
   util.List.of(new 
DescribeShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(util.List.of(1)))



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

Review Comment:
   ```suggestion
        * The Persister to persist the share partition state.
   ```



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -3185,9 +3187,87 @@ class KafkaApis(val requestChannel: RequestChannel,
 
   def handleDescribeShareGroupOffsetsRequest(request: RequestChannel.Request): 
Unit = {
     val describeShareGroupOffsetsRequest = 
request.body[DescribeShareGroupOffsetsRequest]
-    // TODO: Implement the DescribeShareGroupOffsetsRequest handling
-    requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+
+    if (!isShareGroupProtocolEnabled) {
+      requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
+      CompletableFuture.completedFuture[Unit](())
+    } else if (!authHelper.authorize(request.context, READ, GROUP, 
describeShareGroupOffsetsRequest.data.groupId)) {
+      requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception))
+      CompletableFuture.completedFuture[Unit](())
+    } else {
+      val topicNamesToIds = metadataCache.topicNamesToIds()
+      val topicIdToNames = metadataCache.topicIdsToNames()
+
+      val readStateSummaryData = 
getReadShareGroupStateSummaryRequestFromDescribeShareGroupOffsetsRequest(
+        describeShareGroupOffsetsRequest.data(),
+        topicNamesToIds
+      )
+      groupCoordinator.describeShareGroupOffsets(
+        request.context,
+        readStateSummaryData,
+      ).handle[Unit] { (response, exception) =>
+        if (exception != null) {
+          requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(exception))
+        } else {
+          requestHelper.sendMaybeThrottle(
+            request,
+            new DescribeShareGroupOffsetsResponse(
+              
getDescribeShareGroupOffsetsResponseFromReadShareGroupStateSummaryResponse(response,
 topicIdToNames)
+            )
+          )
+        }
+      }
+    }
+  }
+
+  private def 
getReadShareGroupStateSummaryRequestFromDescribeShareGroupOffsetsRequest(describeShareGroupOffsetsRequestData:
 DescribeShareGroupOffsetsRequestData,
+                                                                               
        topicNamesId: util.Map[String, Uuid]
+                                                                               
       ): ReadShareGroupStateSummaryRequestData = {
+    val readStateSummaryTopics = 
describeShareGroupOffsetsRequestData.topics.asScala.map(
+      topic => {
+        val partitions = topic.partitions.asScala.map(
+          partitionIndex => {
+            new PartitionData()
+              .setPartition(partitionIndex)
+              .setLeaderEpoch(0)
+          }
+        ).asJava
+        new ReadStateSummaryData()
+          .setTopicId(topicNamesId.get(topic.topicName()))
+          .setPartitions(partitions)
+      }
+    ).asJava

Review Comment:
   Can't we avoid `asScala` and `asJava` conversions? Seems these conversions 
are not required.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -3185,9 +3187,87 @@ class KafkaApis(val requestChannel: RequestChannel,
 
   def handleDescribeShareGroupOffsetsRequest(request: RequestChannel.Request): 
Unit = {
     val describeShareGroupOffsetsRequest = 
request.body[DescribeShareGroupOffsetsRequest]
-    // TODO: Implement the DescribeShareGroupOffsetsRequest handling
-    requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+
+    if (!isShareGroupProtocolEnabled) {
+      requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
+      CompletableFuture.completedFuture[Unit](())
+    } else if (!authHelper.authorize(request.context, READ, GROUP, 
describeShareGroupOffsetsRequest.data.groupId)) {
+      requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception))
+      CompletableFuture.completedFuture[Unit](())
+    } else {
+      val topicNamesToIds = metadataCache.topicNamesToIds()
+      val topicIdToNames = metadataCache.topicIdsToNames()
+
+      val readStateSummaryData = 
getReadShareGroupStateSummaryRequestFromDescribeShareGroupOffsetsRequest(
+        describeShareGroupOffsetsRequest.data(),
+        topicNamesToIds
+      )
+      groupCoordinator.describeShareGroupOffsets(
+        request.context,
+        readStateSummaryData,
+      ).handle[Unit] { (response, exception) =>
+        if (exception != null) {
+          requestHelper.sendMaybeThrottle(request, 
describeShareGroupOffsetsRequest.getErrorResponse(exception))
+        } else {
+          requestHelper.sendMaybeThrottle(
+            request,
+            new DescribeShareGroupOffsetsResponse(
+              
getDescribeShareGroupOffsetsResponseFromReadShareGroupStateSummaryResponse(response,
 topicIdToNames)
+            )
+          )
+        }
+      }
+    }
+  }
+
+  private def 
getReadShareGroupStateSummaryRequestFromDescribeShareGroupOffsetsRequest(describeShareGroupOffsetsRequestData:
 DescribeShareGroupOffsetsRequestData,
+                                                                               
        topicNamesId: util.Map[String, Uuid]
+                                                                               
       ): ReadShareGroupStateSummaryRequestData = {
+    val readStateSummaryTopics = 
describeShareGroupOffsetsRequestData.topics.asScala.map(
+      topic => {
+        val partitions = topic.partitions.asScala.map(
+          partitionIndex => {
+            new PartitionData()
+              .setPartition(partitionIndex)
+              .setLeaderEpoch(0)
+          }
+        ).asJava
+        new ReadStateSummaryData()
+          .setTopicId(topicNamesId.get(topic.topicName()))
+          .setPartitions(partitions)
+      }
+    ).asJava
+
+    val result = new ReadShareGroupStateSummaryRequestData()
+      .setGroupId(describeShareGroupOffsetsRequestData.groupId())
+      .setTopics(readStateSummaryTopics)
+    result
+  }
+
+  private def 
getDescribeShareGroupOffsetsResponseFromReadShareGroupStateSummaryResponse(readShareGroupStateSummaryResponseData:
 ReadShareGroupStateSummaryResponseData,
+                                                                               
          topicIdNames: util.Map[Uuid, String]
+                                                                               
         ): DescribeShareGroupOffsetsResponseData = {
+    val describeShareGroupOffsetsResponseData = 
readShareGroupStateSummaryResponseData.results().asScala.map(
+      readStateSummaryResult => {
+        val partitions = readStateSummaryResult.partitions().asScala.map(
+          partitionResult => {
+            new DescribeShareGroupOffsetsResponsePartition()
+              .setPartitionIndex(partitionResult.partition())
+              .setStartOffset(partitionResult.startOffset())
+              .setLeaderEpoch(partitionResult.stateEpoch())
+              .setErrorCode(partitionResult.errorCode())
+              .setErrorMessage(partitionResult.errorMessage())
+          }
+        ).asJava
+        new DescribeShareGroupOffsetsResponseTopic()
+          .setTopicId(readStateSummaryResult.topicId())
+          .setTopicName(topicIdNames.get(readStateSummaryResult.topicId()))
+          .setPartitions(partitions)
+      }
+    ).asJava

Review Comment:
   Same as here about conversions.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -935,6 +958,53 @@ public 
CompletableFuture<OffsetFetchResponseData.OffsetFetchResponseGroup> fetch
         }
     }
 
+    /**
+     * See {@link GroupCoordinator#describeShareGroupOffsets(RequestContext, 
ReadShareGroupStateSummaryRequestData)}.
+     */
+    @Override
+    public CompletableFuture<ReadShareGroupStateSummaryResponseData> 
describeShareGroupOffsets(
+        RequestContext context,
+        ReadShareGroupStateSummaryRequestData requestData
+    ) {
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                new ReadShareGroupStateSummaryResponseData()
+                    
.setResults(ReadShareGroupStateSummaryRequest.getErrorReadShareGroupStateSummary(
+                        requestData.topics(),
+                        Errors.COORDINATOR_NOT_AVAILABLE
+                    ))
+            );
+        }
+        CompletableFuture<ReadShareGroupStateSummaryResponseData> future = new 
CompletableFuture<>();
+        
persister.readSummary(ReadShareGroupStateSummaryParameters.from(requestData))
+            .whenComplete((result, error) -> {
+                if (error != null) {
+                    log.error("Failed to read summary of the share partition");
+                    future.completeExceptionally(error);
+                    return;

Review Comment:
   Scenario not covered in unit tests.



##########
server-common/src/main/java/org/apache/kafka/server/share/persister/ReadShareGroupStateSummaryParameters.java:
##########
@@ -58,4 +59,16 @@ public ReadShareGroupStateSummaryParameters build() {
             return new 
ReadShareGroupStateSummaryParameters(groupTopicPartitionData);
         }
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || getClass() != o.getClass()) return false;
+        ReadShareGroupStateSummaryParameters that = 
(ReadShareGroupStateSummaryParameters) o;
+        return Objects.equals(groupTopicPartitionData, 
that.groupTopicPartitionData);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(groupTopicPartitionData);
+    }

Review Comment:
   Why do we need to implement these, am I missing some usage of them? Can't 
see the class usage where these overrides are required.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -10403,6 +10407,181 @@ class KafkaApisTest extends Logging {
     })
   }
 
+  @Test
+  def testDescribeShareGroupOffsetsReturnsUnsupportedVersion(): Unit = {
+    val describeShareGroupOffsetsRequest = new 
DescribeShareGroupOffsetsRequestData().setGroupId("group").setTopics(
+      List(new 
DescribeShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(List(1).map(Int.box).asJava)).asJava

Review Comment:
   Same else where.



-- 
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: jira-unsubscr...@kafka.apache.org

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

Reply via email to