[multistage] Introduce InMemoryMailboxService for Optimizing Join Performance#9484
[multistage] Introduce InMemoryMailboxService for Optimizing Join Performance#9484walterddr merged 12 commits intoapache:masterfrom
Conversation
walterddr
left a comment
There was a problem hiding this comment.
i did a partial review. overall the approach looks good. there are some details we need to iron out and clear up
also please rebase
...ker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcReceivingMailbox.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/InMemoryChannel.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/InMemoryChannel.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcReceivingMailbox.java
Outdated
Show resolved
Hide resolved
...ry-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java
Outdated
Show resolved
Hide resolved
7b88262 to
ab7acfb
Compare
...ntime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentStreamObserver.java
Show resolved
Hide resolved
| if (waitForInitialize()) { | ||
| mailboxContent = _contentStreamObserver.poll(); | ||
| _totalMsgReceived.incrementAndGet(); | ||
| } else { | ||
| return null; | ||
| } | ||
| return mailboxContent; | ||
| return fromMailboxContent(mailboxContent); |
There was a problem hiding this comment.
if (!waitForInitialize()) {
return null;
}
mailboxContent = _contentStreamObserver.poll();
_totalMsgReceived.incrementAndGet();
return fromMailboxContent(mailboxContent);
walterddr
left a comment
There was a problem hiding this comment.
looks good to me overall
i have one suggestion on the wrapper around in-memory mailbox.
I will give another look once we address these. thank you for the contribution!
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcReceivingMailbox.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcReceivingMailbox.java
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/InMemoryChannel.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/InMemoryChannel.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryMailboxService.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/InMemoryChannel.java
Outdated
Show resolved
Hide resolved
...ntime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentStreamObserver.java
Show resolved
Hide resolved
|
@walterddr : Removed |
Codecov Report
@@ Coverage Diff @@
## master #9484 +/- ##
============================================
+ Coverage 60.33% 60.35% +0.02%
- Complexity 5148 5161 +13
============================================
Files 1926 1931 +5
Lines 103209 103450 +241
Branches 15670 15691 +21
============================================
+ Hits 62266 62442 +176
- Misses 36271 36321 +50
- Partials 4672 4687 +15
Flags with carried forward coverage won't be shown. Click here to find out more.
📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more |
agavra
left a comment
There was a problem hiding this comment.
Nice! The solution is really elegant here :) My comments are mostly around concurrency and making sure the non-happy path works properly.
| 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<>(); |
There was a problem hiding this 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!
There was a problem hiding this comment.
i think this was my original split in grpc mailbox ( this was necessary as the channels and mailbox are separately managed)
but in this case all 3 are guaranteed 1-1 mapping b/c they are all in the same JVM
I am ok with both approaches since
- grouping them make more clear code smell, we can precisely managed them together because it is a special property of the in-mem mailbox
- separating them makes logical sense as separation of concern (e.g. the sender normally won't have access to the receiving side info, this just happen to be the case for in-mem mailbox)
| } | ||
|
|
||
| @Override | ||
| public void shutdown() { |
There was a problem hiding this comment.
when do the maps get cleaned up? (esp. regarding a cancelled query or one that times out)
There was a problem hiding this comment.
This is a known leak. @walterddr is planning to fix it with query cancellation afaik
There was a problem hiding this comment.
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
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryReceivingMailbox.java
Outdated
Show resolved
Hide resolved
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemoryReceivingMailbox.java
Show resolved
Hide resolved
|
|
||
| @Override | ||
| public boolean isClosed() { | ||
| return _closed && _queue.size() == 0; |
There was a problem hiding this 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
There was a problem hiding this comment.
in GRPC isClosed() <=> isCompleted(); i think the wording might be misleading in this case
There was a problem hiding this 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:
- 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
_closedis marked true. - If for some reason the sender was not able to send the EOS block, MailboxReceiveOperator will time out after the query timeout is reached.
There was a problem hiding this comment.
upon checking the usage of the interface method isClosed(). it is actually in the non-threadsafe mailboxreceiveoperator.
- i think for now we can keep this b/c grpc mailbox does basically the same thing - it should really be called "shouldAnotherBlockBeExpected"
- later with Support non-blocking MailboxReceivedOperator #9615 we should redefine this API (or create a new one)
| } | ||
|
|
||
| @Override | ||
| public void complete() { |
There was a problem hiding this 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?
There was a problem hiding this comment.
it is guarantee that the last message is the EOS message which serves the purpose of marking it "closed/completed".
this is probably also the reason why the close flag didn't set to volatile b/c the thread that process the EOS content is also the one that sets the flag (not the otherway around)
There was a problem hiding this comment.
it is guarantee that the last message is the EOS
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
There was a problem hiding this comment.
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.
There was a problem hiding this 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. meaning the mailbox could still be receiving data afterwards, just no thread will dequeue from that receiving buffer. exactly the scenario i described in #9626
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
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java
Show resolved
Hide resolved
walterddr
left a comment
There was a problem hiding this comment.
lgtm overall thank you!
there are sevearl remaining items but I think we can follow them up later
This does the following:
MailboxServiceusesTransferableBlock. The serde toMailboxContentis moved within GrpcMailbox.InMemoryMailboxService. If the stage is local, then theTransferableBlockcan be passed from sender to receiver simply using a queue.MultiplexingMailboxServicewhich can choose either gRPC based mailbox-service or in-memory mailbox-service.Optimizations:
These optimizations are most helpful if the stages deal with large data (Amdahl's basically)
cc: @walterddr @siddharthteotia