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

[GitHub] [pinot] ankitsultana opened a new pull request, #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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

   First PR in a series of PRs to fix all reliability issues with the join framework.
   
   cc: @walterddr 


-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/JsonMailboxIdentifier.java:
##########
@@ -106,6 +116,16 @@ public boolean isLocal() {
     return _fromAddress.equals(_toAddress);
   }
 
+  @Override

Review Comment:
   Edit: JsonIgnore is not needed for these fields since it's okay to allow the getters to be used for serde by jackson. The ignoreUnknown properties at the class level takes care of the behavior I was looking for. See this class' UT for more. https://fasterxml.github.io/jackson-annotations/javadoc/2.5/com/fasterxml/jackson/annotation/JsonIgnore.html



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -67,19 +67,21 @@ public class MailboxReceiveOperator extends MultiStageOperator {
   private int _serverIdx;
   private TransferableBlock _upstreamErrorBlock;
 
-  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, long stageId,
-      VirtualServerAddress receiver) {
+  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, int senderStageId,
+      int receiverStageId, VirtualServerAddress receiver) {
     return new JsonMailboxIdentifier(
-        String.format("%s_%s", jobId, stageId),
+        String.format("%s_%s", jobId, senderStageId),

Review Comment:
   but yes we can follow up separately



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10285?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 [#10285](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6752735) into [master](https://codecov.io/gh/apache/pinot/commit/addb6525fca86331dfc468209164bab1636289fb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (addb652) will **decrease** coverage by `56.67%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10285       +/-   ##
   =============================================
   - Coverage     70.39%   13.72%   -56.67%     
   + Complexity     5777      182     -5595     
   =============================================
     Files          2015     1961       -54     
     Lines        109318   106864     -2454     
     Branches      16615    16324      -291     
   =============================================
   - Hits          76954    14670    -62284     
   - Misses        26961    91035    +64074     
   + Partials       5403     1159     -4244     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `13.72% <0.00%> (+<0.01%)` | :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=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...che/pinot/query/mailbox/JsonMailboxIdentifier.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9Kc29uTWFpbGJveElkZW50aWZpZXIuamF2YQ==) | `0.00% <0.00%> (-74.29%)` | :arrow_down: |
   | [...va/org/apache/pinot/query/runtime/QueryRunner.java](https://codecov.io/gh/apache/pinot/pull/10285?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/10285?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.24%)` | :arrow_down: |
   | [...ot/query/runtime/operator/MailboxSendOperator.java](https://codecov.io/gh/apache/pinot/pull/10285?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%> (-68.75%)` | :arrow_down: |
   | [.../pinot/query/runtime/plan/PhysicalPlanVisitor.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL1BoeXNpY2FsUGxhblZpc2l0b3IuamF2YQ==) | `0.00% <0.00%> (-92.31%)` | :arrow_down: |
   | [...rg/apache/pinot/query/service/QueryDispatcher.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvc2VydmljZS9RdWVyeURpc3BhdGNoZXIuamF2YQ==) | `0.00% <0.00%> (-80.59%)` | :arrow_down: |
   | [...src/main/java/org/apache/pinot/sql/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvRmlsdGVyS2luZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...in/java/org/apache/pinot/spi/utils/BytesUtils.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQnl0ZXNVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/common/CustomObject.java](https://codecov.io/gh/apache/pinot/pull/10285?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vQ3VzdG9tT2JqZWN0LmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1589 more](https://codecov.io/gh/apache/pinot/pull/10285?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


[GitHub] [pinot] walterddr commented on a diff in pull request #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -67,19 +67,21 @@ public class MailboxReceiveOperator extends MultiStageOperator {
   private int _serverIdx;
   private TransferableBlock _upstreamErrorBlock;
 
-  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, long stageId,
-      VirtualServerAddress receiver) {
+  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, int senderStageId,
+      int receiverStageId, VirtualServerAddress receiver) {
     return new JsonMailboxIdentifier(
-        String.format("%s_%s", jobId, stageId),
+        String.format("%s_%s", jobId, senderStageId),

Review Comment:
   it will be backward incompatible anyway since you changed the JSON format? or are we doing unknown field as ignore?



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -67,19 +67,21 @@ public class MailboxReceiveOperator extends MultiStageOperator {
   private int _serverIdx;
   private TransferableBlock _upstreamErrorBlock;
 
-  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, long stageId,
-      VirtualServerAddress receiver) {
+  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, int senderStageId,
+      int receiverStageId, VirtualServerAddress receiver) {
     return new JsonMailboxIdentifier(
-        String.format("%s_%s", jobId, stageId),
+        String.format("%s_%s", jobId, senderStageId),

Review Comment:
   i dont know if we still want to keep the senderStage in the jobId format. since now you have the sender/receiver id already in the mailboxIdentifier impl



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java:
##########
@@ -153,10 +155,10 @@ protected TransferableBlock getNextBlock() {
   }
 
   private static JsonMailboxIdentifier toMailboxId(
-      VirtualServer destination, long jobId, int stageId, VirtualServerAddress sender) {
+      VirtualServer destination, long jobId, int stageId, int receiverStageId, VirtualServerAddress sender) {

Review Comment:
   nit: make it consistent, plz change it to senderStageId/receiverStageId



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdentifier.java:
##########
@@ -50,4 +50,8 @@ public interface MailboxIdentifier {
    * @return true if sender and receiver are in the same JVM.
    */
   boolean isLocal();
+
+  int getSenderStageId();
+
+  int getReceiverStageId();

Review Comment:
   neither API is used please at least add test



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdentifier.java:
##########
@@ -50,4 +50,8 @@ public interface MailboxIdentifier {
    * @return true if sender and receiver are in the same JVM.
    */
   boolean isLocal();
+
+  int getSenderStageId();
+
+  int getReceiverStageId();

Review Comment:
   Done



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java:
##########
@@ -153,10 +155,10 @@ protected TransferableBlock getNextBlock() {
   }
 
   private static JsonMailboxIdentifier toMailboxId(
-      VirtualServer destination, long jobId, int stageId, VirtualServerAddress sender) {
+      VirtualServer destination, long jobId, int stageId, int receiverStageId, VirtualServerAddress sender) {

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] ankitsultana commented on a diff in pull request #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -67,19 +67,21 @@ public class MailboxReceiveOperator extends MultiStageOperator {
   private int _serverIdx;
   private TransferableBlock _upstreamErrorBlock;
 
-  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, long stageId,
-      VirtualServerAddress receiver) {
+  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, int senderStageId,
+      int receiverStageId, VirtualServerAddress receiver) {
     return new JsonMailboxIdentifier(
-        String.format("%s_%s", jobId, stageId),
+        String.format("%s_%s", jobId, senderStageId),

Review Comment:
   Yeah the json ignore thing should help with no downtimes during rollout.



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/QueryDispatcher.java:
##########
@@ -196,12 +196,13 @@ private static DataSchema toResultSchema(DataSchema inputSchema, List<Pair<Integ
 
   @VisibleForTesting
   public static MailboxReceiveOperator createReduceStageOperator(MailboxService<TransferableBlock> mailboxService,
-      List<VirtualServer> sendingInstances, long jobId, int stageId, DataSchema dataSchema, VirtualServerAddress server,
+      List<VirtualServer> sendingInstances, long jobId, int stageId, int reducerStageId, DataSchema dataSchema,
+      VirtualServerAddress server,

Review Comment:
   self-review: Move timeoutMs in this line.



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdentifier.java:
##########
@@ -50,4 +50,8 @@ public interface MailboxIdentifier {
    * @return true if sender and receiver are in the same JVM.
    */
   boolean isLocal();
+

Review Comment:
   self-review: Add doc comments.



##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/GrpcMailboxServiceTest.java:
##########
@@ -78,7 +78,8 @@ public void testHappyPath()
     JsonMailboxIdentifier mailboxId = new JsonMailboxIdentifier(
         "happypath",
         new VirtualServerAddress("localhost", _mailboxService1.getMailboxPort(), 0),
-        new VirtualServerAddress("localhost", _mailboxService2.getMailboxPort(), 0));
+        new VirtualServerAddress("localhost", _mailboxService2.getMailboxPort(), 0),

Review Comment:
   self-review: Use static constants for sender/receiver stage-id here and in other places.



##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/RoundRobinSchedulerTest.java:
##########
@@ -33,9 +33,12 @@
 
 
 public class RoundRobinSchedulerTest {
+  private static final int DEFAULT_RECEIVER_STAGE_ID = 1;

Review Comment:
   self-review: Add DEFAULT_SENDER_STAGE_ID variable for consistency



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/JsonMailboxIdentifier.java:
##########
@@ -106,6 +116,16 @@ public boolean isLocal() {
     return _fromAddress.equals(_toAddress);
   }
 
+  @Override

Review Comment:
   self-review: Add JsonIgnore annotations and make getters order same as declarations.



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -67,19 +67,21 @@ public class MailboxReceiveOperator extends MultiStageOperator {
   private int _serverIdx;
   private TransferableBlock _upstreamErrorBlock;
 
-  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, long stageId,
-      VirtualServerAddress receiver) {
+  private static MailboxIdentifier toMailboxId(VirtualServer sender, long jobId, int senderStageId,
+      int receiverStageId, VirtualServerAddress receiver) {
     return new JsonMailboxIdentifier(
-        String.format("%s_%s", jobId, stageId),
+        String.format("%s_%s", jobId, senderStageId),

Review Comment:
   Yeah I wanted to change but it would cause a downtime during a rollout. Can we follow up on this in a separate PR?



-- 
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 #10285: [multistage] Store Sender/Receiver Stage Id in MailboxId

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


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