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


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java:
##########
@@ -0,0 +1,56 @@
+/**
+ * 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 InMemorySendingMailbox implements 
SendingMailbox<TransferableBlock> {
+  private final BlockingQueue<TransferableBlock> _queue;
+  private final String _mailboxId;
+
+  public InMemorySendingMailbox(String mailboxId, 
BlockingQueue<TransferableBlock> queue) {
+    _mailboxId = mailboxId;
+    _queue = queue;
+  }
+
+  @Override
+  public String getMailboxId() {
+    return _mailboxId;
+  }
+
+  @Override
+  public void send(TransferableBlock data)
+      throws UnsupportedOperationException {
+    try {
+      if (!_queue.offer(
+          data, InMemoryMailboxService.DEFAULT_CHANNEL_TIMEOUT_SECONDS, 
TimeUnit.SECONDS)) {
+        throw new RuntimeException(String.format("Timed out when sending block 
in mailbox=%s", _mailboxId));
+      }
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Interrupted trying to send data through the 
channel", e);
+    }
+  }
+
+  @Override
+  public void complete() {

Review Comment:
   > is that true even in the error scenarios? I was worried about cases like 
timeout, where we'd want to timeout quickly instead of blocking for 120s
   
   - if the error is sent from the sender then yes that will be the last 
message;
   - if the error occurs after a normal message was delivered then the error 
will originate from where the error was thrown, from that point the originated 
operator will stop returning any new data blocks
   
   so yes it is guaranteed in the operator level; but it is not guarantee in 
the mailbox level
   
   > The 120s timeout was definitely a bug. Reduced it to 1s. I have mentioned 
some timeout scenario handling in the other comment regarding _queue.size() == 
0.
   
   bare in mind that the 100ms timeout for GRPCReceivingMailbox is only the 
timeout upon initialization; from that point there's no pull-with-time-out. it 
is a pushed model managed by GRPC
   
   
   



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