You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/09/03 07:17:14 UTC

[2/2] lucene-solr:jira/http2: SslContextFactory instance should not be shared between clients and servers

SslContextFactory instance should not be shared between clients and servers


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b378bb5c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b378bb5c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b378bb5c

Branch: refs/heads/jira/http2
Commit: b378bb5c071bb52238bd78c995bbfd07f9a68de8
Parents: 6dced78
Author: Cao Manh Dat <da...@apache.org>
Authored: Mon Sep 3 14:16:46 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Mon Sep 3 14:16:46 2018 +0700

----------------------------------------------------------------------
 .../solr/client/solrj/embedded/SSLConfig.java   | 166 ------------------
 .../handler/component/HttpShardHandler.java     |   4 -
 .../org/apache/solr/cloud/SSLMigrationTest.java |   2 +-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java |  16 +-
 .../solr/client/solrj/embedded/SSLConfig.java   | 166 ++++++++++++++++++
 .../solr/client/solrj/impl/Http2SolrClient.java |  42 +++--
 .../solr/BaseDistributedSearchTestCase.java     |   2 +-
 .../java/org/apache/solr/SolrJettyTestBase.java |   4 +-
 .../java/org/apache/solr/SolrTestCaseHS.java    |   2 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   5 +-
 .../cloud/AbstractFullDistribZkTestBase.java    |   6 +-
 .../org/apache/solr/util/SSLTestConfig.java     | 173 ++++++++-----------
 12 files changed, 285 insertions(+), 303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
deleted file mode 100644
index 62c9024..0000000
--- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
+++ /dev/null
@@ -1,166 +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.solr.client.solrj.embedded;
-
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-
-/** 
- * Encapsulates settings related to SSL Configuration for an embedded Jetty Server.
- * NOTE: all other settings are ignored if {@link #isSSLMode} is false.
- * @see #setUseSSL
- */
-public class SSLConfig {
-  
-  private boolean useSsl;
-  private boolean clientAuth;
-  private String keyStore;
-  private String keyStorePassword;
-  private String trustStore;
-  private String trustStorePassword;
-
-  /** NOTE: all other settings are ignored if useSSL is false; trustStore settings are ignored if clientAuth is false */
-  public SSLConfig(boolean useSSL, boolean clientAuth, String keyStore, String keyStorePassword, String trustStore, String trustStorePassword) {
-    this.useSsl = useSSL;
-    this.clientAuth = clientAuth;
-    this.keyStore = keyStore;
-    this.keyStorePassword = keyStorePassword;
-    this.trustStore = trustStore;
-    this.trustStorePassword = trustStorePassword;
-  }
-  
-  public void setUseSSL(boolean useSSL) {
-    this.useSsl = useSSL;
-  }
-  
-  public void setClientAuth(boolean clientAuth) {
-    this.clientAuth = clientAuth;
-  }
-  
-  /** All other settings on this object are ignored unless this is true */
-  public boolean isSSLMode() {
-    return useSsl;
-  }
-  
-  public boolean isClientAuthMode() {
-    return clientAuth;
-  }
-
-  public String getKeyStore() {
-    return keyStore;
-  }
-
-  public String getKeyStorePassword() {
-    return keyStorePassword;
-  }
-
-  public String getTrustStore() {
-    return trustStore;
-  }
-
-  public String getTrustStorePassword() {
-    return trustStorePassword;
-  }
-
-  /**
-   * Returns an SslContextFactory that should be used by a jetty server based on the specified 
-   * SSLConfig param which may be null.
-   *
-   * if the SSLConfig param is non-null, then this method will return the results of 
-   * {@link #createContextFactory()}.
-   * 
-   * If the SSLConfig param is null, then this method will return null unless the 
-   * <code>tests.jettySsl</code> system property is true, in which case standard "javax.net.ssl.*" 
-   * system properties will be used instead, along with "tests.jettySsl.clientAuth".
-   * 
-   * @see #createContextFactory()
-   */
-  public static SslContextFactory createContextFactory(SSLConfig sslConfig) {
-
-    if (sslConfig != null) {
-      return sslConfig.createContextFactory();
-    }
-    // else...
-    if (Boolean.getBoolean("tests.jettySsl")) {
-      return configureSslFromSysProps();
-    }
-    // else...
-    return null;
-  }
-  
-  /**
-   * Returns an SslContextFactory that should be used by a jetty server based on this SSLConfig instance, 
-   * or null if SSL should not be used.
-   *
-   * The default implementation generates a simple factory according to the keystore, truststore, 
-   * and clientAuth properties of this object.
-   *
-   * @see #getKeyStore
-   * @see #getKeyStorePassword
-   * @see #isClientAuthMode
-   * @see #getTrustStore
-   * @see #getTrustStorePassword
-   */
-  public SslContextFactory createContextFactory() {
-
-    if (! isSSLMode()) {
-      return null;
-    }
-    // else...
-    
-    SslContextFactory factory = new SslContextFactory(false);
-    if (getKeyStore() != null)
-      factory.setKeyStorePath(getKeyStore());
-    if (getKeyStorePassword() != null)
-      factory.setKeyStorePassword(getKeyStorePassword());
-    
-    factory.setNeedClientAuth(isClientAuthMode());
-    
-    if (isClientAuthMode()) {
-      if (getTrustStore() != null)
-        factory.setTrustStorePath(getTrustStore());
-      if (getTrustStorePassword() != null)
-        factory.setTrustStorePassword(getTrustStorePassword());
-    }
-    return factory;
-
-  }
-
-  private static SslContextFactory configureSslFromSysProps() {
-
-    SslContextFactory sslcontext = new SslContextFactory(false);
-
-    if (null != System.getProperty("javax.net.ssl.keyStore")) {
-      sslcontext.setKeyStorePath
-          (System.getProperty("javax.net.ssl.keyStore"));
-    }
-    if (null != System.getProperty("javax.net.ssl.keyStorePassword")) {
-      sslcontext.setKeyStorePassword
-          (System.getProperty("javax.net.ssl.keyStorePassword"));
-    }
-    if (null != System.getProperty("javax.net.ssl.trustStore")) {
-      sslcontext.setTrustStorePath
-          (System.getProperty("javax.net.ssl.trustStore"));
-    }
-    if (null != System.getProperty("javax.net.ssl.trustStorePassword")) {
-      sslcontext.setTrustStorePassword
-          (System.getProperty("javax.net.ssl.trustStorePassword"));
-    }
-    sslcontext.setNeedClientAuth(Boolean.getBoolean("tests.jettySsl.clientAuth"));
-
-    return sslcontext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
index 7986a1c..6f60ffc 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
@@ -32,13 +32,9 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
-import org.apache.http.client.HttpClient;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.impl.Http2SolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
-import org.apache.solr.client.solrj.impl.LBHttp2SolrClient;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.util.ClientUtils;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/core/src/test/org/apache/solr/cloud/SSLMigrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SSLMigrationTest.java b/solr/core/src/test/org/apache/solr/cloud/SSLMigrationTest.java
index f3efd63..6475a39 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SSLMigrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SSLMigrationTest.java
@@ -79,7 +79,7 @@ public class SSLMigrationTest extends AbstractFullDistribZkTestBase {
           .stopAtShutdown(false)
           .withServlets(getExtraServlets())
           .withFilters(getExtraRequestFilters())
-          .withSSLConfig(sslConfig)
+          .withSSLConfig(sslConfig.buildServerSSLConfig())
           .build();
 
       Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
index d470bb6..8102e65 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
@@ -103,7 +103,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
   public void testNoSsl() throws Exception {
     final SSLTestConfig sslConfig = new SSLTestConfig(false, false);
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "http");
     checkClusterWithNodeReplacement(sslConfig);
   }
@@ -114,7 +114,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     // options.
     final SSLTestConfig sslConfig = new SSLTestConfig(false, true);
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "http");
     checkClusterWithNodeReplacement(sslConfig);
   }
@@ -122,7 +122,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
   public void testSslAndNoClientAuth() throws Exception {
     final SSLTestConfig sslConfig = new SSLTestConfig(true, false);
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "https");
     checkClusterWithNodeReplacement(sslConfig);
   }
@@ -133,7 +133,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     final SSLTestConfig sslConfig = new SSLTestConfig(true, true);
 
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "https");
     checkClusterWithNodeReplacement(sslConfig);
   }
@@ -142,7 +142,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
   public void testSslWithCheckPeerName() throws Exception {
     final SSLTestConfig sslConfig = new SSLTestConfig(true, false, true);
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "https");
     checkClusterWithNodeReplacement(sslConfig);
   }
@@ -159,7 +159,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
    */
   private void checkClusterWithNodeReplacement(SSLTestConfig sslConfig) throws Exception {
     
-    final JettyConfig config = JettyConfig.builder().withSSLConfig(sslConfig).build();
+    final JettyConfig config = JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build();
     final MiniSolrCloudCluster cluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), config);
     try {
       checkClusterWithCollectionCreations(cluster, sslConfig);
@@ -187,9 +187,9 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     // certs with a bogus hostname/ip and clients shouldn't care...
     final SSLTestConfig sslConfig = new SSLTestConfig(true, false, false);
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     System.setProperty(ZkStateReader.URL_SCHEME, "https");
-    final JettyConfig config = JettyConfig.builder().withSSLConfig(sslConfig).build();
+    final JettyConfig config = JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build();
     final MiniSolrCloudCluster cluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), config);
     try {
       checkClusterWithCollectionCreations(cluster, sslConfig);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/solrj/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java b/solr/solrj/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
new file mode 100644
index 0000000..4091bf7
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java
@@ -0,0 +1,166 @@
+/*
+ * 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.solr.client.solrj.embedded;
+
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+
+/** 
+ * Encapsulates settings related to SSL Configuration.
+ * NOTE: all other settings are ignored if {@link #isSSLMode} is false.
+ * @see #setUseSSL
+ */
+public class SSLConfig {
+  
+  private boolean useSsl;
+  private boolean clientAuth;
+  private String keyStore;
+  private String keyStorePassword;
+  private String trustStore;
+  private String trustStorePassword;
+
+  /** NOTE: all other settings are ignored if useSSL is false; trustStore settings are ignored if clientAuth is false */
+  public SSLConfig(boolean useSSL, boolean clientAuth, String keyStore, String keyStorePassword, String trustStore, String trustStorePassword) {
+    this.useSsl = useSSL;
+    this.clientAuth = clientAuth;
+    this.keyStore = keyStore;
+    this.keyStorePassword = keyStorePassword;
+    this.trustStore = trustStore;
+    this.trustStorePassword = trustStorePassword;
+  }
+  
+  public void setUseSSL(boolean useSSL) {
+    this.useSsl = useSSL;
+  }
+  
+  public void setClientAuth(boolean clientAuth) {
+    this.clientAuth = clientAuth;
+  }
+  
+  /** All other settings on this object are ignored unless this is true */
+  public boolean isSSLMode() {
+    return useSsl;
+  }
+  
+  public boolean isClientAuthMode() {
+    return clientAuth;
+  }
+
+  public String getKeyStore() {
+    return keyStore;
+  }
+
+  public String getKeyStorePassword() {
+    return keyStorePassword;
+  }
+
+  public String getTrustStore() {
+    return trustStore;
+  }
+
+  public String getTrustStorePassword() {
+    return trustStorePassword;
+  }
+
+  /**
+   * Returns an SslContextFactory that should be used by a jetty server based on the specified 
+   * SSLConfig param which may be null.
+   *
+   * if the SSLConfig param is non-null, then this method will return the results of 
+   * {@link #createContextFactory()}.
+   * 
+   * If the SSLConfig param is null, then this method will return null unless the 
+   * <code>tests.jettySsl</code> system property is true, in which case standard "javax.net.ssl.*" 
+   * system properties will be used instead, along with "tests.jettySsl.clientAuth".
+   * 
+   * @see #createContextFactory()
+   */
+  public static SslContextFactory createContextFactory(SSLConfig sslConfig) {
+
+    if (sslConfig != null) {
+      return sslConfig.createContextFactory();
+    }
+    // else...
+    if (Boolean.getBoolean("tests.jettySsl")) {
+      return configureSslFromSysProps();
+    }
+    // else...
+    return null;
+  }
+  
+  /**
+   * Returns an SslContextFactory that should be used by a jetty server based on this SSLConfig instance, 
+   * or null if SSL should not be used.
+   *
+   * The default implementation generates a simple factory according to the keystore, truststore, 
+   * and clientAuth properties of this object.
+   *
+   * @see #getKeyStore
+   * @see #getKeyStorePassword
+   * @see #isClientAuthMode
+   * @see #getTrustStore
+   * @see #getTrustStorePassword
+   */
+  public SslContextFactory createContextFactory() {
+
+    if (! isSSLMode()) {
+      return null;
+    }
+    // else...
+    
+    SslContextFactory factory = new SslContextFactory(false);
+    if (getKeyStore() != null)
+      factory.setKeyStorePath(getKeyStore());
+    if (getKeyStorePassword() != null)
+      factory.setKeyStorePassword(getKeyStorePassword());
+    
+    factory.setNeedClientAuth(isClientAuthMode());
+    
+    if (isClientAuthMode()) {
+      if (getTrustStore() != null)
+        factory.setTrustStorePath(getTrustStore());
+      if (getTrustStorePassword() != null)
+        factory.setTrustStorePassword(getTrustStorePassword());
+    }
+    return factory;
+
+  }
+
+  private static SslContextFactory configureSslFromSysProps() {
+
+    SslContextFactory sslcontext = new SslContextFactory(false);
+
+    if (null != System.getProperty("javax.net.ssl.keyStore")) {
+      sslcontext.setKeyStorePath
+          (System.getProperty("javax.net.ssl.keyStore"));
+    }
+    if (null != System.getProperty("javax.net.ssl.keyStorePassword")) {
+      sslcontext.setKeyStorePassword
+          (System.getProperty("javax.net.ssl.keyStorePassword"));
+    }
+    if (null != System.getProperty("javax.net.ssl.trustStore")) {
+      sslcontext.setTrustStorePath
+          (System.getProperty("javax.net.ssl.trustStore"));
+    }
+    if (null != System.getProperty("javax.net.ssl.trustStorePassword")) {
+      sslcontext.setTrustStorePassword
+          (System.getProperty("javax.net.ssl.trustStorePassword"));
+    }
+    sslcontext.setNeedClientAuth(Boolean.getBoolean("tests.jettySsl.clientAuth"));
+
+    return sslcontext;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index ad1fcf6..8342a4a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -21,6 +21,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.UnsupportedEncodingException;
+import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -44,6 +45,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.V2RequestSupport;
+import org.apache.solr.client.solrj.embedded.SSLConfig;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -82,6 +84,8 @@ import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
 import org.eclipse.jetty.util.Fields;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.util.Utils.getObjectByPath;
 
@@ -90,7 +94,9 @@ import static org.apache.solr.common.util.Utils.getObjectByPath;
  * @lucene.experimental
  */
 public class Http2SolrClient extends SolrClient {
-  private static volatile SslContextFactory defaultSslContextFactory = getDefaultSslContextFactory();
+  private static volatile SSLConfig defaultSSLConfig;
+
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final int MAX_OUTSTANDING_REQUESTS = 1000;
   private static final String AGENT = "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
   private static final String UTF_8 = StandardCharsets.UTF_8.name();
@@ -191,13 +197,23 @@ public class Http2SolrClient extends SolrClient {
     httpClientExecutor.setDaemon(true);
 
     HttpClientTransport transport;
-    if (builder.useHttp1_1 || builder.sslContextFactory != null) {
+    boolean useHttps = serverBaseUrl != null && serverBaseUrl.startsWith("https");
+    SslContextFactory sslContextFactory = null;
+    if (useHttps && builder.sslConfig == null) {
+      LOG.info("SSLConfig is not provided, building SslContextFactory from System.properties");
+      sslContextFactory = getDefaultSslContextFactory();
+    }
+    if (builder.sslConfig != null) {
+      sslContextFactory = builder.sslConfig.createContextFactory();
+    }
+
+    if (builder.useHttp1_1 || sslContextFactory != null) {
       transport = new HttpClientTransportOverHTTP(2);
-      httpClient = new HttpClient(transport, builder.sslContextFactory);
+      httpClient = new HttpClient(transport, sslContextFactory);
     } else {
       HTTP2Client http2client = new HTTP2Client();
       transport = new HttpClientTransportOverHTTP2(http2client);
-      httpClient = new HttpClient(transport, builder.sslContextFactory);
+      httpClient = new HttpClient(transport, null);
     }
     httpClient.setExecutor(httpClientExecutor);
     httpClient.setStrictEventOrdering(false);
@@ -638,7 +654,7 @@ public class Http2SolrClient extends SolrClient {
   public static class Builder {
 
     private HttpClient httpClient;
-    private SslContextFactory sslContextFactory;
+    private SSLConfig sslConfig = defaultSSLConfig;
     private Integer idleTimeout;
     private Integer connectionTimeout;
     private boolean useHttp1_1 = false;
@@ -646,12 +662,11 @@ public class Http2SolrClient extends SolrClient {
     private Request.BeginListener beginListener = request -> {};
 
     public Builder() {
-      this.sslContextFactory = defaultSslContextFactory;
+
     }
 
     public Builder(String baseSolrUrl) {
       this.baseSolrUrl = baseSolrUrl;
-      this.sslContextFactory = defaultSslContextFactory;
     }
 
     public Http2SolrClient build() {
@@ -663,8 +678,8 @@ public class Http2SolrClient extends SolrClient {
       return this;
     }
 
-    public Builder withSslContextFactory(SslContextFactory factory) {
-      this.sslContextFactory = factory;
+    public Builder withSSLConfig(SSLConfig sslConfig) {
+      this.sslConfig = sslConfig;
       return this;
     }
 
@@ -781,14 +796,13 @@ public class Http2SolrClient extends SolrClient {
     parser = processor;
   }
 
-  // public for testing, only used by tests
-  public static void setSslContextFactory(SslContextFactory sslContextFactory) {
-    Http2SolrClient.defaultSslContextFactory = sslContextFactory;
+  public static void setDefaultSSLConfig(SSLConfig sslConfig) {
+    Http2SolrClient.defaultSSLConfig = sslConfig;
   }
 
   // public for testing, only used by tests
   public static void resetSslContextFactory() {
-    Http2SolrClient.defaultSslContextFactory = getDefaultSslContextFactory();
+    Http2SolrClient.defaultSSLConfig = null;
   }
 
   private static SslContextFactory getDefaultSslContextFactory() {
@@ -813,7 +827,7 @@ public class Http2SolrClient extends SolrClient {
 
     String checkPeerNameStr = System.getProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME);
     boolean sslCheckPeerName = true;
-    if (checkPeerNameStr == null && "false".equalsIgnoreCase(checkPeerNameStr)) {
+    if (checkPeerNameStr == null || "false".equalsIgnoreCase(checkPeerNameStr)) {
       sslCheckPeerName = false;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index dfea26a..c37903c 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -421,7 +421,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
         .setContext(context)
         .withFilters(getExtraRequestFilters())
         .withServlets(getExtraServlets())
-        .withSSLConfig(sslConfig)
+        .withSSLConfig(sslConfig.buildServerSSLConfig())
         .build());
 
     jetty.start();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
index 7703ecb..3709242 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
@@ -65,7 +65,7 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
         .setContext(context)
         .stopAtShutdown(stopAtShutdown)
         .withServlets(extraServlets)
-        .withSSLConfig(sslConfig)
+        .withSSLConfig(sslConfig.buildServerSSLConfig())
         .build();
 
     Properties nodeProps = new Properties();
@@ -89,7 +89,7 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
   }
 
   public static JettySolrRunner createJetty(String solrHome) throws Exception {
-    return createJetty(solrHome, new Properties(), JettyConfig.builder().withSSLConfig(sslConfig).build());
+    return createJetty(solrHome, new Properties(), JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build());
   }
 
   public static JettySolrRunner createJetty(String solrHome, Properties nodeProperties, JettyConfig jettyConfig) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
index 2da0c84..32ef9a7 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
@@ -458,7 +458,7 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
             .stopAtShutdown(true)
             .setContext("/solr")
             .setPort(port)
-            .withSSLConfig(sslConfig)
+            .withSSLConfig(sslConfig.buildServerSSLConfig())
             .build();
         Properties nodeProperties = new Properties();
         nodeProperties.setProperty("solrconfig", solrconfigFile);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index f76432a..71daf96 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -84,7 +84,6 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.embedded.SSLConfig;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
@@ -287,7 +286,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     sslConfig = buildSSLConfig();
     // based on randomized SSL config, set SchemaRegistryProvider appropriately
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
-    Http2SolrClient.setSslContextFactory(sslConfig.buildSslContextFactory());
+    Http2SolrClient.setDefaultSSLConfig(sslConfig.buildClientSSLConfig());
     if(isSSLMode()) {
       // SolrCloud tests should usually clear this
       System.setProperty("urlScheme", "https");
@@ -484,7 +483,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   }
 
   protected static JettyConfig buildJettyConfig(String context) {
-    return JettyConfig.builder().setContext(context).withSSLConfig(sslConfig).build();
+    return JettyConfig.builder().setContext(context).withSSLConfig(sslConfig.buildServerSSLConfig()).build();
   }
   
   protected static String buildUrl(final int port, final String context) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index fc645c7..901772c 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -509,7 +509,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         .stopAtShutdown(false)
         .withServlets(getExtraServlets())
         .withFilters(getExtraRequestFilters())
-        .withSSLConfig(sslConfig)
+        .withSSLConfig(sslConfig.buildServerSSLConfig())
         .build();
 
     Properties props = new Properties();
@@ -540,7 +540,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         .stopAtShutdown(false)
         .withServlets(getExtraServlets())
         .withFilters(getExtraRequestFilters())
-        .withSSLConfig(sslConfig)
+        .withSSLConfig(sslConfig.buildServerSSLConfig())
         .build();
 
     Properties props = new Properties();
@@ -579,7 +579,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         .stopAtShutdown(false)
         .withServlets(getExtraServlets())
         .withFilters(getExtraRequestFilters())
-        .withSSLConfig(sslConfig)
+        .withSSLConfig(sslConfig.buildServerSSLConfig())
         .build();
 
     Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b378bb5c/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
index 7d3f69b..003463a 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
@@ -44,18 +44,21 @@ import org.eclipse.jetty.util.security.CertificateUtils;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 
 /**
- * An {@link SSLConfig} that supports reading key/trust store information directly from resource 
- * files provided with the Solr test-framework classes
+ * An SSLConfig that provides {@link SSLConfig} and {@link SchemaRegistryProvider} for both clients and servers
+ * that supports reading key/trust store information directly from resource files provided with the
+ * Solr test-framework classes
  */
-public class SSLTestConfig extends SSLConfig {
+public class SSLTestConfig {
 
   private static final String TEST_KEYSTORE_BOGUSHOST_RESOURCE = "SSLTestConfig.hostname-and-ip-missmatch.keystore";
   private static final String TEST_KEYSTORE_LOCALHOST_RESOURCE = "SSLTestConfig.testing.keystore";
-  private static final String TEST_KEYSTORE_PASSWORD = "secret";
+  private static final String TEST_PASSWORD = "secret";
 
   private final boolean checkPeerName;
   private final Resource keyStore;
   private final Resource trustStore;
+  private boolean useSsl;
+  private boolean clientAuth;
   
   /** Creates an SSLTestConfig that does not use SSL or client authentication */
   public SSLTestConfig() {
@@ -97,7 +100,8 @@ public class SSLTestConfig extends SSLConfig {
    * @see HttpClientUtil#SYS_PROP_CHECK_PEER_NAME
    */
   public SSLTestConfig(boolean useSSL, boolean clientAuth, boolean checkPeerName) {
-    super(useSSL, clientAuth, null, TEST_KEYSTORE_PASSWORD, null, TEST_KEYSTORE_PASSWORD);
+    this.useSsl = useSSL;
+    this.clientAuth = clientAuth;
     this.checkPeerName = checkPeerName;
     
     final String resourceName = checkPeerName
@@ -113,22 +117,14 @@ public class SSLTestConfig extends SSLConfig {
   public boolean getCheckPeerName() {
     return checkPeerName;
   }
-  
-  /** 
-   * NOTE: This method is meaningless in SSLTestConfig.
-   * @return null
-   */
-  @Override
-  public String getKeyStore() {
-    return null;
+
+  /** All other settings on this object are ignored unless this is true */
+  public boolean isSSLMode() {
+    return useSsl;
   }
-  /** 
-   * NOTE: This method is meaningless in SSLTestConfig.
-   * @return null
-   */
-  @Override
-  public String getTrustStore() {
-    return null;
+
+  public boolean isClientAuthMode() {
+    return clientAuth;
   }
   
   /**
@@ -148,101 +144,95 @@ public class SSLTestConfig extends SSLConfig {
     }
   }
 
-  public SslContextFactory buildSslContextFactory() {
-    if (!isSSLMode()) {
-      return null;
-    }
-    SslContextFactory sslContextFactory = new SslContextFactory(false);
-    try {
-      SSLContext sslContext = buildClientSSLContext();
-      sslContextFactory.setSslContext(sslContext);
-      sslContextFactory.setNeedClientAuth(checkPeerName);
-    } catch (KeyManagementException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException e) {
-      throw new IllegalStateException("Unable to setup https scheme for HTTPClient to test SSL.", e);
-    }
-    return sslContextFactory;
-  }
-  
   /**
-   * Builds a new SSLContext for HTTP <b>clients</b> to use when communicating with servers which have 
-   * been configured based on the settings of this object.  
+   * Builds a new SSLContext for HTTP <b>clients</b> to use when communicating with servers which have
+   * been configured based on the settings of this object.
    *
-   * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking 
-   * due to lack of entropy, also explicitly allows the use of self-signed 
+   * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking
+   * due to lack of entropy, also explicitly allows the use of self-signed
    * certificates (since that's what is almost always used during testing).
    */
-  public SSLContext buildClientSSLContext() throws KeyManagementException, 
+  public SSLContext buildClientSSLContext() throws KeyManagementException,
     UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException {
 
     assert isSSLMode();
-    
+
     SSLContextBuilder builder = SSLContexts.custom();
     builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
-    
+
     // NOTE: KeyStore & TrustStore are swapped because they are from configured from server perspective...
     // we are a client - our keystore contains the keys the server trusts, and vice versa
-    builder.loadTrustMaterial(buildKeyStore(keyStore, getKeyStorePassword()), new TrustSelfSignedStrategy()).build();
+    builder.loadTrustMaterial(buildKeyStore(keyStore, TEST_PASSWORD), new TrustSelfSignedStrategy()).build();
 
     if (isClientAuthMode()) {
-      builder.loadKeyMaterial(buildKeyStore(trustStore, getTrustStorePassword()), getTrustStorePassword().toCharArray());
-      
+      builder.loadKeyMaterial(buildKeyStore(trustStore, TEST_PASSWORD), TEST_PASSWORD.toCharArray());
     }
 
     return builder.build();
   }
-  
-  /**
-   * Builds a new SSLContext for jetty servers which have been configured based on the settings of 
-   * this object.
-   *
-   * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking 
-   * due to lack of entropy, also explicitly allows the use of self-signed 
-   * certificates (since that's what is almost always used during testing).
-   * almost always used during testing). 
-   */
-  public SSLContext buildServerSSLContext() throws KeyManagementException, 
-    UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException {
 
-    assert isSSLMode();
-    
-    SSLContextBuilder builder = SSLContexts.custom();
-    builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
-
-    builder.loadKeyMaterial(buildKeyStore(keyStore, getKeyStorePassword()), getKeyStorePassword().toCharArray());
-
-    if (isClientAuthMode()) {
-      builder.loadTrustMaterial(buildKeyStore(trustStore, getTrustStorePassword()), new TrustSelfSignedStrategy()).build();
-      
+  public SSLConfig buildClientSSLConfig() {
+    if (!isSSLMode()) {
+      return null;
     }
 
-    return builder.build();
+    return new SSLConfig(isSSLMode(), isClientAuthMode(), null, null, null, null) {
+      @Override
+      public SslContextFactory createContextFactory() {
+        SslContextFactory factory = new SslContextFactory(false);
+        try {
+          factory.setSslContext(buildClientSSLContext());
+          factory.setNeedClientAuth(checkPeerName);
+        } catch (KeyManagementException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException e) {
+          throw new IllegalStateException("Unable to setup https scheme for HTTPClient to test SSL.", e);
+        }
+        return factory;
+      }
+    };
   }
 
   /**
-   * Returns an SslContextFactory using {@link #buildServerSSLContext} if SSL should be used, else returns null.
+   * Builds a new SSLContext for jetty servers which have been configured based on the settings of
+   * this object.
+   *
+   * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking
+   * due to lack of entropy, also explicitly allows the use of self-signed
+   * certificates (since that's what is almost always used during testing).
+   * almost always used during testing).
    */
-  @Override
-  public SslContextFactory createContextFactory() {
+  public SSLConfig buildServerSSLConfig() {
     if (!isSSLMode()) {
       return null;
     }
-    // else...
 
-    
-    SslContextFactory factory = new SslContextFactory(false);
-    try {
-      factory.setSslContext(buildServerSSLContext());
-    } catch (Exception e) { 
-      throw new RuntimeException("ssl context init failure: " + e.getMessage(), e); 
-    }
-    factory.setNeedClientAuth(isClientAuthMode());
-    return factory;
+    return new SSLConfig(isSSLMode(), isClientAuthMode(), null, null, null, null) {
+      @Override
+      public SslContextFactory createContextFactory() {
+        SslContextFactory factory = new SslContextFactory(false);
+        try {
+          SSLContextBuilder builder = SSLContexts.custom();
+          builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
+
+          builder.loadKeyMaterial(buildKeyStore(keyStore, TEST_PASSWORD), TEST_PASSWORD.toCharArray());
+
+          if (isClientAuthMode()) {
+            builder.loadTrustMaterial(buildKeyStore(trustStore, TEST_PASSWORD), new TrustSelfSignedStrategy()).build();
+
+          }
+          factory.setSslContext(builder.build());
+        } catch (Exception e) {
+          throw new RuntimeException("ssl context init failure: " + e.getMessage(), e);
+        }
+        factory.setNeedClientAuth(isClientAuthMode());
+        return factory;
+      }
+    };
   }
-  
+
   /**
    * Constructs a KeyStore using the specified filename and password
    */
-  protected static KeyStore buildKeyStore(Resource resource, String password) {
+  private static KeyStore buildKeyStore(Resource resource, String password) {
     try {
       return CertificateUtils.getKeyStore(resource, "JKS", null, password);
     } catch (Exception ex) {
@@ -294,23 +284,6 @@ public class SSLTestConfig extends SSLConfig {
         .register("http", PlainConnectionSocketFactory.getSocketFactory()).build();
     }
   };
-  
-  public static boolean toBooleanDefaultIfNull(Boolean bool, boolean valueIfNull) {
-    if (bool == null) {
-      return valueIfNull;
-    }
-    return bool.booleanValue() ? true : false;
-  }
-  
-  public static Boolean toBooleanObject(String str) {
-    if ("true".equalsIgnoreCase(str)) {
-      return Boolean.TRUE;
-    } else if ("false".equalsIgnoreCase(str)) {
-      return Boolean.FALSE;
-    }
-    // no match
-    return null;
-  }
 
   /**
    * A mocked up instance of SecureRandom that just uses {@link Random} under the covers.