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


##########
core/src/main/java/kafka/server/share/SharePartitionManager.java:
##########
@@ -300,11 +301,25 @@ public CompletableFuture<Map<TopicIdPartition, 
ShareAcknowledgeResponseData.Part
 
         CompletableFuture<Void> allFutures = CompletableFuture.allOf(
             futures.values().toArray(new CompletableFuture[0]));
-        return allFutures.thenApply(v -> {
+        return allFutures.handle((unused, exception) -> {
             Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> 
result = new HashMap<>();
-            futures.forEach((topicIdPartition, future) -> 
result.put(topicIdPartition, new ShareAcknowledgeResponseData.PartitionData()
-                .setPartitionIndex(topicIdPartition.partition())
-                .setErrorCode(future.join().code())));
+            futures.forEach((topicIdPartition, future) -> {
+                try {
+                    Errors error = future.join();
+                    result.put(topicIdPartition,
+                            new ShareAcknowledgeResponseData.PartitionData()
+                                    
.setPartitionIndex(topicIdPartition.partition())
+                                    .setErrorCode(error.code())
+                                    .setErrorMessage(error.message()));
+                } catch (CompletionException e) {
+                    Throwable t = e.getCause();
+                    result.put(topicIdPartition,
+                            new ShareAcknowledgeResponseData.PartitionData()
+                                    
.setPartitionIndex(topicIdPartition.partition())
+                                    
.setErrorCode(Errors.forException(t).code())
+                                    .setErrorMessage(t.getMessage()));
+                }
+            });

Review Comment:
   Hmmm, rather than having try/catch which results from `future.join` we 
should consider declaring `Map<TopicIdPartition, CompletableFuture<Errors>> 
futures = new HashMap<>();` as `Map<TopicIdPartition, 
CompletableFuture<Throwable>> futures = new HashMap<>();`. Rather capturing 
error we should have original `Throwable` itself. And check here if 
`future.join()` results in non-null data then have the error code by calling 
`Errors.forException(throwable)` and set the error message from throwable, 
otherwise set to None error code in response.



-- 
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