-
Notifications
You must be signed in to change notification settings - Fork 1.5k
[multistage] Introduce InMemoryMailboxService for Optimizing Join Performance #9484
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
9b6c6bd
8186653
4a4f1f4
116bd8a
ab7acfb
0ebf757
ed23523
fce4d7a
afbbea2
7746b11
69eac60
e345e4b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,95 @@ | ||
| /** | ||
| * 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.BlockingQueue; | ||
| 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 = 1; | ||
|
|
||
| private final ConcurrentHashMap<String, InMemoryMailboxState> _mailboxStateMap = new ConcurrentHashMap<>(); | ||
|
|
||
| public InMemoryMailboxService(String hostname, int mailboxPort) { | ||
| _hostname = hostname; | ||
| _mailboxPort = mailboxPort; | ||
| } | ||
|
|
||
| @Override | ||
| public void start() { | ||
| } | ||
|
|
||
| @Override | ||
| public void shutdown() { | ||
| } | ||
|
|
||
| @Override | ||
| public String getHostname() { | ||
| return _hostname; | ||
| } | ||
|
|
||
| @Override | ||
| public int getMailboxPort() { | ||
| return _mailboxPort; | ||
| } | ||
|
|
||
| public SendingMailbox<TransferableBlock> getSendingMailbox(MailboxIdentifier mailboxId) { | ||
| Preconditions.checkState(mailboxId.isLocal(), "Cannot use in-memory mailbox service for non-local transport"); | ||
| String mId = mailboxId.toString(); | ||
| return _mailboxStateMap.computeIfAbsent(mId, this::newMailboxState)._sendingMailbox; | ||
| } | ||
|
|
||
| public ReceivingMailbox<TransferableBlock> getReceivingMailbox(MailboxIdentifier mailboxId) { | ||
| Preconditions.checkState(mailboxId.isLocal(), "Cannot use in-memory mailbox service for non-local transport"); | ||
| String mId = mailboxId.toString(); | ||
| return _mailboxStateMap.computeIfAbsent(mId, this::newMailboxState)._receivingMailbox; | ||
| } | ||
|
|
||
| InMemoryMailboxState newMailboxState(String mailboxId) { | ||
| BlockingQueue<TransferableBlock> queue = createDefaultChannel(); | ||
| return new InMemoryMailboxState(new InMemorySendingMailbox(mailboxId, queue), | ||
| new InMemoryReceivingMailbox(mailboxId, queue), queue); | ||
| } | ||
|
|
||
| private ArrayBlockingQueue<TransferableBlock> createDefaultChannel() { | ||
| return new ArrayBlockingQueue<>(DEFAULT_CHANNEL_CAPACITY); | ||
| } | ||
|
|
||
| static class InMemoryMailboxState { | ||
| ReceivingMailbox<TransferableBlock> _receivingMailbox; | ||
| SendingMailbox<TransferableBlock> _sendingMailbox; | ||
| BlockingQueue<TransferableBlock> _queue; | ||
|
|
||
| InMemoryMailboxState(SendingMailbox<TransferableBlock> sendingMailbox, | ||
| ReceivingMailbox<TransferableBlock> receivingMailbox, BlockingQueue<TransferableBlock> queue) { | ||
| _receivingMailbox = receivingMailbox; | ||
| _sendingMailbox = sendingMailbox; | ||
| _queue = queue; | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,66 @@ | ||
| /** | ||
| * 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 volatile 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); | ||
ankitsultana marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // If the poll timed out, we return a null since MailboxReceiveOperator can continue to check other mailboxes | ||
| if (block == null) { | ||
| return null; | ||
| } | ||
| if (block.isEndOfStreamBlock()) { | ||
| _closed = true; | ||
| } | ||
| return block; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isInitialized() { | ||
| return true; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isClosed() { | ||
| return _closed && _queue.size() == 0; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do we check that the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. in GRPC isClosed() <=> isCompleted(); i think the wording might be misleading in this case
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ideally there shouldn't be any case where To better understand this, this is how I believe some of the scenarios will be handled:
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. upon checking the usage of the interface method
|
||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
when do the maps get cleaned up? (esp. regarding a cancelled query or one that times out)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a known leak. @walterddr is planning to fix it with query cancellation afaik
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah we had a discussion and i created a tracker but never get to it.
but upon thinking although the affect is not much (just a simple object reference), it would be a big problem when stuff error out, see my comment in the new issue: #9626