zhaohaidao commented on code in PR #14271: URL: https://github.com/apache/kafka/pull/14271#discussion_r1314219340
########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -428,9 +430,44 @@ public CompletableFuture<ListGroupsResponseData> listGroups( return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception()); } - return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception( - "This API is not implemented yet." - )); + List<CompletableFuture<ListGroupsResponseData>> futures = new java.util.ArrayList<>(Collections.emptyList()); + for (TopicPartition tp : runtime.partitions()) { + futures.add(runtime.scheduleReadOperation( + "list_groups", + tp, + (coordinator, lastCommittedOffset) -> coordinator.listGroups(context, request, lastCommittedOffset) + ).exceptionally(exception -> { + if (!(exception instanceof KafkaException)) { + log.error("ListGroups request {} hit an unexpected exception: {}", + request, exception.getMessage()); + } + return new ListGroupsResponseData() + .setErrorCode(Errors.forException(exception).code()); + })); + } + CompletableFuture<ListGroupsResponseData> responseFuture = new CompletableFuture<>(); + List<ListGroupsResponseData.ListedGroup> listedGroups = new ArrayList<>(); + futures.forEach(CompletableFuture::join); + for (CompletableFuture<ListGroupsResponseData> future : futures) { + try { + ListGroupsResponseData data = future.get(); + if (data.errorCode() != Errors.NONE.code()) { + responseFuture.complete(data); + return responseFuture; + } + listedGroups.addAll(future.get().groups()); + } catch (InterruptedException | ExecutionException e) { + log.error("ListGroups request {} hit an unexpected exception: {}", + request, e.getMessage()); + if (!responseFuture.isDone()) { + responseFuture.complete(new ListGroupsResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())); + return responseFuture; + } + } + } Review Comment: I tried to simplify and improve a version based on my own understanding, but I’m not sure if it’s consistent with what you expected. ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -428,9 +430,44 @@ public CompletableFuture<ListGroupsResponseData> listGroups( return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception()); } - return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception( - "This API is not implemented yet." - )); + List<CompletableFuture<ListGroupsResponseData>> futures = new java.util.ArrayList<>(Collections.emptyList()); + for (TopicPartition tp : runtime.partitions()) { + futures.add(runtime.scheduleReadOperation( + "list_groups", + tp, + (coordinator, lastCommittedOffset) -> coordinator.listGroups(context, request, lastCommittedOffset) + ).exceptionally(exception -> { + if (!(exception instanceof KafkaException)) { + log.error("ListGroups request {} hit an unexpected exception: {}", + request, exception.getMessage()); + } + return new ListGroupsResponseData() + .setErrorCode(Errors.forException(exception).code()); + })); + } + CompletableFuture<ListGroupsResponseData> responseFuture = new CompletableFuture<>(); + List<ListGroupsResponseData.ListedGroup> listedGroups = new ArrayList<>(); + futures.forEach(CompletableFuture::join); + for (CompletableFuture<ListGroupsResponseData> future : futures) { + try { + ListGroupsResponseData data = future.get(); + if (data.errorCode() != Errors.NONE.code()) { + responseFuture.complete(data); + return responseFuture; + } + listedGroups.addAll(future.get().groups()); + } catch (InterruptedException | ExecutionException e) { + log.error("ListGroups request {} hit an unexpected exception: {}", + request, e.getMessage()); + if (!responseFuture.isDone()) { + responseFuture.complete(new ListGroupsResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())); + return responseFuture; + } + } + } Review Comment: I tried to simplify and improve a version based on my own understanding, but I’m not sure if it’s consistent with what you expected. -- 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