From a4103960a4eb7803af6fabd9fe54a3137f82aff0 Mon Sep 17 00:00:00 2001 From: Michael Marshall Date: Wed, 20 Apr 2022 22:25:17 -0500 Subject: [PATCH] Add KeyStore support in WebSocket, Function Worker HTTPS Servers (#15084) * Add KeyStore support in WebSocket, Function Worker HTTPS Servers * Avoid leaking worker config password * Fix checkstyle * Replace broker with appropriate text in annotations * Update python script for new configs Co-authored-by: Lari Hotari ### Motivation We support configuring KeyStores for the broker and the proxy, but not the WebSocket or the Function Worker. By adding this support, users are able to provide KeyStores of type PCKS12 or JKS, which adds flexibility. Further, these KeyStores simplify support for additional algorithms because we rely on the TLS provider to load the KeyStore instead of loading keys ourselves. ### Modifications * Add `KeyStoreSSLContext`s to the function worker server * Add `KeyStoreSSLContext`s to the web socket server * Add configurations to the function worker, the web socket, and the proxy configuration files to simply configuration * Rely on `toString`, not `ObjectMapper`, when converting the `WorkerConfig` to a string so that we don't log the KeyStore password. (Add a test to verify this logic. Note that we don't want the `ObjectMapper` to ignore the field because we use mappers when converting configuration classes.) ### Verifying this change I manually verified that this change works in a minikube cluster. The underlying method named `KeyStoreSSLContext#createSslContextFactory` is already used and tested, so I don't believe we need additional testing on that component. ### Does this pull request potentially affect one of the following parts: This change adds a new way to configure TLS in the WebSocket and Function Worker HTTPS Servers. As such, it adds new configuration. This configuration is named in the same way that the broker and proxy configuration is named, so it is consistent. ### Documentation I've documented the new configuration in the appropriate configuration files. --- conf/broker.conf | 2 + conf/functions_worker.yml | 40 +++++++++++ conf/proxy.conf | 27 +++++++ conf/websocket.conf | 37 ++++++++++ docker/pulsar/scripts/gen-yml-from-env.py | 4 +- .../pulsar/broker/ServiceConfiguration.java | 2 +- .../pulsar/functions/worker/WorkerConfig.java | 72 +++++++++++++++++++ .../worker/WorkerApiV2ResourceConfigTest.java | 18 ++++- .../functions/worker/PulsarWorkerService.java | 9 +-- .../functions/worker/rest/WorkerServer.java | 33 +++++++-- .../pulsar/websocket/service/ProxyServer.java | 35 ++++++--- .../service/WebSocketProxyConfiguration.java | 58 +++++++++++++++ 12 files changed, 312 insertions(+), 25 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 8c503590869a2..72b0b43334b9c 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -621,6 +621,8 @@ tlsRequireTrustedClientCertOnConnect=false tlsProvider= ### --- KeyStore TLS config variables --- ### +## Note that some of the above TLS configs also apply to the KeyStore TLS configuration. + # Enable TLS with KeyStore type configuration in broker. tlsEnabledWithKeyStore=false diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 24d46e1b61a53..4254d3fa8cbf3 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -317,6 +317,46 @@ tlsAllowInsecureConnection: false tlsEnableHostnameVerification: false # Tls cert refresh duration in seconds (set 0 to check on every new connection) tlsCertRefreshCheckDurationSec: 300 +# Whether client certificates are required for TLS. Connections are rejected if the client +# certificate isn't trusted. +tlsRequireTrustedClientCertOnConnect: false + +### --- KeyStore TLS config variables --- ### +## Note that some of the above TLS configs also apply to the KeyStore TLS configuration. + +# TLS Provider for KeyStore type +tlsProvider: + +# Enable TLS with KeyStore type configuration in function worker. +tlsEnabledWithKeyStore: false + +# TLS KeyStore type configuration in function worker: JKS, PKCS12 +tlsKeyStoreType: JKS + +# TLS KeyStore path in function worker +tlsKeyStore: + +# TLS KeyStore password for function worker +tlsKeyStorePassword: + +# TLS TrustStore type configuration in function worker: JKS, PKCS12 +tlsTrustStoreType: JKS + +# TLS TrustStore path in function worker +tlsTrustStore: + +# TLS TrustStore password in function worker, default value is empty password +tlsTrustStorePassword: + +# Specify the tls protocols the function worker's web service will use to negotiate during TLS handshake +# (a comma-separated list of protocol names). +# Examples:- [TLSv1.3, TLSv1.2] +webServiceTlsProtocols: + +# Specify the tls cipher the function worker will use to negotiate during TLS Handshake +# (a comma-separated list of ciphers). +# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] +webServiceTlsCiphers: ######################## # State Management diff --git a/conf/proxy.conf b/conf/proxy.conf index 470ce8155cd55..1d48b78fbecf3 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -72,6 +72,33 @@ webServicePort=8080 # Port to use to server HTTPS request webServicePortTls= +### --- KeyStore TLS config variables --- ### +## Note that some of the above TLS configs also apply to the KeyStore TLS configuration. + +# TLS Provider for KeyStore type +tlsProvider= + +# Enable TLS with KeyStore type configuration in proxy. +tlsEnabledWithKeyStore=false + +# TLS KeyStore type configuration in proxy: JKS, PKCS12 +tlsKeyStoreType=JKS + +# TLS KeyStore path in proxy +tlsKeyStore= + +# TLS KeyStore password for proxy +tlsKeyStorePassword= + +# TLS TrustStore type configuration in proxy: JKS, PKCS12 +tlsTrustStoreType=JKS + +# TLS TrustStore path in proxy +tlsTrustStore= + +# TLS TrustStore password in proxy, default value is empty password +tlsTrustStorePassword= + # Specify the tls protocols the proxy's web service will use to negotiate during TLS handshake # (a comma-separated list of protocol names). # Examples:- [TLSv1.3, TLSv1.2] diff --git a/conf/websocket.conf b/conf/websocket.conf index c871b202021ef..09102905f4310 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -135,6 +135,43 @@ tlsRequireTrustedClientCertOnConnect=false # Tls cert refresh duration in seconds (set 0 to check on every new connection) tlsCertRefreshCheckDurationSec=300 +### --- KeyStore TLS config variables --- ### +## Note that some of the above TLS configs also apply to the KeyStore TLS configuration. + +# TLS Provider for KeyStore type +tlsProvider= + +# Enable TLS with KeyStore type configuration in WebSocket. +tlsEnabledWithKeyStore=false + +# TLS KeyStore type configuration in WebSocket: JKS, PKCS12 +tlsKeyStoreType=JKS + +# TLS KeyStore path in WebSocket +tlsKeyStore= + +# TLS KeyStore password for WebSocket +tlsKeyStorePassword= + +# TLS TrustStore type configuration in WebSocket: JKS, PKCS12 +tlsTrustStoreType=JKS + +# TLS TrustStore path in WebSocket +tlsTrustStore= + +# TLS TrustStore password in WebSocket, default value is empty password +tlsTrustStorePassword= + +# Specify the tls protocols the proxy's web service will use to negotiate during TLS handshake +# (a comma-separated list of protocol names). +# Examples:- [TLSv1.3, TLSv1.2] +webServiceTlsProtocols= + +# Specify the tls cipher the proxy will use to negotiate during TLS Handshake +# (a comma-separated list of ciphers). +# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] +webServiceTlsCiphers= + ### --- Deprecated config variables --- ### # Deprecated. Use configurationStoreServers diff --git a/docker/pulsar/scripts/gen-yml-from-env.py b/docker/pulsar/scripts/gen-yml-from-env.py index f2e10312a3a4d..4534ea9c16819 100755 --- a/docker/pulsar/scripts/gen-yml-from-env.py +++ b/docker/pulsar/scripts/gen-yml-from-env.py @@ -47,7 +47,9 @@ 'proxyRoles', 'schemaRegistryCompatibilityCheckers', 'brokerClientTlsCiphers', - 'brokerClientTlsProtocols' + 'brokerClientTlsProtocols', + 'webServiceTlsCiphers', + 'webServiceTlsProtocols', ] PF_ENV_PREFIX = 'PF_' diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index aaeb0df74c353..381ee573be382 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2547,7 +2547,7 @@ public class ServiceConfiguration implements PulsarConfiguration { @FieldContext( category = CATEGORY_KEYSTORE_TLS, - doc = "TLS Provider for Specify the SSL provider for the broker service: \n" + doc = "Specify the TLS provider for the broker service: \n" + "When using TLS authentication with CACert, the valid value is either OPENSSL or JDK.\n" + "When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc." ) diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index 73ba951177710..7fe566dae7cd3 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -36,9 +36,11 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.TreeSet; import lombok.AccessLevel; import lombok.Data; import lombok.Getter; +import lombok.ToString; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; @@ -74,6 +76,8 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { @Category private static final String CATEGORY_SECURITY = "Common Security Settings (applied for both worker and client)"; @Category + private static final String CATEGORY_KEYSTORE_TLS = "KeyStoreTLS"; + @Category private static final String CATEGORY_WORKER_SECURITY = "Worker Security Settings"; @Category private static final String CATEGORY_CLIENT_SECURITY = "Security settings for clients talking to brokers"; @@ -445,6 +449,74 @@ public boolean isBrokerClientAuthenticationEnabled() { doc = "Tls cert refresh duration in seconds (set 0 to check on every new connection)" ) private long tlsCertRefreshCheckDurationSec = 300; + + /**** --- KeyStore TLS config variables. --- ****/ + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "Enable TLS with KeyStore type configuration in function worker" + ) + private boolean tlsEnabledWithKeyStore = false; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "Specify the TLS provider for the function worker service: \n" + + "When using TLS authentication with CACert, the valid value is either OPENSSL or JDK.\n" + + "When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc." + ) + private String tlsProvider = null; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS KeyStore type configuration in function worker: JKS, PKCS12" + ) + private String tlsKeyStoreType = "JKS"; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS KeyStore path in function worker" + ) + private String tlsKeyStore = null; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS KeyStore password for function worker" + ) + @ToString.Exclude + private String tlsKeyStorePassword = null; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS TrustStore type configuration in function worker: JKS, PKCS12" + ) + private String tlsTrustStoreType = "JKS"; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS TrustStore path in function worker" + ) + private String tlsTrustStore = null; + + @FieldContext( + category = CATEGORY_KEYSTORE_TLS, + doc = "TLS TrustStore password for function worker, null means empty password." + ) + @ToString.Exclude + private String tlsTrustStorePassword = null; + + @FieldContext( + category = CATEGORY_WORKER_SECURITY, + doc = "Specify the tls protocols the proxy's web service will use to negotiate during TLS Handshake.\n\n" + + "Example:- [TLSv1.3, TLSv1.2]" + ) + private Set webServiceTlsProtocols = new TreeSet<>(); + + @FieldContext( + category = CATEGORY_WORKER_SECURITY, + doc = "Specify the tls cipher the proxy's web service will use to negotiate during TLS Handshake.\n\n" + + "Example:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]" + ) + private Set webServiceTlsCiphers = new TreeSet<>(); + @FieldContext( category = CATEGORY_WORKER_SECURITY, doc = "Enforce authentication" diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/worker/WorkerApiV2ResourceConfigTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/worker/WorkerApiV2ResourceConfigTest.java index 6cfaea9e2427a..bf45dd958c823 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/worker/WorkerApiV2ResourceConfigTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/worker/WorkerApiV2ResourceConfigTest.java @@ -25,10 +25,11 @@ import static org.testng.Assert.assertTrue; import java.net.URL; +import java.util.Locale; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.pulsar.functions.auth.KubernetesSecretsTokenAuthProvider; import org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntimeFactory; -import org.apache.pulsar.functions.worker.WorkerConfig; import org.testng.annotations.Test; /** @@ -121,4 +122,19 @@ public void testLoadResourceRestrictionsConfig() throws Exception { assertTrue(newK8SWc.isFunctionInstanceResourceChangeInLockStep()); } + + @Test + public void testPasswordsNotLeakedOnToString() throws Exception { + URL yamlUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); + WorkerConfig wc = WorkerConfig.load(yamlUrl.toURI().getPath()); + assertFalse(wc.toString().toLowerCase(Locale.ROOT).contains("password"), "Stringified config must not contain password"); + } + + @Test + public void testPasswordsPresentOnObjectMapping() throws Exception { + URL yamlUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); + WorkerConfig wc = WorkerConfig.load(yamlUrl.toURI().getPath()); + assertTrue((new ObjectMapper().writeValueAsString(wc)).toLowerCase(Locale.ROOT).contains("password"), + "ObjectMapper output must include passwords for proper serialization"); + } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 7d3f505b35add..bb8e27f221c5a 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -20,8 +20,6 @@ import static org.apache.pulsar.common.policies.data.PoliciesUtil.getBundles; import static org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl.removeIdentifierFromMetadataURL; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import io.netty.util.concurrent.DefaultThreadFactory; @@ -402,12 +400,7 @@ public void start(AuthenticationService authenticationService, workerStatsManager.startupTimeStart(); log.info("/** Starting worker id={} **/", workerConfig.getWorkerId()); - - try { - log.info("Worker Configs: {}", new ObjectMapper().writeValueAsString(workerConfig)); - } catch (JsonProcessingException e) { - log.warn("Failed to print worker configs with error {}", e.getMessage(), e); - } + log.info("Worker Configs: {}", workerConfig); try { DistributedLogConfiguration dlogConf = WorkerUtils.getDlogConf(workerConfig); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index feec5e3d4e508..b9aa35274be2b 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.common.util.SecurityUtility; +import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; @@ -125,12 +126,32 @@ private void init() { if (this.workerConfig.getTlsEnabled()) { try { - SslContextFactory sslCtxFactory = SecurityUtility.createSslContextFactory( - this.workerConfig.isTlsAllowInsecureConnection(), this.workerConfig.getTlsTrustCertsFilePath(), - this.workerConfig.getTlsCertificateFilePath(), this.workerConfig.getTlsKeyFilePath(), - this.workerConfig.isTlsRequireTrustedClientCertOnConnect(), - true, - this.workerConfig.getTlsCertRefreshCheckDurationSec()); + SslContextFactory sslCtxFactory; + if (workerConfig.isTlsEnabledWithKeyStore()) { + sslCtxFactory = KeyStoreSSLContext.createSslContextFactory( + workerConfig.getTlsProvider(), + workerConfig.getTlsKeyStoreType(), + workerConfig.getTlsKeyStore(), + workerConfig.getTlsKeyStorePassword(), + workerConfig.isTlsAllowInsecureConnection(), + workerConfig.getTlsTrustStoreType(), + workerConfig.getTlsTrustStore(), + workerConfig.getTlsTrustStorePassword(), + workerConfig.isTlsRequireTrustedClientCertOnConnect(), + workerConfig.getWebServiceTlsCiphers(), + workerConfig.getWebServiceTlsProtocols(), + workerConfig.getTlsCertRefreshCheckDurationSec() + ); + } else { + sslCtxFactory = SecurityUtility.createSslContextFactory( + workerConfig.isTlsAllowInsecureConnection(), + workerConfig.getTlsTrustCertsFilePath(), + workerConfig.getTlsCertificateFilePath(), + workerConfig.getTlsKeyFilePath(), + workerConfig.isTlsRequireTrustedClientCertOnConnect(), + true, + workerConfig.getTlsCertRefreshCheckDurationSec()); + } httpsConnector = new ServerConnector(server, sslCtxFactory); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); connectors.add(httpsConnector); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index 924f8226f56d3..a83cf3f0363fc 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.SecurityUtility; +import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; @@ -81,14 +82,32 @@ public ProxyServer(WebSocketProxyConfiguration config) // TLS enabled connector if (config.getWebServicePortTls().isPresent()) { try { - SslContextFactory sslCtxFactory = SecurityUtility.createSslContextFactory( - config.isTlsAllowInsecureConnection(), - config.getTlsTrustCertsFilePath(), - config.getTlsCertificateFilePath(), - config.getTlsKeyFilePath(), - config.isTlsRequireTrustedClientCertOnConnect(), - true, - config.getTlsCertRefreshCheckDurationSec()); + SslContextFactory sslCtxFactory; + if (config.isTlsEnabledWithKeyStore()) { + sslCtxFactory = KeyStoreSSLContext.createSslContextFactory( + config.getTlsProvider(), + config.getTlsKeyStoreType(), + config.getTlsKeyStore(), + config.getTlsKeyStorePassword(), + config.isTlsAllowInsecureConnection(), + config.getTlsTrustStoreType(), + config.getTlsTrustStore(), + config.getTlsTrustStorePassword(), + config.isTlsRequireTrustedClientCertOnConnect(), + config.getWebServiceTlsCiphers(), + config.getWebServiceTlsProtocols(), + config.getTlsCertRefreshCheckDurationSec() + ); + } else { + sslCtxFactory = SecurityUtility.createSslContextFactory( + config.isTlsAllowInsecureConnection(), + config.getTlsTrustCertsFilePath(), + config.getTlsCertificateFilePath(), + config.getTlsKeyFilePath(), + config.isTlsRequireTrustedClientCertOnConnect(), + true, + config.getTlsCertRefreshCheckDurationSec()); + } connectorTls = new ServerConnector(server, sslCtxFactory); connectorTls.setPort(config.getWebServicePortTls().get()); connectors.add(connectorTls); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 0b29dd15e0772..da5e8a4fd187c 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -24,6 +24,7 @@ import java.util.TreeSet; import lombok.Getter; import lombok.Setter; +import lombok.ToString; import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; import org.apache.pulsar.common.configuration.FieldContext; import org.apache.pulsar.common.configuration.PulsarConfiguration; @@ -187,6 +188,63 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "TLS cert refresh duration (in seconds). 0 means checking every new connection.") private long tlsCertRefreshCheckDurationSec = 300; + /**** --- KeyStore TLS config variables. --- ****/ + @FieldContext( + doc = "Enable TLS with KeyStore type configuration for WebSocket" + ) + private boolean tlsEnabledWithKeyStore = false; + + @FieldContext( + doc = "Specify the TLS provider for the WebSocket service: \n" + + "When using TLS authentication with CACert, the valid value is either OPENSSL or JDK.\n" + + "When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc." + ) + private String tlsProvider = null; + + @FieldContext( + doc = "TLS KeyStore type configuration in WebSocket: JKS, PKCS12" + ) + private String tlsKeyStoreType = "JKS"; + + @FieldContext( + doc = "TLS KeyStore path in WebSocket" + ) + private String tlsKeyStore = null; + + @FieldContext( + doc = "TLS KeyStore password for WebSocket" + ) + @ToString.Exclude + private String tlsKeyStorePassword = null; + + @FieldContext( + doc = "TLS TrustStore type configuration in WebSocket: JKS, PKCS12" + ) + private String tlsTrustStoreType = "JKS"; + + @FieldContext( + doc = "TLS TrustStore path in WebSocket" + ) + private String tlsTrustStore = null; + + @FieldContext( + doc = "TLS TrustStore password for WebSocket, null means empty password." + ) + @ToString.Exclude + private String tlsTrustStorePassword = null; + + @FieldContext( + doc = "Specify the tls protocols the proxy's web service will use to negotiate during TLS Handshake.\n\n" + + "Example:- [TLSv1.3, TLSv1.2]" + ) + private Set webServiceTlsProtocols = new TreeSet<>(); + + @FieldContext( + doc = "Specify the tls cipher the proxy's web service will use to negotiate during TLS Handshake.\n\n" + + "Example:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]" + ) + private Set webServiceTlsCiphers = new TreeSet<>(); + @FieldContext(doc = "Key-value properties. Types are all String") private Properties properties = new Properties();