dajac commented on code in PR #15196: URL: https://github.com/apache/kafka/pull/15196#discussion_r1453621960
########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ########## @@ -524,37 +495,39 @@ public CompletableFuture<ListGroupsResponseData> listGroups( ); } - final CompletableFuture<ListGroupsResponseData> future = new CompletableFuture<>(); - final List<ListGroupsResponseData.ListedGroup> results = new ArrayList<>(); final Set<TopicPartition> existingPartitionSet = runtime.partitions(); - final AtomicInteger cnt = new AtomicInteger(existingPartitionSet.size()); if (existingPartitionSet.isEmpty()) { return CompletableFuture.completedFuture(new ListGroupsResponseData()); } + final List<CompletableFuture<List<ListGroupsResponseData.ListedGroup>>> futures = + new ArrayList<>(); + for (TopicPartition tp : existingPartitionSet) { - runtime.scheduleReadOperation( + futures.add(runtime.scheduleReadOperation( "list-groups", tp, (coordinator, lastCommittedOffset) -> coordinator.listGroups(request.statesFilter(), lastCommittedOffset) - ).handle((groups, exception) -> { - if (exception == null) { - synchronized (results) { - results.addAll(groups); - } + ).exceptionally(exception -> { + exception = Errors.maybeUnwrapException(exception); + if (exception instanceof NotCoordinatorException) { + return Collections.emptyList(); } else { - if (!(exception instanceof NotCoordinatorException)) { - future.complete(new ListGroupsResponseData().setErrorCode(Errors.forException(exception).code())); - } - } - if (cnt.decrementAndGet() == 0) { - future.complete(new ListGroupsResponseData().setGroups(results)); + throw new CompletionException(exception); } - return null; - }); + })); } - return future; + + return FutureUtils + .combineFutures(futures, ArrayList::new, List::addAll) + .thenApply(groups -> new ListGroupsResponseData().setGroups(groups)) + .exceptionally(exception -> handleOperationException( + "ListGroups", + request, + exception, + (error, __) -> new ListGroupsResponseData().setErrorCode(error.code()) + )); Review Comment: Good observation! I was wondering if failing immediately is really important after all. The read operations for the others should be reasonably quick so the gain is small. Overall, I feel like the new code is easier to reason about and less prone to bugs. What do you think? -- 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