rohangarg commented on code in PR #13475:
URL: https://github.com/apache/druid/pull/13475#discussion_r1037161873


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -155,7 +152,17 @@ CompletableFuture<Either<Long, ClusterByPartitions>> 
inMemoryFullSketchMerging(
             mergedStatisticsCollector.clear();
           }
         }
-      });
+      }));
+    });
+
+    partitionFuture.whenComplete((result, exception) -> {

Review Comment:
   in a corner case, it could happen that `partitionFuture` finishes before all 
futures are queued up in the executor. To avoid that one simple solution could 
be to have a `CountDownLatch` with value as `workerCount`. The `whenComplete` 
would then wait for latch to become 0 before cancelling all the futures.
   Maybe we could also have a better solution with cancellation - will update 
on this thread if I find any utility.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -118,22 +120,17 @@ CompletableFuture<Either<Long, ClusterByPartitions>> 
inMemoryFullSketchMerging(
     final int workerCount = workerTaskIds.size();
     // Guarded by synchronized mergedStatisticsCollector
     final Set<Integer> finishedWorkers = new HashSet<>();
+    final Set<Future<?>> futuresToCancel = ConcurrentHashMap.newKeySet();

Review Comment:
   this could be a list - not sure if equals is correctly done for futures



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -155,7 +152,17 @@ CompletableFuture<Either<Long, ClusterByPartitions>> 
inMemoryFullSketchMerging(
             mergedStatisticsCollector.clear();

Review Comment:
   we can call this catch block only once by checking `partitionFuture.isDone` 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to