zhaohaidao commented on code in PR #14271: URL: https://github.com/apache/kafka/pull/14271#discussion_r1318745164
########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -428,9 +432,51 @@ 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 ArrayList<>(); + for (TopicPartition tp : runtime.partitions()) { + futures.add(runtime.scheduleReadOperation( + "list-groups", + tp, + (coordinator, lastCommittedOffset) -> coordinator.listGroups(request.statesFilter(), lastCommittedOffset) + ).exceptionally(exception -> { + if (!(exception instanceof KafkaException)) { + log.error("ListGroups request {} hit an unexpected exception: {}", + request, exception.getMessage()); + throw new RuntimeException(exception); + } + if (exception instanceof CoordinatorLoadInProgressException) { + throw new RuntimeException(exception); + } else if (exception instanceof NotCoordinatorException) { + log.warn("ListGroups request {} hit a NotCoordinatorException exception: {}", + request, exception.getMessage()); + return new ListGroupsResponseData().setGroups(Collections.emptyList()); + } else { + return new ListGroupsResponseData().setErrorCode(Errors.forException(exception).code()); + } + })); + } + CompletableFuture<ListGroupsResponseData> responseFuture = new CompletableFuture<>(); + List<ListGroupsResponseData.ListedGroup> listedGroups = new ArrayList<>(); + AtomicInteger succeedFutureCount = new AtomicInteger(); + FutureUtils.drainFutures(futures, (data, t) -> { + synchronized (runtime) { + if (t != null) { + responseFuture.completeExceptionally(new UnknownServerException(t.getMessage())); + } else { + if (data.errorCode() != Errors.NONE.code()) { + if (!responseFuture.isDone()) { + responseFuture.complete(data); + } + } else { + listedGroups.addAll(data.groups()); + if (succeedFutureCount.addAndGet(1) == runtime.partitions().size()) { + responseFuture.complete(new ListGroupsResponseData().setGroups(listedGroups)); + } + } + } + } + }); + return responseFuture; Review Comment: Thank you for your suggestions. The code is much simpler and more elegant. -- 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