You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2018/04/22 22:01:06 UTC

[2/2] lucene-solr:branch_7x: SOLR-9304: Fix Solr's HTTP handling to respect '-Dsolr.ssl.checkPeerName=false' aka SOLR_SSL_CHECK_PEER_NAME

SOLR-9304: Fix Solr's HTTP handling to respect '-Dsolr.ssl.checkPeerName=false' aka SOLR_SSL_CHECK_PEER_NAME

(cherry picked from commit 4e0e8e979b66abdf0778fc0ea86ae5ef5d8f2f91)


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

Branch: refs/heads/branch_7x
Commit: 2dffe4ed59e6bf7230b9d1a363bc208182afa5a6
Parents: b16b380
Author: Chris Hostetter <ho...@apache.org>
Authored: Sun Apr 22 13:38:37 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Sun Apr 22 13:39:30 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   5 +-
 solr/bin/solr                                   |   4 +
 solr/bin/solr.cmd                               |   3 +
 solr/bin/solr.in.cmd                            |  12 ++-
 solr/bin/solr.in.sh                             |  16 ++-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java |  59 ++++++++++
 solr/solr-ref-guide/src/enabling-ssl.adoc       |  21 +++-
 .../solr/client/solrj/impl/HttpClientUtil.java  |  59 +++++++++-
 .../client/solrj/impl/HttpClientUtilTest.java   | 108 +++++++++++++++++++
 .../org/apache/solr/util/SSLTestConfig.java     |  89 ++++++++-------
 ...estConfig.hostname-and-ip-missmatch.keystore | Bin 0 -> 2246 bytes
 .../resources/SSLTestConfig.testing.keystore    | Bin 2208 -> 2207 bytes
 .../src/resources/create-keystores.sh           |  37 +++++++
 13 files changed, 362 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e791e26..f792b53 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -156,7 +156,10 @@ Bug Fixes
 * SOLR-6286: TestReplicationHandler.doTestReplicateAfterCoreReload(): stop checking for identical
   commits before/after master core reload; and make non-nightly mode test 10 docs instead of 0.
   (shalin, hossman, Mark Miller, Steve Rowe)
- 
+
+* SOLR-9304: Fix Solr's HTTP handling to respect '-Dsolr.ssl.checkPeerName=false' aka SOLR_SSL_CHECK_PEER_NAME
+  (Shawn Heisey, Carlton Findley, Robby Pond, hossman)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index 3cda782..68d1140 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -224,6 +224,10 @@ if [ "$SOLR_SSL_ENABLED" == "true" ]; then
     fi
   fi
 
+  if [ -n "$SOLR_SSL_CHECK_PEER_NAME" ]; then
+    SOLR_SSL_OPTS+=" -Dsolr.ssl.checkPeerName=$SOLR_SSL_CHECK_PEER_NAME"
+  fi
+
   if [ -n "$SOLR_SSL_CLIENT_TRUST_STORE" ]; then
     SOLR_SSL_OPTS+=" -Djavax.net.ssl.trustStore=$SOLR_SSL_CLIENT_TRUST_STORE"
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/bin/solr.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index e9f6c45..7235a4c 100644
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -111,6 +111,9 @@ IF "%SOLR_SSL_ENABLED%"=="true" (
      set "SOLR_SSL_OPTS=!SOLR_SSL_OPTS! -Djavax.net.ssl.trustStoreType=%SOLR_SSL_TRUST_STORE_TYPE%"
     )
   )
+  IF DEFINED SOLR_SSL_CHECK_PEER_NAME (
+   set "SOLR_SSL_OPTS=!SOLR_SSL_OPTS! -Dsolr.ssl.checkPeerName=%SOLR_SSL_CHECK_PEER_NAME%"
+  )
 ) ELSE (
   set SOLR_SSL_OPTS=
 )

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/bin/solr.in.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.in.cmd b/solr/bin/solr.in.cmd
index a1771ad..86ad708 100644
--- a/solr/bin/solr.in.cmd
+++ b/solr/bin/solr.in.cmd
@@ -103,20 +103,26 @@ REM Uncomment to set SSL-related system properties
 REM Be sure to update the paths to the correct keystore for your environment
 REM set SOLR_SSL_KEY_STORE=etc/solr-ssl.keystore.jks
 REM set SOLR_SSL_KEY_STORE_PASSWORD=secret
-REM set SOLR_SSL_KEY_STORE_TYPE=JKS
 REM set SOLR_SSL_TRUST_STORE=etc/solr-ssl.keystore.jks
 REM set SOLR_SSL_TRUST_STORE_PASSWORD=secret
-REM set SOLR_SSL_TRUST_STORE_TYPE=JKS
+REM Require clients to authenticate
 REM set SOLR_SSL_NEED_CLIENT_AUTH=false
+REM Enable clients to authenticate (but not require)
 REM set SOLR_SSL_WANT_CLIENT_AUTH=false
+REM SSL Certificates contain host/ip "peer name" information that is validated by default. Setting
+REM this to false can be useful to disable these checks when re-using a certificate on many hosts
+REM set SOLR_SSL_CHECK_PEER_NAME=true
+REM Override Key/Trust Store types if necessary
+REM set SOLR_SSL_KEY_STORE_TYPE=JKS
+REM set SOLR_SSL_TRUST_STORE_TYPE=JKS
 
 REM Uncomment if you want to override previously defined SSL values for HTTP client
 REM otherwise keep them commented and the above values will automatically be set for HTTP clients
 REM set SOLR_SSL_CLIENT_KEY_STORE=
 REM set SOLR_SSL_CLIENT_KEY_STORE_PASSWORD=
-REM set SOLR_SSL_CLIENT_KEY_STORE_TYPE=
 REM set SOLR_SSL_CLIENT_TRUST_STORE=
 REM set SOLR_SSL_CLIENT_TRUST_STORE_PASSWORD=
+REM set SOLR_SSL_CLIENT_KEY_STORE_TYPE=
 REM set SOLR_SSL_CLIENT_TRUST_STORE_TYPE=
 
 REM Sets path of Hadoop credential provider (hadoop.security.credential.provider.path property) and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/bin/solr.in.sh
----------------------------------------------------------------------
diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh
index 7cf6a84..9b15bea 100644
--- a/solr/bin/solr.in.sh
+++ b/solr/bin/solr.in.sh
@@ -118,22 +118,28 @@
 #SOLR_SSL_ENABLED=true
 # Uncomment to set SSL-related system properties
 # Be sure to update the paths to the correct keystore for your environment
-#SOLR_SSL_KEY_STORE=/home/shalin/work/oss/shalin-lusolr/solr/server/etc/solr-ssl.keystore.jks
+#SOLR_SSL_KEY_STORE=etc/solr-ssl.keystore.jks
 #SOLR_SSL_KEY_STORE_PASSWORD=secret
-#SOLR_SSL_KEY_STORE_TYPE=JKS
-#SOLR_SSL_TRUST_STORE=/home/shalin/work/oss/shalin-lusolr/solr/server/etc/solr-ssl.keystore.jks
+#SOLR_SSL_TRUST_STORE=etc/solr-ssl.keystore.jks
 #SOLR_SSL_TRUST_STORE_PASSWORD=secret
-#SOLR_SSL_TRUST_STORE_TYPE=JKS
+# Require clients to authenticate
 #SOLR_SSL_NEED_CLIENT_AUTH=false
+# Enable clients to authenticate (but not require)
 #SOLR_SSL_WANT_CLIENT_AUTH=false
+# SSL Certificates contain host/ip "peer name" information that is validated by default. Setting
+# this to false can be useful to disable these checks when re-using a certificate on many hosts
+#SOLR_SSL_CHECK_PEER_NAME=true
+# Override Key/Trust Store types if necessary
+#SOLR_SSL_KEY_STORE_TYPE=JKS
+#SOLR_SSL_TRUST_STORE_TYPE=JKS
 
 # Uncomment if you want to override previously defined SSL values for HTTP client
 # otherwise keep them commented and the above values will automatically be set for HTTP clients
 #SOLR_SSL_CLIENT_KEY_STORE=
 #SOLR_SSL_CLIENT_KEY_STORE_PASSWORD=
-#SOLR_SSL_CLIENT_KEY_STORE_TYPE=
 #SOLR_SSL_CLIENT_TRUST_STORE=
 #SOLR_SSL_CLIENT_TRUST_STORE_PASSWORD=
+#SOLR_SSL_CLIENT_KEY_STORE_TYPE=
 #SOLR_SSL_CLIENT_TRUST_STORE_TYPE=
 
 # Sets path of Hadoop credential provider (hadoop.security.credential.provider.path property) and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/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 98f952a..7a6606a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
@@ -17,6 +17,7 @@
 package org.apache.solr.cloud;
 
 import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.List;
@@ -32,6 +33,8 @@ import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.TestRuleRestoreSystemProperties;
+
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
@@ -46,6 +49,9 @@ import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.util.SSLTestConfig;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestRule;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,6 +78,10 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
   public static final int NUM_SERVERS = 3;
   public static final String CONF_NAME = MethodHandles.lookup().lookupClass().getName();
   
+  @Rule
+  public TestRule syspropRestore = new TestRuleRestoreSystemProperties
+    (HttpClientUtil.SYS_PROP_CHECK_PEER_NAME);
+  
   @Before
   public void before() {
     // undo the randomization of our super class
@@ -120,6 +130,13 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     checkClusterWithNodeReplacement(sslConfig);
   }
   
+  public void testSslWithCheckPeerName() throws Exception {
+    final SSLTestConfig sslConfig = new SSLTestConfig(true, false, true);
+    HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
+    System.setProperty(ZkStateReader.URL_SCHEME, "https");
+    checkClusterWithNodeReplacement(sslConfig);
+  }
+  
   /**
    * Constructs a cluster with the specified sslConfigs, runs {@link #checkClusterWithCollectionCreations}, 
    * then verifies that if we modify the default SSLContext (mimicing <code>javax.net.ssl.*</code> 
@@ -142,6 +159,8 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
       // our test config doesn't use SSL, and reset HttpClientUtil to it's defaults so it picks up our
       // SSLContext that way.
       SSLContext.setDefault( sslConfig.isSSLMode() ? sslConfig.buildClientSSLContext() : DEFAULT_SSL_CONTEXT);
+      System.setProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME,
+                         Boolean.toString(sslConfig.getCheckPeerName()));
       HttpClientUtil.resetHttpClientBuilder();
       
       // recheck that we can communicate with all the jetty instances in our cluster
@@ -151,6 +170,46 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     }
   }
 
+  /** Sanity check that our test scaffolding for validating SSL peer names fails when it should */
+  public void testSslWithInvalidPeerName() throws Exception {
+    // NOTE: first initialize the cluster w/o peer name checks, which means our server will use
+    // certs with a bogus hostname/ip and clients shouldn't care...
+    final SSLTestConfig sslConfig = new SSLTestConfig(true, false, false);
+    HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());
+    System.setProperty(ZkStateReader.URL_SCHEME, "https");
+    final JettyConfig config = JettyConfig.builder().withSSLConfig(sslConfig).build();
+    final MiniSolrCloudCluster cluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), config);
+    try {
+      checkClusterWithCollectionCreations(cluster, sslConfig);
+      
+      // now initialize a client that still uses the existing SSLContext/Provider, so it will accept
+      // our existing certificate, but *does* care about validating the peer name
+      System.setProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME, "true");
+      HttpClientUtil.resetHttpClientBuilder();
+
+      // and validate we get failures when trying to talk to our cluster...
+      final List<JettySolrRunner> jettys = cluster.getJettySolrRunners();
+      for (JettySolrRunner jetty : jettys) {
+        final String baseURL = jetty.getBaseUrl().toString();
+        // verify new solr clients validate peer name and can't talk to this server
+        Exception ex = expectThrows(SolrServerException.class, () -> {
+            try (HttpSolrClient client = getRandomizedHttpSolrClient(baseURL)) {
+              CoreAdminRequest req = new CoreAdminRequest();
+              req.setAction( CoreAdminAction.STATUS );
+              client.request(req);
+            }
+          });
+        assertTrue("Expected an root cause SSL Exception, got: " + ex.toString(),
+                   ex.getCause() instanceof SSLException);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+
+
+    
+  }
+
   /**
    * General purpose cluster sanity check...
    * <ol>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/solr-ref-guide/src/enabling-ssl.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/enabling-ssl.adoc b/solr/solr-ref-guide/src/enabling-ssl.adoc
index b641bfd..35bc1d8 100644
--- a/solr/solr-ref-guide/src/enabling-ssl.adoc
+++ b/solr/solr-ref-guide/src/enabling-ssl.adoc
@@ -77,6 +77,11 @@ NOTE: If you setup Solr as a service on Linux using the steps outlined in <<taki
 .bin/solr.in.sh example SOLR_SSL_* configuration
 [source,bash]
 ----
+# Enables HTTPS. It is implictly true if you set SOLR_SSL_KEY_STORE. Use this config
+# to enable https module with custom jetty configuration.
+SOLR_SSL_ENABLED=true
+# Uncomment to set SSL-related system properties
+# Be sure to update the paths to the correct keystore for your environment
 SOLR_SSL_KEY_STORE=etc/solr-ssl.keystore.jks
 SOLR_SSL_KEY_STORE_PASSWORD=secret
 SOLR_SSL_TRUST_STORE=etc/solr-ssl.keystore.jks
@@ -85,7 +90,10 @@ SOLR_SSL_TRUST_STORE_PASSWORD=secret
 SOLR_SSL_NEED_CLIENT_AUTH=false
 # Enable clients to authenticate (but not require)
 SOLR_SSL_WANT_CLIENT_AUTH=false
-# Define Key Store type if necessary
+# SSL Certificates contain host/ip "peer name" information that is validated by default. Setting
+# this to false can be useful to disable these checks when re-using a certificate on many hosts
+SOLR_SSL_CHECK_PEER_NAME=true
+# Override Key/Trust Store types if necessary
 SOLR_SSL_KEY_STORE_TYPE=JKS
 SOLR_SSL_TRUST_STORE_TYPE=JKS
 ----
@@ -100,6 +108,11 @@ Similarly, when you start Solr on Windows, the `bin\solr.cmd` script includes th
 .bin\solr.in.cmd example SOLR_SSL_* configuration
 [source,text]
 ----
+REM Enables HTTPS. It is implictly true if you set SOLR_SSL_KEY_STORE. Use this config
+REM to enable https module with custom jetty configuration.
+set SOLR_SSL_ENABLED=true
+REM Uncomment to set SSL-related system properties
+REM Be sure to update the paths to the correct keystore for your environment
 set SOLR_SSL_KEY_STORE=etc/solr-ssl.keystore.jks
 set SOLR_SSL_KEY_STORE_PASSWORD=secret
 set SOLR_SSL_TRUST_STORE=etc/solr-ssl.keystore.jks
@@ -108,6 +121,12 @@ REM Require clients to authenticate
 set SOLR_SSL_NEED_CLIENT_AUTH=false
 REM Enable clients to authenticate (but not require)
 set SOLR_SSL_WANT_CLIENT_AUTH=false
+REM SSL Certificates contain host/ip "peer name" information that is validated by default. Setting
+REM this to false can be useful to disable these checks when re-using a certificate on many hosts
+set SOLR_SSL_CHECK_PEER_NAME=true
+REM Override Key/Trust Store types if necessary
+set SOLR_SSL_KEY_STORE_TYPE=JKS
+set SOLR_SSL_TRUST_STORE_TYPE=JKS
 ----
 
 === Run Single Node Solr using SSL

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
index d064a06..e08f85f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
@@ -47,6 +47,7 @@ import org.apache.http.config.RegistryBuilder;
 import org.apache.http.conn.ConnectionKeepAliveStrategy;
 import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
 import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.http.entity.HttpEntityWrapper;
 import org.apache.http.impl.client.BasicCredentialsProvider;
@@ -56,6 +57,7 @@ import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.protocol.HttpRequestExecutor;
+import org.apache.http.ssl.SSLContexts;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ObjectReleaseTracker;
@@ -93,7 +95,16 @@ public class HttpClientUtil {
   public static final String PROP_BASIC_AUTH_USER = "httpBasicAuthUser";
   // Basic auth password 
   public static final String PROP_BASIC_AUTH_PASS = "httpBasicAuthPassword";
-  
+
+  /**
+   * System property consulted to determine if the default {@link SchemaRegistryProvider} 
+   * will require hostname validation of SSL Certificates.  The default behavior is to enforce 
+   * peer name validation.
+   * <p>
+   * This property will have no effect if {@link #setSchemaRegistryProvider} is used to override
+   * the default {@link SchemaRegistryProvider} 
+   * </p>
+   */
   public static final String SYS_PROP_CHECK_PEER_NAME = "solr.ssl.checkPeerName";
   
   // * NOTE* The following params configure the default request config and this
@@ -181,6 +192,9 @@ public class HttpClientUtil {
     httpClientBuilder = newHttpClientBuilder;
   }
 
+  /**
+   * @see #SYS_PROP_CHECK_PEER_NAME
+   */
   public static void setSchemaRegistryProvider(SchemaRegistryProvider newRegistryProvider) {
     schemaRegistryProvider = newRegistryProvider;
   }
@@ -188,7 +202,10 @@ public class HttpClientUtil {
   public static SolrHttpClientBuilder getHttpClientBuilder() {
     return httpClientBuilder;
   }
-  
+
+  /**
+   * @see #SYS_PROP_CHECK_PEER_NAME
+   */
   public static SchemaRegistryProvider getSchemaRegisteryProvider() {
     return schemaRegistryProvider;
   }
@@ -205,9 +222,22 @@ public class HttpClientUtil {
       // except that we explicitly use SSLConnectionSocketFactory.getSystemSocketFactory()
       // to pick up the system level default SSLContext (where javax.net.ssl.* properties
       // related to keystore & truststore are specified)
-      RegistryBuilder<ConnectionSocketFactory> builder = RegistryBuilder.<ConnectionSocketFactory>create();
+      RegistryBuilder<ConnectionSocketFactory> builder = RegistryBuilder.<ConnectionSocketFactory> create();
       builder.register("http", PlainConnectionSocketFactory.getSocketFactory());
-      builder.register("https", SSLConnectionSocketFactory.getSystemSocketFactory());
+
+      // logic to turn off peer host check
+      SSLConnectionSocketFactory sslConnectionSocketFactory = null;
+      boolean sslCheckPeerName = toBooleanDefaultIfNull(
+          toBooleanObject(System.getProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME)), true);
+      if (sslCheckPeerName) {
+        sslConnectionSocketFactory = SSLConnectionSocketFactory.getSystemSocketFactory();
+      } else {
+        sslConnectionSocketFactory = new SSLConnectionSocketFactory(SSLContexts.createSystemDefault(),
+                                                                    NoopHostnameVerifier.INSTANCE);
+        logger.debug(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME + "is false, hostname checks disabled.");
+      }
+      builder.register("https", sslConnectionSocketFactory);
+
       return builder.build();
     }
   }
@@ -459,5 +489,26 @@ public class HttpClientUtil {
     cookiePolicy = policyName;
   }
 
+  /**
+   * @lucene.internal
+   */
+  static boolean toBooleanDefaultIfNull(Boolean bool, boolean valueIfNull) {
+    if (bool == null) {
+      return valueIfNull;
+    }
+    return bool.booleanValue() ? true : false;
+  }
 
+  /**
+   * @lucene.internal
+   */
+  static Boolean toBooleanObject(String str) {
+    if ("true".equalsIgnoreCase(str)) {
+      return Boolean.TRUE;
+    } else if ("false".equalsIgnoreCase(str)) {
+      return Boolean.FALSE;
+    }
+    // no match
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpClientUtilTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpClientUtilTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpClientUtilTest.java
new file mode 100644
index 0000000..ce2f8b7
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpClientUtilTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.impl;
+
+import javax.net.ssl.HostnameVerifier;
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.impl.HttpClientUtil.SchemaRegistryProvider;
+
+import org.apache.commons.lang.reflect.FieldUtils;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.ssl.DefaultHostnameVerifier;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestRuleRestoreSystemProperties;
+
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.rules.TestRule;
+import org.junit.Test;
+
+public class HttpClientUtilTest extends LuceneTestCase {
+
+  @Rule
+  public TestRule syspropRestore = new TestRuleRestoreSystemProperties
+    (HttpClientUtil.SYS_PROP_CHECK_PEER_NAME);
+  
+  @After
+  public void resetHttpClientBuilder() {
+    HttpClientUtil.resetHttpClientBuilder();
+  }
+    
+  public void testSSLSystemProperties() throws IOException {
+    
+    assertNotNull("HTTPS scheme could not be created using system defaults",
+                  HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry().lookup("https"));
+
+    assertSSLHostnameVerifier(DefaultHostnameVerifier.class, HttpClientUtil.getSchemaRegisteryProvider());
+
+    System.setProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME, "true");
+    resetHttpClientBuilder();
+    assertSSLHostnameVerifier(DefaultHostnameVerifier.class, HttpClientUtil.getSchemaRegisteryProvider());
+
+    System.setProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME, "");
+    resetHttpClientBuilder();
+    assertSSLHostnameVerifier(DefaultHostnameVerifier.class, HttpClientUtil.getSchemaRegisteryProvider());
+    
+    System.setProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME, "false");
+    resetHttpClientBuilder();
+    assertSSLHostnameVerifier(NoopHostnameVerifier.class, HttpClientUtil.getSchemaRegisteryProvider());
+  }
+
+  private void assertSSLHostnameVerifier(Class<? extends HostnameVerifier> expected,
+                                         SchemaRegistryProvider provider) {
+    ConnectionSocketFactory socketFactory = provider.getSchemaRegistry().lookup("https");
+    assertNotNull("unable to lookup https", socketFactory);
+    assertTrue("socketFactory is not an SSLConnectionSocketFactory: " + socketFactory.getClass(),
+               socketFactory instanceof SSLConnectionSocketFactory);
+    SSLConnectionSocketFactory sslSocketFactory = (SSLConnectionSocketFactory) socketFactory;
+    try {
+      Object hostnameVerifier = FieldUtils.readField(sslSocketFactory, "hostnameVerifier", true);
+      assertNotNull("sslSocketFactory has null hostnameVerifier", hostnameVerifier);
+      assertEquals("sslSocketFactory does not have expected hostnameVerifier impl",
+                   expected, hostnameVerifier.getClass());
+    } catch (IllegalAccessException e) {
+      throw new AssertionError("Unexpected access error reading hostnameVerifier field", e);
+    }
+  }
+  
+  @Test
+  public void testToBooleanDefaultIfNull() throws Exception {
+    assertFalse(HttpClientUtil.toBooleanDefaultIfNull(Boolean.FALSE, true));
+    assertTrue(HttpClientUtil.toBooleanDefaultIfNull(Boolean.TRUE, false));
+    assertFalse(HttpClientUtil.toBooleanDefaultIfNull(null, false));
+    assertTrue(HttpClientUtil.toBooleanDefaultIfNull(null, true));
+  }
+
+  @Test
+  public void testToBooleanObject() throws Exception {
+    assertEquals(Boolean.TRUE, HttpClientUtil.toBooleanObject("true"));
+    assertEquals(Boolean.TRUE, HttpClientUtil.toBooleanObject("TRUE"));
+    assertEquals(Boolean.TRUE, HttpClientUtil.toBooleanObject("tRuE"));
+
+    assertEquals(Boolean.FALSE, HttpClientUtil.toBooleanObject("false"));
+    assertEquals(Boolean.FALSE, HttpClientUtil.toBooleanObject("FALSE"));
+    assertEquals(Boolean.FALSE, HttpClientUtil.toBooleanObject("fALSE"));
+
+    assertEquals(null, HttpClientUtil.toBooleanObject("t"));
+    assertEquals(null, HttpClientUtil.toBooleanObject("f"));
+    assertEquals(null, HttpClientUtil.toBooleanObject("foo"));
+    assertEquals(null, HttpClientUtil.toBooleanObject(null));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/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 8268fcd..3b03f6e 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
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.util;
 
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
 import java.security.KeyManagementException;
 import java.security.KeyStore;
 import java.security.KeyStoreException;
@@ -27,15 +25,17 @@ import java.security.SecureRandomSpi;
 import java.security.UnrecoverableKeyException;
 import java.util.Random;
 
+import javax.net.ssl.SSLContext;
+
 import org.apache.http.config.Registry;
 import org.apache.http.config.RegistryBuilder;
 import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
 import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
-import org.apache.http.conn.ssl.SSLContextBuilder;
-import org.apache.http.conn.ssl.SSLContexts;
-import org.apache.http.conn.ssl.SSLSocketFactory;
 import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
+import org.apache.http.ssl.SSLContextBuilder;
+import org.apache.http.ssl.SSLContexts;
 import org.apache.solr.client.solrj.embedded.SSLConfig;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpClientUtil.SchemaRegistryProvider;
@@ -49,9 +49,11 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
  */
 public class SSLTestConfig extends SSLConfig {
 
-  private static final String TEST_KEYSTORE_RESOURCE = "SSLTestConfig.testing.keystore";
+  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 final boolean checkPeerName;
   private final Resource keyStore;
   private final Resource trustStore;
   
@@ -59,44 +61,59 @@ public class SSLTestConfig extends SSLConfig {
   public SSLTestConfig() {
     this(false, false);
   }
+  
+  /**
+   * Create an SSLTestConfig based on a few caller specified options, 
+   * implicitly assuming <code>checkPeerName=false</code>.  
+   * <p>
+   * As needed, keystore/truststore information will be pulled from a hardcoded resource 
+   * file provided by the solr test-framework
+   * </p>
+   *
+   * @param useSSL - whether SSL should be required.
+   * @param clientAuth - whether client authentication should be required.
+   */
+  public SSLTestConfig(boolean useSSL, boolean clientAuth) {
+    this(useSSL, clientAuth, false);
+  }
 
-  /** 
+  // NOTE: if any javadocs below change, update create-keystores.sh
+  /**
    * Create an SSLTestConfig based on a few caller specified options.  As needed, 
-   * keystore/truststore information will be pulled from a hardocded resource file provided 
-   * by the solr test-framework.
+   * keystore/truststore information will be pulled from a hardcoded resource files provided 
+   * by the solr test-framework based on the value of <code>checkPeerName</code>:
+   * <ul>
+   * <li><code>true</code> - A keystore resource file will be used that specifies 
+   *     a CN of <code>localhost</code> and a SAN IP of <code>127.0.0.1</code>, to 
+   *     ensure that all connections should be valid regardless of what machine runs the tests.</li> 
+   * <li><code>false</code> - A keystore resource file will be used that specifies 
+   *     a bogus hostname in the CN and reserved IP as the SAN, since no (valid) tests using this 
+   *     SSLTestConfig should care what CN/SAN are.</li> 
+   * </ul>
    *
-   * @param useSSL - wether SSL should be required.
+   * @param useSSL - whether SSL should be required.
    * @param clientAuth - whether client authentication should be required.
+   * @param checkPeerName - whether the client should validate the 'peer name' of the SSL Certificate (and which testing Cert should be used)
+   * @see HttpClientUtil#SYS_PROP_CHECK_PEER_NAME
    */
-  public SSLTestConfig(boolean useSSL, boolean clientAuth) {
+  public SSLTestConfig(boolean useSSL, boolean clientAuth, boolean checkPeerName) {
     super(useSSL, clientAuth, null, TEST_KEYSTORE_PASSWORD, null, TEST_KEYSTORE_PASSWORD);
-    trustStore = keyStore = Resource.newClassPathResource(TEST_KEYSTORE_RESOURCE);
+    this.checkPeerName = checkPeerName;
+    
+    final String resourceName = checkPeerName
+      ? TEST_KEYSTORE_LOCALHOST_RESOURCE : TEST_KEYSTORE_BOGUSHOST_RESOURCE;
+    trustStore = keyStore = Resource.newClassPathResource(resourceName);
     if (null == keyStore || ! keyStore.exists() ) {
       throw new IllegalStateException("Unable to locate keystore resource file in classpath: "
-                                      + TEST_KEYSTORE_RESOURCE);
+                                      + resourceName);
     }
   }
 
-  /**
-   * Helper utility for building resources from arbitrary user input paths/urls
-   * if input is null, returns null; otherwise attempts to build Resource and verifies that Resource exists.
-   */
-  private static final Resource tryNewResource(String userInput, String type) {
-    if (null == userInput) {
-      return null;
-    }
-    Resource result;
-    try {
-      result = Resource.newResource(userInput);
-    } catch (IOException e) {
-      throw new IllegalArgumentException("Can't build " + type + " Resource: " + e.getMessage(), e);
-    }
-    if (! result.exists()) {
-      throw new IllegalArgumentException(type + " Resource does not exist " + result.getName());
-    }
-    return result;
+  /** If true, then servers hostname/ip should be validated against the SSL Cert metadata */
+  public boolean getCheckPeerName() {
+    return checkPeerName;
   }
-
+  
   /** 
    * NOTE: This method is meaningless in SSLTestConfig.
    * @return null
@@ -175,7 +192,7 @@ public class SSLTestConfig extends SSLConfig {
     
     SSLContextBuilder builder = SSLContexts.custom();
     builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
-    
+
     builder.loadKeyMaterial(buildKeyStore(keyStore, getKeyStorePassword()), getKeyStorePassword().toCharArray());
 
     if (isClientAuthMode()) {
@@ -229,11 +246,9 @@ public class SSLTestConfig extends SSLConfig {
     }
     SSLConnectionSocketFactory sslConnectionFactory;
     try {
-      boolean sslCheckPeerName = toBooleanDefaultIfNull(toBooleanObject(System.getProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME)), true);
       SSLContext sslContext = buildClientSSLContext();
-      if (sslCheckPeerName == false) {
-        sslConnectionFactory = new SSLConnectionSocketFactory
-          (sslContext, SSLSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER);
+      if (checkPeerName == false) {
+        sslConnectionFactory = new SSLConnectionSocketFactory(sslContext, NoopHostnameVerifier.INSTANCE);
       } else {
         sslConnectionFactory = new SSLConnectionSocketFactory(sslContext);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/test-framework/src/resources/SSLTestConfig.hostname-and-ip-missmatch.keystore
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/resources/SSLTestConfig.hostname-and-ip-missmatch.keystore b/solr/test-framework/src/resources/SSLTestConfig.hostname-and-ip-missmatch.keystore
new file mode 100644
index 0000000..691a3be
Binary files /dev/null and b/solr/test-framework/src/resources/SSLTestConfig.hostname-and-ip-missmatch.keystore differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/test-framework/src/resources/SSLTestConfig.testing.keystore
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/resources/SSLTestConfig.testing.keystore b/solr/test-framework/src/resources/SSLTestConfig.testing.keystore
index bcc6ec0..4fdb494 100644
Binary files a/solr/test-framework/src/resources/SSLTestConfig.testing.keystore and b/solr/test-framework/src/resources/SSLTestConfig.testing.keystore differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2dffe4ed/solr/test-framework/src/resources/create-keystores.sh
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/resources/create-keystores.sh b/solr/test-framework/src/resources/create-keystores.sh
new file mode 100755
index 0000000..0b43f28
--- /dev/null
+++ b/solr/test-framework/src/resources/create-keystores.sh
@@ -0,0 +1,37 @@
+#!/bin/bash -ex
+
+# 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.
+
+############
+ 
+# This script shows how the keystore files used for solr tests were generated.
+#
+# Running this script should only be necessary if the keystore files need to be
+# replaced, which shouldn't be required until sometime around the year 4751.
+
+# NOTE: if anything below changes, sanity check SSLTestConfig constructor javadocs
+
+echo "### remove old keystores"
+rm -f SSLTestConfig.testing.keystore SSLTestConfig.hostname-and-ip-missmatch.keystore
+
+echo "### create 'localhost' keystore and keys"
+keytool -keystore SSLTestConfig.testing.keystore -storepass "secret" -alias solrtest -keypass "secret" -genkey -keyalg RSA -dname "cn=localhost, ou=SolrTest, o=lucene.apache.org, c=US" -ext "san=dns:localhost,ip:127.0.0.1" -validity 999999
+
+# See https://tools.ietf.org/html/rfc5737
+echo "### create 'Bogus Host' keystore and keys"
+keytool -keystore SSLTestConfig.hostname-and-ip-missmatch.keystore -storepass "secret" -alias solrtest -keypass "secret" -genkey -keyalg RSA -dname "cn=bogus.hostname.tld, ou=SolrTest, o=lucene.apache.org, c=US" -ext "san=dns:bogus.hostname.tld,ip:192.0.2.0" -validity 999999
+
+