You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2016/02/17 16:55:30 UTC

[1/2] incubator-geode git commit: Fixing a suspect NPE string

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 23edf7e13 -> 197555616


Fixing a suspect NPE string

Once in a while a test fails with an NPE in method
Connection.scheduleAckTimeouts() using the variable ackConnectionGroup.
This variable is set to null in setInUse().


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

Branch: refs/heads/develop
Commit: d632bfb73155673b2ee06be0e3657033901e114d
Parents: 23edf7e
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Feb 17 07:51:56 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Feb 17 07:51:56 2016 -0800

----------------------------------------------------------------------
 .../main/java/com/gemstone/gemfire/internal/tcp/Connection.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d632bfb7/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
index 74660da..988ca33 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -2673,11 +2673,12 @@ public class Connection implements Runnable {
               }
             }
           }
-          if (sentAlert) {
+          List group = ackConnectionGroup;
+          if (sentAlert && group != null) {
             // since transmission and ack-receipt are performed serially, we don't
             // want to complain about all receivers out just because one was slow.  We therefore reset
             // the time stamps and give others more time
-            for (Iterator it=ackConnectionGroup.iterator(); it.hasNext(); ) {
+            for (Iterator it=group.iterator(); it.hasNext(); ) {
               Connection con = (Connection)it.next();
               if (con != Connection.this) {
                 con.transmissionStartTime += con.ackSATimeout;


[2/2] incubator-geode git commit: GEODE-965: redundant cache server created during auto-reconnect

Posted by bs...@apache.org.
GEODE-965: redundant cache server created during auto-reconnect

If cluster configuration is enabled but a cache.xml is actually being
used to construct the cache then it is possible that after an auto-
reconnect a redundant CacheServer will be created.

There is an assumption made in writing the auto-reconnect code that if
cluster-configuration is enabled the cache will not be constructed
using a cache.xml file.

This change-set marks CacheServers that are created by gfsh or so
called "default" servers created by cache.xml processing so that
auto-reconnect will know whether they should be handled specially.


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

Branch: refs/heads/develop
Commit: 197555616a479a5dde5c8939bdb52efd751cc901
Parents: d632bfb
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Feb 17 07:54:17 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Feb 17 07:54:17 2016 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/ServerLauncher.java     |  3 ++
 .../internal/InternalDistributedSystem.java     | 53 ++++++++++++++------
 .../gms/mgr/GMSMembershipManager.java           |  9 ++--
 .../gemfire/internal/cache/CacheServerImpl.java | 11 ++++
 .../internal/cache/CacheServerLauncher.java     |  4 +-
 .../internal/cache/GemFireCacheImpl.java        |  0
 .../cache/tier/sockets/CacheServerHelper.java   | 46 +++++++++--------
 .../internal/cache/xmlcache/CacheCreation.java  |  9 +++-
 .../ReconnectedCacheServerDUnitTest.java        | 27 ++++++++++
 9 files changed, 121 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
old mode 100644
new mode 100755
index c991cc1..4a2ce52
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
@@ -53,6 +53,7 @@ import com.gemstone.gemfire.internal.cache.CacheConfig;
 import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.ObjectUtils;
 import com.gemstone.gemfire.internal.lang.StringUtils;
@@ -948,6 +949,8 @@ public final class ServerLauncher extends AbstractLauncher<String> {
       if (getHostNameForClients() != null) {
         cacheServer.setHostnameForClients(getHostNameForClients());
       }
+      
+      CacheServerHelper.setIsDefaultServer(cacheServer);
 
       cacheServer.start();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index 276efec..a193699 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -56,6 +56,7 @@ import com.gemstone.gemfire.admin.AlertLevel;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.execute.internal.FunctionServiceManager;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
@@ -2813,21 +2814,9 @@ public class InternalDistributedSystem
               config.setCacheXMLDescription(cacheXML);
             }
             cache = GemFireCacheImpl.create(this.reconnectDS, config);
-            if (cacheServerCreation != null) {
-              for (CacheServerCreation bridge: cacheServerCreation) {
-                CacheServerImpl impl = (CacheServerImpl)cache.addCacheServer();
-                impl.configureFrom(bridge);
-                try {
-                  if (!impl.isRunning()) {
-                    impl.start();
-                  }
-                } catch (IOException ex) {
-                  throw new GemFireIOException(
-                      LocalizedStrings.CacheCreation_WHILE_STARTING_CACHE_SERVER_0
-                          .toLocalizedString(impl), ex);
-                }
-              }
-            }
+            
+            createAndStartCacheServers(cacheServerCreation, cache);
+
             if (cache.getCachePerfStats().getReliableRegionsMissing() == 0){
               reconnectAttemptCounter = 0;
               logger.info("Reconnected properly");
@@ -2851,6 +2840,40 @@ public class InternalDistributedSystem
     }
   }
 
+
+  /**
+   * after an auto-reconnect we may need to recreate a cache server
+   * and start it
+   */
+  public void createAndStartCacheServers(
+      List<CacheServerCreation> cacheServerCreation, GemFireCacheImpl cache) {
+
+    List<CacheServer> servers = cache.getCacheServers();
+    
+    // if there used to be a cache server but now there isn't one we need
+    // to recreate it.
+    if (servers.isEmpty() && cacheServerCreation != null) {
+      for (CacheServerCreation bridge: cacheServerCreation) {
+        CacheServerImpl impl = (CacheServerImpl)cache.addCacheServer();
+        impl.configureFrom(bridge);
+      }
+    }
+    
+    servers = cache.getCacheServers();
+    for (CacheServer server: servers) {
+      try {
+        if (!server.isRunning()) {
+          server.start();
+        }
+      } catch (IOException ex) {
+        throw new GemFireIOException(
+            LocalizedStrings.CacheCreation_WHILE_STARTING_CACHE_SERVER_0
+                .toLocalizedString(server), ex);
+      }
+    }
+    
+  }
+
   /**
    * Validates that the configuration provided is the same as the configuration for this
    * InternalDistributedSystem

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index d75b28d..edfee10 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -84,6 +84,7 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
@@ -1615,9 +1616,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
         // we need to retain a cache-server description if this JVM was started by gfsh
         List<CacheServerCreation> list = new ArrayList<CacheServerCreation>(cache.getCacheServers().size());
         for (Iterator it = cache.getCacheServers().iterator(); it.hasNext(); ) {
-          CacheServer cs = (CacheServer)it.next();
-          CacheServerCreation bsc = new CacheServerCreation(cache, cs);
-          list.add(bsc);
+          CacheServerImpl cs = (CacheServerImpl)it.next();
+          if (cs.isDefaultServer()) {
+            CacheServerCreation bsc = new CacheServerCreation(cache, cs);
+            list.add(bsc);
+          }
         }
         cache.getCacheConfig().setCacheServerCreation(list);
         logger.info("CacheServer configuration saved");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
index cd4f3e4..000120b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
@@ -114,6 +114,9 @@ public class CacheServerImpl
   
   private List<GatewayTransportFilter> gatewayTransportFilters = Collections.EMPTY_LIST;
   
+  /** is this a server created by a launcher as opposed to by an application or XML? */
+  private boolean isDefaultServer;
+  
   /**
    * Needed because this guy is an advisee
    * @since 5.7
@@ -261,6 +264,14 @@ public class CacheServerImpl
     return this.clientSubscriptionConfig;
   }
 
+  public boolean isDefaultServer() {
+    return isDefaultServer;
+  }
+
+  public void setIsDefaultServer() {
+    this.isDefaultServer = true;
+  }
+
   /**
    * Sets the configuration of <b>this</b> <code>CacheServer</code> based on
    * the configuration of <b>another</b> <code>CacheServer</code>.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
index ac198e9..c5de84b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
@@ -48,8 +48,8 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.PureJavaMode;
 import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.PureLogWriter;
 import com.gemstone.gemfire.internal.process.StartupStatus;
 import com.gemstone.gemfire.internal.process.StartupStatusListener;
 import com.gemstone.gemfire.internal.util.IOUtils;
@@ -876,6 +876,8 @@ public class CacheServerLauncher  {
     if ((disable == null || !disable) && cache.getCacheServers().size() == 0) {
       // Create and add a cache server
       CacheServer server = cache.addCacheServer();
+      
+      CacheServerHelper.setIsDefaultServer(server);
 
       // Set its port if necessary
       Integer serverPort = CacheServerLauncher.getServerPort();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerHelper.java
index b120b57..b0b0be1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerHelper.java
@@ -17,8 +17,10 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 
@@ -40,41 +42,48 @@ import java.util.zip.GZIPOutputStream;
  * @author Barry Oglesby
  * @since 3.5
  */
-public class CacheServerHelper
-  {
-  public static byte[] serialize(Object obj) throws IOException
-  {
+public class CacheServerHelper {
+  
+  public static void setIsDefaultServer(CacheServer server) {
+    if (server instanceof CacheServerImpl) {
+      ((CacheServerImpl)server).setIsDefaultServer();
+    }
+  }
+  
+  public static boolean isDefaultServer(CacheServer server) {
+    if ( !(server instanceof CacheServerImpl) ) {
+      return false;
+    }
+    return ((CacheServerImpl)server).isDefaultServer();
+  }
+  
+  public static byte[] serialize(Object obj) throws IOException {
     return serialize(obj, false);
   }
 
-  public static byte[] serialize(Object obj, boolean zipObject) throws IOException
-  {
+  public static byte[] serialize(Object obj, boolean zipObject) throws IOException {
     return zipObject
       ? zip(obj)
       : BlobHelper.serializeToBlob(obj);
   }
 
-  public static Object deserialize(byte[] blob) throws IOException, ClassNotFoundException
-  {
+  public static Object deserialize(byte[] blob) throws IOException, ClassNotFoundException {
     return deserialize(blob, false);
   }
 
-  public static Object deserialize(byte[] blob, boolean unzipObject) throws IOException, ClassNotFoundException
-  {
+  public static Object deserialize(byte[] blob, boolean unzipObject) throws IOException, ClassNotFoundException {
     return unzipObject
       ? unzip(blob)
       : BlobHelper.deserializeBlob(blob);
   }
 
-  public static Object deserialize(byte[] blob, Version version, boolean unzipObject) throws IOException, ClassNotFoundException
-  {
+  public static Object deserialize(byte[] blob, Version version, boolean unzipObject) throws IOException, ClassNotFoundException {
     return unzipObject
       ? unzip(blob)
       : BlobHelper.deserializeBlob(blob, version, null);
   }
   
-  public static byte[] zip(Object obj) throws IOException
-  {
+  public static byte[] zip(Object obj) throws IOException {
 //logger.info("CacheServerHelper: Zipping object to blob: " + obj);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     GZIPOutputStream gz = new GZIPOutputStream(baos);
@@ -87,8 +96,7 @@ public class CacheServerHelper
     return blob;
   }
 
-  public static Object unzip(byte[] blob) throws IOException, ClassNotFoundException
-  {
+  public static Object unzip(byte[] blob) throws IOException, ClassNotFoundException {
 //logger.info("CacheServerHelper: Unzipping blob to object: " + blob);
     ByteArrayInputStream bais = new ByteArrayInputStream(blob);
     GZIPInputStream gs = new GZIPInputStream(bais);
@@ -107,8 +115,7 @@ public class CacheServerHelper
    * @param s
    * @return byte[]
    */
-  public static byte[] toUTF(String s)
-  {
+  public static byte[] toUTF(String s) {
     HeapDataOutputStream hdos = new HeapDataOutputStream(s);
     return hdos.toByteArray();
   }
@@ -118,8 +125,7 @@ public class CacheServerHelper
    * @param bytearr
    * @return String 
    */
-  public static String fromUTF(byte[] bytearr)
-  {
+  public static String fromUTF(byte[] bytearr) {
     int utflen = bytearr.length;
     int c, char2, char3;
     int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
old mode 100644
new mode 100755
index f7063bc..9456f9a
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
@@ -588,7 +588,8 @@ public class CacheCreation implements InternalCache {
    * Also adds a default server to the param declarativeCacheServers if a serverPort is specified.
    */
   protected void startCacheServers(List declarativeCacheServers, Cache cache, Integer serverPort, String serverBindAdd, Boolean disableDefaultServer) {
-
+    CacheServerCreation defaultServer = null;
+    
     if (declarativeCacheServers.size() > 1
         && (serverPort != null || serverBindAdd != null)) {
       throw new RuntimeException(
@@ -611,7 +612,8 @@ public class CacheCreation implements InternalCache {
       }
       
       if (!existingCacheServer) {
-        declarativeCacheServers.add(new CacheServerCreation((GemFireCacheImpl)cache, false));
+        defaultServer = new CacheServerCreation((GemFireCacheImpl)cache, false);
+        declarativeCacheServers.add(defaultServer);
       }
     }
     
@@ -634,6 +636,9 @@ public class CacheCreation implements InternalCache {
 
       CacheServerImpl impl = (CacheServerImpl)cache.addCacheServer();
       impl.configureFrom(declaredCacheServer);
+      if (declaredCacheServer == defaultServer) {
+        impl.setIsDefaultServer();
+      }
 
       if (serverPort != null && serverPort != CacheServer.DEFAULT_PORT) {
         impl.setPort(serverPort);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/19755561/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectedCacheServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectedCacheServerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectedCacheServerDUnitTest.java
index 2b97a9a..6daf213 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectedCacheServerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectedCacheServerDUnitTest.java
@@ -16,8 +16,10 @@
  */
 package com.gemstone.gemfire.cache30;
 
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 
 
@@ -65,4 +67,29 @@ public class ReconnectedCacheServerDUnitTest extends CacheTestCase {
     assertNotNull(gc.getCacheConfig().getCacheServerCreation());
   }
 
+  public void testDefaultCacheServerNotCreatedOnReconnect() {
+    
+    assertFalse(Boolean.getBoolean("gemfire.autoReconnect-useCacheXMLFile"));
+    
+    GemFireCacheImpl gc = (GemFireCacheImpl)cache;
+
+    // fool the system into thinking cluster-config is being used
+    GMSMembershipManager mgr = (GMSMembershipManager)MembershipManagerHelper
+        .getMembershipManager(gc.getDistributedSystem());
+    final boolean sharedConfigEnabled = true;
+    mgr.saveCacheXmlForReconnect(sharedConfigEnabled);
+
+    // the cache server config should now be stored in the cache's config
+    assertFalse(gc.getCacheServers().isEmpty());
+    int numServers = gc.getCacheServers().size();
+
+    assertNotNull(gc.getCacheConfig().getCacheServerCreation());
+
+    InternalDistributedSystem system = gc.getDistributedSystem();
+    system.createAndStartCacheServers(gc.getCacheConfig().getCacheServerCreation(), gc);
+
+    assertEquals("found these cache servers:" + gc.getCacheServers(),
+        numServers, gc.getCacheServers().size());
+      
+  }
 }