chia7712 commented on code in PR #18929: URL: https://github.com/apache/kafka/pull/18929#discussion_r2083236898
########## clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java: ########## @@ -78,6 +78,24 @@ public static AlterShareGroupOffsetsRequest parse(Readable readable, short versi ); } + public static AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic getErrorAlterShareGroup( Review Comment: Maybe we could inline this method ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -1172,6 +1204,37 @@ public CompletableFuture<List<DescribedGroup>> shareGroupDescribe( return FutureUtils.combineFutures(futures, ArrayList::new, List::addAll); } + /** + * See {@link GroupCoordinator#alterShareGroupOffsets(AuthorizableRequestContext, String, AlterShareGroupOffsetsRequestData)}. + */ + @Override + public CompletableFuture<AlterShareGroupOffsetsResponseData> alterShareGroupOffsets(AuthorizableRequestContext context, String groupId, AlterShareGroupOffsetsRequestData request) { + if (!isActive.get() || metadataImage == null) { + return CompletableFuture.completedFuture( + AlterShareGroupOffsetsRequest.from(Errors.COORDINATOR_NOT_AVAILABLE)); Review Comment: Could we return `CompletableFuture.failedFuture`? The exception could be handled by `handleAlterShareGroupOffsetsRequest` instead ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -1172,6 +1204,37 @@ public CompletableFuture<List<DescribedGroup>> shareGroupDescribe( return FutureUtils.combineFutures(futures, ArrayList::new, List::addAll); } + /** + * See {@link GroupCoordinator#alterShareGroupOffsets(AuthorizableRequestContext, String, AlterShareGroupOffsetsRequestData)}. + */ + @Override + public CompletableFuture<AlterShareGroupOffsetsResponseData> alterShareGroupOffsets(AuthorizableRequestContext context, String groupId, AlterShareGroupOffsetsRequestData request) { + if (!isActive.get() || metadataImage == null) { + return CompletableFuture.completedFuture( + AlterShareGroupOffsetsRequest.from(Errors.COORDINATOR_NOT_AVAILABLE)); + } + + if (groupId == null) { + return CompletableFuture.completedFuture( + AlterShareGroupOffsetsRequest.from(Errors.GROUP_ID_NOT_FOUND)); Review Comment: ditto ########## core/src/main/scala/kafka/server/KafkaApis.scala: ########## @@ -3620,7 +3620,50 @@ class KafkaApis(val requestChannel: RequestChannel, def handleAlterShareGroupOffsetsRequest(request: RequestChannel.Request): Unit = { val alterShareGroupOffsetsRequest = request.body[AlterShareGroupOffsetsRequest] - requestHelper.sendMaybeThrottle(request, alterShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) + val groupId = alterShareGroupOffsetsRequest.data.groupId + + if (!isShareGroupProtocolEnabled) { + requestHelper.sendMaybeThrottle(request, alterShareGroupOffsetsRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) + } else if (!authHelper.authorize(request.context, READ, GROUP, groupId)) { + requestHelper.sendMaybeThrottle(request, alterShareGroupOffsetsRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) + } else { + val responseBuilder = new AlterShareGroupOffsetsResponse.Builder() + val authorizedTopicPartitions = new AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestTopicCollection() + + alterShareGroupOffsetsRequest.data.topics.forEach(topic => { + val invalidTopicError = checkValidTopic(topic.topicName()) + val topicError = invalidTopicError.orElse { + if (!authHelper.authorize(request.context, READ, TOPIC, topic.topicName())) { + Some(new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED)) + } else if (!metadataCache.contains(topic.topicName())) { + Some(new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION)) + } else { + None + } + } + topicError match { + case Some(error) => + topic.partitions().forEach(partition => responseBuilder.addPartition(topic.topicName(), partition.partitionIndex(), error.error)) + case None => + authorizedTopicPartitions.add(topic) + } + }) + + val data = new AlterShareGroupOffsetsRequestData() + .setGroupId(groupId) + .setTopics(authorizedTopicPartitions) + groupCoordinator.alterShareGroupOffsets( + request.context, + groupId, + data + ).handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, alterShareGroupOffsetsRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, responseBuilder.merge(response).build()) + } + } + } CompletableFuture.completedFuture[Unit](()) Review Comment: not sure why share-related APIs don't return `CompletableFuture` to chain `.exceptionally(handleError)`. @AndrewJSchofield @apoorvmittal10 Is it expected behavior? -- 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