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/07/27 22:59:40 UTC

[GitHub] [pinot] xiangfu0 opened a new pull request, #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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

   fix for https://github.com/apache/pinot/issues/11187


-- 
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 merged pull request #11201: [multistage] Return empty block if MailboxRecv nodes have no paired MailboxSender

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


-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -59,13 +59,15 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi
     int workerId = context.getServer().workerId();
     MailboxMetadata senderMailBoxMetadatas =
         context.getStageMetadata().getWorkerMetadataList().get(workerId).getMailBoxInfosMap().get(senderStageId);
-    Preconditions.checkState(senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty(),
-        "Failed to find mailbox for stage: %s",
-        senderStageId);
-    _mailboxIds = MailboxIdUtils.toMailboxIds(requestId, senderMailBoxMetadatas);
-    _mailboxes = _mailboxIds.stream()
-        .map(mailboxId -> _mailboxService.getReceivingMailbox(mailboxId))
-        .collect(Collectors.toCollection(ArrayDeque::new));
+    if (senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) {

Review Comment:
   ```suggestion
       Preconditions.checkState((senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) || expectEmptySendingMailbox) ...
   ```
   
   
   and set the expectEmptySendingMailbox from MailboxAssignmentVisitor when previous stage has no server assigned. ?



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -129,6 +130,11 @@ private void assignWorkersToLeafFragment(PlanFragment fragment, DispatchablePlan
         Preconditions.checkState(tableTypeToSegmentListMap.put(tableType, serverEntry.getValue()) == null,
             "Entry for server {} and table type: {} already exist!", serverEntry.getKey(), tableType);
       }
+
+      // attach unavailable segments to metadata
+      if (!routingTable.getUnavailableSegments().isEmpty()) {
+        metadata.addTableToUnavailableSegmentsMap(logicalTableName, routingTable.getUnavailableSegments());
+      }

Review Comment:
   In the last step of query planning, we need to construct `DispatchableSubPlan` from `PlanFragment` `subPlanRoot` and  `DispatchablePlanContext` `dispatchablePlanContext`
   ```
   private static DispatchableSubPlan finalizeDispatchableSubPlan(PlanFragment subPlanRoot,
         DispatchablePlanContext dispatchablePlanContext) {
       return new DispatchableSubPlan(dispatchablePlanContext.getResultFields(),
           dispatchablePlanContext.constructDispatchablePlanFragmentList(subPlanRoot),
           dispatchablePlanContext.getTableNames());
     }
   ```
   
   So if we don't populate unavailable segments to `PlanFragment` , then we need to add a new field into `DispatchableSubPlan`



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -129,6 +130,11 @@ private void assignWorkersToLeafFragment(PlanFragment fragment, DispatchablePlan
         Preconditions.checkState(tableTypeToSegmentListMap.put(tableType, serverEntry.getValue()) == null,
             "Entry for server {} and table type: {} already exist!", serverEntry.getKey(), tableType);
       }
+
+      // attach unavailable segments to metadata
+      if (!routingTable.getUnavailableSegments().isEmpty()) {
+        metadata.addTableToUnavailableSegmentsMap(logicalTableName, routingTable.getUnavailableSegments());
+      }

Review Comment:
   cant we simply attach to `DispatchablePlanContext` this way no need to change anything in fragment



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/DispatchablePlanFragment.java:
##########
@@ -46,17 +47,22 @@ public class DispatchablePlanFragment {
   // used for passing custom properties to build StageMetadata on the server.
   private final Map<String, String> _customProperties;
 
+  // Used for passing unavailable segments to generate partial results.
+  private final Map<String, Set<String>> _tableToUnavailableSegments;

Review Comment:
   
   ```suggestion
     private final Map<String, Collection<String>> _tableToUnavailableSegments;
   ```



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -129,6 +130,11 @@ private void assignWorkersToLeafFragment(PlanFragment fragment, DispatchablePlan
         Preconditions.checkState(tableTypeToSegmentListMap.put(tableType, serverEntry.getValue()) == null,
             "Entry for server {} and table type: {} already exist!", serverEntry.getKey(), tableType);
       }
+
+      // attach unavailable segments to metadata
+      if (!routingTable.getUnavailableSegments().isEmpty()) {
+        metadata.addTableToUnavailableSegmentsMap(logicalTableName, routingTable.getUnavailableSegments());
+      }

Review Comment:
   make sense.



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -129,6 +130,11 @@ private void assignWorkersToLeafFragment(PlanFragment fragment, DispatchablePlan
         Preconditions.checkState(tableTypeToSegmentListMap.put(tableType, serverEntry.getValue()) == null,
             "Entry for server {} and table type: {} already exist!", serverEntry.getKey(), tableType);
       }
+
+      // attach unavailable segments to metadata
+      if (!routingTable.getUnavailableSegments().isEmpty()) {
+        metadata.addTableToUnavailableSegmentsMap(logicalTableName, routingTable.getUnavailableSegments());
+      }

Review Comment:
   In the last step of planner, we need to construct DispatchableSubPlan from PlanFragment subPlanRoot and  DispatchablePlanContext dispatchablePlanContext
   ```
   private static DispatchableSubPlan finalizeDispatchableSubPlan(PlanFragment subPlanRoot,
         DispatchablePlanContext dispatchablePlanContext) {
       return new DispatchableSubPlan(dispatchablePlanContext.getResultFields(),
           dispatchablePlanContext.constructDispatchablePlanFragmentList(subPlanRoot),
           dispatchablePlanContext.getTableNames());
     }
   ```
   
   So if we don't do it here, then we need to add a new field into `DispatchableSubPlan`



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11201?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11201](https://app.codecov.io/gh/apache/pinot/pull/11201?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (717aea9) into [master](https://app.codecov.io/gh/apache/pinot/commit/2904f1bce19f93b4e6f2f6492e741a5adc02dd82?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (2904f1b) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #11201     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2224     2169     -55     
     Lines      119351   116798   -2553     
     Branches    18069    17757    -312     
   =========================================
     Hits          137      137             
   + Misses     119194   116641   -2553     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `?` | |
   | unittests2temurin20 | `0.11% <0.00%> (ø)` | |
   
   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.
   
   | [Files Changed](https://app.codecov.io/gh/apache/pinot/pull/11201?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...y/runtime/operator/BaseMailboxReceiveOperator.java](https://app.codecov.io/gh/apache/pinot/pull/11201?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9CYXNlTWFpbGJveFJlY2VpdmVPcGVyYXRvci5qYXZh) | `0.00% <0.00%> (ø)` | |
   
   ... and [57 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11201/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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -59,13 +59,15 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi
     int workerId = context.getServer().workerId();
     MailboxMetadata senderMailBoxMetadatas =
         context.getStageMetadata().getWorkerMetadataList().get(workerId).getMailBoxInfosMap().get(senderStageId);
-    Preconditions.checkState(senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty(),
-        "Failed to find mailbox for stage: %s",
-        senderStageId);
-    _mailboxIds = MailboxIdUtils.toMailboxIds(requestId, senderMailBoxMetadatas);
-    _mailboxes = _mailboxIds.stream()
-        .map(mailboxId -> _mailboxService.getReceivingMailbox(mailboxId))
-        .collect(Collectors.toCollection(ArrayDeque::new));
+    if (senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) {

Review Comment:
   Preconditions.checkState((senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) || expectEmptySendingMailbox) ...
   
   and set the expectEmptySendingMailbox from MailboxAssignmentVisitor when previous stage has no server assigned. ?



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -129,6 +130,11 @@ private void assignWorkersToLeafFragment(PlanFragment fragment, DispatchablePlan
         Preconditions.checkState(tableTypeToSegmentListMap.put(tableType, serverEntry.getValue()) == null,
             "Entry for server {} and table type: {} already exist!", serverEntry.getKey(), tableType);
       }
+
+      // attach unavailable segments to metadata
+      if (!routingTable.getUnavailableSegments().isEmpty()) {
+        metadata.addTableToUnavailableSegmentsMap(logicalTableName, routingTable.getUnavailableSegments());
+      }

Review Comment:
   yeah adding a new field to dispatchable subplan is what i meant. 
   
   e.g. 
   1. `WorkerManager` here attach all unavailable segments to `dispatchablePlanContext._unavailableSegmentMap`
   2. `PinotDispatchPlanner#finalizeDispatchableSubPlan` copies the map to `DispachableSubPlan._unavailableSegmentMap`
   
   there's no need for planFragment to know this as they cannot do anything anyway. 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] walterddr commented on a diff in pull request #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java:
##########
@@ -212,6 +218,12 @@ private void assignWorkersToIntermediateFragment(PlanFragment fragment, Dispatch
     } else {
       serverInstances = fetchServersForIntermediateStage(tableNames);
     }
+    if (serverInstances.isEmpty()) {
+      LOGGER.warn("[RequestId: {}] No server instance found for intermediate stage for tables: {}",

Review Comment:
   logger error?



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -59,13 +59,15 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi
     int workerId = context.getServer().workerId();
     MailboxMetadata senderMailBoxMetadatas =
         context.getStageMetadata().getWorkerMetadataList().get(workerId).getMailBoxInfosMap().get(senderStageId);
-    Preconditions.checkState(senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty(),
-        "Failed to find mailbox for stage: %s",
-        senderStageId);
-    _mailboxIds = MailboxIdUtils.toMailboxIds(requestId, senderMailBoxMetadatas);
-    _mailboxes = _mailboxIds.stream()
-        .map(mailboxId -> _mailboxService.getReceivingMailbox(mailboxId))
-        .collect(Collectors.toCollection(ArrayDeque::new));
+    if (senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) {

Review Comment:
   Need to modify some tests for this behavior.



-- 
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 #11201: Return empty block if MailboxRecv nodes have no paired MailboxSender

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -59,13 +59,15 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi
     int workerId = context.getServer().workerId();
     MailboxMetadata senderMailBoxMetadatas =
         context.getStageMetadata().getWorkerMetadataList().get(workerId).getMailBoxInfosMap().get(senderStageId);
-    Preconditions.checkState(senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty(),
-        "Failed to find mailbox for stage: %s",
-        senderStageId);
-    _mailboxIds = MailboxIdUtils.toMailboxIds(requestId, senderMailBoxMetadatas);
-    _mailboxes = _mailboxIds.stream()
-        .map(mailboxId -> _mailboxService.getReceivingMailbox(mailboxId))
-        .collect(Collectors.toCollection(ArrayDeque::new));
+    if (senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) {

Review Comment:
   discussed offline. this should be allowed. instead we should make a empty server check on WorkerManager to ensure that no empty server plan for intermediate stage is allowed



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