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/13 19:17:42 UTC

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

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-397 [created] 7beba45f1


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.


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

Branch: refs/heads/feature/GEODE-397
Commit: 18c0eec8f44b1a1edb39cccb139217f5fe8d0142
Parents: 17fdf57
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 15:31:49 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 13 10:00:56 2015 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/AcceptorImpl.java        | 36 ++++----------------
 1 file changed, 6 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/18c0eec8/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);


[2/2] 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 were used
to call isHostReachable which is a method that was deadcoded.
So all those calls have been commented out. One of call of
getDefaultInstance was deleted (in ConnectionTable) because it
was never used.


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

Branch: refs/heads/feature/GEODE-397
Commit: 7beba45f1f0bf126fd5e6750190dbdb5733d9c70
Parents: 18c0eec
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 15:51:52 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 13 10:01:12 2015 -0700

----------------------------------------------------------------------
 .../client/internal/ConnectionFactoryImpl.java  | 24 ++++++-
 .../cache/client/internal/ConnectionImpl.java   | 37 +++-------
 .../gemfire/internal/SocketCreator.java         | 76 ++++++++++----------
 .../cache/tier/sockets/CacheClientUpdater.java  | 13 ++--
 .../gemfire/internal/tcp/ConnectionTable.java   |  1 -
 .../org/jgroups/stack/GossipClient.java         |  9 ++-
 6 files changed, 85 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7beba45f/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/7beba45f/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..59c6c17 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,13 @@ 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());
-    }
+// commented out because isHostReachable always returns true
+//    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,10 +126,11 @@ 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;
-      }
+// commented out because isHostReachable always returns true
+//      SocketCreator sc = SocketCreator.getDefaultInstance();
+//      if (!sc.isHostReachable(this.theSocket.getInetAddress())) {
+//        return;
+//      }
 
       boolean sendCloseMsg = !TEST_DURABLE_CLIENT_CRASH;
       if (sendCloseMsg) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7beba45f/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..1469c7c 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,44 +1028,46 @@ 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) {
+
+// commenting out isHostReachable since it was turned into deadcode
+// a long time ago because it could block.
+//  /** 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 (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;
-  }
+//    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 {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7beba45f/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..1e1633d 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,12 @@ 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());
-      }
+// commented out because isHostReachable always returns true
+//      if (!socketCreator.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/7beba45f/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/7beba45f/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..a6985e6 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
@@ -284,7 +284,9 @@ public class GossipClient  {
                     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())) {
+                // commented out because isHostReachable always returns true
+                //if (JChannel.getGfFunctions().getSockCreator().isHostReachable(entry.getIpAddress()))
+                {
 
                   // Get GemFire version from IPAddress first
                   _getVersionForAddress(entry);
@@ -550,8 +552,9 @@ public class GossipClient  {
       if (log.isTraceEnabled())
         log.trace("GEMFIRE_VERSION --> " + entry.getIpAddress() + ':'
                 + +entry.getPort());
-      if (JChannel.getGfFunctions().getSockCreator()
-              .isHostReachable(entry.getIpAddress())) {
+      // commented out because isHostReachable always returns true
+      //if (JChannel.getGfFunctions().getSockCreator().isHostReachable(entry.getIpAddress()))
+      {
   
         ConnectTimerTask timeoutTask = new ConnectTimerTask(); // GemStoneAddition
         Socket socket = JChannel.getGfFunctions().getSockCreator().connect(