You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/03/15 19:39:06 UTC

[28/45] incubator-geode git commit: GEODE-1025: Replacing direct dependency on spring data gemfire with an SPI

GEODE-1025: Replacing direct dependency on spring data gemfire with an SPI

Removing the direct use of Spring Data Gemfire from within the geode
code. Replacing it with a SPI that allows the user to override the
behavior of ServerLauncher.start with their own implementation that
creates a cache.

Spring Data Gemfire can provide an implementation that bootstraps the
cache using spring, if spring-xml-location is specified.

Users should implement ServerLauncherCacheProvider. The cache will be
created by the first provider that returns a non null cache. If no
providers create a cache, the cache will be created by the
ServerLauncher itself.

It would be nice to move the spring-xml-location parameter itself into
the interface, but this is hard to do because gfsh determines the
parameters to start-server by looking at the arguments in the command
class. So I left spring-xml-location alone.


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

Branch: refs/heads/feature/GEODE-1050
Commit: 7734781f9ada7a7dc660cdcea46126b6129c0e70
Parents: 87b4eba
Author: Dan Smith <up...@apache.org>
Authored: Mon Feb 29 18:24:42 2016 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Fri Mar 11 11:02:08 2016 -0800

----------------------------------------------------------------------
 geode-assembly/build.gradle                     | 17 ++--
 .../LauncherLifecycleCommandsDUnitTest.java     | 10 +-
 geode-core/build.gradle                         |  2 -
 .../gemfire/distributed/ServerLauncher.java     | 56 ++++-------
 .../ServerLauncherCacheProvider.java            | 34 +++++++
 .../DefaultServerLauncherCacheProvider.java     | 57 +++++++++++
 .../gemfire/distributed/LauncherTestSuite.java  |  2 +-
 .../MockServerLauncherCacheProvider.java        | 42 +++++++++
 .../ServerLauncherWithProviderJUnitTest.java    | 92 ++++++++++++++++++
 .../ServerLauncherWithSpringJUnitTest.java      | 99 --------------------
 ...fire.distributed.ServerLauncherCacheProvider |  1 +
 .../resources/spring/spring-gemfire-context.xml | 42 ---------
 gradle/rat.gradle                               |  1 +
 13 files changed, 263 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index 14febbd..22a5561 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -41,8 +41,16 @@ gradle.taskGraph.whenReady( { graph ->
   }
 })
 
+configurations {
+  bundled {
+    description 'A dependency that is shipped with geode, but is not required to compile'
+  }
+}
+
 dependencies {
   provided project(':geode-core')
+  bundled 'org.springframework.data:spring-data-commons:' + project.'spring-data-commons.version'
+  bundled 'org.springframework.data:spring-data-gemfire:' + project.'spring-data-gemfire.version'
   
   archives project(':geode-common')  
   archives project(':geode-json')  
@@ -284,12 +292,9 @@ distributions {
         from project(":geode-cq").configurations.archives.allArtifacts.files
 
         from project(":geode-core").configurations.runtime
-        // Copying from provided configuration is only for supporting Spring Data GemFire.
-        // If there are more dependencies added to provided configuration, this will need
-        // to change
-        from (project(":geode-core").configurations.provided) {
-          include 'spring-data-gemfire-*'
-        }
+
+        from configurations.bundled
+
         from project(":geode-core").configurations.archives.allArtifacts.files
         from project(":geode-core").webJar
         from project(":geode-core").raJar

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index 7c6929c..f0eaaac 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -48,6 +48,7 @@ import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 import org.junit.FixMethodOrder;
+import org.junit.Ignore;
 import org.junit.runners.MethodSorters;
 
 import javax.management.MBeanServerConnection;
@@ -804,7 +805,8 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
   }
 
-  public void test013StartServerWithSpring() {
+  @Ignore("Disabled until GEODE-1025, SGF-476 are resolved")
+  public void IGNORE_test013StartServerWithSpring() {
     String pathname = (getClass().getSimpleName() + "_" + getTestMethodName());
     File workingDirectory = new File(pathname);
 
@@ -832,9 +834,13 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     assertNotNull(springGemFireServer);
 
     ServerState serverState = springGemFireServer.status();
-
+    
     assertNotNull(serverState);
     assertEquals(Status.ONLINE, serverState.getStatus());
+    
+    //Ensure the member name is what is set through spring
+    String logFile = serverState.getLogFile();
+    assertTrue("Log file name was not configured from spring context: " + logFile, logFile.contains("spring_server.log"));
 
     // Now that the GemFire Server bootstrapped with Spring started up OK, stop it!
     stopServer(springGemFireServer.getWorkingDirectory());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index a72d0bf..6ecedef 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -74,8 +74,6 @@ dependencies {
   compile 'org.fusesource.jansi:jansi:' + project.'jansi.version'
   compile 'org.slf4j:slf4j-api:' + project.'slf4j-api.version'
   runtime 'org.springframework:spring-aop:' + project.'springframework.version'
-  compile 'org.springframework.data:spring-data-commons:' + project.'spring-data-commons.version'
-  provided 'org.springframework.data:spring-data-gemfire:' + project.'spring-data-gemfire.version'
   compile 'org.springframework:spring-beans:' + project.'springframework.version' //only used in AbstractCommandsController
   compile 'org.springframework:spring-context:' + project.'springframework.version'
   compile 'org.springframework:spring-context-support:' + project.'springframework.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
index 4f43370..9c62346 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
@@ -29,6 +29,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.ServiceLoader;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -43,7 +44,7 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.internal.DefaultServerLauncherCacheProvider;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.GemFireVersion;
@@ -72,7 +73,6 @@ import com.gemstone.gemfire.internal.process.ProcessLauncherContext;
 import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.StartupStatusListener;
 import com.gemstone.gemfire.internal.process.UnableToControlProcessException;
-import com.gemstone.gemfire.internal.util.CollectionUtils;
 import com.gemstone.gemfire.internal.util.IOUtils;
 import com.gemstone.gemfire.lang.AttachAPINotFoundException;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
@@ -85,8 +85,6 @@ import joptsimple.OptionException;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
 
-import org.springframework.data.gemfire.support.SpringContextBootstrappingInitializer;
-
 /**
  * The ServerLauncher class is a launcher class with main method to start a GemFire Server (implying a GemFire Cache
  * Server process).
@@ -98,7 +96,7 @@ import org.springframework.data.gemfire.support.SpringContextBootstrappingInitia
  * @since 7.0
  */
 @SuppressWarnings({ "unused" })
-public final class ServerLauncher extends AbstractLauncher<String> {
+public class ServerLauncher extends AbstractLauncher<String> {
 
   /**
    * @deprecated This is specific to the internal implementation and may go away in a future release.
@@ -148,6 +146,8 @@ public final class ServerLauncher extends AbstractLauncher<String> {
 
   private static final AtomicReference<ServerLauncher> INSTANCE = new AtomicReference<>();
 
+  private static final ServerLauncherCacheProvider DEFAULT_CACHE_PROVIDER = new DefaultServerLauncherCacheProvider();
+
   private volatile transient boolean debug;
 
   private final transient ControlNotificationHandler controlHandler;
@@ -313,7 +313,7 @@ public final class ServerLauncher extends AbstractLauncher<String> {
    * @return a CacheConfig object with additional GemFire Cache configuration meta-data used on startup to configure
    * the Cache.
    */
-  final CacheConfig getCacheConfig() {
+  public final CacheConfig getCacheConfig() {
     final CacheConfig copy = new CacheConfig();
     copy.setDeclarativeConfig(this.cacheConfig);
     return copy;
@@ -716,7 +716,7 @@ public final class ServerLauncher extends AbstractLauncher<String> {
 
         try {
           final Properties gemfireProperties = getDistributedSystemProperties(getProperties());
-          this.cache = (isSpringXmlLocationSpecified() ? startWithSpring() : startWithGemFireApi(gemfireProperties));
+          this.cache = createCache(gemfireProperties);
           
           //Set the resource manager options
           if (this.criticalHeapPercentage != null) {
@@ -787,40 +787,16 @@ public final class ServerLauncher extends AbstractLauncher<String> {
     }
   }
 
-  private Cache startWithSpring() {
-    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME, getMemberName());
-
-    new SpringContextBootstrappingInitializer().init(CollectionUtils.createProperties(Collections.singletonMap(
-      SpringContextBootstrappingInitializer.CONTEXT_CONFIG_LOCATIONS_PARAMETER, getSpringXmlLocation())));
-
-    return SpringContextBootstrappingInitializer.getApplicationContext().getBean(Cache.class);
-  }
-
-  private Cache startWithGemFireApi(final Properties gemfireProperties ) {
-    final CacheConfig cacheConfig = getCacheConfig();
-    final CacheFactory cacheFactory = new CacheFactory(gemfireProperties);
-
-    if (cacheConfig.pdxPersistentUserSet) {
-      cacheFactory.setPdxPersistent(cacheConfig.isPdxPersistent());
-    }
-
-    if (cacheConfig.pdxDiskStoreUserSet) {
-      cacheFactory.setPdxDiskStore(cacheConfig.getPdxDiskStore());
-    }
-
-    if (cacheConfig.pdxIgnoreUnreadFieldsUserSet) {
-      cacheFactory.setPdxIgnoreUnreadFields(cacheConfig.getPdxIgnoreUnreadFields());
-    }
-
-    if (cacheConfig.pdxReadSerializedUserSet) {
-      cacheFactory.setPdxReadSerialized(cacheConfig.isPdxReadSerialized());
-    }
-
-    if (cacheConfig.pdxSerializerUserSet) {
-      cacheFactory.setPdxSerializer(cacheConfig.getPdxSerializer());
+  private Cache createCache(Properties gemfireProperties) {
+    ServiceLoader<ServerLauncherCacheProvider> loader = ServiceLoader.load(ServerLauncherCacheProvider.class);
+    for(ServerLauncherCacheProvider provider : loader) {
+      Cache cache = provider.createCache(gemfireProperties, this);
+      if(cache != null) {
+        return cache;
+      }
     }
-
-    return cacheFactory.create();
+    
+    return DEFAULT_CACHE_PROVIDER.createCache(gemfireProperties, this);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncherCacheProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncherCacheProvider.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncherCacheProvider.java
new file mode 100644
index 0000000..5dad5fe
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncherCacheProvider.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.Cache;
+
+/**
+ * ServerLauncherCacheProvider is an extension point for overriding
+ * the behavior of a server started with {@link ServerLauncher} or
+ * the gfsh start server command. 
+ * 
+ * 
+ */
+public interface ServerLauncherCacheProvider {
+
+  Cache createCache(Properties gemfireProperties, ServerLauncher serverLauncher);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DefaultServerLauncherCacheProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DefaultServerLauncherCacheProvider.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DefaultServerLauncherCacheProvider.java
new file mode 100644
index 0000000..6f155ad
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DefaultServerLauncherCacheProvider.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal;
+
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.ServerLauncher;
+import com.gemstone.gemfire.distributed.ServerLauncherCacheProvider;
+import com.gemstone.gemfire.internal.cache.CacheConfig;
+
+public class DefaultServerLauncherCacheProvider
+    implements ServerLauncherCacheProvider {
+
+  @Override
+  public Cache createCache(Properties gemfireProperties, ServerLauncher serverLauncher) {
+    final CacheConfig cacheConfig = serverLauncher.getCacheConfig();
+    final CacheFactory cacheFactory = new CacheFactory(gemfireProperties);
+
+    if (cacheConfig.pdxPersistentUserSet) {
+      cacheFactory.setPdxPersistent(cacheConfig.isPdxPersistent());
+    }
+
+    if (cacheConfig.pdxDiskStoreUserSet) {
+      cacheFactory.setPdxDiskStore(cacheConfig.getPdxDiskStore());
+    }
+
+    if (cacheConfig.pdxIgnoreUnreadFieldsUserSet) {
+      cacheFactory.setPdxIgnoreUnreadFields(cacheConfig.getPdxIgnoreUnreadFields());
+    }
+
+    if (cacheConfig.pdxReadSerializedUserSet) {
+      cacheFactory.setPdxReadSerialized(cacheConfig.isPdxReadSerialized());
+    }
+
+    if (cacheConfig.pdxSerializerUserSet) {
+      cacheFactory.setPdxSerializer(cacheConfig.getPdxSerializer());
+    }
+
+    return cacheFactory.create();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/java/com/gemstone/gemfire/distributed/LauncherTestSuite.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LauncherTestSuite.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LauncherTestSuite.java
index 73a2337..21edb4a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LauncherTestSuite.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LauncherTestSuite.java
@@ -37,7 +37,7 @@ import org.junit.runners.Suite;
    ServerLauncherLocalFileJUnitTest.class,
    ServerLauncherRemoteJUnitTest.class,
    ServerLauncherRemoteFileJUnitTest.class,
-   ServerLauncherWithSpringJUnitTest.class,
+   ServerLauncherWithProviderJUnitTest.class,
 })
 /**
  * Suite of tests for the Launcher classes.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/java/com/gemstone/gemfire/distributed/MockServerLauncherCacheProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/MockServerLauncherCacheProvider.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/MockServerLauncherCacheProvider.java
new file mode 100644
index 0000000..c910832
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/MockServerLauncherCacheProvider.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.Cache;
+
+public class MockServerLauncherCacheProvider
+    implements ServerLauncherCacheProvider {
+  private static Cache cache;
+
+  
+  public static Cache getCache() {
+    return cache;
+  }
+
+  public static void setCache(Cache cache) {
+    MockServerLauncherCacheProvider.cache = cache;
+  }
+
+  @Override
+  public Cache createCache(Properties gemfireProperties,
+      ServerLauncher serverLauncher) {
+    return cache;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithProviderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithProviderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithProviderJUnitTest.java
new file mode 100644
index 0000000..33d3562
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithProviderJUnitTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.process.ProcessType;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Extracted from ServerLauncherLocalJUnitTest.
+ * 
+ * @author John Blum
+ * @author Kirk Lund
+ */
+@Category(IntegrationTest.class)
+public class ServerLauncherWithProviderJUnitTest extends AbstractServerLauncherJUnitTestCase {
+
+  @Before
+  public final void setUpServerLauncherWithSpringTest() throws Exception {
+    disconnectFromDS();
+    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName()+"-");
+  }
+
+  @After
+  public final void tearDownServerLauncherWithSpringTest() throws Exception {
+    MockServerLauncherCacheProvider.setCache(null);
+    disconnectFromDS();
+    
+  }
+
+  // NOTE make sure bugs like Trac #51201 never happen again!!!
+  @Test
+  public void testBootstrapGemFireServerWithProvider() throws Throwable {
+    Cache mockCache = Mockito.mock(Cache.class);
+    MockServerLauncherCacheProvider.setCache(mockCache);
+    this.launcher = new Builder()
+      .setDisableDefaultServer(true)
+      .setForce(true)
+      .setMemberName(getUniqueName())
+      .setSpringXmlLocation("spring/spring-gemfire-context.xml")
+      .set(DistributionConfig.MCAST_PORT_NAME, "0")
+      .build();
+
+    assertNotNull(this.launcher);
+
+    try {
+      assertEquals(Status.ONLINE, this.launcher.start().getStatus());
+
+      waitForServerToStart(this.launcher);
+
+      Cache cache = this.launcher.getCache();
+
+      assertEquals(mockCache, cache);
+    }
+    catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+
+    try {
+      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
+      assertNull(this.launcher.getCache());
+    }
+    catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
deleted file mode 100644
index bf82af8..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.data.gemfire.support.SpringContextBootstrappingInitializer;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
-import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.process.ProcessType;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Extracted from ServerLauncherLocalJUnitTest.
- * 
- * @author John Blum
- * @author Kirk Lund
- */
-@Category(IntegrationTest.class)
-public class ServerLauncherWithSpringJUnitTest extends AbstractServerLauncherJUnitTestCase {
-
-  @Before
-  public final void setUpServerLauncherWithSpringTest() throws Exception {
-    disconnectFromDS();
-    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName()+"-");
-  }
-
-  @After
-  public final void tearDownServerLauncherWithSpringTest() throws Exception {    
-    disconnectFromDS();
-    SpringContextBootstrappingInitializer.getApplicationContext().close();
-  }
-
-  // NOTE make sure bugs like Trac #51201 never happen again!!!
-  @Test
-  public void testBootstrapGemFireServerWithSpring() throws Throwable {
-    this.launcher = new Builder()
-      .setDisableDefaultServer(true)
-      .setForce(true)
-      .setMemberName(getUniqueName())
-      .setSpringXmlLocation("spring/spring-gemfire-context.xml")
-      .set(DistributionConfig.MCAST_PORT_NAME, "0")
-      .build();
-
-    assertNotNull(this.launcher);
-
-    try {
-      assertEquals(Status.ONLINE, this.launcher.start().getStatus());
-
-      waitForServerToStart(this.launcher);
-
-      Cache cache = this.launcher.getCache();
-
-      assertNotNull(cache);
-      assertTrue(cache.getCopyOnRead());
-      assertEquals(0.95f, cache.getResourceManager().getCriticalHeapPercentage(), 0);
-      assertEquals(0.85f, cache.getResourceManager().getEvictionHeapPercentage(), 0);
-      assertFalse(cache.getPdxIgnoreUnreadFields());
-      assertTrue(cache.getPdxPersistent());
-      assertTrue(cache.getPdxReadSerialized());
-    }
-    catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      assertNull(this.launcher.getCache());
-    }
-    catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/resources/META-INF/services/com.gemstone.gemfire.distributed.ServerLauncherCacheProvider
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/META-INF/services/com.gemstone.gemfire.distributed.ServerLauncherCacheProvider b/geode-core/src/test/resources/META-INF/services/com.gemstone.gemfire.distributed.ServerLauncherCacheProvider
new file mode 100644
index 0000000..36bf10d
--- /dev/null
+++ b/geode-core/src/test/resources/META-INF/services/com.gemstone.gemfire.distributed.ServerLauncherCacheProvider
@@ -0,0 +1 @@
+com.gemstone.gemfire.distributed.MockServerLauncherCacheProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/geode-core/src/test/resources/spring/spring-gemfire-context.xml
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/spring/spring-gemfire-context.xml b/geode-core/src/test/resources/spring/spring-gemfire-context.xml
deleted file mode 100644
index a728493..0000000
--- a/geode-core/src/test/resources/spring/spring-gemfire-context.xml
+++ /dev/null
@@ -1,42 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:gfe="http://www.springframework.org/schema/gemfire"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/gemfire http://www.springframework.org/schema/gemfire/spring-gemfire.xsd
-        http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util.xsd
-  ">
-
-  <util:properties id="gemfireProperties">
-    <prop key="name">SpringGemFireServerBootstrap</prop>
-    <prop key="mcast-port">0</prop>
-    <prop key="log-level">config</prop>
-    <prop key="http-service-port">0</prop>
-  </util:properties>
-
-  <gfe:cache properties-ref="gemfireProperties" lazy-init="false" copy-on-read="true"
-             critical-heap-percentage="0.95" eviction-heap-percentage="0.85"
-             pdx-ignore-unread-fields="false" pdx-persistent="true" pdx-read-serialized="true"/>
-
-  <gfe:partitioned-region id="Example" persistent="false"/>
-
-</beans>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7734781f/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 68e04c4..d54f9c4 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -127,6 +127,7 @@ rat {
     '**/META-INF/services/org.xml.sax.ext.EntityResolver2',
     '**/META-INF/services/com.gemstone.gemfire.internal.cache.CacheService',
     '**/META-INF/services/com.gemstone.gemfire.internal.cache.xmlcache.XmlParser',
+    '**/META-INF/services/com.gemstone.gemfire.distributed.ServerLauncherCacheProvider',
     '**/META-INF/services/org.springframework.shell.core.CommandMarker',
 
     // --- Other Licenses ---