Skip to content
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

Add configs to specify keepAlive and shutdownTimeout for GrpcQueryClient #13546

Merged
merged 1 commit into from
Jul 9, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,20 @@ public class GrpcConfig {
public static final String GRPC_TLS_PREFIX = "tls";
public static final String CONFIG_USE_PLAIN_TEXT = "usePlainText";
public static final String CONFIG_MAX_INBOUND_MESSAGE_BYTES_SIZE = "maxInboundMessageSizeBytes";

private static final String CONFIG_CHANNEL_SHUTDOWN_TIMEOUT_SECONDS = "channelShutdownTimeoutSeconds";
private static final int DEFAULT_CHANNEL_SHUTDOWN_TIMEOUT_SECONDS = 10;

// KeepAlive configs
private static final String CONFIG_CHANNEL_KEEP_ALIVE_TIME_SECONDS = "channelKeepAliveTimeSeconds";
private static final int DEFAULT_CHANNEL_KEEP_ALIVE_TIME_SECONDS = -1; // Set value > 0 to enable keep alive

private static final String CONFIG_CHANNEL_KEEP_ALIVE_TIMEOUT_SECONDS = "channelKeepAliveTimeoutSeconds";
private static final int DEFAULT_CHANNEL_KEEP_ALIVE_TIMEOUT_SECONDS = 20; // 20 seconds

private static final String CONFIG_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS = "channelKeepAliveWithoutCalls";
private static final boolean DEFAULT_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS = true;

// Default max message size to 128MB
public static final int DEFAULT_MAX_INBOUND_MESSAGE_BYTES_SIZE = 128 * 1024 * 1024;
// Default use plain text for transport
Expand Down Expand Up @@ -69,6 +83,23 @@ public boolean isUsePlainText() {
return Boolean.parseBoolean(_pinotConfig.getProperty(CONFIG_USE_PLAIN_TEXT, DEFAULT_IS_USE_PLAIN_TEXT));
}

public int getChannelShutdownTimeoutSecond() {
return _pinotConfig.getProperty(CONFIG_CHANNEL_SHUTDOWN_TIMEOUT_SECONDS, DEFAULT_CHANNEL_SHUTDOWN_TIMEOUT_SECONDS);
}

public int getChannelKeepAliveTimeSeconds() {
return _pinotConfig.getProperty(CONFIG_CHANNEL_KEEP_ALIVE_TIME_SECONDS, DEFAULT_CHANNEL_KEEP_ALIVE_TIME_SECONDS);
}

public int getChannelKeepAliveTimeoutSeconds() {
return _pinotConfig.getProperty(CONFIG_CHANNEL_KEEP_ALIVE_TIMEOUT_SECONDS,
DEFAULT_CHANNEL_KEEP_ALIVE_TIMEOUT_SECONDS);
}

public boolean isChannelKeepAliveWithoutCalls() {
return _pinotConfig.getProperty(CONFIG_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS, DEFAULT_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS);
}

public TlsConfig getTlsConfig() {
return _tlsConfig;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,30 +45,43 @@

public class GrpcQueryClient implements Closeable {
private static final Logger LOGGER = LoggerFactory.getLogger(GrpcQueryClient.class);
private static final int DEFAULT_CHANNEL_SHUTDOWN_TIMEOUT_SECOND = 10;

// 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 _managedChannel;
private final PinotQueryServerGrpc.PinotQueryServerBlockingStub _blockingStub;
private final int _channelShutdownTimeoutSeconds;

public GrpcQueryClient(String host, int port) {
this(host, port, new GrpcConfig(Collections.emptyMap()));
}

public GrpcQueryClient(String host, int port, GrpcConfig config) {
ManagedChannelBuilder<?> channelBuilder;
if (config.isUsePlainText()) {
_managedChannel =
channelBuilder =
ManagedChannelBuilder.forAddress(host, port).maxInboundMessageSize(config.getMaxInboundMessageSizeBytes())
.usePlaintext().build();
.usePlaintext();
} else {
_managedChannel =
channelBuilder =
NettyChannelBuilder.forAddress(host, port).maxInboundMessageSize(config.getMaxInboundMessageSizeBytes())
.sslContext(buildSslContext(config.getTlsConfig())).build();
.sslContext(buildSslContext(config.getTlsConfig()));
}

// Set keep alive configs, if enabled
int channelKeepAliveTimeSeconds = config.getChannelKeepAliveTimeSeconds();
if (channelKeepAliveTimeSeconds > 0) {
channelBuilder.keepAliveTime(channelKeepAliveTimeSeconds, TimeUnit.SECONDS)
.keepAliveTimeout(config.getChannelKeepAliveTimeoutSeconds(), TimeUnit.SECONDS)
.keepAliveWithoutCalls(config.isChannelKeepAliveWithoutCalls());
}

_managedChannel = channelBuilder.build();
_blockingStub = PinotQueryServerGrpc.newBlockingStub(_managedChannel);
_channelShutdownTimeoutSeconds = config.getChannelShutdownTimeoutSecond();
}

private SslContext buildSslContext(TlsConfig tlsConfig) {
Expand Down Expand Up @@ -103,7 +116,7 @@ public void close() {
if (!_managedChannel.isShutdown()) {
try {
_managedChannel.shutdownNow();
if (!_managedChannel.awaitTermination(DEFAULT_CHANNEL_SHUTDOWN_TIMEOUT_SECOND, TimeUnit.SECONDS)) {
if (!_managedChannel.awaitTermination(_channelShutdownTimeoutSeconds, TimeUnit.SECONDS)) {
LOGGER.warn("Timed out forcefully shutting down connection: {}. ", _managedChannel);
}
} catch (Exception e) {
Expand Down
Loading