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


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java:
##########
@@ -113,20 +117,30 @@ public void tearDown() throws Exception
   public void 
test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks()
 throws Exception
   {
     // Store futures in a queue
-    final Queue<ListenableFuture<ClusterByStatisticsSnapshot>> futureQueue = 
new ConcurrentLinkedQueue<>();
+    final Queue<Future<?>> futureQueue = new ConcurrentLinkedQueue<>();
     final List<String> workerIds = ImmutableList.of("0", "1", "2", "3");
     final CountDownLatch latch = new CountDownLatch(workerIds.size());
 
-    target = spy(new WorkerSketchFetcher(workerClient, 
ClusterStatisticsMergeMode.PARALLEL, 300_000_000));
+    target = spy(
+        new WorkerSketchFetcher(
+            workerClient,
+            ClusterStatisticsMergeMode.PARALLEL,
+            300_000_000,
+            executorService
+        )
+    );
+
+    // When submitting futures from the executor, add it to the list first.
+    doAnswer(invocation -> {
+      Future<?> future = spy((Future<?>) invocation.callRealMethod());
+      futureQueue.add(future);
+      return future;
+    }).when(executorService).submit(any(Runnable.class));
 
-    // When fetching snapshots, return a mock and add future to queue
     doAnswer(invocation -> {
-      ListenableFuture<ClusterByStatisticsSnapshot> snapshotListenableFuture =
-          
spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class)));
-      futureQueue.add(snapshotListenableFuture);
       latch.countDown();
       latch.await();
-      return snapshotListenableFuture;
+      return Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class));
     }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), 
anyInt());

Review Comment:
   1. shouldn't it be `eq("1")` in the first argument, since below we're adding 
a different mock for `eq("2")` ? 
   2. Can we return a custom future from here which is never done and also 
captures the `cancel` invocation to increment the cancel counter - that could 
remove the need for spying executor service and creating new testing 
constructors for fetcher? 
   3. Why are we doing `latch.await()` in every thread and using sleep in the 
main test thread? Didn't understand that part



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

Review Comment:
   is there a reason for making this `ConcurrentLinkedQueue` instead of a 
simple list?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java:
##########
@@ -231,6 +260,7 @@ public void submitFetchingTasksForNextTimeChunk()
             
stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes);
         // Guarded by synchronized mergedStatisticsCollector
         Set<Integer> finishedWorkers = new HashSet<>();
+        final Queue<Future<?>> executorFutures = new ConcurrentLinkedQueue<>();

Review Comment:
   same as above



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