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/12/12 22:38:48 UTC

[09/19] geode git commit: [GEODE-2196] Add test for Cluster Config. Refactor LocatorServerStartupRule.

[GEODE-2196] Add test for Cluster Config. Refactor LocatorServerStartupRule.

* Add ClusterConfigDUnitTest
* Refactor LocatorServerStartupRule
* this closes #309


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

Branch: refs/heads/feature/GEODE-1027
Commit: adfd41f437d2a30255ae8368e355b497d6bb0164
Parents: 213d8bc
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Dec 8 15:03:54 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Dec 9 09:47:07 2016 -0800

----------------------------------------------------------------------
 .../internal/DistributionConfig.java            |   2 +-
 .../internal/DistributionConfigImpl.java        |   2 +-
 .../ConnectToLocatorSSLDUnitTest.java           |   6 +-
 .../configuration/ClusterConfigDUnitTest.java   | 243 +++++++++++++++++++
 .../ClusterConfigWithoutSecurityDUnitTest.java  |   8 +-
 .../security/PeerAuthenticatorDUnitTest.java    |  12 +-
 ...eerSecurityWithEmbeddedLocatorDUnitTest.java |  18 +-
 .../SecurityClusterConfigDUnitTest.java         |  13 +-
 .../SecurityWithoutClusterConfigDUnitTest.java  |   7 +-
 .../security/StartServerAuthorizationTest.java  |  33 ++-
 .../dunit/rules/LocatorServerStartupRule.java   | 108 +++++----
 .../test/dunit/rules/LocatorStarterRule.java    |  11 +-
 .../apache/geode/test/dunit/rules/Member.java   |  62 +++++
 .../test/dunit/rules/ServerStarterRule.java     |  32 ++-
 .../internal/configuration/cluster_config.zip   | Bin 0 -> 4666 bytes
 .../LuceneClusterConfigurationDUnitTest.java    |  48 ++--
 16 files changed, 474 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
index 5eb070d..c2a395d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
@@ -384,7 +384,7 @@ public interface DistributionConfig extends Config, LogConfig {
    * Default will be the current working directory as determined by
    * <code>System.getProperty("user.dir")</code>.
    */
-  File DEFAULT_DEPLOY_WORKING_DIR = new File(".");
+  File DEFAULT_DEPLOY_WORKING_DIR = new File(System.getProperty("user.dir"));
 
   /**
    * Returns the value of the {@link ConfigurationProperties#USER_COMMAND_PACKAGES} property

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
index 0c6603d..fa6d13f 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
@@ -124,7 +124,7 @@ public class DistributionConfigImpl extends AbstractDistributionConfig implement
    */
   private File logFile = DEFAULT_LOG_FILE;
 
-  protected File deployWorkingDir = DEFAULT_DEPLOY_WORKING_DIR;
+  protected File deployWorkingDir = new File(System.getProperty("user.dir"));
 
   /**
    * The level at which log messages are logged

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
index 751425a..142ce17 100644
--- a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
@@ -83,14 +83,14 @@ public class ConnectToLocatorSSLDUnitTest extends JUnit4DistributedTestCase {
   }
 
   public void setUpLocatorAndConnect(Properties securityProps) throws Exception {
-    lsRule.getLocatorVM(0, securityProps);
+    lsRule.startLocatorVM(0, securityProps);
 
     // saving the securityProps to a file
     OutputStream out = new FileOutputStream(securityPropsFile);
     securityProps.store(out, "");
 
-    GfshShellConnectionRule gfshConnector =
-        new GfshShellConnectionRule(lsRule.getPort(0), GfshShellConnectionRule.PortType.locator);
+    GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule(
+        lsRule.getMember(0).getPort(), GfshShellConnectionRule.PortType.locator);
 
     // when we connect too soon, we would get "Failed to retrieve RMIServer stub:
     // javax.naming.CommunicationException [Root exception is java.rmi.NoSuchObjectException: no

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDUnitTest.java
new file mode 100644
index 0000000..418999a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDUnitTest.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.configuration;
+
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_CONFIGURATION_DIR;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOAD_CLUSTER_CONFIGURATION_FROM_DIR;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE_SIZE_LIMIT;
+import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.distributed.internal.SharedConfiguration;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.JarClassLoader;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.management.internal.configuration.utils.ZipUtils;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.Member;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Properties;
+
+@Category(DistributedTest.class)
+public class ClusterConfigDUnitTest extends JUnit4DistributedTestCase {
+  private static final String EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME = "cluster_config.zip";
+  private static final String[] CONFIG_NAMES = new String[] {"cluster", "group1", "group2"};
+
+  private static final ExpectedConfig NO_GROUP =
+      new ExpectedConfig().maxLogFileSize("5000").regions("regionForCluster").jars("cluster.jar");
+
+  private static final ExpectedConfig GROUP1 = new ExpectedConfig().maxLogFileSize("6000")
+      .regions("regionForCluster", "regionForGroup1").jars("cluster.jar", "group1.jar");
+
+  private static final ExpectedConfig GROUP2 = new ExpectedConfig().maxLogFileSize("7000")
+      .regions("regionForCluster", "regionForGroup2").jars("cluster.jar", "group2.jar");
+
+  private static final ExpectedConfig GROUP1_AND_2 = new ExpectedConfig().maxLogFileSize("7000")
+      .regions("regionForCluster", "regionForGroup1", "regionForGroup2")
+      .jars("cluster.jar", "group1.jar", "group2.jar");
+
+
+  private String locatorString;
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @Before
+  public void setupFirstLocatorWithClusterConfigFromDirectory() throws Exception {
+    File locatorDir = lsRule.getRootFolder().newFolder("locator-0");
+
+    // The unzip should yield a cluster config directory structure like:
+    // tempFolder/locator-0/cluster_config/cluster/cluster.xml
+    // tempFolder/locator-0/cluster_config/cluster/cluster.properties
+    // tempFolder/locator-0/cluster_config/cluster/cluster.jar
+    // tempFolder/locator-0/cluster_config/group1/ {group1.xml, group1.properties, group1.jar}
+    // tempFolder/locator-0/cluster_config/group2/ ...
+    ZipUtils.unzip(getClass().getResource(EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME).getPath(),
+        locatorDir.getCanonicalPath());
+
+    Properties locatorProps = new Properties();
+    locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
+    locatorProps.setProperty(LOAD_CLUSTER_CONFIGURATION_FROM_DIR, "true");
+    locatorProps.setProperty(CLUSTER_CONFIGURATION_DIR, locatorDir.getCanonicalPath());
+
+    Member firstLocator = lsRule.startLocatorVM(0, locatorProps);
+    locatorString = "localhost[" + firstLocator.getPort() + "]";
+
+    verifyLocatorConfigExistsInFileSystem(firstLocator.getWorkingDir());
+    firstLocator.invoke(this::verifyLocatorConfigExistsInInternalRegion);
+  }
+
+  @Test
+  public void secondLocatorLoadsClusterConfigFromFirstLocator() throws IOException {
+    Properties secondLocatorProps = new Properties();
+    secondLocatorProps.setProperty(LOCATORS, locatorString);
+    secondLocatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
+    Member secondLocator = lsRule.startLocatorVM(1, secondLocatorProps);
+
+    verifyLocatorConfig(secondLocator);
+  }
+
+  @Test
+  public void serverWithZeroOrOneGroupsLoadCorrectConfigFromLocator() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(LOCATORS, locatorString);
+    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
+
+    Member serverWithNoGroup = lsRule.startServerVM(1, serverProps);
+    verifyServerConfig(NO_GROUP, serverWithNoGroup);
+
+    serverProps.setProperty(GROUPS, "group1");
+    Member serverForGroup1 = lsRule.startServerVM(2, serverProps);
+    verifyServerConfig(GROUP1, serverForGroup1);
+
+    serverProps.setProperty(GROUPS, "group2");
+    Member serverForGroup2 = lsRule.startServerVM(3, serverProps);
+    verifyServerConfig(GROUP2, serverForGroup2);
+  }
+
+  @Test
+  public void oneServerWithMultipleGroupsLoadsCorrectConfigFromLocator() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(LOCATORS, locatorString);
+    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
+    serverProps.setProperty(GROUPS, "group1,group2");
+    Member serverWithNoGroup = lsRule.startServerVM(1, serverProps);
+
+    serverWithNoGroup.invoke(() -> this.verifyServerConfig(GROUP1_AND_2, serverWithNoGroup));
+  }
+
+  private void verifyLocatorConfig(Member locator) {
+    verifyLocatorConfigExistsInFileSystem(locator.getWorkingDir());
+    locator.invoke(this::verifyLocatorConfigExistsInInternalRegion);
+  }
+
+  private void verifyServerConfig(ExpectedConfig expectedConfig, Member server)
+      throws ClassNotFoundException {
+    verifyServerJarFilesExistInFileSystem(server.getWorkingDir(), expectedConfig.jars);
+    server.invoke(() -> this.verifyServerConfigInMemory(expectedConfig));
+  }
+
+  private void verifyLocatorConfigExistsInFileSystem(File workingDir) {
+    File clusterConfigDir = new File(workingDir, "cluster_config");
+    assertThat(clusterConfigDir).exists();
+
+    for (String configName : CONFIG_NAMES) {
+      File configDir = new File(clusterConfigDir, configName);
+      assertThat(configDir).exists();
+
+      File jar = new File(configDir, configName + ".jar");
+      File properties = new File(configDir, configName + ".properties");
+      File xml = new File(configDir, configName + ".xml");
+      assertThat(configDir.listFiles()).contains(jar, properties, xml);
+    }
+  }
+
+  private void verifyLocatorConfigExistsInInternalRegion() throws Exception {
+    InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
+    SharedConfiguration sc = internalLocator.getSharedConfiguration();
+
+    for (String configName : CONFIG_NAMES) {
+      Configuration config = sc.getConfiguration(configName);
+      assertThat(config).isNotNull();
+    }
+  }
+
+  private void verifyServerConfigInMemory(ExpectedConfig expectedConfig)
+      throws ClassNotFoundException {
+    Cache cache = LocatorServerStartupRule.serverStarter.cache;
+    for (String region : expectedConfig.regions) {
+      assertThat(cache.getRegion(region)).isNotNull();
+    }
+    Properties props = cache.getDistributedSystem().getProperties();
+    assertThat(props.getProperty(LOG_FILE_SIZE_LIMIT)).isEqualTo(expectedConfig.maxLogFileSize);
+
+    for (String jar : expectedConfig.jars) {
+      JarClassLoader jarClassLoader = findJarClassLoader(jar);
+      assertThat(jarClassLoader).isNotNull();
+      assertThat(jarClassLoader.loadClass(nameOfClassContainedInJar(jar))).isNotNull();
+    }
+  }
+
+  private void verifyServerJarFilesExistInFileSystem(File workingDir, String[] jarNames) {
+    assertThat(workingDir.listFiles()).isNotEmpty();
+
+    for (String jarName : jarNames) {
+      assertThat(workingDir.listFiles()).filteredOn((File file) -> file.getName().contains(jarName))
+          .isNotEmpty();
+    }
+  }
+
+  private String nameOfClassContainedInJar(String jarName) {
+    switch (jarName) {
+      case "cluster.jar":
+        return "Cluster";
+      case "group1.jar":
+        return "Group1";
+      case "group2.jar":
+        return "Group2";
+      default:
+        throw new IllegalArgumentException(
+            EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME + " does not contain a jar named " + jarName);
+    }
+  }
+
+  private JarClassLoader findJarClassLoader(final String jarName) {
+    Collection<ClassLoader> classLoaders = ClassPathLoader.getLatest().getClassLoaders();
+    for (ClassLoader classLoader : classLoaders) {
+      if (classLoader instanceof JarClassLoader
+          && ((JarClassLoader) classLoader).getJarName().equals(jarName)) {
+        return (JarClassLoader) classLoader;
+      }
+    }
+    return null;
+  }
+
+  private static class ExpectedConfig implements Serializable {
+    public String maxLogFileSize;
+    public String[] regions;
+    public String[] jars;
+
+    public ExpectedConfig maxLogFileSize(String maxLogFileSize) {
+      this.maxLogFileSize = maxLogFileSize;
+      return this;
+    }
+
+    public ExpectedConfig regions(String... regions) {
+      this.regions = regions;
+      return this;
+    }
+
+    public ExpectedConfig jars(String... jars) {
+      this.jars = jars;
+      return this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/ClusterConfigWithoutSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/ClusterConfigWithoutSecurityDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/ClusterConfigWithoutSecurityDUnitTest.java
index 1ad6799..fe37f9e 100644
--- a/geode-core/src/test/java/org/apache/geode/security/ClusterConfigWithoutSecurityDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/ClusterConfigWithoutSecurityDUnitTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.*;
 
 import java.util.Properties;
 
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.junit.After;
 import org.junit.Before;
@@ -33,7 +34,6 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 
@@ -50,7 +50,7 @@ public class ClusterConfigWithoutSecurityDUnitTest extends JUnit4DistributedTest
         .addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toString());
     IgnoredException
         .addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2.toString());
-    lsRule.getLocatorVM(0, new Properties());
+    lsRule.startLocatorVM(0, new Properties());
   }
 
   @After
@@ -71,7 +71,7 @@ public class ClusterConfigWithoutSecurityDUnitTest extends JUnit4DistributedTest
 
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
-    serverStarter.startServer(lsRule.getPort(0));
+    serverStarter.startServer(lsRule.getMember(0).getPort());
     DistributedSystem ds = serverStarter.cache.getDistributedSystem();
 
     // after cache is created, the configuration won't chagne
@@ -93,7 +93,7 @@ public class ClusterConfigWithoutSecurityDUnitTest extends JUnit4DistributedTest
 
     ServerStarterRule serverStarter = new ServerStarterRule(props);
 
-    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
         .isInstanceOf(GemFireConfigException.class)
         .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toLocalizedString());
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
index 90cc6f1..a7752ae 100644
--- a/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
@@ -16,10 +16,13 @@
 package org.apache.geode.security;
 
 import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import java.util.Properties;
 
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.Member;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.junit.Before;
 import org.junit.Rule;
@@ -29,7 +32,6 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.security.templates.DummyAuthenticator;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 
@@ -42,23 +44,23 @@ public class PeerAuthenticatorDUnitTest extends JUnit4DistributedTestCase {
   public void before() throws Exception {
     Properties props = new Properties();
     props.setProperty(SECURITY_PEER_AUTHENTICATOR, DummyAuthenticator.class.getName());
-    lsRule.getLocatorVM(0, props);
+    lsRule.startLocatorVM(0, props);
   }
 
   @Test
   public void testPeerAuthenticator() throws Exception {
 
-    int locatorPort = lsRule.getPort(0);
+    int locatorPort = lsRule.getMember(0).getPort();
     Properties server1Props = new Properties();
     server1Props.setProperty("security-username", "user");
     server1Props.setProperty("security-password", "user");
-    lsRule.getServerVM(1, server1Props, locatorPort);
+    lsRule.startServerVM(1, server1Props, locatorPort);
 
 
     Properties server2Props = new Properties();
     server2Props.setProperty("security-username", "bogus");
     server2Props.setProperty("security-password", "user");
-    VM server2 = lsRule.getNodeVM(2);
+    VM server2 = getHost(0).getVM(2);
 
     server2.invoke(() -> {
       ServerStarterRule serverStarter = new ServerStarterRule(server2Props);

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/PeerSecurityWithEmbeddedLocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PeerSecurityWithEmbeddedLocatorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/PeerSecurityWithEmbeddedLocatorDUnitTest.java
index d69e379..66c5be6 100644
--- a/geode-core/src/test/java/org/apache/geode/security/PeerSecurityWithEmbeddedLocatorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/PeerSecurityWithEmbeddedLocatorDUnitTest.java
@@ -16,10 +16,13 @@
 package org.apache.geode.security;
 
 import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import java.util.Properties;
 
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -28,7 +31,6 @@ import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.security.templates.DummyAuthenticator;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
@@ -47,19 +49,19 @@ public class PeerSecurityWithEmbeddedLocatorDUnitTest extends JUnit4DistributedT
     Properties server0Props = new Properties();
     server0Props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
     server0Props.setProperty("start-locator", "localhost[" + locatorPort + "]");
-    lsRule.getServerVM(0, server0Props);
+    lsRule.startServerVM(0, server0Props);
 
 
     Properties server1Props = new Properties();
     server1Props.setProperty("security-username", "cluster");
     server1Props.setProperty("security-password", "cluster");
-    lsRule.getServerVM(1, server1Props, locatorPort);
+    lsRule.startServerVM(1, server1Props, locatorPort);
 
     Properties server2Props = new Properties();
     server2Props.setProperty("security-username", "user");
     server2Props.setProperty("security-password", "wrongPwd");
 
-    VM server2 = lsRule.getNodeVM(2);
+    VM server2 = getHost(0).getVM(2);
     server2.invoke(() -> {
       ServerStarterRule serverStarter = new ServerStarterRule(server2Props);
       assertThatThrownBy(() -> serverStarter.startServer(locatorPort))
@@ -68,6 +70,8 @@ public class PeerSecurityWithEmbeddedLocatorDUnitTest extends JUnit4DistributedT
     });
   }
 
+
+
   @Test
   public void testPeerAuthenticator() throws Exception {
     int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
@@ -75,19 +79,19 @@ public class PeerSecurityWithEmbeddedLocatorDUnitTest extends JUnit4DistributedT
     Properties server0Props = new Properties();
     server0Props.setProperty(SECURITY_PEER_AUTHENTICATOR, DummyAuthenticator.class.getName());
     server0Props.setProperty("start-locator", "localhost[" + locatorPort + "]");
-    lsRule.getServerVM(0, server0Props);
+    lsRule.startServerVM(0, server0Props);
 
 
     Properties server1Props = new Properties();
     server1Props.setProperty("security-username", "user");
     server1Props.setProperty("security-password", "user");
-    lsRule.getServerVM(1, server1Props, locatorPort);
+    lsRule.startServerVM(1, server1Props, locatorPort);
 
     Properties server2Props = new Properties();
     server2Props.setProperty("security-username", "bogus");
     server2Props.setProperty("security-password", "user");
 
-    VM server2 = lsRule.getNodeVM(2);
+    VM server2 = getHost(0).getVM(2);
     server2.invoke(() -> {
       ServerStarterRule serverStarter = new ServerStarterRule(server2Props);
       assertThatThrownBy(() -> serverStarter.startServer(locatorPort))

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
index 220449d..cd6518e 100644
--- a/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
@@ -34,6 +34,7 @@ import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -58,7 +59,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
     props.setProperty(JMX_MANAGER_START, "false");
     props.setProperty(JMX_MANAGER_PORT, 0 + "");
     props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
-    lsRule.getLocatorVM(0, props);
+    lsRule.startLocatorVM(0, props);
   }
 
   @Test
@@ -71,7 +72,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
 
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
-    serverStarter.startServer(lsRule.getPort(0));
+    serverStarter.startServer(lsRule.getMember(0).getPort());
     DistributedSystem ds = serverStarter.cache.getDistributedSystem();
 
     // after cache is created, we got the security props passed in by cluster config
@@ -93,7 +94,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
 
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
-    serverStarter.startServer(lsRule.getPort(0));
+    serverStarter.startServer(lsRule.getMember(0).getPort());
     DistributedSystem ds = serverStarter.cache.getDistributedSystem();
 
     // after cache is created, we got the security props passed in by cluster config
@@ -115,7 +116,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
 
-    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
         .isInstanceOf(GemFireConfigException.class)
         .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toLocalizedString());
 
@@ -134,7 +135,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
 
-    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
         .isInstanceOf(GemFireConfigException.class)
         .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toLocalizedString());
 
@@ -152,7 +153,7 @@ public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
 
     ServerStarterRule serverStarter = new ServerStarterRule(props);
 
-    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+    assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
         .isInstanceOf(GemFireConfigException.class)
         .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2.toLocalizedString());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
index 17b1031..72654cb 100644
--- a/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
@@ -20,7 +20,9 @@ import static org.junit.Assert.*;
 
 import java.util.Properties;
 
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -29,7 +31,6 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
@@ -51,7 +52,7 @@ public class SecurityWithoutClusterConfigDUnitTest extends JUnit4DistributedTest
     props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
     props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
     props.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
-    lsRule.getLocatorVM(0, props);
+    lsRule.startLocatorVM(0, props);
   }
 
   @Test
@@ -69,7 +70,7 @@ public class SecurityWithoutClusterConfigDUnitTest extends JUnit4DistributedTest
 
     // initial security properties should only contain initial set of values
     ServerStarterRule serverStarter = new ServerStarterRule(props);
-    serverStarter.startServer(lsRule.getPort(0));
+    serverStarter.startServer(lsRule.getMember(0).getPort());
     DistributedSystem ds = serverStarter.cache.getDistributedSystem();
     assertEquals(3, ds.getSecurityProperties().size());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java b/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
index dfa0f3d..24d0db5 100644
--- a/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
@@ -15,22 +15,23 @@
 
 package org.apache.geode.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.Properties;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.apache.geode.test.dunit.Host.getHost;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
+import java.util.Properties;
 
 @Category({DistributedTest.class, SecurityTest.class})
 public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
@@ -42,7 +43,7 @@ public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
   public void before() throws Exception {
     Properties props = new Properties();
     props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
-    lsRule.getLocatorVM(0, props);
+    lsRule.startLocatorVM(0, props);
   }
 
   @Test
@@ -52,10 +53,10 @@ public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
     props.setProperty("security-username", "user");
     props.setProperty("security-password", "wrongPswd");
 
-    VM server = lsRule.getNodeVM(1);
+    VM server = getHost(0).getVM(1);
     server.invoke(() -> {
       ServerStarterRule serverStarter = new ServerStarterRule(props);
-      assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+      assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
           .isInstanceOf(GemFireSecurityException.class).hasMessageContaining(
               "Security check failed. Authentication error. Please check your credentials");
     });
@@ -69,15 +70,13 @@ public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
     props.setProperty("security-username", "user");
     props.setProperty("security-password", "user");
 
-    VM server = lsRule.getNodeVM(1);
+    VM server = getHost(0).getVM(1);
     server.invoke(() -> {
       ServerStarterRule serverStarter = new ServerStarterRule(props);
-      assertThatThrownBy(() -> serverStarter.startServer(lsRule.getPort(0)))
+      assertThatThrownBy(() -> serverStarter.startServer(lsRule.getMember(0).getPort()))
           .isInstanceOf(GemFireSecurityException.class)
           .hasMessageContaining("user not authorized for CLUSTER:MANAGE");
     });
-
-
   }
 
   @Test
@@ -88,7 +87,7 @@ public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
     props.setProperty("security-username", "cluster");
     props.setProperty("security-password", "cluster");
 
-    lsRule.getServerVM(1, props);
+    lsRule.startServerVM(1, props);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
index 6053e1e..c56f7ab 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
@@ -11,6 +11,7 @@
  * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
+ *
  */
 
 package org.apache.geode.test.dunit.rules;
@@ -21,57 +22,72 @@ import static org.apache.geode.test.dunit.Host.getHost;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.rules.ExternalResource;
+import org.junit.rules.TemporaryFolder;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Properties;
 
 
 /**
- * this rule can help you start up locator/server in different VMs you can multiple locators/servers
- * combination
+ * A rule to help you start locators and servers inside of a
+ * <a href="https://cwiki.apache.org/confluence/display/GEODE/Distributed-Unit-Tests">DUnit
+ * test</a>. This rule will start Servers and Locators inside of the four remote {@link VM}s created
+ * by the DUnit framework.
  */
 public class LocatorServerStartupRule extends ExternalResource implements Serializable {
 
-  // these are only avaialbe in each VM
+  /**
+   * This is only available in each Locator/Server VM, not in the controller (test) VM.
+   */
   public static ServerStarterRule serverStarter;
+
+  /**
+   * This is only available in each Locator/Server VM, not in the controller (test) VM.
+   */
   public static LocatorStarterRule locatorStarter;
-  public int[] ports = new int[4];
-  private Host host = getHost(0);
+
+  private TemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+  private Member[] members;
 
   @Before
-  public void before() {
-    after();
+  public void before() throws IOException {
+    temporaryFolder.create();
+    Invoke.invokeInEveryVM("Stop each VM", this::stop);
+    members = new Member[4];
   }
 
   @After
   public void after() {
-    stop();
-    Invoke.invokeInEveryVM("Stop each VM", () -> stop());
+    temporaryFolder.delete();
+    Invoke.invokeInEveryVM("Stop each VM", this::stop);
   }
 
   /**
-   * Returns getHost(0).getVM(0) as a locator instance with the given configuration properties.
-   * 
-   * @param locatorProperties
+   * Starts a locator instance with the given configuration properties inside
+   * {@code getHost(0).getVM(index)}.
    *
    * @return VM locator vm
-   *
-   * @throws IOException
    */
-  public VM getLocatorVM(int index, Properties locatorProperties) throws IOException {
-    VM locatorVM = host.getVM(index);
-    locatorProperties.setProperty(NAME, "locator-" + index);
+  public Member startLocatorVM(int index, Properties locatorProperties) throws IOException {
+    String name = "locator-" + index;
+    locatorProperties.setProperty(NAME, name);
+    File workingDir = createWorkingDirForMember(name);
+
+    VM locatorVM = getHost(0).getVM(index);
     int locatorPort = locatorVM.invoke(() -> {
+      System.setProperty("user.dir", workingDir.getCanonicalPath());
       locatorStarter = new LocatorStarterRule(locatorProperties);
       locatorStarter.startLocator();
       return locatorStarter.locator.getPort();
     });
-    ports[index] = locatorPort;
-    return locatorVM;
+    members[index] = new Member(locatorVM, locatorPort, workingDir);
+    return members[index];
   }
 
   /**
@@ -79,48 +95,41 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
    * 
    * @return VM node vm
    */
-
-  public VM getServerVM(int index, Properties properties) {
-    return getServerVM(index, properties, 0);
+  public Member startServerVM(int index, Properties properties) throws IOException {
+    return startServerVM(index, properties, 0);
   }
 
   /**
-   * starts a cache server that connect to the locator running at the given port.
-   * 
-   * @param index
-   * @param properties
-   * @param locatorPort
-   * @return
+   * Starts a cache server that connect to the locator running at the given port.
    */
-  public VM getServerVM(int index, Properties properties, int locatorPort) {
-    VM nodeVM = getNodeVM(index);
-    properties.setProperty(NAME, "server-" + index);
-    int port = nodeVM.invoke(() -> {
+  public Member startServerVM(int index, Properties properties, int locatorPort)
+      throws IOException {
+    String name = "server-" + index;
+    properties.setProperty(NAME, name);
+    File workingDir = createWorkingDirForMember(name);
+
+    VM serverVM = getHost(0).getVM(index);
+    int port = serverVM.invoke(() -> {
+      System.setProperty("user.dir", workingDir.getCanonicalPath());
       serverStarter = new ServerStarterRule(properties);
       serverStarter.startServer(locatorPort);
       return serverStarter.server.getPort();
     });
-    ports[index] = port;
-    return nodeVM;
+    members[index] = new Member(serverVM, port, workingDir);
+    return members[index];
   }
 
-
-
   /**
-   * this will simply returns the node
-   * 
-   * @param index
-   * @return
+   * Returns the {@link Member} running inside the VM with the specified {@code index}
    */
-  public VM getNodeVM(int index) {
-    return host.getVM(index);
+  public Member getMember(int index) {
+    return members[index];
   }
 
-  public int getPort(int index) {
-    return ports[index];
+  public TemporaryFolder getRootFolder() {
+    return temporaryFolder;
   }
 
-
   public final void stop() {
     if (serverStarter != null) {
       serverStarter.after();
@@ -130,4 +139,13 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     }
   }
 
+  private File createWorkingDirForMember(String dirName) throws IOException {
+    File workingDir = new File(temporaryFolder.getRoot(), dirName);
+    if (!workingDir.exists()) {
+      temporaryFolder.newFolder(dirName);
+    }
+
+    return workingDir;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
index 3329a40..6fc0d03 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
@@ -33,11 +33,14 @@ import java.util.concurrent.TimeUnit;
 /**
  * This is a rule to start up a locator in your current VM. It's useful for your Integration Tests.
  *
- * If you need a rule to start a server/locator in different VM for Distribution tests, You should
- * use LocatorServerStartupRule
+ * If you need a rule to start a server/locator in different VMs for Distributed tests, You should
+ * use {@link LocatorServerStartupRule}.
  *
- * You may choose to use this class not as a rule or use it in your own rule, (see
- * LocatorServerStartupRule) you will need to call startLocator() and after() manually in that case.
+ * <p>
+ * You may choose to use this class not as a rule or use it in your own rule (see
+ * {@link LocatorServerStartupRule}), in which case you will need to call startLocator() and after()
+ * manually.
+ * </p>
  */
 
 public class LocatorStarterRule extends ExternalResource implements Serializable {

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
new file mode 100644
index 0000000..10b6743
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ *
+ */
+
+package org.apache.geode.test.dunit.rules;
+
+import org.apache.geode.test.dunit.SerializableRunnableIF;
+import org.apache.geode.test.dunit.VM;
+
+import java.io.File;
+import java.io.Serializable;
+
+/**
+ * A server or locator inside a DUnit {@link VM}.
+ */
+public class Member implements Serializable {
+  private VM vm;
+  private int port;
+  private File workingDir;
+
+  public Member(VM vm, int port, File workingDir) {
+    this.vm = vm;
+    this.port = port;
+    this.workingDir = workingDir;
+  }
+
+  /**
+   * The VM object is an RMI stub which lets us execute code in the JVM of this member.
+   * 
+   * @return the {@link VM}
+   */
+  public VM getVM() {
+    return vm;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public File getWorkingDir() {
+    return workingDir;
+  }
+
+  /**
+   * Invokes {@code runnable.run()} in the {@code VM} of this member.
+   */
+  public void invoke(final SerializableRunnableIF runnable) {
+    this.vm.invoke(runnable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
index 8da0778..c94185a 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
@@ -25,6 +25,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.junit.rules.ExternalResource;
 
 import java.io.Serializable;
@@ -34,11 +35,13 @@ import java.util.Properties;
 /**
  * This is a rule to start up a server in your current VM. It's useful for your Integration Tests.
  *
- * If you need a rule to start a server/locator in different VM for Distribution tests, You should
- * use LocatorServerStartupRule
- *
- * You may choose to use this class not as a rule or use it in your own rule, (see
- * LocatorServerStartupRule) you will need to call startServer and after() manually in that case.
+ * If you need a rule to start a server/locator in different VMs for Distributed tests, You should
+ * use {@link LocatorServerStartupRule}.
+ * <p>
+ * You may choose to use this class not as a rule or use it in your own rule (see
+ * {@link LocatorServerStartupRule}), in which case you will need to call startLocator() and after()
+ * manually.
+ * </p>
  */
 public class ServerStarterRule extends ExternalResource implements Serializable {
 
@@ -66,10 +69,12 @@ public class ServerStarterRule extends ExternalResource implements Serializable
     if (!properties.containsKey(NAME)) {
       properties.setProperty(NAME, this.getClass().getName());
     }
-    if (locatorPort > 0) {
-      properties.setProperty(LOCATORS, "localhost[" + locatorPort + "]");
-    } else {
-      properties.setProperty(LOCATORS, "");
+    if (!properties.containsKey(LOCATORS)) {
+      if (locatorPort > 0) {
+        properties.setProperty(LOCATORS, "localhost[" + locatorPort + "]");
+      } else {
+        properties.setProperty(LOCATORS, "");
+      }
     }
     if (properties.containsKey(JMX_MANAGER_PORT)) {
       int jmxPort = Integer.parseInt(properties.getProperty(JMX_MANAGER_PORT));
@@ -84,7 +89,6 @@ public class ServerStarterRule extends ExternalResource implements Serializable
     CacheFactory cf = new CacheFactory(properties);
     cf.setPdxReadSerialized(pdxPersistent);
     cf.setPdxPersistent(pdxPersistent);
-
     cache = cf.create();
     server = cache.addCacheServer();
     server.setPort(0);
@@ -102,9 +106,13 @@ public class ServerStarterRule extends ExternalResource implements Serializable
 
   @Override
   public void after() {
-    if (cache != null)
+    if (cache != null) {
       cache.close();
-    if (server != null)
+      cache = null;
+    }
+    if (server != null) {
       server.stop();
+      server = null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster_config.zip
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster_config.zip b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster_config.zip
new file mode 100644
index 0000000..37cacbf
Binary files /dev/null and b/geode-core/src/test/resources/org/apache/geode/management/internal/configuration/cluster_config.zip differ

http://git-wip-us.apache.org/repos/asf/geode/blob/adfd41f4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index 02b7fe4..0b0e936 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -26,6 +26,8 @@ import java.util.Arrays;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.Member;
 import org.apache.lucene.analysis.Analyzer;
 import org.junit.Rule;
 import org.junit.Test;
@@ -45,7 +47,6 @@ import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 
@@ -59,8 +60,8 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
   @Test
   public void indexGetsCreatedUsingClusterConfiguration() throws Exception {
-    VM locator = startLocatorWithClusterConfigurationEnabled();
-    VM vm1 = startNodeUsingClusterConfiguration(1, false);
+    Member locator = startLocatorWithClusterConfigurationEnabled();
+    Member vm1 = startNodeUsingClusterConfiguration(1, false);
 
     // Connect Gfsh to locator.
     createAndConnectGfshToLocator();
@@ -72,7 +73,7 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
     // Start vm2. This should have lucene index created using cluster
     // configuration.
-    VM vm2 = startNodeUsingClusterConfiguration(2, false);
+    Member vm2 = startNodeUsingClusterConfiguration(2, false);
     vm2.invoke(() -> {
       LuceneService luceneService = LuceneServiceProvider.get(ls.serverStarter.cache);
       final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
@@ -83,8 +84,8 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
   @Test
   public void indexWithAnalyzerGetsCreatedUsingClusterConfiguration() throws Exception {
-    VM locator = startLocatorWithClusterConfigurationEnabled();
-    VM vm1 = startNodeUsingClusterConfiguration(1, false);
+    Member locator = startLocatorWithClusterConfigurationEnabled();
+    Member vm1 = startNodeUsingClusterConfiguration(1, false);
 
     // Connect Gfsh to locator.
     createAndConnectGfshToLocator();
@@ -97,7 +98,7 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
     // Start vm2. This should have lucene index created using cluster
     // configuration.
-    VM vm2 = startNodeUsingClusterConfiguration(2, false);
+    Member vm2 = startNodeUsingClusterConfiguration(2, false);
     vm2.invoke(() -> {
       LuceneService luceneService = LuceneServiceProvider.get(ls.serverStarter.cache);
       final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
@@ -115,14 +116,14 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
   @Test
   public void indexGetsCreatedOnGroupOfNodes() throws Exception {
-    VM locator = startLocatorWithClusterConfigurationEnabled();
+    Member locator = startLocatorWithClusterConfigurationEnabled();
 
     // Start vm1, vm2 in group
-    VM vm1 = startNodeUsingClusterConfiguration(1, true);
-    VM vm2 = startNodeUsingClusterConfiguration(2, true);
+    Member vm1 = startNodeUsingClusterConfiguration(1, true);
+    Member vm2 = startNodeUsingClusterConfiguration(2, true);
 
     // Start vm3 outside the group. The Lucene index should not be present here.
-    VM vm3 = startNodeUsingClusterConfiguration(3, true);
+    Member vm3 = startNodeUsingClusterConfiguration(3, true);
 
     // Connect Gfsh to locator.
     createAndConnectGfshToLocator();
@@ -151,14 +152,14 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
   @Test
   public void indexNotCreatedOnNodeOutSideTheGroup() throws Exception {
-    VM locator = startLocatorWithClusterConfigurationEnabled();
+    Member locator = startLocatorWithClusterConfigurationEnabled();
 
     // Start vm1, vm2 in group
-    VM vm1 = startNodeUsingClusterConfiguration(1, true);
-    VM vm2 = startNodeUsingClusterConfiguration(2, true);
+    Member vm1 = startNodeUsingClusterConfiguration(1, true);
+    Member vm2 = startNodeUsingClusterConfiguration(2, true);
 
     // Start vm3 outside the group. The Lucene index should not be present here.
-    VM vm3 = startNodeUsingClusterConfiguration(3, false);
+    Member vm3 = startNodeUsingClusterConfiguration(3, false);
 
     // Connect Gfsh to locator.
     createAndConnectGfshToLocator();
@@ -187,10 +188,10 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
 
   @Test
   public void indexAreCreatedInValidGroupOfNodesJoiningLater() throws Exception {
-    VM locator = startLocatorWithClusterConfigurationEnabled();
+    Member locator = startLocatorWithClusterConfigurationEnabled();
 
     // Start vm1 in group
-    VM vm1 = startNodeUsingClusterConfiguration(1, true);
+    Member vm1 = startNodeUsingClusterConfiguration(1, true);
     // Connect Gfsh to locator.
     createAndConnectGfshToLocator();
 
@@ -200,10 +201,10 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
     createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, groupName);
 
     // Start vm2 in group
-    VM vm2 = startNodeUsingClusterConfiguration(2, true);
+    Member vm2 = startNodeUsingClusterConfiguration(2, true);
 
     // Start vm3 outside the group. The Lucene index should not be present here.
-    VM vm3 = startNodeUsingClusterConfiguration(3, false);
+    Member vm3 = startNodeUsingClusterConfiguration(3, false);
 
     // VM2 should have lucene index created using gfsh execution
     vm2.invoke(() -> {
@@ -226,7 +227,8 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
     connect(jmxHost, jmxPort, httpPort, gfsh);
   }
 
-  private VM startNodeUsingClusterConfiguration(int vmIndex, boolean addGroup) throws Exception {
+  private Member startNodeUsingClusterConfiguration(int vmIndex, boolean addGroup)
+      throws Exception {
     File dir = this.temporaryFolder.newFolder();
     Properties nodeProperties = new Properties();
     nodeProperties.setProperty(USE_CLUSTER_CONFIGURATION, "true");
@@ -234,10 +236,10 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
     if (addGroup) {
       nodeProperties.setProperty(GROUPS, groupName);
     }
-    return ls.getServerVM(vmIndex, nodeProperties, ls.getPort(0));
+    return ls.startServerVM(vmIndex, nodeProperties, ls.getMember(0).getPort());
   }
 
-  private VM startLocatorWithClusterConfigurationEnabled() throws Exception {
+  private Member startLocatorWithClusterConfigurationEnabled() throws Exception {
     try {
       jmxHost = InetAddress.getLocalHost().getHostName();
     } catch (UnknownHostException ignore) {
@@ -258,7 +260,7 @@ public class LuceneClusterConfigurationDUnitTest extends CliCommandTestBase {
     locatorProps.setProperty(JMX_MANAGER_PORT, String.valueOf(jmxPort));
     locatorProps.setProperty(HTTP_SERVICE_PORT, String.valueOf(httpPort));
     locatorProps.setProperty(CLUSTER_CONFIGURATION_DIR, dir.getCanonicalPath());
-    return ls.getLocatorVM(0, locatorProps);
+    return ls.startLocatorVM(0, locatorProps);
   }
 
   private void createLuceneIndexUsingGfsh(boolean addGroup) throws Exception {