You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "xiangfu0 (via GitHub)" <gi...@apache.org> on 2023/05/21 08:21:12 UTC

[GitHub] [pinot] xiangfu0 opened a new pull request, #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

xiangfu0 opened a new pull request, #10791:
URL: https://github.com/apache/pinot/pull/10791

   - Refactor StageMetadata from `pinot-query-planner` to `pinot-query-runtime` module


-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-common/src/main/proto/worker.proto:
##########
@@ -70,14 +70,15 @@ message QueryResponse {
 
 message StagePlan {
   int32 stageId = 1;
-  string virtualAddress = 2;
-  StageNode stageRoot = 3;
-  StageMetadata stageMetadata = 4;
+  StageNode stageRoot = 2;
+  StageMetadata stageMetadata = 3;
 }
 
 message StageMetadata {
   repeated WorkerMetadata workerMetadata = 1;
   map<string, string> customProperty = 2;
+  QueryServerInstance queryServerInstance = 3;
+  repeated int32 workerIds = 4;

Review Comment:
   why do we need these 2? 
   
   - queryServerInstance is not useful as when data is being sent. hostname and server port is already knew to the broker and channel is being selected. mailbox port is known to that particular server on its own JVM
   - workerId list is actually part of the repeated WorkerMetadata (workerID is part of the virtualAddress)



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#10791](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (4bcf0ff) into [master](https://app.codecov.io/gh/apache/pinot/commit/e302318d33d41facdaff0c03cfbc96e423f41d9d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (e302318) will **decrease** coverage by `20.74%`.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head 4bcf0ff differs from pull request most recent head 31b2c9d. Consider uploading reports for the commit 31b2c9d to get more accurate results
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10791       +/-   ##
   =============================================
   - Coverage     34.43%   13.70%   -20.74%     
   + Complexity      462      439       -23     
   =============================================
     Files          2159     2105       -54     
     Lines        116044   113557     -2487     
     Branches      17569    17272      -297     
   =============================================
   - Hits          39961    15564    -24397     
   - Misses        72604    96720    +24116     
   + Partials       3479     1273     -2206     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests2 | `13.70% <0.00%> (+0.02%)` | :arrow_up: |
   
   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=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [.../pinot/query/planner/DispatchablePlanFragment.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcGxhbm5lci9EaXNwYXRjaGFibGVQbGFuRnJhZ21lbnQuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...va/org/apache/pinot/query/runtime/QueryRunner.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9RdWVyeVJ1bm5lci5qYXZh) | `0.00% <ø> (ø)` | |
   | [...pinot/query/runtime/plan/DistributedStagePlan.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL0Rpc3RyaWJ1dGVkU3RhZ2VQbGFuLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...ot/query/runtime/plan/OpChainExecutionContext.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL09wQ2hhaW5FeGVjdXRpb25Db250ZXh0LmphdmE=) | `0.00% <ø> (ø)` | |
   | [...e/pinot/query/runtime/plan/PlanRequestContext.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL1BsYW5SZXF1ZXN0Q29udGV4dC5qYXZh) | `0.00% <ø> (ø)` | |
   | [...apache/pinot/query/runtime/plan/StageMetadata.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL1N0YWdlTWV0YWRhdGEuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../query/runtime/plan/serde/QueryPlanSerDeUtils.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL3NlcmRlL1F1ZXJ5UGxhblNlckRlVXRpbHMuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../runtime/plan/server/ServerPlanRequestContext.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL3NlcnZlci9TZXJ2ZXJQbGFuUmVxdWVzdENvbnRleHQuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [.../pinot/query/service/dispatch/QueryDispatcher.java](https://app.codecov.io/gh/apache/pinot/pull/10791?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvc2VydmljZS9kaXNwYXRjaC9RdWVyeURpc3BhdGNoZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   
   ... and [769 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/10791/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :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=apache)
   


-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
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: 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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   let's consider separate address this issue into a different PR.
   this one is already quite large and hard to review. 
   
   i guess the problem i have with this is it adds unnecessary info to the proto 



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   let's consider separate address this issue into a different PR.
   this one is already quite large and hard to review. and the approach here doesn't make it any worse than previous (which serialized one worker per request)
   
   based on a quick read on this new change adds unnecessary info to the proto 



-- 
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] xiangfu0 commented on a diff in pull request #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   This is good point, I added one more commit to this PR, you can check the diff: https://github.com/apache/pinot/pull/10791/commits/88ce2598618dfc69f7db96302e6983303962cccc
   
   



-- 
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] xiangfu0 commented on a diff in pull request #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   The basic idea is to carry all the workIds within one stagePlan then expand it at the worker side to generate one DistributedStagePlan per worker.



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java:
##########
@@ -51,12 +53,23 @@ public static DistributedStagePlan deserialize(Worker.StagePlan stagePlan) {
     return distributedStagePlan;
   }
 
-  public static Worker.StagePlan serialize(DistributedStagePlan distributedStagePlan) {
+  public static List<DistributedStagePlan> deserialize(Worker.QueryRequest request) {
+    List<DistributedStagePlan> distributedStagePlans = new ArrayList<>();
+    for (Worker.StagePlan stagePlan : request.getStagePlanList()) {
+      distributedStagePlans.add(deserialize(stagePlan));
+    }
+    return distributedStagePlans;
+  }
+
+  public static Worker.StagePlan serialize(DispatchableSubPlan dispatchableSubPlan, int stageId,
+      VirtualServerAddress serverAddress) {

Review Comment:
   @ankitsultana i remember you mentioned to me that dispatching 1-worker per request is extremely heavy in GRPC payload gen. 
   does this approach help? 
   e.g. 
   1. the dispatchable plan with the planFragment(stage) + all the worker metadata will be **serialized** together in 1 GRPC request (1 request for all worker)
   2. the request will be **deserialized** into a list of distributed stage plans (e.g. 1 plan per worker)



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   actually we decided to include this in the PR, b/c both changes require proto modification so it is best to avoid multiple changes to wire protocol. 
   - updated the PR by force pushing
   - added several follow up comments and will address in follow up PRs. PTAL



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryServer.java:
##########
@@ -83,32 +93,30 @@ 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;
-    long requestId = -1;
+    requestMetadataMap = request.getMetadataMap();
+    long requestId = Long.parseLong(requestMetadataMap.get(QueryConfig.KEY_OF_BROKER_REQUEST_ID));
     try {
-      distributedStagePlan = QueryPlanSerDeUtils.deserialize(request.getStagePlan());
-      requestMetadataMap = request.getMetadataMap();
-      requestId = Long.parseLong(requestMetadataMap.get(QueryConfig.KEY_OF_BROKER_REQUEST_ID));
+      distributedStagePlans = QueryPlanSerDeUtils.deserializeStagePlan(request);
     } catch (Exception e) {
       LOGGER.error("Caught exception while deserializing the request: {}, payload: {}", requestId, request, e);
       responseObserver.onError(Status.INVALID_ARGUMENT.withDescription("Bad request").withCause(e).asException());

Review Comment:
   on Error should be onNext(error payload) then onComplete(), broker doesn't handle onError at the moment



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java:
##########
@@ -24,44 +24,51 @@
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import org.apache.commons.lang.StringUtils;
 import org.apache.pinot.common.proto.Worker;
+import org.apache.pinot.query.planner.DispatchablePlanFragment;
+import org.apache.pinot.query.planner.DispatchableSubPlan;
 import org.apache.pinot.query.planner.plannode.AbstractPlanNode;
 import org.apache.pinot.query.planner.plannode.StageNodeSerDeUtils;
 import org.apache.pinot.query.routing.MailboxMetadata;
-import org.apache.pinot.query.routing.StageMetadata;
+import org.apache.pinot.query.routing.QueryServerInstance;
 import org.apache.pinot.query.routing.VirtualServerAddress;
 import org.apache.pinot.query.routing.WorkerMetadata;
 import org.apache.pinot.query.runtime.plan.DistributedStagePlan;
+import org.apache.pinot.query.runtime.plan.StageMetadata;
 
 
 /**
  * This utility class serialize/deserialize between {@link Worker.StagePlan} elements to Planner elements.
  */
 public class QueryPlanSerDeUtils {
+  private static final Pattern VIRTUAL_SERVER_PATTERN = Pattern.compile(
+      "(?<virtualid>[0-9]+)@(?<host>[^:]+):(?<port>[0-9]+)");
 
   private QueryPlanSerDeUtils() {
     // do not instantiate.
   }
 
-  public static DistributedStagePlan deserialize(Worker.StagePlan stagePlan) {
-    DistributedStagePlan distributedStagePlan = new DistributedStagePlan(stagePlan.getStageId());
-    distributedStagePlan.setServer(protoToAddress(stagePlan.getVirtualAddress()));
-    distributedStagePlan.setStageRoot(StageNodeSerDeUtils.deserializeStageNode(stagePlan.getStageRoot()));
-    distributedStagePlan.setStageMetadata(fromProtoStageMetadata(stagePlan.getStageMetadata()));
-    return distributedStagePlan;
+  public static List<DistributedStagePlan> deserializeStagePlan(Worker.QueryRequest request) {

Review Comment:
   intellij refactor error
   ```suggestion
     public static List<DistributedStagePlan> deserialize(Worker.QueryRequest request) {
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   actually we decided to include this in the PR, b/c both changes require proto modification so it is best to avoid multiple changes to wire protocol. updated the PR by force pushing



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java:
##########
@@ -79,6 +86,25 @@ public static String addressToProto(VirtualServerAddress serverAddress) {
     return String.format("%s@%s:%s", serverAddress.workerId(), serverAddress.hostname(), serverAddress.port());
   }
 
+  private static List<DistributedStagePlan> deserializeStagePlan(Worker.StagePlan stagePlan) {
+    List<DistributedStagePlan> distributedStagePlans = new ArrayList<>();
+    String serverAddress = stagePlan.getStageMetadata().getServerAddress();
+    String[] hostPort = StringUtils.split(serverAddress, ':');
+    String hostname = hostPort[0];
+    int port = Integer.parseInt(hostPort[1]);

Review Comment:
   - host-port should be at StagePlan level not metadata level
   - host-port parsing should be a util



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
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 
   - short term: we don't catch this and simple let the submission go through successfully. then return onCompleted once. 
   - long term: have a fork-join on all submitted futures, return success if all future return success and error when something is wrong. 



-- 
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 #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryServer.java:
##########
@@ -48,6 +51,8 @@ public class QueryServer extends PinotQueryWorkerGrpc.PinotQueryWorkerImplBase {
   private Server _server = null;
   private final QueryRunner _queryRunner;
 
+  private final ExecutorService _executorService = Executors.newCachedThreadPool();

Review Comment:
   let's make this submission threadpool configurable. 



-- 
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] ankitsultana commented on a diff in pull request #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java:
##########
@@ -51,12 +53,23 @@ public static DistributedStagePlan deserialize(Worker.StagePlan stagePlan) {
     return distributedStagePlan;
   }
 
-  public static Worker.StagePlan serialize(DistributedStagePlan distributedStagePlan) {
+  public static List<DistributedStagePlan> deserialize(Worker.QueryRequest request) {
+    List<DistributedStagePlan> distributedStagePlans = new ArrayList<>();
+    for (Worker.StagePlan stagePlan : request.getStagePlanList()) {
+      distributedStagePlans.add(deserialize(stagePlan));
+    }
+    return distributedStagePlans;
+  }
+
+  public static Worker.StagePlan serialize(DispatchableSubPlan dispatchableSubPlan, int stageId,
+      VirtualServerAddress serverAddress) {

Review Comment:
   Left a comment in the other part of the code where the serialize is being called (num-workers * num-stages) times per query.



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -139,22 +140,24 @@ int submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeout
         for (Map.Entry<QueryServerInstance, List<Integer>> queryServerEntry
             : dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap().entrySet()) {
           QueryServerInstance queryServerInstance = queryServerEntry.getKey();
+          Worker.QueryRequest.Builder queryRequestBuilder = Worker.QueryRequest.newBuilder();
+          String host = queryServerInstance.getHostname();
+          int servicePort = queryServerInstance.getQueryServicePort();
+          int mailboxPort = queryServerInstance.getQueryMailboxPort();
           for (int workerId : queryServerEntry.getValue()) {
-            String host = queryServerInstance.getHostname();
-            int servicePort = queryServerInstance.getQueryServicePort();
-            int mailboxPort = queryServerInstance.getQueryMailboxPort();
             VirtualServerAddress virtualServerAddress = new VirtualServerAddress(host, mailboxPort, workerId);
-            DispatchClient client = getOrCreateDispatchClient(host, servicePort);
             dispatchCalls++;
-            int finalStageId = stageId;
-            _executorService.submit(() -> client.submit(Worker.QueryRequest.newBuilder().setStagePlan(
-                        QueryPlanSerDeUtils.serialize(
-                            constructDistributedStagePlan(dispatchableSubPlan, finalStageId, virtualServerAddress)))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_ID, String.valueOf(requestId))
-                    .putMetadata(QueryConfig.KEY_OF_BROKER_REQUEST_TIMEOUT_MS, String.valueOf(timeoutMs))
-                    .putAllMetadata(queryOptions).build(), finalStageId, queryServerInstance, deadline,
-                dispatchCallbacks::offer));
+            queryRequestBuilder.addStagePlan(
+                QueryPlanSerDeUtils.serialize(dispatchableSubPlan, stageId, virtualServerAddress));

Review Comment:
   This would serialize the entire sub plan for each worker and stageId combination. Big use-cases would usually have 256 partitions (i.e. workers) and ~10 or so stages.
   
   So this means we'll call this ~2000 times. For low qps use-cases this should be fine but relatively higher use-cases might start getting bottlenecked.
   
   @walterddr : I remember we had discussed this a couple of months ago. Was wondering if we are planning to fix this anytime soon. cc: @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] walterddr merged pull request #10791: [multistage] Refactor StageMetadata from pinot-query-planner to pinot-query-runner module

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr merged PR #10791:
URL: https://github.com/apache/pinot/pull/10791


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