ankitsultana commented on code in PR #9484:
URL: https://github.com/apache/pinot/pull/9484#discussion_r998844286


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryReceivingMailbox.java:
##########
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.mailbox;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+public class InMemoryReceivingMailbox implements 
ReceivingMailbox<TransferableBlock> {
+  private final String _mailboxId;
+  private final BlockingQueue<TransferableBlock> _queue;
+  private boolean _closed;
+
+  public InMemoryReceivingMailbox(String mailboxId, 
BlockingQueue<TransferableBlock> queue) {
+    _mailboxId = mailboxId;
+    _queue = queue;
+    _closed = false;
+  }
+
+  @Override
+  public String getMailboxId() {
+    return _mailboxId;
+  }
+
+  @Override
+  public TransferableBlock receive()
+      throws Exception {
+    TransferableBlock block = _queue.poll(
+        InMemoryMailboxService.DEFAULT_CHANNEL_TIMEOUT_SECONDS, 
TimeUnit.SECONDS);
+    if (block == null) {
+      throw new RuntimeException(String.format("Timed out waiting for data 
block on mailbox=%s", _mailboxId));
+    }
+    if (block.isEndOfStreamBlock()) {
+      _closed = true;
+    }
+    return block;
+  }
+
+  @Override
+  public boolean isInitialized() {
+    return true;
+  }
+
+  @Override
+  public boolean isClosed() {
+    return _closed && _queue.size() == 0;

Review Comment:
   Ideally there shouldn't be any case where `_closed` is true but 
`_queue.size()` is not 0. Only case where it may happen is when the sender 
sends something immediately after sending the EOS block. We can also throw an 
exception here in that case.
   
   To better understand this, this is how I believe some of the scenarios will 
be handled:
   
   1. If the sender has died, it would have sent an end of stream block and 
then terminated on its end. The receiver here would then get the end of stream 
block and send it upstream to MailboxReceiveOperator. The queue should be empty 
after `_closed` is marked true.
   2. If for some reason the sender was not able to send the EOS block, 
MailboxReceiveOperator will time out after the query timeout is reached.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to