You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/10/18 21:34:42 UTC

[GitHub] [pinot] agavra commented on a diff in pull request #9484: [multistage] Introduce InMemoryMailboxService for Optimizing Join Performance

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


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

Review Comment:
   nit: volatile - are there any other concerns with concurrency here?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryMailboxService.java:
##########
@@ -0,0 +1,85 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+public class InMemoryMailboxService implements MailboxService<TransferableBlock> {
+  // channel manager
+  private final String _hostname;
+  private final int _mailboxPort;
+  static final int DEFAULT_CHANNEL_CAPACITY = 5;
+  // TODO: This should come from a config and should be consistent with the timeout for GrpcMailboxService
+  static final int DEFAULT_CHANNEL_TIMEOUT_SECONDS = 120;
+
+  // maintaining a list of registered mailboxes.
+  private final ConcurrentHashMap<String, ReceivingMailbox<TransferableBlock>> _receivingMailboxMap =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, SendingMailbox<TransferableBlock>> _sendingMailboxMap =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, ArrayBlockingQueue<TransferableBlock>> _channelMap =
+      new ConcurrentHashMap<>();
+
+  public InMemoryMailboxService(String hostname, int mailboxPort) {
+    _hostname = hostname;
+    _mailboxPort = mailboxPort;
+  }
+
+  @Override
+  public void start() {
+  }
+
+  @Override
+  public void shutdown() {

Review Comment:
   when do the maps get cleaned up? (esp. regarding a cancelled query or one that times out)



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryMailboxService.java:
##########
@@ -0,0 +1,85 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+public class InMemoryMailboxService implements MailboxService<TransferableBlock> {
+  // channel manager
+  private final String _hostname;
+  private final int _mailboxPort;
+  static final int DEFAULT_CHANNEL_CAPACITY = 5;
+  // TODO: This should come from a config and should be consistent with the timeout for GrpcMailboxService
+  static final int DEFAULT_CHANNEL_TIMEOUT_SECONDS = 120;
+
+  // maintaining a list of registered mailboxes.
+  private final ConcurrentHashMap<String, ReceivingMailbox<TransferableBlock>> _receivingMailboxMap =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, SendingMailbox<TransferableBlock>> _sendingMailboxMap =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, ArrayBlockingQueue<TransferableBlock>> _channelMap =
+      new ConcurrentHashMap<>();

Review Comment:
   it's usually safer to wrap `ReceivingMailbox`, `SendingMailbox` and `ArrayBlockingQueue` all in a single `MailboxState` class, and then maintain just a single `Map<String, MailboxState>` that sets all three of these at once in a single `computeIfAbsent` call.
   
   I know it's unlikely that there will be a race, but it's easier to reason about when it's guaranteed there won't be!



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

Review Comment:
   nit `throws UnsupportedOperationException` can be removed?



##########
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:
   why do we check that the `_queue.size() == 0`? if I cancel the query or it times out and I close the mailbox I shouldn't be forced to process the queue



##########
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:
   do we want to send a sentinel block here to make sure that the receiving side will terminate (instead of waiting for `DEFAULT_CHANNEL_TIMEOUT_SECONDS`)? or is there any other mechanism to make sure that in the non-happy path the receiver won't need to wait the whole timeout?



##########
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);

Review Comment:
   we should consider matching the behavior of `GrpcReceivingMailbox` and time out after a very short interval (e.g. 100ms) and then return `null`. in this case, the `MailboxReceiveOperator` will just cycle back through so long as this mailbox isn't closed
   
   we'll probably end up changing the threading model, but it would be nice to not make this block essentially indefinitely in case there's data in a different mailbox for us to read from



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MultiplexingMailboxService.java:
##########
@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+public class MultiplexingMailboxService implements MailboxService<TransferableBlock> {

Review Comment:
   +1 very clean!



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