walterddr commented on code in PR #10791:
URL: https://github.com/apache/pinot/pull/10791#discussion_r1201473967


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryServer.java:
##########
@@ -83,28 +88,33 @@ public void shutdown() {
   @Override
   public void submit(Worker.QueryRequest request, 
StreamObserver<Worker.QueryResponse> responseObserver) {
     // Deserialize the request
-    DistributedStagePlan distributedStagePlan;
+    List<DistributedStagePlan> distributedStagePlans;
     Map<String, String> requestMetadataMap;
     try {
-      distributedStagePlan = 
QueryPlanSerDeUtils.deserialize(request.getStagePlan());
+      distributedStagePlans = QueryPlanSerDeUtils.deserialize(request);
       requestMetadataMap = request.getMetadataMap();
     } catch (Exception e) {
       LOGGER.error("Caught exception while deserializing the request: {}", 
request, e);
       responseObserver.onError(Status.INVALID_ARGUMENT.withDescription("Bad 
request").withCause(e).asException());
       return;
     }
-
-    try {
-      _queryRunner.processQuery(distributedStagePlan, requestMetadataMap);
-      responseObserver.onNext(Worker.QueryResponse.newBuilder()
-          .putMetadata(QueryConfig.KEY_OF_SERVER_RESPONSE_STATUS_OK, 
"").build());
-      responseObserver.onCompleted();
-    } catch (Throwable t) {
-      responseObserver.onNext(Worker.QueryResponse.newBuilder()
-          .putMetadata(QueryConfig.KEY_OF_SERVER_RESPONSE_STATUS_ERROR, 
QueryException.getTruncatedStackTrace(t))
-          .build());
-      responseObserver.onCompleted();
-    }
+    distributedStagePlans.forEach(distributedStagePlan -> {
+          _executorService.submit(() -> {
+            try {
+              _queryRunner.processQuery(distributedStagePlan, 
requestMetadataMap);
+              responseObserver.onNext(Worker.QueryResponse.newBuilder()
+                  .putMetadata(QueryConfig.KEY_OF_SERVER_RESPONSE_STATUS_OK, 
"").build());
+              responseObserver.onCompleted();
+            } catch (Throwable t) {
+              responseObserver.onNext(Worker.QueryResponse.newBuilder()
+                  .putMetadata(QueryConfig.KEY_OF_SERVER_RESPONSE_STATUS_ERROR,
+                      QueryException.getTruncatedStackTrace(t))
+                  .build());
+              responseObserver.onCompleted();

Review Comment:
   this is not the right way to handle it. we should only return onNext() once. 
if we were to submit multiple server request for running then we should wait 
until all of them returns then return a single success or any error should 
gathered together and return 1 error. this potentially can return multiple 
onNext with errors. 
   
   I suggest we don't catch this and simple let the submission go through 
successfully. then return onCompleted once. 



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