ankitsultana commented on code in PR #13645: URL: https://github.com/apache/pinot/pull/13645#discussion_r1728177518
########## pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/GrpcMailboxServer.java: ########## @@ -37,17 +54,33 @@ * send by the sender of the sender/receiver pair. */ public class GrpcMailboxServer extends PinotMailboxGrpc.PinotMailboxImplBase { + private static final Logger LOGGER = LoggerFactory.getLogger(GrpcMailboxServer.class); Review Comment: The PR title is aimed at QueryServer. Do you also need this for the GrpcMailboxServer? If so you also need to enable this in `GrpcSendingMailbox`. If you don't need this then you can revert the changes in this file. ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java: ########## @@ -71,7 +88,17 @@ public void start() { LOGGER.info("Starting QueryServer"); try { if (_server == null) { - _server = ServerBuilder.forPort(_port).addService(this).maxInboundMessageSize(MAX_INBOUND_MESSAGE_SIZE).build(); + if (_tlsConfig == null) { + try { + _server = + ServerBuilder.forPort(_port).addService(this).maxInboundMessageSize(MAX_INBOUND_MESSAGE_SIZE).build(); Review Comment: Why is try-catch required now? Since this line is the same as before. ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java: ########## @@ -36,12 +52,28 @@ */ class DispatchClient { private static final StreamObserver<Worker.CancelResponse> NO_OP_CANCEL_STREAM_OBSERVER = new CancelObserver(); + private static final Logger LOGGER = LoggerFactory.getLogger(DispatchClient.class); + // the key is the hashCode of the TlsConfig, the value is the SslContext + // We don't use TlsConfig as the map key because the TlsConfig is mutable, which means the hashCode can change. If the + // hashCode changes and the map is resized, the SslContext of the old hashCode will be lost. + private static final Map<Integer, SslContext> CLIENT_SSL_CONTEXTS_CACHE = new ConcurrentHashMap<>(); private final ManagedChannel _channel; private final PinotQueryWorkerGrpc.PinotQueryWorkerStub _dispatchStub; public DispatchClient(String host, int port) { - _channel = ManagedChannelBuilder.forAddress(host, port).usePlaintext().build(); + this(host, port, new GrpcConfig(Collections.emptyMap())); + } + public DispatchClient(String host, int port, GrpcConfig grpcConfig) { + if (grpcConfig.isUsePlainText()) { + _channel = + ManagedChannelBuilder.forAddress(host, port).maxInboundMessageSize(grpcConfig.getMaxInboundMessageSizeBytes()) Review Comment: Why are we setting max inbound message size here? I don't think it's required since dispatch client's response is usually very small ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java: ########## @@ -58,4 +90,26 @@ public void cancel(long requestId) { Worker.CancelRequest cancelRequest = Worker.CancelRequest.newBuilder().setRequestId(requestId).build(); _dispatchStub.cancel(cancelRequest, NO_OP_CANCEL_STREAM_OBSERVER); } + private SslContext buildSslContext(TlsConfig tlsConfig) { Review Comment: lack of new line and duplicate code. You can perhaps create a new utils class in `org.apache.pinot.common.utils.grpc`? cc: @Jackie-Jiang if he has a better idea. -- 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-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org