You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2015/10/16 22:58:15 UTC

[1/3] incubator-geode git commit: GEODE-396: Fix server to use server/gateway ssl config

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 21743ec13 -> e45f5e3aa


GEODE-396: Fix server to use server/gateway ssl config

The AcceptorImpl now keeps the non-default SocketCreator it makes
with the server/gateway ssl and uses it for all connections
from the client instead of the default cluster ssl config.

The extra logging done by the server that included the keystore
and truststore passwords (in clear text) has been removed.

Added unit test that reproduced this bug.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/03e593de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/03e593de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/03e593de

Branch: refs/heads/develop
Commit: 03e593dee378aad86ac31b524dd30ccad8405bfb
Parents: 21743ec
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 15:31:49 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Oct 16 11:26:24 2015 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/AcceptorImpl.java        |  36 +--
 .../internal/SSLNoClientAuthDUnitTest.java      | 271 +++++++++++++++++++
 .../cache/client/internal/default.keystore      | Bin 0 -> 1115 bytes
 3 files changed, 277 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/03e593de/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index 74cdfa9..b5fd228 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -11,8 +11,6 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
 import java.net.BindException;
 import java.net.Inet6Address;
 import java.net.InetAddress;
@@ -270,6 +268,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
 
   private boolean isGatewayReceiver;
   private List<GatewayTransportFilter> gatewayTransportFilters;
+  private final SocketCreator socketCreator; 
   /**
    * Initializes this acceptor thread to listen for connections on the given
    * port.
@@ -386,43 +385,20 @@ public class AcceptorImpl extends Acceptor implements Runnable
 
     {
       final int backLog = Integer.getInteger(BACKLOG_PROPERTY_NAME, DEFAULT_BACKLOG).intValue();
-      SocketCreator sc = null;
       DistributionConfig config = ((InternalDistributedSystem)c.getDistributedSystem()).getConfig();
       if(!isGatewayReceiver) {
         //If configured use SSL properties for cache-server
-        sc = SocketCreator.createNonDefaultInstance(config.getServerSSLEnabled(),
+        this.socketCreator = SocketCreator.createNonDefaultInstance(config.getServerSSLEnabled(),
             config.getServerSSLRequireAuthentication(),
             config.getServerSSLProtocols(),
             config.getServerSSLCiphers(),
             config.getServerSSLProperties());
-        if(config.getServerSSLEnabled()) {
-          StringWriter sw = new StringWriter();
-          PrintWriter writer = new PrintWriter(sw);
-          config.getServerSSLProperties().list(writer);          
-          logger.info(
-              "Starting CacheServer with SSL config : Authentication Required {} Ciphers {} Protocols {} Other Properties {} ",
-                  config.getServerSSLRequireAuthentication(),
-                  config.getServerSSLCiphers(),
-                  config.getServerSSLProtocols(),
-                  sw.toString());
-        }
       } else {
-        sc = SocketCreator.createNonDefaultInstance(config.getGatewaySSLEnabled(),
+        this.socketCreator = SocketCreator.createNonDefaultInstance(config.getGatewaySSLEnabled(),
             config.getGatewaySSLRequireAuthentication(),
             config.getGatewaySSLProtocols(),
             config.getGatewaySSLCiphers(),
             config.getGatewaySSLProperties());
-        if(config.getGatewaySSLEnabled()) {
-          StringWriter sw = new StringWriter();
-          PrintWriter writer = new PrintWriter(sw);
-          config.getGatewaySSLProperties().list(writer);          
-          logger.info(
-              "Starting Gateway with SSL config : Authentication Required {} Ciphers {} Protocols {} Other Properties {} ",
-                  config.getGatewaySSLRequireAuthentication(),
-                  config.getGatewaySSLCiphers(),
-                  config.getGatewaySSLProtocols(),
-                  sw.toString());
-        }
       }
       
       final GemFireCacheImpl gc;
@@ -435,7 +411,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
       final long tilt = System.currentTimeMillis() + 120 * 1000;
 
       if (isSelector()) {
-        if (sc.useSSL()) {
+        if (this.socketCreator.useSSL()) {
           throw new IllegalArgumentException(LocalizedStrings.AcceptorImpl_SELECTOR_THREAD_POOLING_CAN_NOT_BE_USED_WITH_CLIENTSERVER_SSL_THE_SELECTOR_CAN_BE_DISABLED_BY_SETTING_MAXTHREADS0.toLocalizedString());
         }
         ServerSocketChannel channel = ServerSocketChannel.open();
@@ -486,7 +462,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
         // immediately restarted, which sometimes results in a bind exception
         for (;;) {
           try {
-            this.serverSock = sc.createServerSocket(port, backLog,
+            this.serverSock = this.socketCreator.createServerSocket(port, backLog,
                 getBindAddress(), this.gatewayTransportFilters,
                 socketBufferSize);
             break;
@@ -1323,7 +1299,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
             break;
           }
         }
-        SocketCreator.getDefaultInstance().configureServerSSLSocket(s);
+        this.socketCreator.configureServerSSLSocket(s);
         this.loggedAcceptError = false;
 
         handOffNewClientConnection(s);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/03e593de/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/SSLNoClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/SSLNoClientAuthDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/SSLNoClientAuthDUnitTest.java
new file mode 100644
index 0000000..390c285
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/SSLNoClientAuthDUnitTest.java
@@ -0,0 +1,271 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.cache.client.internal;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Properties;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.security.AuthenticationRequiredException;
+import com.gemstone.gemfire.util.test.TestUtil;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * Test for GEODE-396
+ */
+public class SSLNoClientAuthDUnitTest extends DistributedTestCase {
+  
+  private static final long serialVersionUID = 1L;
+  private Cache cache;
+  private CacheServer cacheServer;
+  private ClientCache clientCache;
+  private int cacheServerPort;
+  private String hostName;
+  
+  private static final String DEFAULT_STORE = "default.keystore";
+  
+  private static SSLNoClientAuthDUnitTest instance = new SSLNoClientAuthDUnitTest("SSLNoClientAuthDUnitTest");
+  
+  
+  public void setUp() throws Exception {
+    disconnectAllFromDS();
+    super.setUp();
+  }
+
+  public SSLNoClientAuthDUnitTest(String name) {
+    super(name);
+  }  
+
+  public Cache createCache(Properties props) throws Exception
+  {
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    cache = new CacheFactory(props).create();
+    if (cache == null) {
+      throw new Exception("CacheFactory.create() returned null ");
+    }
+    return cache;
+  }
+  
+  private void createServer() throws IOException{
+    cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    cacheServer = cache.addCacheServer();
+    cacheServer.setPort(cacheServerPort);
+    cacheServer.start();
+    hostName = cacheServer.getHostnameForClients();
+  }
+  
+  public int getCacheServerPort(){
+    return cacheServerPort;
+  }
+  
+  public String getCacheServerHost(){
+    return hostName;
+  }
+  
+  public void stopCacheServer(){
+    this.cacheServer.stop();
+  }
+  
+  
+  @SuppressWarnings("rawtypes")
+  public void setUpServerVM(boolean cacheServerSslenabled) throws Exception {
+    Properties gemFireProps = new Properties();
+
+    String cacheServerSslprotocols = "any";
+    String cacheServerSslciphers = "any";
+    boolean cacheServerSslRequireAuth = false;
+    gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
+        String.valueOf(cacheServerSslenabled));
+    gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
+        cacheServerSslprotocols);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
+        cacheServerSslciphers);
+    gemFireProps.put(
+        DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
+        String.valueOf(cacheServerSslRequireAuth));
+
+    String keyStore = TestUtil.getResourcePath(SSLNoClientAuthDUnitTest.class, DEFAULT_STORE);
+    String trustStore = TestUtil.getResourcePath(SSLNoClientAuthDUnitTest.class, DEFAULT_STORE);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, keyStore);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, trustStore);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+    
+    StringWriter sw = new StringWriter();
+    PrintWriter writer = new PrintWriter(sw);
+    gemFireProps.list(writer);
+    System.out.println("Starting cacheserver ds with following properties \n" + sw);
+    createCache(gemFireProps);
+    
+    RegionFactory factory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    Region r = factory.create("serverRegion");
+    r.put("serverkey", "servervalue");
+  }
+  
+  public void setUpClientVM(String host, int port,
+      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth,
+      String keyStore, String trustStore) {
+
+    Properties gemFireProps = new Properties();
+
+    String cacheServerSslprotocols = "any";
+    String cacheServerSslciphers = "any";
+
+    String keyStorePath = TestUtil.getResourcePath(SSLNoClientAuthDUnitTest.class, keyStore);
+    String trustStorePath = TestUtil.getResourcePath(SSLNoClientAuthDUnitTest.class, trustStore);
+    //using new server-ssl-* properties
+    gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
+        String.valueOf(cacheServerSslenabled));
+    gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
+        cacheServerSslprotocols);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
+        cacheServerSslciphers);
+    gemFireProps.put(
+        DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
+        String.valueOf(cacheServerSslRequireAuth));
+
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, keyStorePath);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, trustStorePath);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+
+    StringWriter sw = new StringWriter();
+    PrintWriter writer = new PrintWriter(sw);
+    gemFireProps.list(writer);
+    System.out.println("Starting client ds with following properties \n" + sw.getBuffer());
+    
+    ClientCacheFactory clientCacheFactory = new ClientCacheFactory(gemFireProps);
+    clientCacheFactory.addPoolServer(host, port);
+    clientCache = clientCacheFactory.create();
+    
+    ClientRegionFactory<String,String> regionFactory = clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+    Region<String, String> region = regionFactory.create("serverRegion");  
+    assertNotNull(region);
+  }
+  
+  public void doClientRegionTest(){
+    Region<String, String> region = clientCache.getRegion("serverRegion");
+    assertEquals("servervalue",region.get("serverkey"));
+    region.put("clientkey", "clientvalue");
+    assertEquals("clientvalue",region.get("clientkey"));
+  }
+  
+  public void doServerRegionTest(){
+    Region<String, String> region = cache.getRegion("serverRegion");
+    assertEquals("servervalue",region.get("serverkey"));    
+    assertEquals("clientvalue",region.get("clientkey"));
+  }
+  
+  
+  public static void setUpServerVMTask(boolean cacheServerSslenabled) throws Exception{
+    instance.setUpServerVM(cacheServerSslenabled);
+  }
+  
+  public static void createServerTask() throws Exception {
+    instance.createServer();
+  }
+  
+  public static void setUpClientVMTask(String host, int port,
+      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth, String keyStore, String trustStore)
+      throws Exception {
+    instance.setUpClientVM(host, port, cacheServerSslenabled, cacheServerSslRequireAuth, keyStore, trustStore);
+  }
+  
+  public static void doClientRegionTestTask() {
+    instance.doClientRegionTest();
+  }
+  
+  public static void doServerRegionTestTask() {
+    instance.doServerRegionTest();
+  }
+  
+  public static Object[] getCacheServerEndPointTask() {
+    Object[] array = new Object[2];
+    array[0] = instance.getCacheServerHost();
+    array[1] = instance.getCacheServerPort();
+    return array;
+  }
+  
+  public static void closeCacheTask(){
+    if (instance != null && instance.cache != null) {
+      instance.cache.close();
+    }
+  }
+  
+  public static void closeClientCacheTask(){
+    if (instance != null && instance.clientCache != null) {
+      instance.clientCache.close();
+    }
+  }
+  
+  /**
+   * Test for GEODE-396
+   */
+  public void testSSLServerWithNoAuth() throws Exception {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+
+    boolean cacheServerSslenabled = true;
+    boolean cacheClientSslenabled = true;
+    boolean cacheClientSslRequireAuth = true;
+
+    serverVM.invoke(SSLNoClientAuthDUnitTest.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled});
+    serverVM.invoke(SSLNoClientAuthDUnitTest.class, "createServerTask");
+
+    Object array[] = (Object[])serverVM.invoke(SSLNoClientAuthDUnitTest.class, "getCacheServerEndPointTask"); 
+    String hostName = (String)array[0];
+    int port = (Integer) array[1];
+    Object params[] = new Object[6];
+    params[0] = hostName;
+    params[1] = port;
+    params[2] = cacheClientSslenabled;
+    params[3] = cacheClientSslRequireAuth;
+    params[4] = DEFAULT_STORE;
+    params[5] = DEFAULT_STORE;
+    //getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
+    try {
+      clientVM.invoke(SSLNoClientAuthDUnitTest.class, "setUpClientVMTask", params);
+      clientVM.invoke(SSLNoClientAuthDUnitTest.class, "doClientRegionTestTask");
+      serverVM.invoke(SSLNoClientAuthDUnitTest.class, "doServerRegionTestTask");
+    } catch (Exception rmiException) {
+      Throwable e = rmiException.getCause();
+      //getLogWriter().info("ExceptionCause at clientVM " + e);
+      fail("Unexpected Exception " + e);
+    }
+  }
+  
+  public void tearDown2() throws Exception
+  {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+    clientVM.invoke(SSLNoClientAuthDUnitTest.class, "closeClientCacheTask");
+    serverVM.invoke(SSLNoClientAuthDUnitTest.class, "closeCacheTask");
+    super.tearDown2();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/03e593de/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/default.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/default.keystore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/default.keystore
new file mode 100644
index 0000000..9dbc135
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/default.keystore differ


[3/3] incubator-geode git commit: GEODE-405: sameAs needs to ignore ssl-* properties

Posted by ds...@apache.org.
GEODE-405: sameAs needs to ignore ssl-* properties

Since toProperties does not include ssl-* properties
sameAs needs to also ignore ssl-* properties.
Since the ssl-* are always copied to cluster-ssl-*
and sameAs compares these it will still return false
if the ssl-* properties had different values.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/e45f5e3a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/e45f5e3a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/e45f5e3a

Branch: refs/heads/develop
Commit: e45f5e3aa5cdb9e47016154df5eae220a9a1d9ae
Parents: fbe8c37
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Oct 14 16:23:56 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Oct 16 13:51:47 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/AbstractConfig.java        |  6 +++++
 .../InternalDistributedSystemJUnitTest.java     | 24 ++++++++++++++++++++
 2 files changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e45f5e3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
index 75e4314..2bb047d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
@@ -276,6 +276,12 @@ public abstract class AbstractConfig implements Config {
     String[] validAttributeNames = getAttributeNames();
     for (int i=0; i < validAttributeNames.length; i++) {
       String attName = validAttributeNames[i];
+      if (this.isDeprecated(attName)) {
+        // since toProperties skips isDeprecated sameAs
+        // needs to also skip them.
+        // See GEODE-405.
+        continue;
+      }
       Object thisAtt = this.getAttributeObject(attName);
       Object otherAtt = other.getAttributeObject(attName);
       if (thisAtt == otherAtt) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e45f5e3a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
index 88a0654..3af05cb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
@@ -704,6 +704,7 @@ public class InternalDistributedSystemJUnitTest
 
     try {
       sys.validateSameProperties(config2.toProperties(), true);
+      fail("should have detected different mcast-ports");
     } catch (IllegalStateException iex) {
       // This passes the test
     }
@@ -712,6 +713,29 @@ public class InternalDistributedSystemJUnitTest
       sys.disconnect();
     }
   }
+  @Test
+  public void testDeprecatedSSLProps() {
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    props.setProperty("ssl-enabled", "true");
+    Config config1 = new DistributionConfigImpl(props, false);
+    Properties props1 = config1.toProperties();
+    // For the deprecated ssl-* properties a decision was made
+    // to not include them in the result of "toProperties".
+    // The cause of this is: com.gemstone.gemfire.internal.AbstractConfig.isDeprecated(String)
+    // and its use in toProperties.
+    // The other thing that is done is the ssl-* props are copied to cluster-ssl-*.
+    // The following two assertions demonstrate this.
+    assertEquals(null, props1.getProperty("ssl-enabled"));
+    assertEquals("true", props1.getProperty("cluster-ssl-enabled"));
+    Config config2 = new DistributionConfigImpl(props1, false);
+    assertEquals(true, config1.sameAs(config2));
+    Properties props3 = new Properties(props1);
+    props3.setProperty("ssl-enabled", "false");
+    Config config3 = new DistributionConfigImpl(props3, false);
+    assertEquals(false, config1.sameAs(config3));
+  }
   public static String getHostAddress(InetAddress addr) {
     String address = addr.getHostAddress();
     if (addr instanceof Inet4Address


[2/3] incubator-geode git commit: GEODE-397: Fix clients to use server ssl config

Posted by ds...@apache.org.
GEODE-397: Fix clients to use server ssl config

Each client pool now creates a single SocketCreator that
uses either the server or gateway ssl config.
That SocketCreator is used for all connections the client
makes to the server. It no longer uses the default cluster
SocketCreator when connecting to the server.
This fix might show some performance improvement because the
old code recreated the SocketCreator every time the client
created a server connection. Now it just happens once for each
pool.

Also since using SocketCreator.getDefaultInstance when it should
have used a non-default instance caused this bug all calls of
getDefaultInstance were reviewed. A number of them used
to call isHostReachable which is a method that was deadcoded.
So all those calls have been removed. One of call of
getDefaultInstance was deleted (in ConnectionTable) because it
was never used.

The existing code CacheServerSSLConnectionDUnit is now named
CacheServerSSLConnectionDUnitTest which causes this test to
be run with the other unit tests. A test for this bug was
added to it.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/fbe8c37a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/fbe8c37a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/fbe8c37a

Branch: refs/heads/develop
Commit: fbe8c37a94b61af411640c455fe4787e19f3d536
Parents: 03e593d
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 15:51:52 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Oct 16 11:27:08 2015 -0700

----------------------------------------------------------------------
 .../client/internal/ConnectionFactoryImpl.java  |  24 +-
 .../cache/client/internal/ConnectionImpl.java   |  28 +-
 .../gemfire/internal/SocketCreator.java         |  40 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |  10 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |   1 -
 .../internal/CacheServerSSLConnectionDUnit.java | 648 -------------------
 .../CacheServerSSLConnectionDUnitTest.java      | 417 ++++++++++++
 .../cache/client/internal/cacheserver.cer       | Bin 0 -> 782 bytes
 .../cache/client/internal/cacheserver.keystore  | Bin 0 -> 1253 bytes
 .../client/internal/cacheserver.truststore      | Bin 0 -> 844 bytes
 .../gemfire/cache/client/internal/client.cer    | Bin 0 -> 782 bytes
 .../cache/client/internal/client.keystore       | Bin 0 -> 1251 bytes
 .../cache/client/internal/client.truststore     | Bin 0 -> 846 bytes
 .../cache/client/internal/trusted.keystore      | Bin 0 -> 1078 bytes
 .../org/jgroups/stack/GossipClient.java         |   6 +-
 .../org/jgroups/stack/SockCreatorImpl.java      |   5 -
 .../gemstone/org/jgroups/util/SockCreator.java  |   8 +-
 17 files changed, 452 insertions(+), 735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionFactoryImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionFactoryImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionFactoryImpl.java
index 02a1fc0..b1c6185 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionFactoryImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionFactoryImpl.java
@@ -20,8 +20,10 @@ import com.gemstone.gemfire.cache.client.ServerRefusedConnectionException;
 import com.gemstone.gemfire.cache.client.internal.ServerBlackList.FailureTracker;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientUpdater;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -51,6 +53,7 @@ public class ConnectionFactoryImpl implements ConnectionFactory {
   private final boolean usedByGateway;
   private final ServerBlackList blackList;
   private final CancelCriterion cancelCriterion;
+  private final SocketCreator socketCreator;
   private ConnectionSource source;
   private int readTimeout;
   private InternalDistributedSystem ds;
@@ -85,6 +88,22 @@ public class ConnectionFactoryImpl implements ConnectionFactory {
     this.blackList = new ServerBlackList(pingInterval);
     this.cancelCriterion = cancelCriterion;
     this.pool = pool;
+    DistributionConfig config = InternalDistributedSystem.getConnectedInstance().getConfig();
+    if (this.usedByGateway || (this.gatewaySender != null)) {
+      this.socketCreator = SocketCreator.createNonDefaultInstance(config.getGatewaySSLEnabled(),
+          config.getGatewaySSLRequireAuthentication(), config.getGatewaySSLProtocols(),
+          config.getGatewaySSLCiphers(), config.getGatewaySSLProperties());
+      if (sender!= null && !sender.getGatewayTransportFilters().isEmpty()) {
+        this.socketCreator.initializeTransportFilterClientSocketFactory(sender);
+      }
+    } else {
+      //If configured use SSL properties for cache-server
+      this.socketCreator = SocketCreator.createNonDefaultInstance(config.getServerSSLEnabled(),
+          config.getServerSSLRequireAuthentication(),
+          config.getServerSSLProtocols(),
+          config.getServerSSLCiphers(),
+          config.getServerSSLProperties());
+    }
   }
   
   public void start(ScheduledExecutorService background) {
@@ -114,7 +133,8 @@ public class ConnectionFactoryImpl implements ConnectionFactory {
     try {
       HandShake connHandShake = new HandShake(handshake);
       connection.connect(endpointManager, location, connHandShake,
-                         socketBufferSize, handShakeTimeout, readTimeout, getCommMode(forQueue), this.gatewaySender);
+                         socketBufferSize, handShakeTimeout, readTimeout, 
+                         getCommMode(forQueue), this.gatewaySender, this.socketCreator);
       failureTracker.reset();
       connection.setHandShake(connHandShake);
       authenticateIfRequired(connection);
@@ -271,7 +291,7 @@ public class ConnectionFactoryImpl implements ConnectionFactory {
 //  Launch the thread
     CacheClientUpdater updater = new CacheClientUpdater(clientUpdateName,
         endpoint.getLocation(), isPrimary, ds, new HandShake(this.handshake), qManager,
-        endpointManager, endpoint, handShakeTimeout);
+        endpointManager, endpoint, handShakeTimeout, this.socketCreator);
     
     if(!updater.isConnected()) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
index 749a765..7880821 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
@@ -87,29 +87,8 @@ public class ConnectionImpl implements Connection {
   
   public ServerQueueStatus connect(EndpointManager endpointManager,
       ServerLocation location, HandShake handShake, int socketBufferSize,
-      int handShakeTimeout, int readTimeout, byte communicationMode, GatewaySender sender)
+      int handShakeTimeout, int readTimeout, byte communicationMode, GatewaySender sender, SocketCreator sc)
       throws IOException {
-    SocketCreator sc = SocketCreator.getDefaultInstance();
-    DistributionConfig config = ds.getConfig();
-    if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
-      sc = SocketCreator.createNonDefaultInstance(config.getGatewaySSLEnabled(),
-          config.getGatewaySSLRequireAuthentication(), config.getGatewaySSLProtocols(),
-          config.getGatewaySSLCiphers(), config.getGatewaySSLProperties());
-      if (sender!= null && !sender.getGatewayTransportFilters().isEmpty()) {
-        sc.initializeTransportFilterClientSocketFactory(sender);
-      }
-    } else {
-      //If configured use SSL properties for cache-server
-      sc = SocketCreator.createNonDefaultInstance(config.getServerSSLEnabled(),
-          config.getServerSSLRequireAuthentication(),
-          config.getServerSSLProtocols(),
-          config.getServerSSLCiphers(),
-          config.getServerSSLProperties());
-    }
-    if (!sc
-        .isHostReachable(InetAddress.getByName(location.getHostName()))) {
-      throw new NoRouteToHostException("Server is not reachable: " + location.getHostName());
-    }
     theSocket = sc.connectForClient(
         location.getHostName(), location.getPort(), handShakeTimeout, socketBufferSize);
     theSocket.setTcpNoDelay(true);
@@ -142,11 +121,6 @@ public class ConnectionImpl implements Connection {
     
     try {
       // if a forced-disconnect has occurred, we can't send messages to anyone
-      SocketCreator sc = SocketCreator.getDefaultInstance();
-      if (!sc.isHostReachable(this.theSocket.getInetAddress())) {
-        return;
-      }
-
       boolean sendCloseMsg = !TEST_DURABLE_CLIENT_CRASH;
       if (sendCloseMsg) {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
index 940936f..b73a986 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
@@ -1028,45 +1028,7 @@ public class SocketCreator  implements com.gemstone.org.jgroups.util.SockCreator
 //         rw.readLock().unlock();
 //       }
   }
-  
-  /** has the isReachable method been looked up already? */
-  volatile boolean isReachableChecked;
-  
-  /** InetAddress.isReachable() is in v1.5 and later */
-  volatile Method isReachableMethod;
-  
-  public boolean isHostReachable(InetAddress host) {
-    boolean result = true;
-    try {
-      Method m = null;
-      if (isReachableChecked) {
-        m = isReachableMethod;
-      }
-      else {
-        // deadcoded - InetAddress.isReachable uses the ECHO port
-        // if we don't have root permission, and the ECHO port may
-        // be blocked
-        //m = InetAddress.class.getMethod("isReachable", new Class[] { int.class });
-        //isReachableMethod = m;
-        isReachableChecked = true;
-      }
-      if (m != null) {
-        result = ((Boolean)m.invoke(host, new Object[] {Integer.valueOf(250)})).booleanValue();
-        return result;
-      }
-    }
-    catch (InvocationTargetException e) {
-    }
-//    catch (NoSuchMethodException e) {
-//    }
-    catch (IllegalAccessException e) {
-    }
-    // any other bright ideas?  attempts to connect a socket to a missing
-    // machine may hang, so don't try the echo port or anything requiring
-    // full Sockets
-    return result;
-  }
-  
+
   /** Will be a server socket... this one simply registers the listeners. */
   public void configureServerSSLSocket( Socket socket ) throws IOException {
 //       rw.readLock().lockInterruptibly();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
index 07dc030..3915465 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
@@ -278,7 +278,8 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
       String name, ServerLocation location,
       boolean primary, DistributedSystem ids,
       HandShake handshake, QueueManager qManager, EndpointManager eManager,
-      Endpoint endpoint, int handshakeTimeout) throws AuthenticationRequiredException,
+      Endpoint endpoint, int handshakeTimeout,
+      SocketCreator socketCreator) throws AuthenticationRequiredException,
       AuthenticationFailedException, ServerRefusedConnectionException {
     super(LoggingThreadGroup.createThreadGroup("Client update thread"), name);
     this.setDaemon(true);
@@ -308,12 +309,7 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
       int socketBufferSize = Integer.getInteger(
           "BridgeServer.SOCKET_BUFFER_SIZE", 32768).intValue();
 
-      if (!SocketCreator.getDefaultInstance()
-          .isHostReachable(InetAddress.getByName(location.getHostName()))) {
-        throw new NoRouteToHostException("Server is not reachable: " + location.getHostName());
-      }
-
-      mySock = SocketCreator.getDefaultInstance().connectForClient(
+      mySock = socketCreator.connectForClient(
           location.getHostName(), location.getPort(), handshakeTimeout, socketBufferSize);
       mySock.setTcpNoDelay(true);
       mySock.setSendBufferSize(socketBufferSize);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
index 508eba2..bbd490d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
@@ -804,7 +804,6 @@ public class ConnectionTable  {
         }
       }
       // now close any sockets being formed
-      SocketCreator sc = SocketCreator.getDefaultInstance();
       synchronized(connectingSockets) {
         for (Iterator it = connectingSockets.entrySet().iterator(); it.hasNext(); ) {
           Map.Entry entry = (Map.Entry)it.next();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnit.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnit.java
deleted file mode 100644
index 7bf7c7e..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnit.java
+++ /dev/null
@@ -1,648 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
- * This product is protected by U.S. and international copyright
- * and intellectual property laws. Pivotal products are covered by
- * one or more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.cache.client.internal;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Properties;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.client.ClientCache;
-import com.gemstone.gemfire.cache.client.ClientCacheFactory;
-import com.gemstone.gemfire.cache.client.ClientRegionFactory;
-import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
-import com.gemstone.gemfire.cache.client.ServerOperationException;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.security.AuthenticationRequiredException;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-/**
- * Tests cacheserver ssl support added. See https://svn.gemstone.com/trac/gemfire/ticket/48995 for details
- * @author tushark
- *
- */
-public class CacheServerSSLConnectionDUnit extends DistributedTestCase {
-  
-  private static final long serialVersionUID = 1L;
-  private Cache cache;
-  private DistributedSystem ds;
-  private CacheServer cacheServer;
-  private ClientCache clientCache;
-  private int cacheServerPort;
-  private String hostName;
-  private int locatorPort;
-  private Locator locator;
-  
-  private static final String jtests = System.getProperty("JTESTS");
-  
-  private static final String TRUSTED_STORE = "trusted.keystore";
-  private static final String CLIENT_KEY_STORE = "client.keystore";
-  private static final String CLIENT_TRUST_STORE = "client.truststore";
-  private static final String SERVER_KEY_STORE = "cacheserver.keystore";
-  private static final String SERVER_TRUST_STORE = "cacheserver.truststore";
-  
-  private static CacheServerSSLConnectionDUnit instance = new CacheServerSSLConnectionDUnit("CacheServerSSLConnectionDUnit");
-  
-  
-  public void setUp() throws Exception {
-    disconnectAllFromDS();
-    super.setUp();
-  }
-
-  public CacheServerSSLConnectionDUnit(String name) {
-    super(name);
-  }  
-
-  public Cache createCache(Properties props) throws Exception
-  {
-    ds = getSystem(props);
-    cache = CacheFactory.create(ds);
-    if (cache == null) {
-      throw new Exception("CacheFactory.create() returned null ");
-    }
-    return cache;
-  }
-  
-  private void createServer() throws IOException{
-    cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    cacheServer = cache.addCacheServer();
-    cacheServer.setPort(cacheServerPort);
-    cacheServer.start();
-    hostName = cacheServer.getHostnameForClients();
-  }
-  
-  public int getCacheServerPort(){
-    return cacheServerPort;
-  }
-  
-  public String getCacheServerHost(){
-    return hostName;
-  }
-  
-  public void stopCacheServer(){
-    this.cacheServer.stop();
-  }
-  
-  
-  @SuppressWarnings("rawtypes")
-  public void setUpServerVM(boolean cacheServerSslenabled, boolean p2pProps, String locators) throws Exception {
-    Properties gemFireProps = new Properties();
-
-    String cacheServerSslprotocols = "any";
-    String cacheServerSslciphers = "any";
-    boolean cacheServerSslRequireAuth = true;
-    
-    String filePathPrefix = jtests
-    + File.separator
-    + makePath(new String[] { "com", "gemstone", "gemfire", "cache",
-        "client", "internal" });
-    
-    if (!p2pProps) {
-      gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
-          String.valueOf(cacheServerSslenabled));
-      gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
-          cacheServerSslprotocols);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
-          cacheServerSslciphers);
-      gemFireProps.put(
-          DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
-          String.valueOf(cacheServerSslRequireAuth));
-      
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, filePathPrefix
-          + SERVER_KEY_STORE);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
-      gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, filePathPrefix
-          + SERVER_TRUST_STORE);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
-    } else {
-      
-      gemFireProps.put(DistributionConfig.SSL_ENABLED_NAME,
-          String.valueOf(cacheServerSslenabled));
-      gemFireProps.put(DistributionConfig.SSL_PROTOCOLS_NAME,
-          cacheServerSslprotocols);
-      gemFireProps.put(DistributionConfig.SSL_CIPHERS_NAME,
-          cacheServerSslciphers);
-      gemFireProps.put(
-          DistributionConfig.SSL_REQUIRE_AUTHENTICATION_NAME,
-          String.valueOf(cacheServerSslRequireAuth));
-
-      gemFireProps.put("javax.net.ssl.keyStoreType", "jks");
-      gemFireProps.put("javax.net.ssl.keyStore", filePathPrefix + TRUSTED_STORE);
-      gemFireProps.put("javax.net.ssl.keyStorePassword", "password");
-      gemFireProps.put("javax.net.ssl.trustStore", filePathPrefix + TRUSTED_STORE);
-      gemFireProps.put("javax.net.ssl.trustStorePassword", "password");
-      gemFireProps.put(DistributionConfig.LOCATORS_NAME, locators);
-    }
-    
-    StringWriter sw = new StringWriter();
-    PrintWriter writer = new PrintWriter(sw);
-    gemFireProps.list(writer);
-    System.out.println("Starting cacheserver ds with following properties \n" + sw);
-    createCache(gemFireProps);
-    
-    RegionFactory factory = cache.createRegionFactory(RegionShortcut.REPLICATE);
-    Region r = factory.create("serverRegion");
-    r.put("serverkey", "servervalue");
-  }
-  
-  public static String makePath(String[] strings) {
-    StringBuilder sb = new StringBuilder();
-    for(int i=0;i<strings.length;i++){
-      sb.append(strings[i]);      
-      sb.append(File.separator);
-    }
-    return sb.toString();
-  }
-
-  public void setUpClientVM(String host, int port,
-      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth,
-      boolean p2pProps, String locators, String keyStore, String trustStore) {
-
-    Properties gemFireProps = new Properties();
-
-    String cacheServerSslprotocols = "any";
-    String cacheServerSslciphers = "any";
-
-    String filePathPrefix = jtests
-    + File.separator
-    + makePath(new String[] { "com", "gemstone", "gemfire", "cache",
-        "client", "internal" });
-
-    //using new server-ssl-* properties
-    if (!p2pProps) {
-      gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
-          String.valueOf(cacheServerSslenabled));
-      gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
-          cacheServerSslprotocols);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
-          cacheServerSslciphers);
-      gemFireProps.put(
-          DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
-          String.valueOf(cacheServerSslRequireAuth));
-      
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, filePathPrefix
-          + keyStore);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
-      gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, filePathPrefix
-          + trustStore);
-      gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
-    } else {
-      //using p2p ssl-* properties
-      gemFireProps.put(DistributionConfig.SSL_ENABLED_NAME,
-          String.valueOf(cacheServerSslenabled));
-      gemFireProps.put(DistributionConfig.SSL_PROTOCOLS_NAME,
-          cacheServerSslprotocols);
-      gemFireProps.put(DistributionConfig.SSL_CIPHERS_NAME,
-          cacheServerSslciphers);
-      gemFireProps.put(
-          DistributionConfig.SSL_REQUIRE_AUTHENTICATION_NAME,
-          String.valueOf(cacheServerSslRequireAuth));
-
-      gemFireProps.put("javax.net.ssl.keyStoreType", "jks");
-      gemFireProps.put("javax.net.ssl.keyStore", filePathPrefix+ keyStore);
-      gemFireProps.put("javax.net.ssl.keyStorePassword", "password");
-      gemFireProps.put("javax.net.ssl.trustStore", filePathPrefix + trustStore);
-      gemFireProps.put("javax.net.ssl.trustStorePassword", "password");
-    }
-
-    StringWriter sw = new StringWriter();
-    PrintWriter writer = new PrintWriter(sw);
-    gemFireProps.list(writer);
-    System.out.println("Starting client ds with following properties \n" + sw.getBuffer());
-    
-    ClientCacheFactory clientCacheFactory = new ClientCacheFactory(gemFireProps);
-    clientCacheFactory.addPoolServer(host, port);
-    clientCache = clientCacheFactory.create();
-    
-    ClientRegionFactory<String,String> regionFactory = clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
-    Region<String, String> region = regionFactory.create("serverRegion");  
-    assertNotNull(region);
-  }
-  
-  public void doClientRegionTest(){
-    Region<String, String> region = clientCache.getRegion("serverRegion");
-    assertEquals("servervalue",region.get("serverkey"));
-    region.put("clientkey", "clientvalue");
-    assertEquals("clientvalue",region.get("clientkey"));
-  }
-  
-  public void doServerRegionTest(){
-    Region<String, String> region = cache.getRegion("serverRegion");
-    assertEquals("servervalue",region.get("serverkey"));    
-    assertEquals("clientvalue",region.get("clientkey"));
-  }
-  
-  
-  public static void setUpServerVMTask(boolean cacheServerSslenabled, boolean p2pProps, String locators) throws Exception{
-    instance.setUpServerVM(cacheServerSslenabled,p2pProps,locators);
-  }
-  
-  public static void createServerTask() throws Exception {
-    instance.createServer();
-  }
-  
-  public static void setUpClientVMTask(String host, int port,
-      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth, boolean p2pProps, String locators, String keyStore, String trustStore)
-      throws Exception {
-    instance.setUpClientVM(host, port, cacheServerSslenabled,
-        cacheServerSslRequireAuth,p2pProps,locators, keyStore, trustStore);
-  }
-  
-  public static void doClientRegionTestTask() {
-    instance.doClientRegionTest();
-  }
-  
-  public static void doServerRegionTestTask() {
-    instance.doServerRegionTest();
-  }
-  
-  public static Object[] getCacheServerEndPointTask() {
-    Object[] array = new Object[2];
-    array[0] = instance.getCacheServerHost();
-    array[1] = instance.getCacheServerPort();
-    return array;
-  }
-  
-  public static void closeCacheTask(){
-    instance.cache.close();
-  }
-  
-  public static void closeClientCacheTask(){
-    instance.clientCache.close();
-  }
-  
-  public static void setUpSSLLocatorVMTask() throws Exception {
-    
-    Properties props = new Properties();
-    instance.locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME,"0");
-    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
-    
-    String cacheServerSslprotocols = "any";
-    String cacheServerSslciphers = "any";
-    String filePathPrefix = jtests
-    + File.separator
-    + makePath(new String[] { "com", "gemstone", "gemfire", "cache",
-        "client", "internal" });
-    
-    props.put(DistributionConfig.SSL_ENABLED_NAME,
-        String.valueOf(true));
-    props.put(DistributionConfig.SSL_PROTOCOLS_NAME,
-        cacheServerSslprotocols);
-    props.put(DistributionConfig.SSL_CIPHERS_NAME,
-        cacheServerSslciphers);
-    
-    props.put(
-        DistributionConfig.SSL_REQUIRE_AUTHENTICATION_NAME,
-        String.valueOf(true)); 
-    
-    props.put("javax.net.ssl.keyStoreType", "jks");
-    props.put("javax.net.ssl.keyStore", filePathPrefix + TRUSTED_STORE);
-    props.put("javax.net.ssl.keyStorePassword", "password");
-    props.put("javax.net.ssl.trustStore", filePathPrefix + TRUSTED_STORE);
-    props.put("javax.net.ssl.trustStorePassword", "password");
-    StringWriter sw = new StringWriter();
-    PrintWriter writer = new PrintWriter(sw);
-    props.list(writer);
-    getLogWriter().info("Starting locator ds with following properties \n" + sw.getBuffer());
-    
-    try {
-      File logFile = new File(testName + "-locator" + instance.locatorPort
-          + ".log");
-      InetAddress bindAddr = null;
-      try {
-        bindAddr = InetAddress.getLocalHost();
-      } catch (UnknownHostException uhe) {
-        fail("While resolving bind address ", uhe);
-      }
-      props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
-      instance.locator = Locator.startLocatorAndDS(instance.locatorPort, logFile, bindAddr, props);      
-    } catch (IOException ex) {
-      fail("While starting locator on port " + instance.locatorPort, ex);
-    }    
-    //instance.getSystem(props);
-    getLogWriter().info("Locator has started ...");
-  }
-  
-  
-  public static String getLocatorPortTask() {    
-    return instance.locator.getBindAddress().getHostName() + "[" + instance.locatorPort +"]";
-  }
-  
-  public static void resetSSLPropertiesTask() {
-    System.clearProperty("javax.net.ssl.keyStoreType");
-    System.clearProperty("javax.net.ssl.keyStore");
-    System.clearProperty("javax.net.ssl.keyStorePassword");
-    System.clearProperty("javax.net.ssl.trustStore");
-    System.clearProperty("javax.net.ssl.trustStorePassword");    
-  }
-
-  
-  /*
-   * Tests that cacheserver started with ssl-enabled properties accepts connection with client configured with ssl-enabled property
-   */
-  public void testCacheServerSSL_OldWay_P2P() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    VM locatorVM = host.getVM(0);
-    
-    boolean cacheServerSslenabled = true;
-    boolean cacheClientSslenabled = true;
-    boolean cacheClientSslRequireAuth = true;
-    boolean useP2pSSLProperties = true;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    //start SSL Locator
-    locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpSSLLocatorVMTask");
-    String locatorport = (String)locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "getLocatorPortTask");
-    
-    Thread.sleep(10);
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, useP2pSSLProperties, locatorport});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[8];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    params[4] = useP2pSSLProperties;
-    params[5] = locatorport;
-    params[6] = TRUSTED_STORE;
-    params[7] = TRUSTED_STORE;
-    getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-    
-    locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "resetSSLPropertiesTask");
-    
-  }
-  
-  /*
-   * Tests that cacheserver started with ssl-enabled properties  accepts connection with client configured with ssl-enabled property
-   */  
-  public void testCacheServerSSL_ServerOldWay_ClientNewWay() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    VM locatorVM = host.getVM(0);
-    
-    boolean cacheServerSslenabled = true;
-    boolean cacheClientSslenabled = true;
-    boolean cacheClientSslRequireAuth = true;
-    boolean useP2pSSLProperties = true;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    //start SSL Locator
-    locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpSSLLocatorVMTask");
-    String locatorport = (String)locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "getLocatorPortTask");
-    
-    Thread.sleep(10);
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, useP2pSSLProperties, locatorport});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[8];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    params[4] = !useP2pSSLProperties;
-    params[5] = locatorport;
-    params[6] = TRUSTED_STORE;
-    params[7] = TRUSTED_STORE;
-    getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-    
-    locatorVM.invoke(CacheServerSSLConnectionDUnit.class, "resetSSLPropertiesTask");
-    
-  }
-  
-  
-  public void testCacheServerSSL() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    
-    boolean cacheServerSslenabled = true;
-    boolean cacheClientSslenabled = true;
-    boolean cacheClientSslRequireAuth = true;
-    boolean useP2pSSLProperties = true;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, !useP2pSSLProperties,null});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[8];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    params[4] = !useP2pSSLProperties; //using new server-ssl properties not p2p ssl properties
-    params[5] = null;
-    params[6] = CLIENT_KEY_STORE;
-    params[7] = CLIENT_TRUST_STORE;
-    getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-    
-  }
-  
-  
-  public void testNonSSLClient() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    
-    boolean cacheServerSslenabled = true;
-    boolean cacheClientSslenabled = false;
-    boolean cacheClientSslRequireAuth = true;
-    boolean useP2pSSLProperties = true;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, !useP2pSSLProperties, null});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[8];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    params[4] = !useP2pSSLProperties; //using new server-ssl properties not p2p ssl properties
-    params[5] = null;
-    params[6] = TRUSTED_STORE;
-    params[7] = TRUSTED_STORE;
-    try{
-      getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);    
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-      serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-      fail("Test should fail as non-ssl client is tryting to connect to ssl configured server");
-    } catch (Exception rmiException) {
-      Throwable e = rmiException.getCause();
-      getLogWriter().info("ExceptionCause at clientVM " + e);
-      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
-        Throwable t = e.getCause();
-        getLogWriter().info("Cause is " + t);
-        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
-      } else {
-        getLogWriter().error("Unexpected exception ", e);
-        fail("Unexpected Exception...expected "
-            + AuthenticationRequiredException.class);
-      }
-    }
-  }
-  
-  /* It seems it is allowed if client chooses to not insist on authentication. wierd let this test fail??? */
-  /*
-  public void testSSLClientWithNoAuth() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    
-    boolean cacheServerSslenabled = true;
-    boolean cacheClientSslenabled = true;
-    boolean cacheClientSslRequireAuth = false;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, null});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[4];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    try{
-      getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);    
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-      serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-      fail("Test should fail as ssl client with wrong config(SslRequireAuth=false) is trying to connect to ssl configured server");
-    } catch (Exception rmiException) {
-      Throwable e = rmiException.getCause();
-      getLogWriter().info("ExceptionCause at clientVM " + e);
-      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
-        Throwable t = e.getCause();
-        getLogWriter().info("Cause is " + t);
-        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
-      } else {
-        getLogWriter().error("Unexpected exception ", e);
-        fail("Unexpected Exception...expected "
-            + AuthenticationRequiredException.class);
-      }
-    }
-  }*/
-  
-  
-  public void testSSLClientWithNonSSLServer() throws Exception {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    
-    boolean cacheServerSslenabled = false;
-    boolean cacheClientSslenabled = true;
-    boolean cacheClientSslRequireAuth = true;
-    boolean useP2pSSLProperties = true;
-    
-    getLogWriter().info("JTESTS : " + System.getProperty("JTESTS"));
-    
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled, !useP2pSSLProperties, null});
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "createServerTask");
-    
-    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnit.class, "getCacheServerEndPointTask"); 
-    String hostName = (String)array[0];
-    int port = (Integer) array[1];
-    Object params[] = new Object[8];
-    params[0] = hostName;
-    params[1] = port;
-    params[2] = cacheClientSslenabled;
-    params[3] = cacheClientSslRequireAuth;
-    params[4] = !useP2pSSLProperties;
-    params[5] = null;
-    params[6] = TRUSTED_STORE;
-    params[7] = TRUSTED_STORE;
-    try{
-      getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);    
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "setUpClientVMTask", params);
-      clientVM.invoke(CacheServerSSLConnectionDUnit.class, "doClientRegionTestTask");
-      serverVM.invoke(CacheServerSSLConnectionDUnit.class, "doServerRegionTestTask");
-      fail("Test should fail as ssl client with wrong config(SslRequireAuth=false) is tryting to connect to server configured with (SslRequireAuth=true)");
-    }catch (Exception rmiException) {
-      
-      //ignore
-      
-      /*Throwable e = rmiException.getCause();
-      getLogWriter().info("ExceptionCause at clientVM " + e);
-      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
-        Throwable t = e.getCause();
-        getLogWriter().info("Cause is " + t);
-        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
-      } else {
-        getLogWriter().error("Unexpected exception ", e);
-        fail("Unexpected Exception...expected "
-            + AuthenticationRequiredException.class);
-      }*/
-    }
-  }
-  
-  public void tearDown2() throws Exception
-  {
-    final Host host = Host.getHost(0);
-    VM serverVM = host.getVM(1);
-    VM clientVM = host.getVM(2);
-    clientVM.invoke(CacheServerSSLConnectionDUnit.class, "closeClientCacheTask");
-    serverVM.invoke(CacheServerSSLConnectionDUnit.class, "closeCacheTask");
-    super.tearDown2();
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnitTest.java
new file mode 100644
index 0000000..5102a54
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/CacheServerSSLConnectionDUnitTest.java
@@ -0,0 +1,417 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.cache.client.internal;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Properties;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.security.AuthenticationRequiredException;
+import com.gemstone.gemfire.util.test.TestUtil;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * Tests cacheserver ssl support added. See https://svn.gemstone.com/trac/gemfire/ticket/48995 for details
+ * @author tushark
+ *
+ */
+public class CacheServerSSLConnectionDUnitTest extends DistributedTestCase {
+  
+  private static final long serialVersionUID = 1L;
+  private Cache cache;
+  private CacheServer cacheServer;
+  private ClientCache clientCache;
+  private int cacheServerPort;
+  private String hostName;
+  
+  private static final String TRUSTED_STORE = "trusted.keystore";
+  private static final String CLIENT_KEY_STORE = "client.keystore";
+  private static final String CLIENT_TRUST_STORE = "client.truststore";
+  private static final String SERVER_KEY_STORE = "cacheserver.keystore";
+  private static final String SERVER_TRUST_STORE = "cacheserver.truststore";
+  
+  private static CacheServerSSLConnectionDUnitTest instance = new CacheServerSSLConnectionDUnitTest("CacheServerSSLConnectionDUnit");
+  
+  
+  public void setUp() throws Exception {
+    disconnectAllFromDS();
+    super.setUp();
+  }
+
+  public CacheServerSSLConnectionDUnitTest(String name) {
+    super(name);
+  }  
+
+  public Cache createCache(Properties props) throws Exception
+  {
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    cache = new CacheFactory(props).create();
+    if (cache == null) {
+      throw new Exception("CacheFactory.create() returned null ");
+    }
+    return cache;
+  }
+  
+  private void createServer() throws IOException{
+    cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    cacheServer = cache.addCacheServer();
+    cacheServer.setPort(cacheServerPort);
+    cacheServer.start();
+    hostName = cacheServer.getHostnameForClients();
+  }
+  
+  public int getCacheServerPort(){
+    return cacheServerPort;
+  }
+  
+  public String getCacheServerHost(){
+    return hostName;
+  }
+  
+  public void stopCacheServer(){
+    this.cacheServer.stop();
+  }
+  
+  
+  @SuppressWarnings("rawtypes")
+  public void setUpServerVM(boolean cacheServerSslenabled) throws Exception {
+    Properties gemFireProps = new Properties();
+
+    String cacheServerSslprotocols = "any";
+    String cacheServerSslciphers = "any";
+    boolean cacheServerSslRequireAuth = true;
+    gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
+        String.valueOf(cacheServerSslenabled));
+    gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
+        cacheServerSslprotocols);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
+        cacheServerSslciphers);
+    gemFireProps.put(
+        DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
+        String.valueOf(cacheServerSslRequireAuth));
+
+    String keyStore = TestUtil.getResourcePath(CacheServerSSLConnectionDUnitTest.class, SERVER_KEY_STORE);
+    String trustStore = TestUtil.getResourcePath(CacheServerSSLConnectionDUnitTest.class, SERVER_TRUST_STORE);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, keyStore);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, trustStore);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+    
+    StringWriter sw = new StringWriter();
+    PrintWriter writer = new PrintWriter(sw);
+    gemFireProps.list(writer);
+    System.out.println("Starting cacheserver ds with following properties \n" + sw);
+    createCache(gemFireProps);
+    
+    RegionFactory factory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    Region r = factory.create("serverRegion");
+    r.put("serverkey", "servervalue");
+  }
+  
+  public void setUpClientVM(String host, int port,
+      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth,
+      String keyStore, String trustStore, boolean subscription) {
+
+    Properties gemFireProps = new Properties();
+
+    String cacheServerSslprotocols = "any";
+    String cacheServerSslciphers = "any";
+
+    String keyStorePath = TestUtil.getResourcePath(CacheServerSSLConnectionDUnitTest.class, keyStore);
+    String trustStorePath = TestUtil.getResourcePath(CacheServerSSLConnectionDUnitTest.class, trustStore);
+    //using new server-ssl-* properties
+    gemFireProps.put(DistributionConfig.SERVER_SSL_ENABLED_NAME,
+        String.valueOf(cacheServerSslenabled));
+    gemFireProps.put(DistributionConfig.SERVER_SSL_PROTOCOLS_NAME,
+        cacheServerSslprotocols);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_CIPHERS_NAME,
+        cacheServerSslciphers);
+    gemFireProps.put(
+        DistributionConfig.SERVER_SSL_REQUIRE_AUTHENTICATION_NAME,
+        String.valueOf(cacheServerSslRequireAuth));
+
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_TYPE_NAME, "jks");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_NAME, keyStorePath);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_NAME, trustStorePath);
+    gemFireProps.put(DistributionConfig.SERVER_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+
+    StringWriter sw = new StringWriter();
+    PrintWriter writer = new PrintWriter(sw);
+    gemFireProps.list(writer);
+    System.out.println("Starting client ds with following properties \n" + sw.getBuffer());
+    
+    ClientCacheFactory clientCacheFactory = new ClientCacheFactory(gemFireProps);
+    clientCacheFactory.setPoolSubscriptionEnabled(subscription).addPoolServer(host, port);
+    clientCache = clientCacheFactory.create();
+    
+    ClientRegionFactory<String,String> regionFactory = clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+    Region<String, String> region = regionFactory.create("serverRegion");  
+    assertNotNull(region);
+  }
+  
+  public void doClientRegionTest(){
+    Region<String, String> region = clientCache.getRegion("serverRegion");
+    assertEquals("servervalue",region.get("serverkey"));
+    region.put("clientkey", "clientvalue");
+    assertEquals("clientvalue",region.get("clientkey"));
+  }
+  
+  public void doServerRegionTest(){
+    Region<String, String> region = cache.getRegion("serverRegion");
+    assertEquals("servervalue",region.get("serverkey"));    
+    assertEquals("clientvalue",region.get("clientkey"));
+  }
+  
+  
+  public static void setUpServerVMTask(boolean cacheServerSslenabled) throws Exception{
+    instance.setUpServerVM(cacheServerSslenabled);
+  }
+  
+  public static void createServerTask() throws Exception {
+    instance.createServer();
+  }
+  
+  public static void setUpClientVMTask(String host, int port,
+      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth, String keyStore, String trustStore)
+      throws Exception {
+    instance.setUpClientVM(host, port, cacheServerSslenabled,
+        cacheServerSslRequireAuth, keyStore, trustStore, true);
+  }
+  public static void setUpClientVMTaskNoSubscription(String host, int port,
+      boolean cacheServerSslenabled, boolean cacheServerSslRequireAuth, String keyStore, String trustStore)
+      throws Exception {
+    instance.setUpClientVM(host, port, cacheServerSslenabled,
+        cacheServerSslRequireAuth, keyStore, trustStore, false);
+  }
+  
+  public static void doClientRegionTestTask() {
+    instance.doClientRegionTest();
+  }
+  
+  public static void doServerRegionTestTask() {
+    instance.doServerRegionTest();
+  }
+  
+  public static Object[] getCacheServerEndPointTask() {
+    Object[] array = new Object[2];
+    array[0] = instance.getCacheServerHost();
+    array[1] = instance.getCacheServerPort();
+    return array;
+  }
+  
+  public static void closeCacheTask(){
+    if (instance != null && instance.cache != null) {
+      instance.cache.close();
+    }
+  }
+  
+  public static void closeClientCacheTask(){
+    if (instance != null && instance.clientCache != null) {
+      instance.clientCache.close();
+    }
+  }
+  
+  public void testCacheServerSSL() throws Exception {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+    
+    boolean cacheServerSslenabled = true;
+    boolean cacheClientSslenabled = true;
+    boolean cacheClientSslRequireAuth = true;
+    
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled});
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "createServerTask");
+    
+    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "getCacheServerEndPointTask"); 
+    String hostName = (String)array[0];
+    int port = (Integer) array[1];
+    Object params[] = new Object[6];
+    params[0] = hostName;
+    params[1] = port;
+    params[2] = cacheClientSslenabled;
+    params[3] = cacheClientSslRequireAuth;
+    params[4] = CLIENT_KEY_STORE;
+    params[5] = CLIENT_TRUST_STORE;
+    //getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
+    clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpClientVMTask", params);
+    clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doClientRegionTestTask");
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doServerRegionTestTask");
+    
+  }
+  
+  
+  public void testNonSSLClient() throws Exception {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+    
+    boolean cacheServerSslenabled = true;
+    boolean cacheClientSslenabled = false;
+    boolean cacheClientSslRequireAuth = true;
+    
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled});
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "createServerTask");
+    
+    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "getCacheServerEndPointTask"); 
+    String hostName = (String)array[0];
+    int port = (Integer) array[1];
+    Object params[] = new Object[6];
+    params[0] = hostName;
+    params[1] = port;
+    params[2] = cacheClientSslenabled;
+    params[3] = cacheClientSslRequireAuth;
+    params[4] = TRUSTED_STORE;
+    params[5] = TRUSTED_STORE;
+    ExpectedException expect = addExpectedException("javax.net.ssl.SSLException", serverVM);
+    ExpectedException expect2 = addExpectedException("IOException", serverVM);
+    try{
+      //getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);    
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpClientVMTaskNoSubscription", params);
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doClientRegionTestTask");
+      serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doServerRegionTestTask");
+      fail("Test should fail as non-ssl client is trying to connect to ssl configured server");
+    } catch (Exception rmiException) {
+      Throwable e = rmiException.getCause();
+      //getLogWriter().info("ExceptionCause at clientVM " + e);
+      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
+        Throwable t = e.getCause();
+        //getLogWriter().info("Cause is " + t);
+        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
+      } else {
+        //getLogWriter().error("Unexpected exception ", e);
+        fail("Unexpected Exception: " + e + " expected: "
+            + AuthenticationRequiredException.class);
+      }
+    } finally {
+      expect.remove();
+      expect2.remove();
+    }
+  }
+  
+  public void testSSLClientWithNoAuth() throws Exception {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+
+    boolean cacheServerSslenabled = true;
+    boolean cacheClientSslenabled = true;
+    boolean cacheClientSslRequireAuth = false;
+
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled});
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "createServerTask");
+
+    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "getCacheServerEndPointTask"); 
+    String hostName = (String)array[0];
+    int port = (Integer) array[1];
+    Object params[] = new Object[6];
+    params[0] = hostName;
+    params[1] = port;
+    params[2] = cacheClientSslenabled;
+    params[3] = cacheClientSslRequireAuth;
+    params[4] = CLIENT_KEY_STORE;
+    params[5] = CLIENT_TRUST_STORE;
+    //getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);
+    try {
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpClientVMTask", params);
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doClientRegionTestTask");
+      serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doServerRegionTestTask");
+    } catch (Exception rmiException) {
+      Throwable e = rmiException.getCause();
+      //getLogWriter().info("ExceptionCause at clientVM " + e);
+      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
+        Throwable t = e.getCause();
+        //getLogWriter().info("Cause is " + t);
+        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
+      } else {
+        //getLogWriter().error("Unexpected exception ", e);
+        fail("Unexpected Exception...expected "
+            + AuthenticationRequiredException.class);
+      }
+    }
+  }
+  
+  public void testSSLClientWithNonSSLServer() throws Exception {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+    
+    boolean cacheServerSslenabled = false;
+    boolean cacheClientSslenabled = true;
+    boolean cacheClientSslRequireAuth = true;
+    
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpServerVMTask", new Object[]{cacheServerSslenabled});
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "createServerTask");
+    
+    Object array[] = (Object[])serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "getCacheServerEndPointTask"); 
+    String hostName = (String)array[0];
+    int port = (Integer) array[1];
+    Object params[] = new Object[6];
+    params[0] = hostName;
+    params[1] = port;
+    params[2] = cacheClientSslenabled;
+    params[3] = cacheClientSslRequireAuth;
+    params[4] = TRUSTED_STORE;
+    params[5] = TRUSTED_STORE;
+    ExpectedException expect = addExpectedException("javax.net.ssl.SSLHandshakeException", serverVM);
+    try{
+      //getLogWriter().info("Starting client with server endpoint " + hostName + ":" + port);    
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "setUpClientVMTask", params);
+      clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doClientRegionTestTask");
+      serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "doServerRegionTestTask");
+      fail("Test should fail as ssl client with ssl enabled is trying to connect to server with ssl disabled");
+    }catch (Exception rmiException) {
+      
+      //ignore
+      
+      /*Throwable e = rmiException.getCause();
+      getLogWriter().info("ExceptionCause at clientVM " + e);
+      if (e instanceof com.gemstone.gemfire.cache.client.ServerOperationException) {
+        Throwable t = e.getCause();
+        getLogWriter().info("Cause is " + t);
+        assertTrue(t instanceof com.gemstone.gemfire.security.AuthenticationRequiredException);
+      } else {
+        getLogWriter().error("Unexpected exception ", e);
+        fail("Unexpected Exception...expected "
+            + AuthenticationRequiredException.class);
+      }*/
+    } finally {
+      expect.remove();
+    }
+  }
+  
+  public void tearDown2() throws Exception
+  {
+    final Host host = Host.getHost(0);
+    VM serverVM = host.getVM(1);
+    VM clientVM = host.getVM(2);
+    clientVM.invoke(CacheServerSSLConnectionDUnitTest.class, "closeClientCacheTask");
+    serverVM.invoke(CacheServerSSLConnectionDUnitTest.class, "closeCacheTask");
+    super.tearDown2();
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.cer
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.cer b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.cer
new file mode 100644
index 0000000..3c6645e
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.cer differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.keystore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.keystore
new file mode 100644
index 0000000..adbea7b
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.truststore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.truststore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.truststore
new file mode 100644
index 0000000..3920963
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/cacheserver.truststore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.cer
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.cer b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.cer
new file mode 100644
index 0000000..35b64af
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.cer differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.keystore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.keystore
new file mode 100644
index 0000000..38a315d
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.truststore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.truststore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.truststore
new file mode 100644
index 0000000..d598b86
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/client.truststore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/trusted.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/trusted.keystore b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/trusted.keystore
new file mode 100644
index 0000000..e8fd92e
Binary files /dev/null and b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/internal/trusted.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/GossipClient.java
----------------------------------------------------------------------
diff --git a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/GossipClient.java b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/GossipClient.java
index 304bd52..74455fa 100644
--- a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/GossipClient.java
+++ b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/GossipClient.java
@@ -283,8 +283,7 @@ public class GossipClient  {
                 if(log.isTraceEnabled())
                     log.trace("REGISTER_REQ --> " + entry.getIpAddress() + ':' + entry.getPort());
                 //sock=new Socket(entry.getIpAddress(), entry.getPort());
-                // GemStoneAddition - use SocketCreator
-                if (JChannel.getGfFunctions().getSockCreator().isHostReachable(entry.getIpAddress())) {
+                {
 
                   // Get GemFire version from IPAddress first
                   _getVersionForAddress(entry);
@@ -550,8 +549,7 @@ public class GossipClient  {
       if (log.isTraceEnabled())
         log.trace("GEMFIRE_VERSION --> " + entry.getIpAddress() + ':'
                 + +entry.getPort());
-      if (JChannel.getGfFunctions().getSockCreator()
-              .isHostReachable(entry.getIpAddress())) {
+      {
   
         ConnectTimerTask timeoutTask = new ConnectTimerTask(); // GemStoneAddition
         Socket socket = JChannel.getGfFunctions().getSockCreator().connect(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/SockCreatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/SockCreatorImpl.java b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/SockCreatorImpl.java
index 946507a..27ca179 100755
--- a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/SockCreatorImpl.java
+++ b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/stack/SockCreatorImpl.java
@@ -32,11 +32,6 @@ public class SockCreatorImpl implements SockCreator {
   }
 
   @Override
-  public boolean isHostReachable(InetAddress ipAddress) {
-    return true;
-  }
-
-  @Override
   public Socket connect(InetAddress ipAddress, int port, int i,
       ConnectionWatcher watcher, boolean clientToServer) throws IOException {
     Socket socket = new Socket();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbe8c37a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/util/SockCreator.java
----------------------------------------------------------------------
diff --git a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/util/SockCreator.java b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/util/SockCreator.java
index 3ff053d..8b063f3 100755
--- a/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/util/SockCreator.java
+++ b/gemfire-jgroups/src/main/java/com/gemstone/org/jgroups/util/SockCreator.java
@@ -4,6 +4,12 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.Socket;
 
+/** This interface does not define how to create socks
+ * but defines a factory for creating sockets.
+ * One of its implementations had already used
+ * the name "SocketCreator" at the time the interface
+ * was created.
+ */
 public interface SockCreator {
 
   boolean useSSL();
@@ -11,8 +17,6 @@ public interface SockCreator {
   Socket connect(InetAddress ipAddress, int port, int connectTimeout,
       ConnectionWatcher watcher, boolean clientToServer, int timeout, boolean useSSL) throws IOException;
 
-  boolean isHostReachable(InetAddress ipAddress);
-
   Socket connect(InetAddress ipAddress, int port, int timeout,
       ConnectionWatcher watcher, boolean clientToServer
       ) throws IOException;