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

[GitHub] [pinot] 61yao opened a new pull request, #10219: [multistage] [bugfix] Populate query deadline across the stack

61yao opened a new pull request, #10219:
URL: https://github.com/apache/pinot/pull/10219

   Set rpc deadline and query deadline across the stack.
   
   Use deadline instead of timeout because it is more accurate than passing the timeout


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


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

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1100749417


##########
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:
   done



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


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

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on PR #10219:
URL: https://github.com/apache/pinot/pull/10219#issuecomment-1414131856

   is this built on top of #10190? if so we can merge the other PR first. 
   also test failure seems legit 


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


[GitHub] [pinot] 61yao closed pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao closed pull request #10219: [multistage] [bugfix] Populate query deadline across the stack
URL: https://github.com/apache/pinot/pull/10219


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


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

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
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


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

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096333935


##########
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:
   +1



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


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

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096208827


##########
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:
   yes then it should failed faster as the compilation shouldn't take too long comparing to the actual execution. 



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


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

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096353817


##########
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:
   Sounds good in this case plz add a comment in the QueryServer to explain the deadline is not only used for dispatch grpc but also for entire l query



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


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

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096176942


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -65,15 +67,16 @@ public QueryDispatcher() {
   public ResultTable submitAndReduce(long requestId, QueryPlan queryPlan,
       MailboxService<TransferableBlock> mailboxService, long timeoutMs, Map<String, String> queryOptions)
       throws Exception {
+    long deadlineNanos = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMs);

Review Comment:
   The timeOut passed here is after deducting whatever was spent in building QueryPlan right ?



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


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

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096225314


##########
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:
   it is easier to use single deadline for all requests. I can do a follow up as optimization to set the plan rpc deadline shorter 



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


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

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096188289


##########
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:
   > we might later on provide a different deadline for dispatch for the compilation.
   
   But I feel this should be subsumed right as part of the overall query deadline. I mean what if in a pathological scenario, dispatch for the compilation itself exhausts 



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


[GitHub] [pinot] 61yao commented on pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao commented on PR #10219:
URL: https://github.com/apache/pinot/pull/10219#issuecomment-1415001499

   > #9571 (comment)
   
   Lingering execution problem is not totally solved yet but it is moving toward that direction. 
   
   If the deadline X is crossed, downstream stage won't send data to upstream anymore since the deadline will exceed for RPC side. 
   
   However, there may be lingering execution and data outside the control of rpc. We need to tackle them one by one to solve the leak issue. 
   
   


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


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

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096176942


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -65,15 +67,16 @@ public QueryDispatcher() {
   public ResultTable submitAndReduce(long requestId, QueryPlan queryPlan,
       MailboxService<TransferableBlock> mailboxService, long timeoutMs, Map<String, String> queryOptions)
       throws Exception {
+    long deadlineNanos = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMs);

Review Comment:
   The `timeOut` passed here is after deducting whatever was spent in building `QueryPlan` right ?



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


[GitHub] [pinot] siddharthteotia commented on pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on PR #10219:
URL: https://github.com/apache/pinot/pull/10219#issuecomment-1416286552

   > #9571
   
   Sounds good. Thanks for clarifying. 


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


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

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao commented on code in PR #10219:
URL: https://github.com/apache/pinot/pull/10219#discussion_r1096222328


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -65,15 +67,16 @@ public QueryDispatcher() {
   public ResultTable submitAndReduce(long requestId, QueryPlan queryPlan,
       MailboxService<TransferableBlock> mailboxService, long timeoutMs, Map<String, String> queryOptions)
       throws Exception {
+    long deadlineNanos = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMs);

Review Comment:
   It is actually after plan. It is a good point. I moved the counting of start time to the place where we receive the request. So it is now counting all of the time including parsing and planning.   



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


[GitHub] [pinot] 61yao commented on pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

Posted by "61yao (via GitHub)" <gi...@apache.org>.
61yao commented on PR #10219:
URL: https://github.com/apache/pinot/pull/10219#issuecomment-1413235271

   @xiangfu0


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


[GitHub] [pinot] codecov-commenter commented on pull request #10219: [multistage] [bugfix] Populate query deadline across the stack

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10219:
URL: https://github.com/apache/pinot/pull/10219#issuecomment-1413311515

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10219](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a8cd74c) into [master](https://codecov.io/gh/apache/pinot/commit/76a38bffab2530ca427e1ee281d50056a6b90633?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (76a38bf) will **decrease** coverage by `35.26%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10219       +/-   ##
   =============================================
   - Coverage     70.40%   35.15%   -35.26%     
   + Complexity     5756      202     -5554     
   =============================================
     Files          2016     2016               
     Lines        109100   109145       +45     
     Branches      16562    16577       +15     
   =============================================
   - Hits          76815    38372    -38443     
   - Misses        26911    67483    +40572     
   + Partials       5374     3290     -2084     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `24.58% <0.00%> (-0.03%)` | :arrow_down: |
   | integration2 | `24.60% <0.00%> (+0.02%)` | :arrow_up: |
   | unittests1 | `?` | |
   | unittests2 | `13.70% <0.00%> (-0.02%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/query/mailbox/GrpcMailboxService.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9HcnBjTWFpbGJveFNlcnZpY2UuamF2YQ==) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...apache/pinot/query/mailbox/GrpcSendingMailbox.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9HcnBjU2VuZGluZ01haWxib3guamF2YQ==) | `0.00% <0.00%> (-82.36%)` | :arrow_down: |
   | [...he/pinot/query/mailbox/InMemoryMailboxService.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9Jbk1lbW9yeU1haWxib3hTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/query/mailbox/InMemorySendingMailbox.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9Jbk1lbW9yeVNlbmRpbmdNYWlsYm94LmphdmE=) | `0.00% <0.00%> (-61.54%)` | :arrow_down: |
   | [...inot/query/mailbox/MultiplexingMailboxService.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9NdWx0aXBsZXhpbmdNYWlsYm94U2VydmljZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/query/runtime/QueryRunner.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9RdWVyeVJ1bm5lci5qYXZh) | `0.00% <0.00%> (-85.00%)` | :arrow_down: |
   | [...query/runtime/operator/MailboxReceiveOperator.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9NYWlsYm94UmVjZWl2ZU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-95.66%)` | :arrow_down: |
   | [...ot/query/runtime/operator/MailboxSendOperator.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9NYWlsYm94U2VuZE9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-91.38%)` | :arrow_down: |
   | [...query/runtime/operator/exchange/BlockExchange.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9leGNoYW5nZS9CbG9ja0V4Y2hhbmdlLmphdmE=) | `0.00% <0.00%> (-94.12%)` | :arrow_down: |
   | [...y/runtime/operator/exchange/BroadcastExchange.java](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9leGNoYW5nZS9Ccm9hZGNhc3RFeGNoYW5nZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1155 more](https://codecov.io/gh/apache/pinot/pull/10219?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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