You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "walterddr (via GitHub)" <gi...@apache.org> on 2023/02/03 16:41:24 UTC

[GitHub] [pinot] walterddr commented on a diff in pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -100,9 +104,8 @@ public int submit(long requestId, QueryPlan queryPlan, long timeoutMs, Map<Strin
           DispatchClient client = getOrCreateDispatchClient(host, servicePort);
           Worker.QueryResponse response = client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
                   QueryPlanSerDeUtils.serialize(constructDistributedStagePlan(queryPlan, stageId, serverInstance)))
-              .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-              .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-              .putAllMetadata(queryOptions).build());
+              .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId)).putAllMetadata(queryOptions)

Review Comment:
   nit: please configure your IDE to not automatically reformat based on checkstyle. 
   
   This will help other reviews to not focus on these auto-reformat changes, thx



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -84,9 +87,10 @@ public ResultTable submitAndReduce(long requestId, QueryPlan queryPlan,
     return resultTable;
   }
 
-  public int submit(long requestId, QueryPlan queryPlan, long timeoutMs, Map<String, String> queryOptions)
+  public int submit(long requestId, QueryPlan queryPlan, long deadlineNanos, Map<String, String> queryOptions)
       throws Exception {
     int reduceStageId = -1;
+    Deadline deadline = Deadline.after(deadlineNanos - System.nanoTime(), TimeUnit.NANOSECONDS);

Review Comment:
   could we put query deadline as not part of the GRPC dispatch request deadline but instead in the `metadata`?
   
   IMO the deadline for the dispatch GRPC call is not related to the mailbox GRPC call or the query execution, as they return immediately after the plan is compiled. we might later on provide a different deadline for dispatch for the compilation. 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryServer.java:
##########
@@ -79,6 +81,7 @@ public void submit(Worker.QueryRequest request, StreamObserver<Worker.QueryRespo
     // Deserialize the request
     DistributedStagePlan distributedStagePlan;
     Map<String, String> requestMetadataMap;
+    long deadlineNanos = Context.current().getDeadline().timeRemaining(TimeUnit.NANOSECONDS) + System.nanoTime();

Review Comment:
   ditto, please change to extract these from metadata



-- 
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-unsubscribe@pinot.apache.org

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


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