Skip to content

Commit

Permalink
Fix for review commenst
Browse files Browse the repository at this point in the history
  • Loading branch information
anandkrshaw committed Sep 14, 2024
1 parent 1b4d5b7 commit d771f61
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public GrpcQueryClient(String host, int port, GrpcConfig config) {
_channelShutdownTimeoutSeconds = config.getChannelShutdownTimeoutSecond();
}

private SslContext buildSslContext(TlsConfig tlsConfig) {
public static SslContext buildSslContext(TlsConfig tlsConfig) {
LOGGER.info("Building gRPC SSL context");
SslContext sslContext = CLIENT_SSL_CONTEXTS_CACHE.computeIfAbsent(tlsConfig.hashCode(), tlsConfigHashCode -> {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,24 +21,17 @@
import io.grpc.Deadline;
import io.grpc.ManagedChannel;
import io.grpc.ManagedChannelBuilder;
import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts;
import io.grpc.netty.shaded.io.grpc.netty.NettyChannelBuilder;
import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext;
import io.grpc.netty.shaded.io.netty.handler.ssl.SslContextBuilder;
import io.grpc.netty.shaded.io.netty.handler.ssl.SslProvider;
import io.grpc.stub.StreamObserver;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import javax.net.ssl.SSLException;
import nl.altindag.ssl.SSLFactory;
import org.apache.pinot.common.config.GrpcConfig;
import org.apache.pinot.common.config.TlsConfig;
import org.apache.pinot.common.proto.PinotQueryWorkerGrpc;
import org.apache.pinot.common.proto.Worker;
import org.apache.pinot.common.utils.tls.PinotInsecureMode;
import org.apache.pinot.common.utils.tls.RenewableTlsUtils;
import org.apache.pinot.common.utils.grpc.GrpcQueryClient;
import org.apache.pinot.query.routing.QueryServerInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -64,13 +57,14 @@ class DispatchClient {
public DispatchClient(String host, int port) {
this(host, port, new GrpcConfig(Collections.emptyMap()));
}

public DispatchClient(String host, int port, GrpcConfig grpcConfig) {
if (grpcConfig.isUsePlainText()) {
_channel = ManagedChannelBuilder.forAddress(host, port).usePlaintext().build();
} else {
_channel =
NettyChannelBuilder.forAddress(host, port)
.sslContext(buildSslContext(grpcConfig.getTlsConfig())).build();
.sslContext(GrpcQueryClient.buildSslContext(grpcConfig.getTlsConfig())).build();
}
_dispatchStub = PinotQueryWorkerGrpc.newStub(_channel);
}
Expand All @@ -88,26 +82,4 @@ 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) {
LOGGER.info("Building gRPC SSL context");
SslContext sslContext = CLIENT_SSL_CONTEXTS_CACHE.computeIfAbsent(tlsConfig.hashCode(), tlsConfigHashCode -> {
try {
SSLFactory sslFactory = RenewableTlsUtils.createSSLFactoryAndEnableAutoRenewalWhenUsingFileStores(tlsConfig,
PinotInsecureMode::isPinotInInsecureMode);
SslContextBuilder sslContextBuilder = SslContextBuilder.forClient();
sslFactory.getKeyManagerFactory().ifPresent(sslContextBuilder::keyManager);
sslFactory.getTrustManagerFactory().ifPresent(sslContextBuilder::trustManager);
if (tlsConfig.getSslProvider() != null) {
sslContextBuilder =
GrpcSslContexts.configure(sslContextBuilder, SslProvider.valueOf(tlsConfig.getSslProvider()));
} else {
sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder);
}
return sslContextBuilder.build();
} catch (SSLException e) {
throw new RuntimeException("Failed to build gRPC SSL context", e);
}
});
return sslContext;
}
}

0 comments on commit d771f61

Please sign in to comment.