gortiz commented on code in PR #15180:
URL: https://github.com/apache/pinot/pull/15180#discussion_r1991928407


##########
pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/BinaryWorkloadScheduler.java:
##########
@@ -94,7 +96,8 @@ public ListenableFuture<byte[]> submit(ServerQueryRequest 
queryRequest) {
     
queryRequest.getTimerContext().startNewPhaseTimer(ServerQueryPhase.SCHEDULER_WAIT);
     if 
(!QueryOptionsUtils.isSecondaryWorkload(queryRequest.getQueryContext().getQueryOptions()))
 {
       QueryExecutorService queryExecutorService = 
_resourceManager.getExecutorService(queryRequest, null);
-      ListenableFutureTask<byte[]> queryTask = 
createQueryFutureTask(queryRequest, queryExecutorService);
+      ExecutorService innerExecutorService = 
QueryThreadContext.contextAwareExecutorService(queryExecutorService);
+      ListenableFutureTask<byte[]> queryTask = 
createQueryFutureTask(queryRequest, innerExecutorService);

Review Comment:
   That is a very good question. That part of the code is really complex to 
read. In fact, I had to debug it to remember why I was doing that.
   
   The key point here is that there are 2 executors here:
   - The one behind `ResourceManager`, which runs the future we just created.
   - `innerExecutorService`, which can be used inside our future to parallelize 
query processing (ie a group by combine)
   
   The one in ResourceManager is not a context aware executor and in fact 
ResourceManager is being used as a SPI to provide some executor features. We 
could delegate the responsibility of keeping the thread context into 
ResourceManager, but that would break current semantics and third party 
ResourceManager would be broken. Therefore I decided to manage the context in 
the ListenableFuture itself.
   
   Once the ListenableFuture has the context set, if it uses 
`innerExecutorService` to parallelize something, it will be able to get the 
context from the thread running the ListenableFuture.



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