Jackie-Jiang commented on code in PR #10681:
URL: https://github.com/apache/pinot/pull/10681#discussion_r1177079909
##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java:
##########
@@ -18,80 +18,117 @@
*/
package org.apache.pinot.query.mailbox;
-import org.apache.pinot.query.runtime.operator.MailboxReceiveOperator;
-import org.apache.pinot.query.runtime.operator.OpChain;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import org.apache.pinot.query.mailbox.channel.ChannelManager;
+import org.apache.pinot.query.mailbox.channel.GrpcMailboxServer;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
- * Mailbox service that handles transfer for mailbox contents.
- *
- * @param <T> type of content supported by this mailbox service.
+ * Mailbox service that handles data transfer.
*/
-public interface MailboxService<T> {
+public class MailboxService {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(MailboxService.class);
+ private static final int DANGLING_RECEIVING_MAILBOX_EXPIRY_SECONDS = 300;
- /**
- * Starting the mailbox service.
- */
- void start();
+ // We use a cache to ensure the receiving mailbox are not leaked in the
cases where the corresponding OpChain is
+ // either never registered or died before the sender finished sending data.
+ private final Cache<String, ReceivingMailbox> _receivingMailboxCache =
+
CacheBuilder.newBuilder().expireAfterAccess(DANGLING_RECEIVING_MAILBOX_EXPIRY_SECONDS,
TimeUnit.SECONDS)
+ .removalListener((RemovalListener<String, ReceivingMailbox>)
notification -> {
+ if (notification.wasEvicted()) {
+ int numPendingBlocks =
notification.getValue().getNumPendingBlocks();
+ if (numPendingBlocks > 0) {
+ LOGGER.warn("Evicting dangling receiving mailbox: {} with {}
pending blocks", notification.getKey(),
+ numPendingBlocks);
+ }
+ }
+ }).build();
- /**
- * Shutting down the mailbox service.
- */
- void shutdown();
+ private final String _hostname;
+ private final int _port;
+ private final PinotConfiguration _config;
+ private final Consumer<String> _receiveMailCallback;
+ private final ChannelManager _channelManager = new ChannelManager();
+
+ private GrpcMailboxServer _grpcMailboxServer;
+
+ public MailboxService(String hostname, int port, PinotConfiguration config,
Consumer<String> receiveMailCallback) {
+ _hostname = hostname;
+ _port = port;
+ _config = config;
Review Comment:
I feel it is okay to ask the caller to pass the port. The reason being the
`MailboxService` will be initialized by multiple components, and there is no
guarantee they use the same config key for the port (that is the reason why we
cannot extract hostname from the config).
--
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]