You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by sa...@apache.org on 2018/08/29 18:29:38 UTC

[geode] branch feature/GEODE-5338 updated: made hostname validation default to true

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

sai_boorlagadda pushed a commit to branch feature/GEODE-5338
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/feature/GEODE-5338 by this push:
     new aa4b2e5  made hostname validation default to true
aa4b2e5 is described below

commit aa4b2e58ca1740627c5bdfbaea09e1b7a7cbcb16
Author: Sai Boorlagadda <sb...@pivotal.io>
AuthorDate: Wed Aug 29 11:28:48 2018 -0700

    made hostname validation default to true
---
 .../internal/CustomSSLProviderDistributedTest.java | 274 +++++++++++++++------
 .../internal/provider/CustomKeyManagerFactory.java |  24 +-
 .../client/internal/provider/CustomProvider.java   |  53 ----
 .../provider/CustomTrustManagerFactory.java        |  24 +-
 .../tcpserver/TCPClientSSLIntegrationTest.java     |  40 ++-
 .../internal/AbstractDistributionConfig.java       |   2 +-
 .../distributed/internal/DistributionConfig.java   |   4 +-
 .../apache/geode/internal/net/SocketCreator.java   |   7 +
 8 files changed, 257 insertions(+), 171 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java
index 4c2748d..a455ebc 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java
@@ -15,98 +15,66 @@
 
 package org.apache.geode.cache.client.internal;
 
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_CIPHERS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_PROTOCOLS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_REQUIRE_AUTHENTICATION;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_USE_DEFAULT_SSLCONTEXT;
+import static org.apache.geode.security.SecurableCommunicationChannels.ALL;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
 
-import java.security.Security;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
 import java.util.Properties;
 
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
+import javax.net.ssl.SSLContext;
+
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
-import org.apache.geode.cache.client.internal.provider.CustomProvider;
-import org.apache.geode.security.SecurableCommunicationChannels;
-import org.apache.geode.test.dunit.SerializableConsumerIF;
-import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.cache.client.NoAvailableServersException;
+import org.apache.geode.cache.client.internal.provider.CustomKeyManagerFactory;
+import org.apache.geode.cache.client.internal.provider.CustomTrustManagerFactory;
+import org.apache.geode.cache.ssl.CertStores;
+import org.apache.geode.cache.ssl.TestSSLUtils.CertificateBuilder;
+import org.apache.geode.distributed.internal.tcpserver.LocatorCancelException;
+import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.categories.ClientServerTest;
-import org.apache.geode.test.junit.rules.GfshCommandRule;
-import org.apache.geode.util.test.TestUtil;
 
 @Category({ClientServerTest.class})
 public class CustomSSLProviderDistributedTest {
   private static MemberVM locator;
   private static MemberVM server;
-  private static ClientVM client;
-
-  private static final String SERVER_KEY_STORE = "cacheserver.keystore";
-  private static final String SERVER_TRUST_STORE = "cacheserver.truststore";
-
-  @ClassRule
-  public static ClusterStartupRule cluster = new ClusterStartupRule();
-
-  @ClassRule
-  public static GfshCommandRule gfsh = new GfshCommandRule();
-
-  private static String serverKeystore =
-      TestUtil.getResourcePath(CustomSSLProviderDistributedTest.class, SERVER_KEY_STORE);
-  private static String serverTruststore =
-      TestUtil.getResourcePath(CustomSSLProviderDistributedTest.class, SERVER_TRUST_STORE);
-
-  private static Properties serverSSLProperties = new Properties() {
-    {
-      setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.ALL);
-      setProperty(SSL_KEYSTORE, serverKeystore);
-      setProperty(SSL_KEYSTORE_PASSWORD, "password");
-      setProperty(SSL_KEYSTORE_TYPE, "JKS");
-      setProperty(SSL_TRUSTSTORE, serverTruststore);
-      setProperty(SSL_TRUSTSTORE_PASSWORD, "password");
-      setProperty(SSL_TRUSTSTORE_TYPE, "JKS");
-      setProperty(SSL_CIPHERS, "any");
-      setProperty(SSL_PROTOCOLS, "any");
-      setProperty(SSL_REQUIRE_AUTHENTICATION, String.valueOf("true"));
-    }
-  };
 
-  private static Properties clientSSLProperties = new Properties() {
-    {
-      setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.SERVER);
-      setProperty(SSL_REQUIRE_AUTHENTICATION, String.valueOf("true"));
-      setProperty(SSL_USE_DEFAULT_SSLCONTEXT, String.valueOf("true"));
-    }
-  };
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  private CustomKeyManagerFactory.PKIXFactory keyManagerFactory;
+  private CustomTrustManagerFactory.PKIXFactory trustManagerFactory;
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  private void setupCluster(Properties locatorSSLProps, Properties serverSSLProps) {
     // create a cluster
-    locator = cluster.startLocatorVM(0, serverSSLProperties);
-    server = cluster.startServerVM(1, serverSSLProperties, locator.getPort());
+    locator = cluster.startLocatorVM(0, locatorSSLProps);
+    server = cluster.startServerVM(1, serverSSLProps, locator.getPort());
 
     // create region
     server.invoke(CustomSSLProviderDistributedTest::createServerRegion);
     locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/region", 1);
-
-    // setup client
-    setupClient(server.getPort(), server.getVM().getHost().getHostName());
   }
 
   private static void createServerRegion() {
@@ -116,38 +84,182 @@ public class CustomSSLProviderDistributedTest {
     r.put("serverkey", "servervalue");
   }
 
-  private static void setupClient(int serverPort, String serverHost) throws Exception {
-    SerializableConsumerIF<ClientCacheFactory> clientSetup = cf -> {
-      // add custom provider
-      Security.insertProviderAt(new CustomProvider(), 2);
-      cf.addPoolServer(serverHost, serverPort);
-    };
+  @Test
+  public void hostNameIsValidatedWhenUsingDefaultContext() throws Exception {
+    CertificateBuilder locatorCertificate = new CertificateBuilder()
+        .commonName("locator")
+        // ClusterStartupRule uses 'localhost' as locator host
+        .sanDnsName(InetAddress.getLoopbackAddress().getHostName())
+        .sanDnsName(InetAddress.getLocalHost().getHostName())
+        .sanIpAddress(InetAddress.getLocalHost())
+        .sanIpAddress(InetAddress.getByName("0.0.0.0")); // to pass on windows
 
-    client = cluster.startClientVM(2, clientSSLProperties, clientSetup);
+    CertificateBuilder serverCertificate = new CertificateBuilder()
+        .commonName("server")
+        .sanDnsName(InetAddress.getLocalHost().getHostName())
+        .sanIpAddress(InetAddress.getLocalHost());
 
-    // create a client region
-    client.invoke(CustomSSLProviderDistributedTest::createClientRegion);
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("client");
+
+    validateClientSSLConnection(locatorCertificate, serverCertificate, clientCertificate, true,
+        true, false, null);
   }
 
-  private static void createClientRegion() {
-    ClientRegionFactory<String, String> regionFactory =
-        ClusterStartupRule.getClientCache().createClientRegionFactory(ClientRegionShortcut.PROXY);
-    Region<String, String> region = regionFactory.create("region");
-    assertThat(region).isNotNull();
+  @Test
+  public void clientCanChooseNotToValidateHostName() throws Exception {
+    CertificateBuilder locatorCertificate = new CertificateBuilder()
+        .commonName("locator");
+
+    CertificateBuilder serverCertificate = new CertificateBuilder()
+        .commonName("server");
+
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("client");
+
+    validateClientSSLConnection(locatorCertificate, serverCertificate, clientCertificate, false,
+        false, true, null);
   }
 
   @Test
-  public void testClientSSLConnection() {
-    client.invoke(CustomSSLProviderDistributedTest::doClientRegionTest);
-    server.invoke(CustomSSLProviderDistributedTest::doServerRegionTest);
+  public void clientConnectionFailsIfNoHostNameInLocatorKey() throws Exception {
+    CertificateBuilder locatorCertificate = new CertificateBuilder()
+        .commonName("locator");
+
+    CertificateBuilder serverCertificate = new CertificateBuilder()
+        .commonName("server");
+
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("client");
+
+    validateClientSSLConnection(locatorCertificate, serverCertificate, clientCertificate, false,
+        false, false, LocatorCancelException.class);
   }
 
-  private static void doClientRegionTest() {
-    Region<String, String> region = ClusterStartupRule.getClientCache().getRegion("region");
-    assertThat("servervalue").isEqualTo(region.get("serverkey"));
+  @Test
+  public void clientConnectionFailsWhenWrongHostNameInLocatorKey() throws Exception {
+    CertificateBuilder locatorCertificate = new CertificateBuilder()
+        .commonName("locator")
+        .sanDnsName("example.com");;
 
-    region.put("clientkey", "clientvalue");
-    assertThat("clientvalue").isEqualTo(region.get("clientkey"));
+    CertificateBuilder serverCertificate = new CertificateBuilder()
+        .commonName("server")
+        .sanDnsName("example.com");;
+
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("client");
+
+    validateClientSSLConnection(locatorCertificate, serverCertificate, clientCertificate, false,
+        false,
+        false,
+        LocatorCancelException.class);
+  }
+
+  @Test
+  public void expectConnectionFailureWhenNoHostNameInServerKey() throws Exception {
+    CertificateBuilder locatorCertificateWithSan = new CertificateBuilder()
+        .commonName("locator")
+        .sanDnsName(InetAddress.getLoopbackAddress().getHostName())
+        .sanDnsName(InetAddress.getLocalHost().getHostName())
+        .sanIpAddress(InetAddress.getLocalHost());
+
+    CertificateBuilder serverCertificateWithNoSan = new CertificateBuilder()
+        .commonName("server");
+
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("client");
+
+    validateClientSSLConnection(locatorCertificateWithSan, serverCertificateWithNoSan,
+        clientCertificate, false, false, false,
+        NoAvailableServersException.class);
+  }
+
+  private void validateClientSSLConnection(CertificateBuilder locatorCertificate,
+      CertificateBuilder serverCertificate, CertificateBuilder clientCertificate,
+      boolean enableHostNameVerficationForLocator, boolean enableHostNameVerificationForServer,
+      boolean disableHostNameVerificationForClient,
+      Class expectedExceptionOnClient)
+      throws GeneralSecurityException, IOException {
+
+    CertStores locatorStore = CertStores.locatorStore();
+    locatorStore.withCertificate(locatorCertificate);
+
+    CertStores serverStore = CertStores.serverStore();
+    serverStore.withCertificate(serverCertificate);
+
+    CertStores clientStore = CertStores.clientStore();
+    clientStore.withCertificate(clientCertificate);
+
+    Properties locatorSSLProps = locatorStore
+        .trustSelf()
+        .trust(serverStore.alias(), serverStore.certificate())
+        .trust(clientStore.alias(), clientStore.certificate())
+        .propertiesWith(ALL, false, enableHostNameVerficationForLocator);
+
+    Properties serverSSLProps = serverStore
+        .trustSelf()
+        .trust(locatorStore.alias(), locatorStore.certificate())
+        .trust(clientStore.alias(), clientStore.certificate())
+        .propertiesWith(ALL, true, enableHostNameVerificationForServer);
+
+    // this props is only to create temp keystore and truststore and get paths
+    Properties clientSSLProps = clientStore
+        .trust(locatorStore.alias(), locatorStore.certificate())
+        .trust(serverStore.alias(), serverStore.certificate())
+        .propertiesWith(ALL, true, true);
+
+    setupCluster(locatorSSLProps, serverSSLProps);
+
+    // setup client
+    keyManagerFactory =
+        new CustomKeyManagerFactory.PKIXFactory(clientSSLProps.getProperty(SSL_KEYSTORE));
+    keyManagerFactory.engineInit(null, null);
+
+    trustManagerFactory =
+        new CustomTrustManagerFactory.PKIXFactory(clientSSLProps.getProperty(SSL_TRUSTSTORE));
+    trustManagerFactory.engineInit((KeyStore) null);
+
+    SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
+    sslContext.init(keyManagerFactory.engineGetKeyManagers(),
+        trustManagerFactory.engineGetTrustManagers(), null);
+    // set default context
+    SSLContext.setDefault(sslContext);
+
+    Properties clientSSLProperties = new Properties();
+    clientSSLProperties.setProperty(SSL_ENABLED_COMPONENTS, ALL);
+    clientSSLProperties.setProperty(SSL_REQUIRE_AUTHENTICATION, String.valueOf("true"));
+    clientSSLProperties.setProperty(SSL_USE_DEFAULT_SSLCONTEXT, String.valueOf("true"));
+
+    if (disableHostNameVerificationForClient) {
+      // client chose to override default
+      clientSSLProperties.setProperty(SSL_ENDPOINT_IDENTIFICATION_ENABLED, String.valueOf("false"));
+    }
+
+    ClientCacheFactory clientCacheFactory = new ClientCacheFactory(clientSSLProperties);
+    clientCacheFactory.addPoolLocator(locator.getVM().getHost().getHostName(), locator.getPort());
+    ClientCache clientCache = clientCacheFactory.create();
+
+    ClientRegionFactory<String, String> regionFactory =
+        clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+
+    if (expectedExceptionOnClient != null) {
+      IgnoredException.addIgnoredException("javax.net.ssl.SSLHandshakeException");
+      IgnoredException.addIgnoredException("java.net.SocketException");
+
+      Region<String, String> clientRegion = regionFactory.create("region");
+      assertThatExceptionOfType(expectedExceptionOnClient)
+          .isThrownBy(() -> clientRegion.put("clientkey", "clientvalue"));
+    } else {
+      // test client can read and write to server
+      Region<String, String> clientRegion = regionFactory.create("region");
+      assertThat("servervalue").isEqualTo(clientRegion.get("serverkey"));
+      clientRegion.put("clientkey", "clientvalue");
+
+      // test server can see data written by client
+      server.invoke(CustomSSLProviderDistributedTest::doServerRegionTest);
+    }
+
+    SocketCreatorFactory.close();
   }
 
   private static void doServerRegionTest() {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java
index 043877b..75cf24a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java
@@ -32,22 +32,23 @@ import javax.net.ssl.KeyManagerFactorySpi;
 import javax.net.ssl.ManagerFactoryParameters;
 import javax.net.ssl.X509ExtendedKeyManager;
 
-import org.apache.geode.util.test.TestUtil;
 
-abstract class CustomKeyManagerFactory extends KeyManagerFactorySpi {
+public abstract class CustomKeyManagerFactory extends KeyManagerFactorySpi {
 
   private final Logger logger = Logger.getLogger(this.getClass().getName());
 
   private final String algorithm;
+  private final String keyStorePath;
   private KeyManagerFactory customKeyManagerFactory;
   private X509ExtendedKeyManager customKeyManager;
 
-  private CustomKeyManagerFactory(String algorithm) {
+  private CustomKeyManagerFactory(String algorithm, String keyStorePath) {
     this.algorithm = algorithm;
+    this.keyStorePath = keyStorePath;
   }
 
   @Override
-  protected final KeyManager[] engineGetKeyManagers() {
+  public final KeyManager[] engineGetKeyManagers() {
     X509ExtendedKeyManager systemKeyManager = getCustomKeyManager();
     return new KeyManager[] {systemKeyManager};
   }
@@ -59,20 +60,17 @@ abstract class CustomKeyManagerFactory extends KeyManagerFactorySpi {
   }
 
   @Override
-  protected final void engineInit(KeyStore keyStore, char[] chars) {
+  public final void engineInit(KeyStore keyStore, char[] chars) {
     // ignore the passed in keystore as it will be null
     init();
   }
 
   private void init() {
-    String CLIENT_KEY_STORE = "client.keystore";
     String SSL_KEYSTORE_TYPE = "JKS";
-    String SSL_KEYSTORE =
-        TestUtil.getResourcePath(CustomKeyManagerFactory.class, CLIENT_KEY_STORE);
     String SSL_KEYSTORE_PASSWORD = "password";
 
     try {
-      FileInputStream fileInputStream = new FileInputStream(SSL_KEYSTORE);
+      FileInputStream fileInputStream = new FileInputStream(keyStorePath);
       KeyStore keyStore = KeyStore.getInstance(SSL_KEYSTORE_TYPE);
       keyStore.load(fileInputStream, SSL_KEYSTORE_PASSWORD.toCharArray());
       this.customKeyManagerFactory = KeyManagerFactory.getInstance(this.algorithm, "SunJSSE");
@@ -97,14 +95,14 @@ abstract class CustomKeyManagerFactory extends KeyManagerFactorySpi {
   }
 
   public static final class PKIXFactory extends CustomKeyManagerFactory {
-    public PKIXFactory() {
-      super("PKIX");
+    public PKIXFactory(String keyStorePath) {
+      super("PKIX", keyStorePath);
     }
   }
 
   public static final class SimpleFactory extends CustomKeyManagerFactory {
-    public SimpleFactory() {
-      super("SunX509");
+    public SimpleFactory(String keyStorePath) {
+      super("SunX509", keyStorePath);
     }
   }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java
deleted file mode 100644
index 6537dce..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java
+++ /dev/null
@@ -1,53 +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.geode.cache.client.internal.provider;
-
-import java.security.Provider;
-import java.util.logging.Logger;
-
-
-public final class CustomProvider extends Provider {
-  private static final long serialVersionUID = -2667509590306131953L;
-
-  private final Logger logger = Logger.getLogger(this.getClass().getName());
-
-  public CustomProvider() {
-    super("Custom Provider", 1.0,
-        "KeyManagerFactory and TrustManagerFactory based on a custom factory implementation");
-
-    this.logger.fine("KeyManager enabled");
-
-    put("KeyManagerFactory.SunX509",
-        "org.apache.geode.cache.client.internal.provider.CustomKeyManagerFactory$SimpleFactory");
-    put("KeyManagerFactory.PKIX",
-        "org.apache.geode.cache.client.internal.provider.CustomKeyManagerFactory$PKIXFactory");
-    put("Alg.Alias.KeyManagerFactory.SunPKIX", "PKIX");
-    put("Alg.Alias.KeyManagerFactory.X509", "PKIX");
-    put("Alg.Alias.KeyManagerFactory.X.509", "PKIX");
-
-    this.logger.fine("TrustManager enabled");
-
-    put("TrustManagerFactory.SunX509",
-        "org.apache.geode.cache.client.internal.provider.CustomTrustManagerFactory$SimpleFactory");
-    put("TrustManagerFactory.PKIX",
-        "org.apache.geode.cache.client.internal.provider.CustomTrustManagerFactory$PKIXFactory");
-    put("Alg.Alias.TrustManagerFactory.SunPKIX", "PKIX");
-    put("Alg.Alias.TrustManagerFactory.X509", "PKIX");
-    put("Alg.Alias.TrustManagerFactory.X.509", "PKIX");
-
-    this.logger.fine("Provider loaded");
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java
index 1d36295..6d11455 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java
@@ -31,34 +31,35 @@ import javax.net.ssl.TrustManagerFactory;
 import javax.net.ssl.TrustManagerFactorySpi;
 import javax.net.ssl.X509ExtendedTrustManager;
 
-import org.apache.geode.util.test.TestUtil;
 
-abstract class CustomTrustManagerFactory extends TrustManagerFactorySpi {
+public abstract class CustomTrustManagerFactory extends TrustManagerFactorySpi {
 
   private final Logger logger = Logger.getLogger(this.getClass().getName());
 
   private final String algorithm;
+  private final String trustStorePath;
   private TrustManagerFactory customTrustManagerFactory;
   private X509ExtendedTrustManager customTrustManager;
 
-  private CustomTrustManagerFactory(String algorithm) {
+  private CustomTrustManagerFactory(String algorithm, String trustStorePath) {
     this.algorithm = algorithm;
+    this.trustStorePath = trustStorePath;
   }
 
   @Override
-  protected final TrustManager[] engineGetTrustManagers() {
+  public final TrustManager[] engineGetTrustManagers() {
     X509ExtendedTrustManager systemTrustManager = getCustomTrustManager();
     return new TrustManager[] {systemTrustManager};
   }
 
   @Override
-  protected final void engineInit(ManagerFactoryParameters managerFactoryParameters) {
+  public final void engineInit(ManagerFactoryParameters managerFactoryParameters) {
     // not supported right now
     throw new UnsupportedOperationException("use engineInit with keystore");
   }
 
   @Override
-  protected final void engineInit(KeyStore keyStore) {
+  public final void engineInit(KeyStore keyStore) {
     // ignore the passed in keystore as it will be null
     init();
   }
@@ -77,9 +78,6 @@ abstract class CustomTrustManagerFactory extends TrustManagerFactorySpi {
   }
 
   private void init() {
-    String trustStoreName = "client.truststore";
-    String trustStorePath =
-        TestUtil.getResourcePath(CustomTrustManagerFactory.class, trustStoreName);
     String trustStoreType = "JKS";
     String trustStorePassword = "password";
 
@@ -96,14 +94,14 @@ abstract class CustomTrustManagerFactory extends TrustManagerFactorySpi {
   }
 
   public static final class PKIXFactory extends CustomTrustManagerFactory {
-    public PKIXFactory() {
-      super("PKIX");
+    public PKIXFactory(String trustStorePath) {
+      super("PKIX", trustStorePath);
     }
   }
 
   public static final class SimpleFactory extends CustomTrustManagerFactory {
-    public SimpleFactory() {
-      super("SunX509");
+    public SimpleFactory(String trustStorePath) {
+      super("SunX509", trustStorePath);
     }
   }
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
index 1613f75..dc55a23 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
@@ -15,7 +15,10 @@
 package org.apache.geode.distributed.internal.tcpserver;
 
 import static org.apache.geode.security.SecurableCommunicationChannels.LOCATOR;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -62,7 +65,7 @@ public class TCPClientSSLIntegrationTest {
   }
 
   private void startServerAndClient(CertificateBuilder serverCertificate,
-      CertificateBuilder clientCertificate)
+      CertificateBuilder clientCertificate, boolean enableHostNameValidation)
       throws GeneralSecurityException, IOException {
 
     CertStores serverStore = CertStores.locatorStore();
@@ -74,11 +77,11 @@ public class TCPClientSSLIntegrationTest {
     Properties serverProperties = serverStore
         .trustSelf()
         .trust(clientStore.alias(), clientStore.certificate())
-        .propertiesWith(LOCATOR, true, true);
+        .propertiesWith(LOCATOR, true, enableHostNameValidation);
 
     Properties clientProperties = clientStore
         .trust(serverStore.alias(), serverStore.certificate())
-        .propertiesWith(LOCATOR, true, true);
+        .propertiesWith(LOCATOR, true, enableHostNameValidation);
 
     startTcpServer(serverProperties);
 
@@ -94,8 +97,11 @@ public class TCPClientSSLIntegrationTest {
     localhost = InetAddress.getLocalHost();
     port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
+    TcpHandler tcpHandler = Mockito.mock(TcpHandler.class);
+    when(tcpHandler.processRequest(any())).thenReturn("Running!");
+
     server = new FakeTcpServer(port, localhost, sslProperties, null,
-        Mockito.mock(TcpHandler.class), Mockito.mock(PoolStatHelper.class),
+        tcpHandler, Mockito.mock(PoolStatHelper.class),
         Thread.currentThread().getThreadGroup(), "server thread");
     server.start();
   }
@@ -109,7 +115,25 @@ public class TCPClientSSLIntegrationTest {
     CertificateBuilder clientCertificate = new CertificateBuilder()
         .commonName("tcp-client");
 
-    startServerAndClient(serverCertificate, clientCertificate);
+    startServerAndClient(serverCertificate, clientCertificate, true);
+    String response =
+        (String) client.requestToServer(localhost, port, Boolean.valueOf(false), 5 * 1000);
+    assertThat(response).isEqualTo("Running!");
+  }
+
+  @Test
+  public void clientChooseToDisableHasHostnameValidation() throws Exception {
+    // no host name in server cert
+    CertificateBuilder serverCertificate = new CertificateBuilder()
+        .commonName("tcp-server");
+
+    CertificateBuilder clientCertificate = new CertificateBuilder()
+        .commonName("tcp-client");
+
+    startServerAndClient(serverCertificate, clientCertificate, false);
+    String response =
+        (String) client.requestToServer(localhost, port, Boolean.valueOf(false), 5 * 1000);
+    assertThat(response).isEqualTo("Running!");
   }
 
   @Test
@@ -120,7 +144,7 @@ public class TCPClientSSLIntegrationTest {
     CertificateBuilder clientCertificate = new CertificateBuilder()
         .commonName("tcp-client");
 
-    startServerAndClient(serverCertificate, clientCertificate);
+    startServerAndClient(serverCertificate, clientCertificate, true);
 
     assertThatExceptionOfType(LocatorCancelException.class)
         .isThrownBy(() -> client.requestToServer(localhost, port, Boolean.valueOf(false), 5 * 1000))
@@ -138,7 +162,7 @@ public class TCPClientSSLIntegrationTest {
     CertificateBuilder clientCertificate = new CertificateBuilder()
         .commonName("tcp-client");
 
-    startServerAndClient(serverCertificate, clientCertificate);
+    startServerAndClient(serverCertificate, clientCertificate, true);
 
     assertThatExceptionOfType(LocatorCancelException.class)
         .isThrownBy(() -> client.requestToServer(localhost, port, Boolean.valueOf(false), 5 * 1000))
@@ -147,7 +171,7 @@ public class TCPClientSSLIntegrationTest {
                 + localhost.getHostName() + " found."));
   }
 
-  private class FakeTcpServer extends TcpServer {
+  private static class FakeTcpServer extends TcpServer {
     private DistributionConfig distributionConfig;
 
     public FakeTcpServer(int port, InetAddress bind_address, Properties sslConfig,
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
index b138010..db64481 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
@@ -1363,7 +1363,7 @@ public abstract class AbstractDistributionConfig extends AbstractConfig
         "User defined fully qualified class name implementing PostProcessor interface for integrated security. Defaults to \"{0}\". Legal values can be any \"class name\" implementing PostProcessor that is present in the classpath.");
 
     m.put(SSL_ENDPOINT_IDENTIFICATION_ENABLED,
-        "If true, clients validate server hostname using server certificate during SSL handshake.");
+        "If true, clients validate server hostname using server certificate during SSL handshake. Defaults to true since GEODE 1.7.0");
 
     m.put(SSL_USE_DEFAULT_SSLCONTEXT,
         "If true allows users to use the default SSLContext. When using default SSLContext, users do not need to configure other ssl properties except ssl-enabled-components and ssl-require-authentication.");
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
index caaf525..7c93f4a 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
@@ -4883,9 +4883,9 @@ public interface DistributionConfig extends Config, LogConfig {
   /**
    * The default {@link ConfigurationProperties#SSL_ENDPOINT_IDENTIFICATION_ENABLED} value.
    * <p>
-   * Actual value of this constant is <code>false</code>.
+   * Actual value of this constant is <code>true</code>.
    */
-  boolean DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ENABLED = false;
+  boolean DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ENABLED = true;
 
   /**
    * Returns the value of the {@link ConfigurationProperties#SSL_ENDPOINT_IDENTIFICATION_ENABLED}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
index 28cedf0..060cb6c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.net;
 
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
+
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.BindException;
@@ -1036,6 +1038,11 @@ public class SocketCreator {
       sslSocket.setEnableSessionCreation(true);
 
       if (sslConfig.doEndpointIdentification()) {
+        if (!sslConfig.useDefaultSSLContext()) {
+          logger.warn("Since GEODE 1.7.0, hostname is validated in the Server's certificate "
+              + "during SSL handshake. You can disable using setting '"
+              + SSL_ENDPOINT_IDENTIFICATION_ENABLED + "' to false.");
+        }
         SSLParameters sslParameters = sslSocket.getSSLParameters();
         sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
         sslSocket.setSSLParameters(sslParameters);