ankitsultana commented on code in PR #10135:
URL: https://github.com/apache/pinot/pull/10135#discussion_r1073672483


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryServer.java:
##########
@@ -89,25 +88,12 @@ public void submit(Worker.QueryRequest request, 
StreamObserver<Worker.QueryRespo
       return;
     }
 
-    // return dispatch successful.
-    // TODO: return meaningful value here.
+    // TODO: break this into parsing and execution, so that responseObserver 
can return upon compilation complete.
+    // compilation complete indicates dispatch successful.
+    _executorService.submit(() -> 
_queryRunner.processQuery(distributedStagePlan, requestMetadataMap));

Review Comment:
   IIUC with this PR the same thread pool is used for:
   
   1. Accepting the dispatched plan and running the leaf stage operator (if the 
dispatched stage is a leaf stage).
   2. Accepting the dispatched plan and submitting the OpChain to the scheduler 
(if it's a non-leaf stage)
   
   One advantage I can see with this approach is that the non-leaf stages will 
simply schedule the OpChain and exit, whereas the leaf stages will get directly 
submitted to the thread-pool and hence start running asap.
   
   However we are getting this advantage implicitly (it relies on the fact that 
we are using the same thread pool).
   
   Long term would we want to use the scheduler itself to schedule the leaf 
stages too? We could add another API in the scheduler that fast-tracks 
execution for leaf stages (by perhaps doing exactly what we are doing here).
   
   Right now we are extracting the scheduler's thread-pool out and then using 
it to submit threads directly which doesn't seem like the cleanest design, 
since the scheduler may also be interested in tracking the leaf stage operators 
later.



-- 
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: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to