You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by pe...@apache.org on 2022/05/05 02:17:54 UTC

[pulsar] 03/03: [improve][broker-web&websocket&proxy&function-worker] Full-support set ssl provider, ciphers and protocols (#13740)

This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch branch-2.8
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit b28f5413c89a1cd0bba7a48d451fe6e925b9b4ba
Author: Zixuan Liu <no...@gmail.com>
AuthorDate: Sun May 1 12:35:46 2022 +0800

    [improve][broker-web&websocket&proxy&function-worker] Full-support set ssl provider, ciphers and protocols (#13740)
    
    Fixes #13734
    
    Pulsar doesn't set ssl provider, ciphers and protocols to the web, websocket and proxy service when `tlsEnabledWithKeyStore=false`
    
    - Add `org.apache.pulsar.jetty.tls` package in pulsar-broker-common for Jetty TLS support
    - Add a new `webServiceTlsProvider=Conscrypt` to broker and proxy config
    - Update `Conscrypt` as the `tlsProvider` value in websocket config
    
    In the old version, we implicitly use the `Conscrypt` provider, now we need to set it explicitly.
    
    (cherry picked from commit bf15e83a3f19b698c1492071792045e47824675c)
---
 conf/broker.conf                                   |   3 +
 conf/functions_worker.yml                          |   6 +-
 conf/proxy.conf                                    |   9 +-
 conf/standalone.conf                               |   3 +
 conf/websocket.conf                                |   8 +-
 .../apache/pulsar/broker/ServiceConfiguration.java |  21 +++
 .../java/org/apache/pulsar/jetty/package-info.java |  19 ++
 .../pulsar/jetty/tls/JettySslContextFactory.java   | 116 +++++++++++++
 .../org/apache/pulsar/jetty/tls/package-info.java  |  19 ++
 .../jetty/tls/JettySslContextFactoryTest.java      | 174 +++++++++++++++++++
 .../JettySslContextFactoryWithKeyStoreTest.java    | 193 +++++++++++++++++++++
 .../src/test/resources/ssl/jetty_client_key.jks    | Bin 0 -> 2679 bytes
 .../src/test/resources/ssl/jetty_client_trust.jks  | Bin 0 -> 1207 bytes
 .../src/test/resources/ssl/jetty_server_key.jks    | Bin 0 -> 2679 bytes
 .../src/test/resources/ssl/jetty_server_trust.jks  | Bin 0 -> 1207 bytes
 .../src/test/resources/ssl/my-ca/ca.pem            |  18 ++
 .../src/test/resources/ssl/my-ca/client-ca.pem     |  19 ++
 .../src/test/resources/ssl/my-ca/client-key.pem    |  28 +++
 .../src/test/resources/ssl/my-ca/server-ca.pem     |  19 ++
 .../src/test/resources/ssl/my-ca/server-key.pem    |  28 +++
 .../org/apache/pulsar/broker/PulsarService.java    |   2 +
 .../org/apache/pulsar/broker/web/WebService.java   |  16 +-
 .../proxy/ProxyPublishConsumeTlsTest.java          |   2 +-
 pulsar-common/pom.xml                              |   5 -
 .../common/util/DefaultSslContextBuilder.java      |  23 ++-
 .../apache/pulsar/common/util/SecurityUtility.java |  74 +++-----
 .../util/keystoretls/KeyStoreSSLContext.java       |  57 +-----
 .../SslContextFactoryWithAutoRefresh.java          |  66 -------
 .../discovery/service/server/ServerManager.java    |  13 +-
 .../pulsar/functions/worker/rest/WorkerServer.java |  17 +-
 .../pulsar/proxy/server/AdminProxyHandler.java     |  14 +-
 .../pulsar/proxy/server/ProxyConfiguration.java    |  21 +++
 .../org/apache/pulsar/proxy/server/WebServer.java  |  15 +-
 .../pulsar/websocket/service/ProxyServer.java      |  13 +-
 .../service/WebSocketProxyConfiguration.java       |   6 +-
 35 files changed, 801 insertions(+), 226 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index affdaaf8dcc..cd304dc858c 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -589,6 +589,9 @@ tlsRequireTrustedClientCertOnConnect=false
 # When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc.
 tlsProvider=
 
+# Specify the TLS provider for the web service: SunJSSE, Conscrypt and etc.
+webServiceTlsProvider=Conscrypt
+
 ### --- KeyStore TLS config variables --- ###
 ## Note that some of the above TLS configs also apply to the KeyStore TLS configuration.
 
diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml
index efdb6b61653..dbcce3d660f 100644
--- a/conf/functions_worker.yml
+++ b/conf/functions_worker.yml
@@ -305,11 +305,11 @@ tlsCertRefreshCheckDurationSec: 300
 # certificate isn't trusted.
 tlsRequireTrustedClientCertOnConnect: false
 
-### --- KeyStore TLS config variables --- ###
+### --- TLS config variables --- ###
 ## Note that some of the above TLS configs also apply to the KeyStore TLS configuration.
 
-# TLS Provider for KeyStore type
-tlsProvider:
+# Specify the TLS provider for the web service: SunJSSE, Conscrypt and etc.
+tlsProvider: Conscrypt
 
 # Enable TLS with KeyStore type configuration in function worker.
 tlsEnabledWithKeyStore: false
diff --git a/conf/proxy.conf b/conf/proxy.conf
index 6033a07b851..c70b5939018 100644
--- a/conf/proxy.conf
+++ b/conf/proxy.conf
@@ -68,12 +68,17 @@ webServicePort=8080
 # Port to use to server HTTPS request
 webServicePortTls=
 
-### --- KeyStore TLS config variables --- ###
+### --- TLS config variables --- ###
 ## Note that some of the above TLS configs also apply to the KeyStore TLS configuration.
 
-# TLS Provider for KeyStore type
+# Specify the TLS provider for the broker service:
+# When using TLS authentication with CACert, the valid value is either OPENSSL or JDK.
+# When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc.
 tlsProvider=
 
+# Specify the TLS provider for the web service, available values can be SunJSSE, Conscrypt and etc.
+webServiceTlsProvider=Conscrypt
+
 # Enable TLS with KeyStore type configuration in proxy.
 tlsEnabledWithKeyStore=false
 
diff --git a/conf/standalone.conf b/conf/standalone.conf
index ded670cc151..8562f1aa8e0 100644
--- a/conf/standalone.conf
+++ b/conf/standalone.conf
@@ -355,6 +355,9 @@ tlsRequireTrustedClientCertOnConnect=false
 # When using TLS authentication with KeyStore, available values can be SunJSSE, Conscrypt and etc.
 tlsProvider=
 
+# Specify the TLS provider for the web service: SunJSSE, Conscrypt and etc.
+webServiceTlsProvider=Conscrypt
+
 ### --- KeyStore TLS config variables --- ###
 # Enable TLS with KeyStore type configuration in broker.
 tlsEnabledWithKeyStore=false
diff --git a/conf/websocket.conf b/conf/websocket.conf
index 85d8bc4564d..e2688516bc2 100644
--- a/conf/websocket.conf
+++ b/conf/websocket.conf
@@ -96,6 +96,7 @@ brokerClientTrustCertsFilePath=
 anonymousUserRole=
 
 ### --- TLS --- ###
+## Note that some of the above TLS configs also apply to the KeyStore TLS configuration.
 
 # Deprecated - use webServicePortTls and brokerClientTlsEnabled instead
 tlsEnabled=false
@@ -119,11 +120,8 @@ 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=
+# Specify the TLS provider for the WebSocket: SunJSSE, Conscrypt and etc.
+tlsProvider=Conscrypt
 
 # Enable TLS with KeyStore type configuration in WebSocket.
 tlsEnabledWithKeyStore=false
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 abd0bffe7eb..1fe4d90532f 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
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import lombok.Getter;
 import lombok.Setter;
@@ -141,6 +142,26 @@ public class ServiceConfiguration implements PulsarConfiguration {
     )
     private Optional<Integer> webServicePortTls = Optional.empty();
 
+    @FieldContext(
+            category = CATEGORY_SERVER,
+            doc = "Specify the TLS provider for the web service: SunJSSE, Conscrypt and etc."
+    )
+    private String webServiceTlsProvider = "Conscrypt";
+
+    @FieldContext(
+            category = CATEGORY_TLS,
+            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<String> webServiceTlsProtocols = new TreeSet<>();
+
+    @FieldContext(
+            category = CATEGORY_TLS,
+            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<String> webServiceTlsCiphers = new TreeSet<>();
+
     @FieldContext(
         category = CATEGORY_SERVER,
         doc = "Hostname or IP address the service binds on"
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/package-info.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/package-info.java
new file mode 100644
index 00000000000..f01bd9198f5
--- /dev/null
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.jetty;
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java
new file mode 100644
index 00000000000..514fa4a0725
--- /dev/null
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.jetty.tls;
+
+import java.util.Set;
+import javax.net.ssl.SSLContext;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.common.util.DefaultSslContextBuilder;
+import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+import org.apache.pulsar.common.util.keystoretls.NetSslContextBuilder;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+
+@Slf4j
+public class JettySslContextFactory {
+    static {
+        // DO NOT EDIT - Load Conscrypt provider
+        if (SecurityUtility.CONSCRYPT_PROVIDER != null) {
+        }
+    }
+
+    public static SslContextFactory.Server createServerSslContextWithKeystore(String sslProviderString,
+                                                                              String keyStoreTypeString,
+                                                                              String keyStore,
+                                                                              String keyStorePassword,
+                                                                              boolean allowInsecureConnection,
+                                                                              String trustStoreTypeString,
+                                                                              String trustStore,
+                                                                              String trustStorePassword,
+                                                                              boolean requireTrustedClientCertOnConnect,
+                                                                              Set<String> ciphers,
+                                                                              Set<String> protocols,
+                                                                              long certRefreshInSec) {
+        NetSslContextBuilder sslCtxRefresher = new NetSslContextBuilder(
+                sslProviderString,
+                keyStoreTypeString,
+                keyStore,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStore,
+                trustStorePassword,
+                requireTrustedClientCertOnConnect,
+                certRefreshInSec);
+
+        return new JettySslContextFactory.Server(sslProviderString, sslCtxRefresher,
+                requireTrustedClientCertOnConnect, ciphers, protocols);
+    }
+
+    public static SslContextFactory createServerSslContext(String sslProviderString, boolean tlsAllowInsecureConnection,
+                                                           String tlsTrustCertsFilePath,
+                                                           String tlsCertificateFilePath,
+                                                           String tlsKeyFilePath,
+                                                           boolean tlsRequireTrustedClientCertOnConnect,
+                                                           Set<String> ciphers,
+                                                           Set<String> protocols,
+                                                           long certRefreshInSec) {
+        DefaultSslContextBuilder sslCtxRefresher =
+                new DefaultSslContextBuilder(tlsAllowInsecureConnection, tlsTrustCertsFilePath, tlsCertificateFilePath,
+                        tlsKeyFilePath, tlsRequireTrustedClientCertOnConnect, certRefreshInSec, sslProviderString);
+
+        return new JettySslContextFactory.Server(sslProviderString, sslCtxRefresher,
+                tlsRequireTrustedClientCertOnConnect, ciphers, protocols);
+    }
+
+    private static class Server extends SslContextFactory.Server {
+        private final SslContextAutoRefreshBuilder<SSLContext> sslCtxRefresher;
+
+        public Server(String sslProviderString, SslContextAutoRefreshBuilder<SSLContext> sslCtxRefresher,
+                      boolean requireTrustedClientCertOnConnect, Set<String> ciphers, Set<String> protocols) {
+            super();
+            this.sslCtxRefresher = sslCtxRefresher;
+
+            if (ciphers != null && ciphers.size() > 0) {
+                this.setIncludeCipherSuites(ciphers.toArray(new String[0]));
+            }
+
+            if (protocols != null && protocols.size() > 0) {
+                this.setIncludeProtocols(protocols.toArray(new String[0]));
+            }
+
+            if (sslProviderString != null && !sslProviderString.equals("")) {
+                setProvider(sslProviderString);
+            }
+
+            if (requireTrustedClientCertOnConnect) {
+                this.setNeedClientAuth(true);
+                this.setTrustAll(false);
+            } else {
+                this.setWantClientAuth(true);
+                this.setTrustAll(true);
+            }
+        }
+
+        @Override
+        public SSLContext getSslContext() {
+            return sslCtxRefresher.get();
+        }
+    }
+}
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/package-info.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/package-info.java
new file mode 100644
index 00000000000..8978699ff79
--- /dev/null
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.jetty.tls;
diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java
new file mode 100644
index 00000000000..c1816674880
--- /dev/null
+++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pulsar.jetty.tls;
+
+import com.google.common.io.Resources;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLHandshakeException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+import org.apache.pulsar.common.util.SecurityUtility;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class JettySslContextFactoryTest {
+
+    @Test
+    public void testJettyTlsServerTls() throws Exception {
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory factory = JettySslContextFactory.createServerSslContext(
+                null,
+                false,
+                Resources.getResource("ssl/my-ca/ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-key.pem").getPath(),
+                true,
+                null,
+                null,
+                600);
+
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https",
+                new SSLConnectionSocketFactory(getClientSslContext(), new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    @Test(expectedExceptions = SSLHandshakeException.class)
+    public void testJettyTlsServerInvalidTlsProtocol() throws Exception {
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory factory = JettySslContextFactory.createServerSslContext(
+                null,
+                false,
+                Resources.getResource("ssl/my-ca/ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-key.pem").getPath(),
+                true,
+                null,
+                new HashSet<String>() {
+                    {
+                        this.add("TLSv1.3");
+                    }
+                },
+                600);
+        factory.setHostnameVerifier((s, sslSession) -> true);
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https", new SSLConnectionSocketFactory(getClientSslContext(),
+                new String[]{"TLSv1.2"}, null, new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    @Test(expectedExceptions = SSLHandshakeException.class)
+    public void testJettyTlsServerInvalidCipher() throws Exception {
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory factory = JettySslContextFactory.createServerSslContext(
+                null,
+                false,
+                Resources.getResource("ssl/my-ca/ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/server-key.pem").getPath(),
+                true,
+                new HashSet<String>() {
+                    {
+                        this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256");
+                    }
+                },
+                new HashSet<String>() {
+                    {
+                        this.add("TLSv1.2");
+                    }
+                },
+                600);
+
+        factory.setHostnameVerifier((s, sslSession) -> true);
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https", new SSLConnectionSocketFactory(getClientSslContext(),
+                new String[]{"TLSv1.2"}, new String[]{"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"},
+                new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    private static SSLContext getClientSslContext() throws GeneralSecurityException, IOException {
+        return SecurityUtility.createSslContext(
+                false,
+                Resources.getResource("ssl/my-ca/ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/client-ca.pem").getPath(),
+                Resources.getResource("ssl/my-ca/client-key.pem").getPath(),
+                null
+        );
+    }
+}
diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java
new file mode 100644
index 00000000000..292bd123fdf
--- /dev/null
+++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.jetty.tls;
+
+import com.google.common.io.Resources;
+import java.io.FileInputStream;
+import java.security.KeyStore;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLHandshakeException;
+import javax.net.ssl.TrustManagerFactory;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class JettySslContextFactoryWithKeyStoreTest {
+
+    @Test
+    public void testJettyTlsServerTls() throws Exception {
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null,
+                "JKS", Resources.getResource("ssl/jetty_server_key.jks").getPath(),
+                "jetty_server_pwd", false, "JKS",
+                Resources.getResource("ssl/jetty_server_trust.jks").getPath(),
+                "jetty_server_pwd", true, null,
+                null, 600);
+        factory.setHostnameVerifier((s, sslSession) -> true);
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https",
+                new SSLConnectionSocketFactory(getClientSslContext(), new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    @Test(expectedExceptions = SSLHandshakeException.class)
+    public void testJettyTlsServerInvalidTlsProtocol() throws Exception {
+        Configurator.setRootLevel(Level.INFO);
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null,
+                "JKS", Resources.getResource("ssl/jetty_server_key.jks").getPath(),
+                "jetty_server_pwd", false, "JKS",
+                Resources.getResource("ssl/jetty_server_trust.jks").getPath(),
+                "jetty_server_pwd", true, null,
+                new HashSet<String>() {
+                    {
+                        this.add("TLSv1.3");
+                    }
+                }, 600);
+        factory.setHostnameVerifier((s, sslSession) -> true);
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https", new SSLConnectionSocketFactory(getClientSslContext(),
+                new String[]{"TLSv1.2"}, null, new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    @Test(expectedExceptions = SSLHandshakeException.class)
+    public void testJettyTlsServerInvalidCipher() throws Exception {
+        Server server = new Server();
+        List<ServerConnector> connectors = new ArrayList<>();
+        SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null,
+                "JKS", Resources.getResource("ssl/jetty_server_key.jks").getPath(),
+                "jetty_server_pwd", false, "JKS",
+                Resources.getResource("ssl/jetty_server_trust.jks").getPath(),
+                "jetty_server_pwd", true, new HashSet<String>() {
+                    {
+                        this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256");
+                    }
+                },
+                new HashSet<String>() {
+                    {
+                        this.add("TLSv1.2");
+                    }
+                }, 600);
+        factory.setHostnameVerifier((s, sslSession) -> true);
+        ServerConnector connector = new ServerConnector(server, factory);
+        connector.setPort(0);
+        connectors.add(connector);
+        server.setConnectors(connectors.toArray(new ServerConnector[0]));
+        server.start();
+        // client connect
+        HttpClientBuilder httpClientBuilder = HttpClients.custom();
+        RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+        registryBuilder.register("https", new SSLConnectionSocketFactory(getClientSslContext(),
+                new String[]{"TLSv1.2"}, new String[]{"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"},
+                new NoopHostnameVerifier()));
+        PoolingHttpClientConnectionManager cm = new PoolingHttpClientConnectionManager(registryBuilder.build());
+        httpClientBuilder.setConnectionManager(cm);
+        CloseableHttpClient httpClient = httpClientBuilder.build();
+        HttpGet httpGet = new HttpGet("https://localhost:" + connector.getLocalPort());
+        httpClient.execute(httpGet);
+        httpClient.close();
+        server.stop();
+    }
+
+    private static SSLContext getClientSslContext() {
+        return getSslContext(Resources.getResource("ssl/jetty_client_key.jks").getPath(),
+                "jetty_client_pwd",
+                Resources.getResource("ssl/jetty_client_trust.jks").getPath(),
+                "jetty_client_pwd");
+    }
+
+    private static SSLContext getSslContext(String keyStorePath, String keyStorePassword,
+                                            String trustStorePath, String trustStorePassword) {
+        try {
+            SSLContext sslContext = SSLContext.getInstance("TLS");
+            // key store
+            KeyManagerFactory keyManagerFactory =
+                    KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            KeyStore keyStore = KeyStore.getInstance("JKS");
+            try (FileInputStream inputStream = new FileInputStream(keyStorePath)) {
+                keyStore.load(inputStream, keyStorePassword.toCharArray());
+            }
+            keyManagerFactory.init(keyStore, keyStorePassword.toCharArray());
+            KeyManager[] keyManagers = keyManagerFactory.getKeyManagers();
+            // trust store
+            TrustManagerFactory trustManagerFactory =
+                    TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            KeyStore trustStore = KeyStore.getInstance("JKS");
+            try (FileInputStream inputStream = new FileInputStream(trustStorePath)) {
+                trustStore.load(inputStream, trustStorePassword.toCharArray());
+            }
+            trustManagerFactory.init(trustStore);
+            sslContext.init(keyManagers, trustManagerFactory.getTrustManagers(), new SecureRandom());
+            return sslContext;
+        } catch (Exception e) {
+            log.error("load ssl context error ", e);
+            return null;
+        }
+    }
+
+}
\ No newline at end of file
diff --git a/pulsar-broker-common/src/test/resources/ssl/jetty_client_key.jks b/pulsar-broker-common/src/test/resources/ssl/jetty_client_key.jks
new file mode 100644
index 00000000000..2b8ea64347d
Binary files /dev/null and b/pulsar-broker-common/src/test/resources/ssl/jetty_client_key.jks differ
diff --git a/pulsar-broker-common/src/test/resources/ssl/jetty_client_trust.jks b/pulsar-broker-common/src/test/resources/ssl/jetty_client_trust.jks
new file mode 100644
index 00000000000..166a2e00fb3
Binary files /dev/null and b/pulsar-broker-common/src/test/resources/ssl/jetty_client_trust.jks differ
diff --git a/pulsar-broker-common/src/test/resources/ssl/jetty_server_key.jks b/pulsar-broker-common/src/test/resources/ssl/jetty_server_key.jks
new file mode 100644
index 00000000000..b6189b75c8a
Binary files /dev/null and b/pulsar-broker-common/src/test/resources/ssl/jetty_server_key.jks differ
diff --git a/pulsar-broker-common/src/test/resources/ssl/jetty_server_trust.jks b/pulsar-broker-common/src/test/resources/ssl/jetty_server_trust.jks
new file mode 100644
index 00000000000..b09cc030a71
Binary files /dev/null and b/pulsar-broker-common/src/test/resources/ssl/jetty_server_trust.jks differ
diff --git a/pulsar-broker-common/src/test/resources/ssl/my-ca/ca.pem b/pulsar-broker-common/src/test/resources/ssl/my-ca/ca.pem
new file mode 100644
index 00000000000..3d5a80e2347
--- /dev/null
+++ b/pulsar-broker-common/src/test/resources/ssl/my-ca/ca.pem
@@ -0,0 +1,18 @@
+-----BEGIN CERTIFICATE-----
+MIIC9DCCAdygAwIBAgIUNbNkV2+K2Hf4Q1V5gdAENZQiLokwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAxMGUHVsc2FyMCAXDTIyMDExNDA0MjgwMFoYDzIxMjIwMTE2
+MDQyODAwWjARMQ8wDQYDVQQDEwZQdWxzYXIwggEiMA0GCSqGSIb3DQEBAQUAA4IB
+DwAwggEKAoIBAQDBR2K5EKVziLqdsz78efEW4lOwKiJ32e97uxn1Z6oKgkgImpVP
+Z9aoJB4EwSnDg+6FV2YULdWPm7C6W33tDmWRaU/Hlo/cOejnK8UmiMu/EyDpE2Wj
+n0RimGmwOkBi2IWIcIzWMmPDZ9kZc65OUeEmwZedKRy62PQyfCeNU4OOHQn3PXjI
+NbXJZD5TvBmn4SJn2RP9EgmIPaBAh/Mng045ZeHHLhwMKC8EOyHc2aB7AL6brymR
+xzsiYWdcJn4mqqMvT82mVvhkgAMOcR4CXYF8eYnsG6ZbDHb13CawcvLVREJZk7AB
+XZi9Rd5xczxHILM8rdkIZfunaG1X5hbih5wJAgMBAAGjQjBAMA4GA1UdDwEB/wQE
+AwIBBjAPBgNVHRMBAf8EBTADAQH/MB0GA1UdDgQWBBTCC1lYG+62cUPjNk9q4jCm
+Ps65njANBgkqhkiG9w0BAQsFAAOCAQEAKV2Lpu5cH5EsG53EWsYxEKvuQZ0LTxCE
+wCDf/NxJaQbzfv0tsbZatMge0vcZ/5r8tZZoOC+pGTwk6MaRbEFH8PmvlH1LIQvu
+Y34/YQZOy8wBTWwaIfFMnYWc0iAFoFt2Lzuq+GOI+svTFp729Ae8r7UxY/f9Lioc
+ttdGr7vA6PpcIMoEIPjVp+m41uL9IDfX8eOxg4gVlwtqpbHdTzMrOz0YY+3qH/WK
+6Qffw4pwitzAEj2zCn2lvGC5cbpd13SAaqtB3xL/Aet0SS2r3g9qDo1RruQhXUng
+06U/Hqtn5K1fNQv3pivi3Jg5z1DfJWHkH37luAoIlOZHRmPK6rhp/g==
+-----END CERTIFICATE-----
diff --git a/pulsar-broker-common/src/test/resources/ssl/my-ca/client-ca.pem b/pulsar-broker-common/src/test/resources/ssl/my-ca/client-ca.pem
new file mode 100644
index 00000000000..adcae3393ad
--- /dev/null
+++ b/pulsar-broker-common/src/test/resources/ssl/my-ca/client-ca.pem
@@ -0,0 +1,19 @@
+-----BEGIN CERTIFICATE-----
+MIIDHDCCAgSgAwIBAgIUJJpmKX3DnbUwJ7tUhCt8MTiwz0owDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAxMGUHVsc2FyMCAXDTIyMDExNDA0MjgwMFoYDzIxMjExMjIx
+MDQyODAwWjARMQ8wDQYDVQQDEwZQdWxzYXIwggEiMA0GCSqGSIb3DQEBAQUAA4IB
+DwAwggEKAoIBAQDZN+CNZ1i1WaXulbwSASOfXErWXhGV9DHqavPp3DohgQdundfS
+648T/X80uWQlyxu4L4j0oc97jtzc1AyZFXj5nocVsveEO9aDjnYCc5NdBNJLQHgl
+IO59fEpTd55NO24g9a8/sxgn0ADCenMlngk1Ou+2QJBONw7W12/WUSUg6ICe+b+x
+qPzgApue16oGw9HxhPwa3oEvVZrEnFIWLjsSWtezhgFHMCH9/ngk0KlRyes/EZCz
+ZgkO5mgii2fmNDg+yuWUfw7Q0x6BJskGIrxisJiJBRR1+DIvJqgqxJsNmeeEQrZK
+YHBukj5RWDFOpOHgqFbPsv45sVKoLrGFrMnNAgMBAAGjajBoMA4GA1UdDwEB/wQE
+AwIFoDATBgNVHSUEDDAKBggrBgEFBQcDAjAMBgNVHRMBAf8EAjAAMB0GA1UdDgQW
+BBSwkx93xjYP4I+dcFF3xS9NLesmFjAUBgNVHREEDTALgglsb2NhbGhvc3QwDQYJ
+KoZIhvcNAQELBQADggEBAAK3ZF63w46pT76QIOeSM3ocUm6izvW/IrxLUESfgRC4
+gg0/5VfPiHHUe6orn15KuPXHe7xCUFqc2oFn5aIU1B/6iOPeNItvMJidU0a3UAiw
+hFK9MSFgESNBiEnu1dE5tPcIIxTyCFQ/8loeY3dsdcNVoguH/2J9v/XcMMga46A1
+wudaaa1nb+ZYnXkRuyObKVJQN7EqC+4edinMOTPBbF9wtRMAMBRHXXENXb9zFthi
+Dbdn4YvadYsNHxh5ar+hQn/HSPMuCUPY/uUqxtBagb6aS0YnSoUscSLs1Jizg5NX
+d+QV8X/5E6W4xWnptUZwVxOemkdnr6A8MH1eQKKFZTM=
+-----END CERTIFICATE-----
diff --git a/pulsar-broker-common/src/test/resources/ssl/my-ca/client-key.pem b/pulsar-broker-common/src/test/resources/ssl/my-ca/client-key.pem
new file mode 100644
index 00000000000..5b08b151c80
--- /dev/null
+++ b/pulsar-broker-common/src/test/resources/ssl/my-ca/client-key.pem
@@ -0,0 +1,28 @@
+-----BEGIN PRIVATE KEY-----
+MIIEvAIBADANBgkqhkiG9w0BAQEFAASCBKYwggSiAgEAAoIBAQDZN+CNZ1i1WaXu
+lbwSASOfXErWXhGV9DHqavPp3DohgQdundfS648T/X80uWQlyxu4L4j0oc97jtzc
+1AyZFXj5nocVsveEO9aDjnYCc5NdBNJLQHglIO59fEpTd55NO24g9a8/sxgn0ADC
+enMlngk1Ou+2QJBONw7W12/WUSUg6ICe+b+xqPzgApue16oGw9HxhPwa3oEvVZrE
+nFIWLjsSWtezhgFHMCH9/ngk0KlRyes/EZCzZgkO5mgii2fmNDg+yuWUfw7Q0x6B
+JskGIrxisJiJBRR1+DIvJqgqxJsNmeeEQrZKYHBukj5RWDFOpOHgqFbPsv45sVKo
+LrGFrMnNAgMBAAECggEATeVZ45uiFja16J9NuG8sJSPluoY1bD8L/3KnUcAmIImy
+7powIXVT8+k+StwI6/ywThbN2FyGmVqcHZz1f5hRr8KH0uJBHOyQetEFxM9Jk1v9
+Rfsymq36mImP5erJnAyp66vvUrqY+P4Ap71duam4x5wBBqyUk1fvPGA5vPOQiwHs
+TN9JHizGobY25fpigWKIMamyE7HWXEUzVdOo83ZiNx53ths+WcF/kqto2v5LtyfJ
+HgoPocfZI8tRz9tfgc8zOkvyjsvgdd6rLhd0r2oExnyQBJdktGFpQZMGambU328u
+NqcdJscjP/HWAHRzuSdOvCMOEn8E5GIjcWEnQqOmSQKBgQDcpb655/UdcVxrv2Ou
+8juucDJMpf6i/UcmlXVXx+3zGSuQZcCC2fupe3JcxPdK7bo65YlC3OoRihggh2sS
+cnFMNHMfyoE3G/doXIr3QyL9UAQt4yb+7Nz7jRXYcg4Ytv+FVS6BSzIDEK17v+es
+GuWDM3JwtigtzYS4tRh7lgmuBwKBgQD8BXp7yIyVv657B8OJJSoeGataziFPhZux
+WKoS3gq24169ZWXwLc+nwrdgvBNrRaHuX+cYh93RF9+2WZrRcRL41XqN938adasY
+zPsfOJa9IOgUzQtGUMSe1/WqvHfcvqZCqYq4u/LSdf+I67woP4tCqqn4E928aIZb
+6PjLH+dUiwKBgH1ntn7y1t1lEKIspPtJsaHzIqNttMvuKAJF7+t0Nkl0hM4NBt1Y
+BzDMeLNBP0vW0YGn89uMs3xEgHH8hV52rO4i4UuwTMCFpJgsAM+H2NsgHz/1WrSI
+6xANn9zk9h4V5CRjxYq2sjYLxI4RBBtNLiTjmKd24F8n78cLJl8XZ2kBAoGAGoHF
+ATH1v2ZaxqvpYApdpK7UfAeEL2YBGyUVNkjOXbAKbec1Uo6u8ZkkSnNdo4G+Z2EE
+4Gqh5PUa3YYNJ4w6D5v8eOQYJUNNDJ26p+z+xcOpRU7PqcSi+YYDW8LY5InU2NwW
+MBnsj0BD8TXCI4WTcx6aI/KK9t8TiqU1Tb/8R8MCgYANVinOLz2enB+Qzu4o88W/
+witKHI3D9+z/uWjp0Q4rwmr3OL4FD9vZWvL4qwbDgpfLirJ4e3UVfN1/FoytAKlk
+Kykf8oDWciCIdxStt/yUpgQv78IL3vM5d9B8Qb7KCRtJ0BIXGJ7Gle3xJeuduZLe
++F+hwI3Dpv5HPqa9o6ttJw==
+-----END PRIVATE KEY-----
diff --git a/pulsar-broker-common/src/test/resources/ssl/my-ca/server-ca.pem b/pulsar-broker-common/src/test/resources/ssl/my-ca/server-ca.pem
new file mode 100644
index 00000000000..df5f69298e2
--- /dev/null
+++ b/pulsar-broker-common/src/test/resources/ssl/my-ca/server-ca.pem
@@ -0,0 +1,19 @@
+-----BEGIN CERTIFICATE-----
+MIIDHDCCAgSgAwIBAgIUVQHD0/oi9Ca50HA7DFLYOO2wEzYwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAxMGUHVsc2FyMCAXDTIyMDExNDA0MjgwMFoYDzIxMjExMjIx
+MDQyODAwWjARMQ8wDQYDVQQDEwZQdWxzYXIwggEiMA0GCSqGSIb3DQEBAQUAA4IB
+DwAwggEKAoIBAQDBcqDkMhjLd9ik//UQijqbajQP5t6dvVZNn9gODQrS9oB/URur
+NzCcPWYPJZfEJlTkV8mlmgq4dBjwghpy5ALOGiERk55JPIN4cy01hQ6j7YSPFvMv
+BjqZvm5dpGDNTr7GY7THegMM1wpk9EaUOm7tBOHtf6ZnANjSMcQM74RCSBt0Koqw
+06CKVDCbgJ5NNE1LgwYeVQAwtQAhY8rqqQKJvCorFbq7OiisFBnz5pRBT6N4kMo1
+9LZo3Oe2F2w9eH9vacQ0NjSOCNXqal9Xl/Pwy9JgKKppwZ/3nCgRc+yfjrnkRz0f
+b+llb2NpR5Ge+tNMakqelE8bDSw/5BPjRPftAgMBAAGjajBoMA4GA1UdDwEB/wQE
+AwIFoDATBgNVHSUEDDAKBggrBgEFBQcDATAMBgNVHRMBAf8EAjAAMB0GA1UdDgQW
+BBRXws5mmLbW+xOLflUyUZ0I0uN96zAUBgNVHREEDTALgglsb2NhbGhvc3QwDQYJ
+KoZIhvcNAQELBQADggEBAKMklpYJIkp4icz9Ea5wWQiRXWb94lGdyCA833VHeGB2
+fKvNXj1d6lEiy26pOjhDmycroKelj70WqOsqVgi4xh4Y9sj6pwb8Q423Tu3qNO1k
+qaScTar2DANSigNzqlSbLshPWQ2ZyDwkvZPuqPgHzOXekzbUGwxgCiySaQkl2mCS
+mBaG3XnESwiMIKkLphEv0MAvTVaImbSRWYEQ4OECwcHXxx+14wK8NLcdDIHcSzki
+8Eq24CxDOeL5QxciGMi5tylsdCpT+D/BXTKiu46yoRjXUsTLYL53yUZZIqQ3A4CV
+enZ/vHhP0Ev9RcRigFTqrBm7EC3b2AUpvqgRMnPwQZo=
+-----END CERTIFICATE-----
diff --git a/pulsar-broker-common/src/test/resources/ssl/my-ca/server-key.pem b/pulsar-broker-common/src/test/resources/ssl/my-ca/server-key.pem
new file mode 100644
index 00000000000..a3f3a36b73c
--- /dev/null
+++ b/pulsar-broker-common/src/test/resources/ssl/my-ca/server-key.pem
@@ -0,0 +1,28 @@
+-----BEGIN PRIVATE KEY-----
+MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDBcqDkMhjLd9ik
+//UQijqbajQP5t6dvVZNn9gODQrS9oB/URurNzCcPWYPJZfEJlTkV8mlmgq4dBjw
+ghpy5ALOGiERk55JPIN4cy01hQ6j7YSPFvMvBjqZvm5dpGDNTr7GY7THegMM1wpk
+9EaUOm7tBOHtf6ZnANjSMcQM74RCSBt0Koqw06CKVDCbgJ5NNE1LgwYeVQAwtQAh
+Y8rqqQKJvCorFbq7OiisFBnz5pRBT6N4kMo19LZo3Oe2F2w9eH9vacQ0NjSOCNXq
+al9Xl/Pwy9JgKKppwZ/3nCgRc+yfjrnkRz0fb+llb2NpR5Ge+tNMakqelE8bDSw/
+5BPjRPftAgMBAAECggEBAJm2JsgMUo1ihn/dbnIdFCKoCgRUs7FtYCVADOJlVKN7
+AXGpFi4/JV4Qn4cLnQNcXfovE2iF9VzJy4NYLgH60YvJUVtxC8Yv0lukUVkEiDST
+p9A3MTa9YVUG7xVzZwPcPVTQpzYV6lSKjpTXUTm5EKk/RvJ7itKv5plmt9x7eYFb
+/JwqXo1Z6C4gfIFR85LWmrCsNUK5T9oooLz88D6+ZH3+fWlr75RDff2kqdLshMTs
+N0Ov7NXcRFeruFs/IPrgTxjBMeNa2LFdYVPeeQ41L4uOI49uVBAmSn1be+THvDoj
+Do+6wTEF/h6/VLoOaIFZZdHlqd4is+xcEg8gwVkCn2ECgYEAxqVvGKc9qaqEVwBx
+U5Ru9OFx0NqEBvkYZRbCg1REcMFd3lqFTHvHiF3pmCp0XgLJKYuy42618IJXhj6D
+Y15/p9jX0025MpnH/AdwpO6x5pv6gb/JOMnHOnq8sI3R+V6TVsv1WZj0sOj94mF0
++Od++bQkUnSlfE4X7v+cJfo/Q8UCgYEA+Uz1yOyI9Dv1dEdBMdBA8MTriYU0uJCV
+dVKzL/uC9XyguVBWu1HX0MvEKyjPRycvLB7TuQqAFLgCtC8EEuPGBpWtyXOm9Jxw
+ToCfUZFuBQeMuf4vZcFgJjiEKTdKBxrvjkhyIhPR6JAy0WUr8Ry+ZtqvmG5NOEz5
+ptm1tznYngkCgYEAlckeyV8p/uqF2biKu3QcamgoU0zB6yQfAfK0fySmasNTzZtC
+EhbvsOLnhgbVMiI1ny8ol5fedtlBuAchOWeDKIQ40as0r3QHuQG/LY6S9Im+zeFY
+kIqNwInWB+cYYkmvHe6zNXlBYLh+4BmOgzTDqPPtw4MTWXTlVSDGlFhrJeUCgYBX
+7rlS4Xt9ChkNpoRsWZROWGbr3rw1zWmqND1X01Lh28+lDZ1J/RguYXET+BUEd+G/
+oi/zuKxsomrxuxOoxgZ3FBx0TgK5jORgDCYl0zIHPB57DBkTvx123cBf+Ux3LR0K
+BqubMXp8mUATc6gIJ6dRCBmfnmhGT4BPRcM+mXy6YQKBgGEGH37VABus+Oi3g1bk
+qEAaUI1asRLJIfbY2ImxEroLIQAbTFuIQUsZTKpT7jJZubjYvy1Fev0LU/n7Kv2w
+7ym41z70ro5uxwUBfJjnF3RtgncNcftn4b3siNzvBfKEBuhegMeS5YAbBIwABUpR
+4mVpm9BLOiX4yENIT6JdUQFc
+-----END PRIVATE KEY-----
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
index 504b53f0123..a8c107fa0b3 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
@@ -1399,6 +1399,8 @@ public class PulsarService implements AutoCloseable {
                                 conf.getBrokerClientAuthenticationParameters());
 
                 if (conf.isBrokerClientTlsEnabled()) {
+                    builder.tlsCiphers(config.getBrokerClientTlsCiphers())
+                            .tlsProtocols(config.getBrokerClientTlsProtocols());
                     if (conf.isBrokerClientTlsEnabledWithKeyStore()) {
                         builder.useKeyStoreTls(true)
                                 .tlsTrustStoreType(conf.getBrokerClientTlsTrustStoreType())
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java
index 4d9ff83e3f9..9033c50e933 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java
@@ -31,8 +31,7 @@ import javax.servlet.DispatcherType;
 import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.common.util.SecurityUtility;
-import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
+import org.apache.pulsar.jetty.tls.JettySslContextFactory;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -101,8 +100,8 @@ public class WebService implements AutoCloseable {
                 SslContextFactory sslCtxFactory;
                 ServiceConfiguration config = pulsar.getConfiguration();
                 if (config.isTlsEnabledWithKeyStore()) {
-                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
-                            config.getTlsProvider(),
+                    sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore(
+                            config.getWebServiceTlsProvider(),
                             config.getTlsKeyStoreType(),
                             config.getTlsKeyStore(),
                             config.getTlsKeyStorePassword(),
@@ -111,15 +110,20 @@ public class WebService implements AutoCloseable {
                             config.getTlsTrustStore(),
                             config.getTlsTrustStorePassword(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec()
                     );
                 } else {
-                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContext(
+                            config.getWebServiceTlsProvider(),
                             config.isTlsAllowInsecureConnection(),
                             config.getTlsTrustCertsFilePath(),
                             config.getTlsCertificateFilePath(),
                             config.getTlsKeyFilePath(),
-                            config.isTlsRequireTrustedClientCertOnConnect(), true,
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec());
                 }
                 httpsConnector = new PulsarServerConnector(server, 1, 1, sslCtxFactory);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java
index b8702522fce..9960d4c03f5 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java
@@ -104,7 +104,7 @@ public class ProxyPublishConsumeTlsTest extends TlsProducerConsumerBase {
 
         SslContextFactory sslContextFactory = new SslContextFactory();
         sslContextFactory.setSslContext(SecurityUtility
-                .createSslContext(false, SecurityUtility.loadCertificatesFromPemFile(TLS_TRUST_CERT_FILE_PATH)));
+                .createSslContext(false, SecurityUtility.loadCertificatesFromPemFile(TLS_TRUST_CERT_FILE_PATH), null));
 
         WebSocketClient consumeClient = new WebSocketClient(sslContextFactory);
         SimpleConsumerSocket consumeSocket = new SimpleConsumerSocket();
diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml
index 2fcd625d808..733ab2eda51 100644
--- a/pulsar-common/pom.xml
+++ b/pulsar-common/pom.xml
@@ -136,11 +136,6 @@
       <artifactId>netty-codec-haproxy</artifactId>
     </dependency>
 
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
index b1e8a14ff95..2e67b02f90b 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
@@ -18,11 +18,8 @@
  */
 package org.apache.pulsar.common.util;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.security.GeneralSecurityException;
 import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLException;
 
 @SuppressWarnings("checkstyle:JavadocType")
 public class DefaultSslContextBuilder extends SslContextAutoRefreshBuilder<SSLContext> {
@@ -31,23 +28,37 @@ public class DefaultSslContextBuilder extends SslContextAutoRefreshBuilder<SSLCo
     protected final boolean tlsAllowInsecureConnection;
     protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath;
     protected final boolean tlsRequireTrustedClientCertOnConnect;
+    private final String providerName;
 
     public DefaultSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
-            String keyFilePath, boolean requireTrustedClientCertOnConnect, long certRefreshInSec)
-            throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
+                                    String keyFilePath, boolean requireTrustedClientCertOnConnect,
+                                    long certRefreshInSec) {
         super(certRefreshInSec);
         this.tlsAllowInsecureConnection = allowInsecure;
         this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
         this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath);
         this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath);
         this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
+        this.providerName = null;
+    }
+
+    public DefaultSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
+                                    String keyFilePath, boolean requireTrustedClientCertOnConnect,
+                                    long certRefreshInSec, String providerName) {
+        super(certRefreshInSec);
+        this.tlsAllowInsecureConnection = allowInsecure;
+        this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
+        this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath);
+        this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath);
+        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
+        this.providerName = providerName;
     }
 
     @Override
     public synchronized SSLContext update() throws GeneralSecurityException {
         this.sslContext = SecurityUtility.createSslContext(tlsAllowInsecureConnection,
                 tlsTrustCertsFilePath.getFileName(), tlsCertificateFilePath.getFileName(),
-                tlsKeyFilePath.getFileName());
+                tlsKeyFilePath.getFileName(), this.providerName);
         return this.sslContext;
     }
 
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
index 73acbe6e742..0bde6e3ed5c 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
@@ -65,7 +65,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.pulsar.common.classification.InterfaceAudience;
 import org.apache.pulsar.common.classification.InterfaceStability;
 import org.apache.pulsar.common.tls.TlsHostnameVerifier;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
 
 /**
  * Helper class for the security domain.
@@ -198,9 +197,10 @@ public class SecurityUtility {
         return provider;
     }
 
-    public static SSLContext createSslContext(boolean allowInsecureConnection, Certificate[] trustCertificates)
+    public static SSLContext createSslContext(boolean allowInsecureConnection, Certificate[] trustCertificates,
+                                              String providerName)
             throws GeneralSecurityException {
-        return createSslContext(allowInsecureConnection, trustCertificates, (Certificate[]) null, (PrivateKey) null);
+        return createSslContext(allowInsecureConnection, trustCertificates, null, null, providerName);
     }
 
     public static SslContext createNettySslContextForClient(SslProvider sslProvider, boolean allowInsecureConnection,
@@ -213,11 +213,11 @@ public class SecurityUtility {
     }
 
     public static SSLContext createSslContext(boolean allowInsecureConnection, String trustCertsFilePath,
-            String certFilePath, String keyFilePath) throws GeneralSecurityException {
+            String certFilePath, String keyFilePath, String providerName) throws GeneralSecurityException {
         X509Certificate[] trustCertificates = loadCertificatesFromPemFile(trustCertsFilePath);
         X509Certificate[] certificates = loadCertificatesFromPemFile(certFilePath);
         PrivateKey privateKey = loadPrivateKeyFromPemFile(keyFilePath);
-        return createSslContext(allowInsecureConnection, trustCertificates, certificates, privateKey);
+        return createSslContext(allowInsecureConnection, trustCertificates, certificates, privateKey, providerName);
     }
 
     /**
@@ -325,18 +325,25 @@ public class SecurityUtility {
     }
 
     public static SSLContext createSslContext(boolean allowInsecureConnection, Certificate[] trustCertficates,
-            Certificate[] certificates, PrivateKey privateKey) throws GeneralSecurityException {
+                                              Certificate[] certificates, PrivateKey privateKey)
+            throws GeneralSecurityException {
+        return createSslContext(allowInsecureConnection, trustCertficates, certificates, privateKey, null);
+    }
+
+    public static SSLContext createSslContext(boolean allowInsecureConnection, Certificate[] trustCertficates,
+                                              Certificate[] certificates, PrivateKey privateKey, String providerName)
+            throws GeneralSecurityException {
         KeyStoreHolder ksh = new KeyStoreHolder();
         TrustManager[] trustManagers = null;
         KeyManager[] keyManagers = null;
+        Provider provider = resolveProvider(providerName);
 
-        trustManagers = setupTrustCerts(ksh, allowInsecureConnection, trustCertficates, CONSCRYPT_PROVIDER);
+        trustManagers = setupTrustCerts(ksh, allowInsecureConnection, trustCertficates, provider);
         keyManagers = setupKeyManager(ksh, privateKey, certificates);
 
-        SSLContext sslCtx = CONSCRYPT_PROVIDER != null ? SSLContext.getInstance("TLS", CONSCRYPT_PROVIDER)
+        SSLContext sslCtx = provider != null ? SSLContext.getInstance("TLS", provider)
                 : SSLContext.getInstance("TLS");
         sslCtx.init(keyManagers, trustManagers, new SecureRandom());
-        sslCtx.getDefaultSSLParameters();
         return sslCtx;
     }
 
@@ -543,51 +550,16 @@ public class SecurityUtility {
         }
     }
 
-    public static SslContextFactory createSslContextFactory(boolean tlsAllowInsecureConnection,
-            String tlsTrustCertsFilePath, String tlsCertificateFilePath, String tlsKeyFilePath,
-            boolean tlsRequireTrustedClientCertOnConnect, boolean autoRefresh, long certRefreshInSec)
-            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
-        SslContextFactory sslCtxFactory = null;
-        if (autoRefresh) {
-            sslCtxFactory = new SslContextFactoryWithAutoRefresh(tlsAllowInsecureConnection, tlsTrustCertsFilePath,
-                tlsCertificateFilePath, tlsKeyFilePath, tlsRequireTrustedClientCertOnConnect, certRefreshInSec);
-        } else {
-            sslCtxFactory = new SslContextFactory();
-            SSLContext sslCtx = createSslContext(tlsAllowInsecureConnection, tlsTrustCertsFilePath,
-                tlsCertificateFilePath, tlsKeyFilePath);
-            sslCtxFactory.setSslContext(sslCtx);
+    public static Provider resolveProvider(String providerName) throws NoSuchAlgorithmException {
+        Provider provider = null;
+        if (!StringUtils.isEmpty(providerName)) {
+            provider = Security.getProvider(providerName);
         }
-        if (tlsRequireTrustedClientCertOnConnect) {
-            sslCtxFactory.setNeedClientAuth(true);
-        } else {
-            sslCtxFactory.setWantClientAuth(true);
-        }
-        sslCtxFactory.setTrustAll(true);
-        return sslCtxFactory;
-    }
 
-    /**
-     * {@link SslContextFactory} that auto-refresh SSLContext.
-     */
-    static class SslContextFactoryWithAutoRefresh extends SslContextFactory {
-
-        private final DefaultSslContextBuilder sslCtxRefresher;
-
-        public SslContextFactoryWithAutoRefresh(boolean tlsAllowInsecureConnection, String tlsTrustCertsFilePath,
-                String tlsCertificateFilePath, String tlsKeyFilePath, boolean tlsRequireTrustedClientCertOnConnect,
-                long certRefreshInSec)
-                throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
-            super();
-            sslCtxRefresher = new DefaultSslContextBuilder(tlsAllowInsecureConnection, tlsTrustCertsFilePath,
-                    tlsCertificateFilePath, tlsKeyFilePath, tlsRequireTrustedClientCertOnConnect, certRefreshInSec);
-            if (CONSCRYPT_PROVIDER != null) {
-                setProvider(CONSCRYPT_PROVIDER.getName());
-            }
+        if (provider == null) {
+            provider = SSLContext.getDefault().getProvider();
         }
 
-        @Override
-        public SSLContext getSslContext() {
-            return sslCtxRefresher.get();
-        }
+        return provider;
     }
 }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
index 987a32b216c..7b06a33601b 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
@@ -38,7 +38,6 @@ import javax.net.ssl.TrustManagerFactory;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.common.util.SecurityUtility;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
 
 /**
  * KeyStoreSSLContext that mainly wrap a SSLContext to provide SSL context for both webservice and netty.
@@ -125,8 +124,10 @@ public class KeyStoreSSLContext {
 
     public SSLContext createSSLContext() throws GeneralSecurityException, IOException {
         SSLContext sslContext;
-        if (sslProviderString != null) {
-            sslContext = SSLContext.getInstance(protocol, sslProviderString);
+
+        Provider provider = SecurityUtility.resolveProvider(sslProviderString);
+        if (provider != null) {
+            sslContext = SSLContext.getInstance(protocol, provider);
         } else {
             sslContext = SSLContext.getInstance(protocol);
         }
@@ -149,8 +150,8 @@ public class KeyStoreSSLContext {
         if (this.allowInsecureConnection) {
             trustManagerFactory = InsecureTrustManagerFactory.INSTANCE;
         } else {
-            trustManagerFactory = sslProviderString != null
-                    ? TrustManagerFactory.getInstance(tmfAlgorithm, sslProviderString)
+            trustManagerFactory = provider != null
+                    ? TrustManagerFactory.getInstance(tmfAlgorithm, provider)
                     : TrustManagerFactory.getInstance(tmfAlgorithm);
             KeyStore trustStore = KeyStore.getInstance(trustStoreTypeString);
             char[] passwordChars = trustStorePassword.toCharArray();
@@ -333,48 +334,4 @@ public class KeyStoreSSLContext {
 
         return keyStoreSSLContext.createSSLContext();
     }
-
-    // for web server. autoRefresh is default true.
-    public static SslContextFactory createSslContextFactory(String sslProviderString,
-                                                            String keyStoreTypeString,
-                                                            String keyStore,
-                                                            String keyStorePassword,
-                                                            boolean allowInsecureConnection,
-                                                            String trustStoreTypeString,
-                                                            String trustStore,
-                                                            String trustStorePassword,
-                                                            boolean requireTrustedClientCertOnConnect,
-                                                            long certRefreshInSec)
-            throws GeneralSecurityException, IOException {
-        SslContextFactory sslCtxFactory;
-
-        if (sslProviderString == null) {
-            Provider provider = SecurityUtility.CONSCRYPT_PROVIDER;
-            if (provider != null) {
-                sslProviderString = provider.getName();
-            }
-        }
-
-        sslCtxFactory = new SslContextFactoryWithAutoRefresh(
-                sslProviderString,
-                keyStoreTypeString,
-                keyStore,
-                keyStorePassword,
-                allowInsecureConnection,
-                trustStoreTypeString,
-                trustStore,
-                trustStorePassword,
-                requireTrustedClientCertOnConnect,
-                certRefreshInSec);
-
-        if (requireTrustedClientCertOnConnect) {
-            sslCtxFactory.setNeedClientAuth(true);
-        } else {
-            sslCtxFactory.setWantClientAuth(true);
-        }
-        sslCtxFactory.setTrustAll(true);
-
-        return sslCtxFactory;
-    }
-}
-
+}
\ No newline at end of file
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java
deleted file mode 100644
index 0882a3a0cb1..00000000000
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pulsar.common.util.keystoretls;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.GeneralSecurityException;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLException;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-
-/**
- * SslContextFactoryWithAutoRefresh that create SSLContext for web server, and refresh in time.
- */
-public class SslContextFactoryWithAutoRefresh extends SslContextFactory {
-    private final NetSslContextBuilder sslCtxRefresher;
-
-    public SslContextFactoryWithAutoRefresh(String sslProviderString,
-                                            String keyStoreTypeString,
-                                            String keyStore,
-                                            String keyStorePassword,
-                                            boolean allowInsecureConnection,
-                                            String trustStoreTypeString,
-                                            String trustStore,
-                                            String trustStorePassword,
-                                            boolean requireTrustedClientCertOnConnect,
-                                            long certRefreshInSec)
-            throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
-        super();
-        sslCtxRefresher = new NetSslContextBuilder(
-                sslProviderString,
-                keyStoreTypeString,
-                keyStore,
-                keyStorePassword,
-                allowInsecureConnection,
-                trustStoreTypeString,
-                trustStore,
-                trustStorePassword,
-                requireTrustedClientCertOnConnect,
-                certRefreshInSec);
-        if (sslProviderString != null) {
-            setProvider(sslProviderString);
-        }
-    }
-
-    @Override
-    public SSLContext getSslContext() {
-        return sslCtxRefresher.get();
-    }
-}
diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
index 0c0bb95251e..86a4eb58cb7 100644
--- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
+++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
@@ -28,6 +28,7 @@ import javax.servlet.Servlet;
 import org.apache.pulsar.common.util.RestException;
 import org.apache.pulsar.common.util.SecurityUtility;
 import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
+import org.apache.pulsar.jetty.tls.JettySslContextFactory;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -74,8 +75,8 @@ public class ServerManager {
             try {
                 SslContextFactory sslCtxFactory;
                 if (config.isTlsEnabledWithKeyStore()) {
-                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
-                            config.getTlsProvider(),
+                    sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore(
+                            null,
                             config.getTlsKeyStoreType(),
                             config.getTlsKeyStore(),
                             config.getTlsKeyStorePassword(),
@@ -84,16 +85,20 @@ public class ServerManager {
                             config.getTlsTrustStore(),
                             config.getTlsTrustStorePassword(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
+                            null,
+                            null,
                             config.getTlsCertRefreshCheckDurationSec()
                     );
                 } else {
-                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContext(
+                            null,
                             config.isTlsAllowInsecureConnection(),
                             config.getTlsTrustCertsFilePath(),
                             config.getTlsCertificateFilePath(),
                             config.getTlsKeyFilePath(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
-                            true,
+                            null,
+                            null,
                             config.getTlsCertRefreshCheckDurationSec());
                 }
                 connectorTls = new ServerConnector(server, 1, 1, sslCtxFactory);
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 7c9a1798b92..66a0f809285 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
@@ -25,12 +25,12 @@ import org.apache.pulsar.broker.authentication.AuthenticationService;
 import org.apache.pulsar.broker.web.AuthenticationFilter;
 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;
 import org.apache.pulsar.functions.worker.rest.api.v2.WorkerStatsApiV2Resource;
+import org.apache.pulsar.jetty.tls.JettySslContextFactory;
+import org.eclipse.jetty.server.ConnectionLimit;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -142,7 +142,7 @@ public class WorkerServer {
             try {
                 SslContextFactory sslCtxFactory;
                 if (workerConfig.isTlsEnabledWithKeyStore()) {
-                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore(
                             workerConfig.getTlsProvider(),
                             workerConfig.getTlsKeyStoreType(),
                             workerConfig.getTlsKeyStore(),
@@ -152,17 +152,22 @@ public class WorkerServer {
                             workerConfig.getTlsTrustStore(),
                             workerConfig.getTlsTrustStorePassword(),
                             workerConfig.isTlsRequireTrustedClientCertOnConnect(),
+                            workerConfig.getWebServiceTlsCiphers(),
+                            workerConfig.getWebServiceTlsProtocols(),
                             workerConfig.getTlsCertRefreshCheckDurationSec()
                     );
                 } else {
-                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContext(
+                            workerConfig.getTlsProvider(),
                             workerConfig.isTlsAllowInsecureConnection(),
                             workerConfig.getTlsTrustCertsFilePath(),
                             workerConfig.getTlsCertificateFilePath(),
                             workerConfig.getTlsKeyFilePath(),
                             workerConfig.isTlsRequireTrustedClientCertOnConnect(),
-                            true,
-                            workerConfig.getTlsCertRefreshCheckDurationSec());
+                            workerConfig.getWebServiceTlsCiphers(),
+                            workerConfig.getWebServiceTlsProtocols(),
+                            workerConfig.getTlsCertRefreshCheckDurationSec()
+                    );
                 }
                 httpsConnector = new ServerConnector(server, sslCtxFactory);
                 httpsConnector.setPort(this.workerConfig.getWorkerPortTls());
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
index 301c57170a8..5c86d2be089 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
@@ -244,15 +244,17 @@ class AdminProxyHandler extends ProxyServlet {
                     AuthenticationDataProvider authData = auth.getAuthData();
                     if (authData.hasDataForTls()) {
                         sslCtx = SecurityUtility.createSslContext(
-                            config.isTlsAllowInsecureConnection(),
-                            trustCertificates,
-                            authData.getTlsCertificates(),
-                            authData.getTlsPrivateKey()
+                                config.isTlsAllowInsecureConnection(),
+                                trustCertificates,
+                                authData.getTlsCertificates(),
+                                authData.getTlsPrivateKey(),
+                                config.getBrokerClientSslProvider()
                         );
                     } else {
                         sslCtx = SecurityUtility.createSslContext(
-                            config.isTlsAllowInsecureConnection(),
-                            trustCertificates
+                                config.isTlsAllowInsecureConnection(),
+                                trustCertificates,
+                                config.getBrokerClientSslProvider()
                         );
                     }
 
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
index 6369755e9c2..b08fea282cb 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.stream.Collectors;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -214,6 +215,26 @@ public class ProxyConfiguration implements PulsarConfiguration {
     )
     private Optional<Integer> webServicePortTls = Optional.empty();
 
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Specify the TLS provider for the web service, available values can be SunJSSE, Conscrypt and etc."
+    )
+    private String webServiceTlsProvider = "Conscrypt";
+
+    @FieldContext(
+            category = CATEGORY_TLS,
+            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<String> webServiceTlsProtocols = new TreeSet<>();
+
+    @FieldContext(
+            category = CATEGORY_TLS,
+            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<String> webServiceTlsCiphers = new TreeSet<>();
+
     @FieldContext(
             category = CATEGORY_SERVER,
             doc = "The directory where nar Extraction happens"
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java
index 62a2063b0b2..f35f183e4fd 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java
@@ -37,8 +37,7 @@ import org.apache.pulsar.broker.web.AuthenticationFilter;
 import org.apache.pulsar.broker.web.JsonMapperProvider;
 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.jetty.tls.JettySslContextFactory;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.HttpConfiguration;
@@ -101,8 +100,8 @@ public class WebServer {
             try {
                 SslContextFactory sslCtxFactory;
                 if (config.isTlsEnabledWithKeyStore()) {
-                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
-                            config.getTlsProvider(),
+                    sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore(
+                            config.getWebServiceTlsProvider(),
                             config.getTlsKeyStoreType(),
                             config.getTlsKeyStore(),
                             config.getTlsKeyStorePassword(),
@@ -111,16 +110,20 @@ public class WebServer {
                             config.getTlsTrustStore(),
                             config.getTlsTrustStorePassword(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec()
                     );
                 } else {
-                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContext(
+                            config.getWebServiceTlsProvider(),
                             config.isTlsAllowInsecureConnection(),
                             config.getTlsTrustCertsFilePath(),
                             config.getTlsCertificateFilePath(),
                             config.getTlsKeyFilePath(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
-                            true,
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec());
                 }
                 connectorTls = new ServerConnector(server, 1, 1, sslCtxFactory);
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 23414cb4532..22fc40cdb1b 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
@@ -36,8 +36,7 @@ import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.web.JsonMapperProvider;
 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.apache.pulsar.jetty.tls.JettySslContextFactory;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -80,7 +79,7 @@ public class ProxyServer {
             try {
                 SslContextFactory sslCtxFactory;
                 if (config.isTlsEnabledWithKeyStore()) {
-                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore(
                             config.getTlsProvider(),
                             config.getTlsKeyStoreType(),
                             config.getTlsKeyStore(),
@@ -90,16 +89,20 @@ public class ProxyServer {
                             config.getTlsTrustStore(),
                             config.getTlsTrustStorePassword(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec()
                     );
                 } else {
-                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                    sslCtxFactory = JettySslContextFactory.createServerSslContext(
+                            config.getTlsProvider(),
                             config.isTlsAllowInsecureConnection(),
                             config.getTlsTrustCertsFilePath(),
                             config.getTlsCertificateFilePath(),
                             config.getTlsKeyFilePath(),
                             config.isTlsRequireTrustedClientCertOnConnect(),
-                            true,
+                            config.getWebServiceTlsCiphers(),
+                            config.getWebServiceTlsProtocols(),
                             config.getTlsCertRefreshCheckDurationSec());
                 }
                 connectorTls = new ServerConnector(server, sslCtxFactory);
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 e62fecf9fab..22a1384af5a 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
@@ -138,11 +138,9 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration {
     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."
+            doc = "Specify the TLS provider for the WebSocket service: SunJSSE, Conscrypt and etc."
     )
-    private String tlsProvider = null;
+    private String tlsProvider = "Conscrypt";
 
     @FieldContext(
             doc = "TLS KeyStore type configuration in WebSocket: JKS, PKCS12"