You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/12/15 23:18:18 UTC

geode git commit: [GEODE-2196] Add test for exportClusterConfig.

Repository: geode
Updated Branches:
  refs/heads/develop b0fbf72f0 -> 9826f5fff


[GEODE-2196] Add test for exportClusterConfig.

- Break ClusterConfigDUnitTest out into multiple test classes.
- Refactor ClusterConfigDUnitTest.ExpectedConfig into a standalone ClusterConfig class.
- Refactor GfshShellConnectionRule to use Awaitility instead of Thread.sleep


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

Branch: refs/heads/develop
Commit: 9826f5fffd0f43ba4d6dec6ed7b90abc9caa57a4
Parents: b0fbf72
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Dec 15 10:36:08 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Dec 15 15:17:15 2016 -0800

----------------------------------------------------------------------
 .../internal/configuration/ClusterConfig.java   | 187 +++++++
 .../configuration/ClusterConfigBaseTest.java    |  57 ++
 .../configuration/ClusterConfigDUnitTest.java   | 545 -------------------
 .../ClusterConfigDeployJarDUnitTest.java        | 187 +++++++
 .../ClusterConfigImportDUnitTest.java           | 142 +++++
 .../ClusterConfigStartMemberDUnitTest.java      | 104 ++++
 .../internal/configuration/ConfigGroup.java     | 118 ++++
 .../dunit/rules/GfshShellConnectionRule.java    |  37 +-
 .../apache/geode/test/dunit/rules/Locator.java  |  28 +
 .../dunit/rules/LocatorServerStartupRule.java   |  33 +-
 .../apache/geode/test/dunit/rules/Member.java   |   2 +-
 .../apache/geode/test/dunit/rules/Server.java   |  27 +
 12 files changed, 893 insertions(+), 574 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
new file mode 100644
index 0000000..f406d51
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
@@ -0,0 +1,187 @@
+/*
+ * 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.LOG_FILE_SIZE_LIMIT;
+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.internal.JarDeployer;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.test.dunit.rules.Locator;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.Member;
+import org.apache.geode.test.dunit.rules.Server;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ClusterConfig implements Serializable {
+  private List<ConfigGroup> groups;
+
+  public ClusterConfig(ConfigGroup... configGroups) {
+    this.groups = new ArrayList<>();
+
+    Collections.addAll(this.groups, configGroups);
+  }
+
+  public String getMaxLogFileSize() {
+    if (this.groups.size() == 0) {
+      return null;
+    }
+    ConfigGroup lastGroupAdded = this.groups.get(this.groups.size() - 1);
+    return lastGroupAdded.getMaxLogFileSize();
+  }
+
+  public List<String> getJarNames() {
+    return groups.stream().flatMap((ConfigGroup configGroup) -> configGroup.getJars().stream())
+        .collect(Collectors.toList());
+  }
+
+  public List<String> getRegions() {
+    return groups.stream().flatMap((ConfigGroup configGroup) -> configGroup.getRegions().stream())
+        .collect(Collectors.toList());
+  }
+
+  public List<ConfigGroup> getGroups() {
+    return Collections.unmodifiableList(groups);
+  }
+
+  public void verify(Locator locator) {
+    verifyLocator(locator);
+  }
+
+  public void verify(Server server) throws ClassNotFoundException {
+    verifyServer(server);
+  }
+
+  public void verifyLocator(Member locator) {
+    Set<String> expectedGroupConfigs =
+        this.getGroups().stream().map(ConfigGroup::getName).collect(Collectors.toSet());
+
+    // verify info exists in memeory
+    locator.invoke(() -> {
+      InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
+      SharedConfiguration sc = internalLocator.getSharedConfiguration();
+
+      // verify no extra configs exist in memory
+      Set<String> actualGroupConfigs = sc.getEntireConfiguration().keySet();
+      assertThat(actualGroupConfigs).isEqualTo(expectedGroupConfigs);
+
+      // verify jars are as expected
+      for (ConfigGroup configGroup : this.getGroups()) {
+        Configuration config = sc.getConfiguration(configGroup.name);
+        assertThat(config.getJarNames()).isEqualTo(configGroup.getJars());
+      }
+
+      // TODO: assert that groupConfig.getXml() contains expected region names
+    });
+
+    File clusterConfigDir = new File(locator.getWorkingDir(), "/cluster_config");
+    Set<String> actualGroupDirs = toSetIgnoringHiddenFiles(clusterConfigDir.list());
+    assertThat(actualGroupDirs).isEqualTo(expectedGroupConfigs);
+
+    for (ConfigGroup configGroup : this.getGroups()) {
+      Set<String> actualFiles =
+          toSetIgnoringHiddenFiles(new File(clusterConfigDir, configGroup.name).list());
+
+      Set<String> expectedFiles = configGroup.getAllFiles();
+      assertThat(actualFiles).isEqualTo(expectedFiles);
+    }
+  }
+
+  public void verifyServer(Member server) throws ClassNotFoundException {
+    // verify files exist in filesystem
+    Set<String> expectedJarNames = this.getJarNames().stream().map(ClusterConfig::getServerJarName)
+        .collect(Collectors.toSet());
+    Set<String> actualJarNames = toSetIgnoringHiddenFiles(
+        server.getWorkingDir().list((dir, filename) -> filename.contains(".jar")));
+    assertThat(actualJarNames).isEqualTo(expectedJarNames);
+
+    // verify config exists in memory
+    server.invoke(() -> {
+      Cache cache = LocatorServerStartupRule.serverStarter.cache;
+
+      // TODO: set compare to fail if there are extra regions
+      for (String region : this.getRegions()) {
+        assertThat(cache.getRegion(region)).isNotNull();
+      }
+
+      if (!StringUtils.isBlank(this.getMaxLogFileSize())) {
+        Properties props = cache.getDistributedSystem().getProperties();
+        assertThat(props.getProperty(LOG_FILE_SIZE_LIMIT)).isEqualTo(this.getMaxLogFileSize());
+      }
+
+      for (String jar : this.getJarNames()) {
+        JarClassLoader jarClassLoader = findJarClassLoader(jar);
+        assertThat(jarClassLoader).isNotNull();
+        assertThat(jarClassLoader.loadClass(nameOfClassContainedInJar(jar))).isNotNull();
+      }
+    });
+  }
+
+  private static 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 Set<String> toSetIgnoringHiddenFiles(String[] array) {
+    return Arrays.stream(array).filter((String name) -> !name.startsWith("."))
+        .collect(Collectors.toSet());
+  }
+
+  private static String getServerJarName(String jarName) {
+    return JarDeployer.JAR_PREFIX + jarName + "#1";
+  }
+
+
+  private static 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(
+            "We don't know what class to expect in the jar named " + jarName);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigBaseTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigBaseTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigBaseTest.java
new file mode 100644
index 0000000..63c88dd
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigBaseTest.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 org.apache.geode.management.internal.configuration;
+
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
+
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.junit.Before;
+import org.junit.Rule;
+
+import java.util.Properties;
+
+public class ClusterConfigBaseTest extends JUnit4DistributedTestCase {
+  public static final String EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME = "cluster_config.zip";
+  public static final String EXPORTED_CLUSTER_CONFIG_PATH =
+      ClusterConfigBaseTest.class.getResource(EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME).getPath();
+
+  public static final ConfigGroup CLUSTER = new ConfigGroup("cluster").regions("regionForCluster")
+      .jars("cluster.jar").maxLogFileSize("5000");
+  public static final ConfigGroup GROUP1 = new ConfigGroup("group1").regions("regionForGroup1")
+      .jars("group1.jar").maxLogFileSize("6000");
+  public static final ConfigGroup GROUP2 = new ConfigGroup("group2").regions("regionForGroup2")
+      .jars("group2.jar").maxLogFileSize("7000");
+
+  public static final ClusterConfig CONFIG_FROM_ZIP = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  protected Properties locatorProps;
+  protected Properties serverProps;
+
+  @Before
+  public void before() throws Exception {
+    locatorProps = new Properties();
+    locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
+
+    serverProps = new Properties();
+    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/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
deleted file mode 100644
index 5655e48..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDUnitTest.java
+++ /dev/null
@@ -1,545 +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 org.apache.geode.management.internal.configuration;
-
-import static java.util.Arrays.stream;
-import static java.util.stream.Collectors.joining;
-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.internal.JarDeployer;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-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.GfshShellConnectionRule;
-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.After;
-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.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-@Category(DistributedTest.class)
-public class ClusterConfigDUnitTest extends JUnit4DistributedTestCase {
-
-  private Properties locatorProps;
-  private Properties serverProps;
-  private GfshShellConnectionRule gfshConnector;
-  @Rule
-  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
-
-  @Before
-  public void before() throws Exception {
-    locatorProps = new Properties();
-    locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
-
-    serverProps = new Properties();
-    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
-  }
-
-  @After
-  public void after() throws Exception {
-    if (gfshConnector != null) {
-      gfshConnector.close();
-    }
-  }
-
-  @Test
-  public void testStartLocator() throws Exception {
-    Member firstLocator = startLocatorWithLoadCCFromDir();
-
-    locatorProps.setProperty(LOCATORS, "localhost[" + firstLocator.getPort() + "]");
-    Member secondLocator = lsRule.startLocatorVM(1, locatorProps);
-
-    verifyClusterConfigZipLoadedInLocator(secondLocator);
-  }
-
-  @Test
-  public void testStartServerWithSingleGroup() throws Exception {
-    Member locator = startLocatorWithLoadCCFromDir();
-
-    Member serverWithNoGroup = lsRule.startServerVM(1, serverProps, locator.getPort());
-    verifyServerConfig(NO_GROUP, serverWithNoGroup);
-
-    serverProps.setProperty(GROUPS, "group1");
-    Member serverForGroup1 = lsRule.startServerVM(2, serverProps, locator.getPort());
-    verifyServerConfig(GROUP1, serverForGroup1);
-
-    serverProps.setProperty(GROUPS, "group2");
-    Member serverForGroup2 = lsRule.startServerVM(3, serverProps, locator.getPort());
-    verifyServerConfig(GROUP2, serverForGroup2);
-  }
-
-  @Test
-  public void testStartServerWithMultipleGroup() throws Exception {
-    Member locator = startLocatorWithLoadCCFromDir();
-
-    serverProps.setProperty(GROUPS, "group1,group2");
-    Member server = lsRule.startServerVM(1, serverProps, locator.getPort());
-
-    verifyServerConfig(GROUP1_AND_2, server);
-  }
-
-  @Test
-  public void testImportWithRunningServer() throws Exception {
-    String zipFilePath = getClass().getResource(EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME).getPath();
-    // set up the locator/servers
-    Member locator = lsRule.startLocatorVM(0, locatorProps);
-    Member server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
-    gfshConnector =
-        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
-    gfshConnector.connect();
-    CommandResult result =
-        gfshConnector.executeCommand("import cluster-configuration --zip-file-name=" + zipFilePath);
-
-    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
-  }
-
-  @Test
-  public void testImportClusterConfig() throws Exception {
-    String zipFilePath = getClass().getResource(EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME).getPath();
-    // set up the locator/servers
-    Member locator = lsRule.startLocatorVM(0, locatorProps);
-    verifyInitialLocatorConfigInFileSystem(locator);
-
-    gfshConnector =
-        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
-    gfshConnector.connect();
-    assertThat(gfshConnector.isConnected()).isTrue();
-
-    CommandResult result =
-        gfshConnector.executeCommand("import cluster-configuration --zip-file-name=" + zipFilePath);
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    // verify that the previous folder is copied to "cluster_configxxxxxx".
-    String workingDirFiles = Arrays.stream(locator.getWorkingDir().listFiles()).map(File::getName)
-        .collect(joining(", "));
-    System.out.println("Locator working dir contains: " + workingDirFiles);
-    assertThat(locator.getWorkingDir().listFiles())
-        .filteredOn((File file) -> file.getName() != "cluster_config")
-        .filteredOn((File file) -> file.getName().startsWith("cluster_config")).isNotEmpty();
-    verifyClusterConfigZipLoadedInLocator(locator);
-
-    // start server1 with no group
-    Member server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
-    verifyServerConfig(NO_GROUP, server1);
-
-    // start server2 in group1
-    serverProps.setProperty(GROUPS, "group1");
-    Member server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
-    verifyServerConfig(GROUP1, server2);
-
-    // start server3 in group1 and group2
-    serverProps.setProperty(GROUPS, "group1,group2");
-    Member server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
-    verifyServerConfig(GROUP1_AND_2, server3);
-  }
-
-  @Test
-  public void testDeployToNoServer() throws Exception {
-    String clusterJarPath = getClass().getResource("cluster.jar").getPath();
-    // set up the locator/servers
-    Member locator = lsRule.startLocatorVM(0, locatorProps);
-
-    gfshConnector =
-        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
-    gfshConnector.connect();
-    assertThat(gfshConnector.isConnected()).isTrue();
-
-    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJarPath);
-    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
-  }
-
-  @Test
-  public void testDeploy() throws Exception {
-    String clusterJar = getClass().getResource("cluster.jar").getPath();
-    String group1Jar = getClass().getResource("group1.jar").getPath();
-    String group2Jar = getClass().getResource("group2.jar").getPath();
-
-    // set up the locator/servers
-    Member locator = lsRule.startLocatorVM(0, locatorProps);
-    // server1 in no group
-    Member server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
-    // server2 in group1
-    serverProps.setProperty(GROUPS, "group1");
-    Member server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
-    // server3 in group1 and group2
-    serverProps.setProperty(GROUPS, "group1,group2");
-    Member server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
-
-    gfshConnector =
-        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
-    gfshConnector.connect();
-    assertThat(gfshConnector.isConnected()).isTrue();
-
-    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJar);
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    ExpectedConfig cluster = new ExpectedConfig().jars("cluster.jar").name("cluster");
-    verifyLocatorConfig(cluster, locator);
-    verifyLocatorConfigNotExist("group1", locator);
-    verifyLocatorConfigNotExist("group2", locator);
-    verifyServerConfig(cluster, server1);
-    verifyServerConfig(cluster, server2);
-    verifyServerConfig(cluster, server3);
-
-    result = gfshConnector.executeCommand("deploy --jar=" + group1Jar + " --group=group1");
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    ExpectedConfig serverGroupOne = new ExpectedConfig().jars("group1.jar", "cluster.jar");
-    ExpectedConfig locatorGroupOne = new ExpectedConfig().jars("group1.jar").name("group1");
-    verifyLocatorConfig(cluster, locator);
-    verifyLocatorConfig(locatorGroupOne, locator);
-    verifyLocatorConfigNotExist("group2", locator);
-    verifyServerConfig(cluster, server1);
-    verifyServerConfig(serverGroupOne, server2);
-    verifyServerConfig(serverGroupOne, server3);
-
-    result = gfshConnector.executeCommand("deploy --jar=" + group2Jar + " --group=group2");
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    ExpectedConfig groupOneAndTwo =
-        new ExpectedConfig().jars("group1.jar", "group2.jar", "cluster.jar");
-    ExpectedConfig locatorGroupTwo = new ExpectedConfig().jars("group2.jar").name("group2");
-    verifyLocatorConfig(cluster, locator);
-    verifyLocatorConfig(locatorGroupOne, locator);
-    verifyLocatorConfig(locatorGroupTwo, locator);
-    verifyServerConfig(cluster, server1);
-    verifyServerConfig(serverGroupOne, server2);
-    verifyServerConfig(groupOneAndTwo, server3);
-  }
-
-  @Test
-  public void testDeployMultiGroup() throws Exception {
-    String clusterJar = getClass().getResource("cluster.jar").getPath();
-    String group1Jar = getClass().getResource("group1.jar").getPath();
-
-    // set up the locator/servers
-    Member locator = lsRule.startLocatorVM(0, locatorProps);
-    // start 2 servers in the both groups
-    serverProps.setProperty(GROUPS, "group1");
-    Member server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
-    serverProps.setProperty(GROUPS, "group2");
-    Member server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
-    serverProps.setProperty(GROUPS, "group1,group2");
-    Member server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
-
-    ExpectedConfig clusterConfig = new ExpectedConfig().name("cluster");
-    ExpectedConfig group1Config = new ExpectedConfig().name("group1");
-    ExpectedConfig group2Config = new ExpectedConfig().name("group2");
-
-    gfshConnector =
-        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
-    gfshConnector.connect();
-    assertThat(gfshConnector.isConnected()).isTrue();
-
-    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJar);
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    // deploy cluster.jar to the cluster
-    clusterConfig.addJar("cluster.jar");
-    verifyLocatorConfig(clusterConfig, locator);
-    verifyLocatorConfigNotExist("group1", locator);
-    verifyLocatorConfigNotExist("group2", locator);
-    verifyServerConfig(clusterConfig, server1);
-    verifyServerConfig(clusterConfig, server2);
-    verifyServerConfig(clusterConfig, server3);
-
-    // deploy group1.jar to both group1 and group2
-    result = gfshConnector.executeCommand("deploy --jar=" + group1Jar + " --group=group1,group2");
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-    group1Config.addJar("group1.jar");
-    group2Config.addJar("group1.jar");
-    ExpectedConfig serverConfig = new ExpectedConfig().jars("cluster.jar", "group1.jar");
-    verifyLocatorConfig(clusterConfig, locator);
-    verifyLocatorConfig(group1Config, locator);
-    verifyLocatorConfig(group2Config, locator);
-    verifyServerConfig(serverConfig, server1);
-    verifyServerConfig(serverConfig, server2);
-    verifyServerConfig(serverConfig, server3);
-
-    // test undeploy cluster
-    result = gfshConnector.executeCommand("undeploy --jar=cluster.jar");
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    clusterConfig.removeJar("cluster.jar");
-    verifyLocatorConfig(clusterConfig, locator);
-    verifyLocatorConfig(group1Config, locator);
-    verifyLocatorConfig(group2Config, locator);
-    serverConfig.removeJar("cluster.jar");
-    verifyServerConfig(serverConfig, server1);
-    verifyServerConfig(serverConfig, server2);
-    verifyServerConfig(serverConfig, server2);
-
-    result = gfshConnector.executeCommand("undeploy --jar=group1.jar --group=group1");
-    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
-
-    group1Config.removeJar("group1.jar");
-    verifyLocatorConfig(clusterConfig, locator);
-    verifyLocatorConfig(group1Config, locator);
-    verifyLocatorConfig(group2Config, locator);
-    // server2 is not in group1, so serverConfig remains unchanged
-    verifyServerConfig(serverConfig, server2);
-
-    // server1 and server3 is in group1, so their group1.jar is removed
-    serverConfig.removeJar("group1.jar");
-    verifyServerConfig(serverConfig, server1);
-    verifyServerConfig(serverConfig, server3);
-  }
-
-
-  private Member startLocatorWithLoadCCFromDir() throws Exception {
-    File locatorDir = lsRule.getRootFolder().newFolder("locator-0");
-    File configDir = new File(locatorDir, "cluster_config");
-
-    // 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(),
-        configDir.getCanonicalPath());
-
-    Properties properties = new Properties();
-    properties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
-    properties.setProperty(LOAD_CLUSTER_CONFIGURATION_FROM_DIR, "true");
-    properties.setProperty(CLUSTER_CONFIGURATION_DIR, locatorDir.getCanonicalPath());
-
-    Member locator = lsRule.startLocatorVM(0, properties);
-    verifyClusterConfigZipLoadedInLocator(locator);
-
-    return locator;
-  }
-
-  private static String getServerJarName(String jarName) {
-    return JarDeployer.JAR_PREFIX + jarName + "#1";
-  }
-
-
-  public static final String EXPORTED_CLUSTER_CONFIG_ZIP_FILENAME = "cluster_config.zip";
-  public static final String[] CONFIG_NAMES = new String[] {"cluster", "group1", "group2"};
-
-  public static final ExpectedConfig NO_GROUP =
-      new ExpectedConfig().maxLogFileSize("5000").regions("regionForCluster").jars("cluster.jar");
-
-  public static final ExpectedConfig GROUP1 = new ExpectedConfig().maxLogFileSize("6000")
-      .regions("regionForCluster", "regionForGroup1").jars("cluster.jar", "group1.jar");
-
-  public static final ExpectedConfig GROUP2 = new ExpectedConfig().maxLogFileSize("7000")
-      .regions("regionForCluster", "regionForGroup2").jars("cluster.jar", "group2.jar");
-
-  public static final ExpectedConfig GROUP1_AND_2 = new ExpectedConfig().maxLogFileSize("7000")
-      .regions("regionForCluster", "regionForGroup1", "regionForGroup2")
-      .jars("cluster.jar", "group1.jar", "group2.jar");
-
-
-  public static void verifyInitialLocatorConfigInFileSystem(Member member) {
-    File clusterConfigDir = new File(member.getWorkingDir(), "cluster_config");
-    assertThat(clusterConfigDir).exists();
-    File configDir = new File(clusterConfigDir, "cluster");
-    assertThat(configDir).exists();
-    File properties = new File(configDir, "cluster.properties");
-    assertThat(properties).exists();
-    File xml = new File(configDir, "cluster.xml");
-    assertThat(xml).exists();
-  }
-
-  public static void verifyClusterConfigZipLoadedInLocator(Member locator) {
-    // verify loaded in memeory
-    locator.invoke(() -> {
-      InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
-      SharedConfiguration sc = internalLocator.getSharedConfiguration();
-
-      for (String configName : CONFIG_NAMES) {
-        Configuration config = sc.getConfiguration(configName);
-        assertThat(config).isNotNull();
-      }
-    });
-
-    // verify loaded into the file system
-    File clusterConfigDir = new File(locator.getWorkingDir(), "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);
-    }
-  }
-
-  public static void verifyServerConfig(ExpectedConfig expectedConfig, Member server)
-      throws ClassNotFoundException {
-    verifyServerJarFilesExistInFileSystem(server.getWorkingDir(), expectedConfig.jars);
-    server.invoke(() -> verifyServerConfigInMemory(expectedConfig));
-  }
-
-  public static void verifyLocatorConfig(ExpectedConfig expectedConfig, Member locator) {
-    // verify info exists in memeory
-    locator.invoke(() -> {
-      InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
-      SharedConfiguration sc = internalLocator.getSharedConfiguration();
-      Configuration config = sc.getConfiguration(expectedConfig.name);
-      assertThat(config.getJarNames()).isEqualTo(expectedConfig.jars);
-    });
-
-    // verify files exists on disc
-    for (String jar : expectedConfig.jars) {
-      assertThat(
-          new File(locator.getWorkingDir(), "/cluster_config/" + expectedConfig.name + "/" + jar))
-              .exists();
-    }
-  }
-
-  public static void verifyLocatorConfigNotExist(String configName, Member locator) {
-    // verify info not in memeory
-    locator.invoke(() -> {
-      InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
-      SharedConfiguration sc = internalLocator.getSharedConfiguration();
-      Configuration config = sc.getConfiguration(configName);
-      assertThat(config).isNull();
-    });
-
-    // verify files does not
-    assertThat(new File(locator.getWorkingDir(), "/cluster_config/" + configName)).doesNotExist();
-  }
-
-  private static void verifyServerConfigInMemory(ExpectedConfig expectedConfig)
-      throws ClassNotFoundException {
-    Cache cache = LocatorServerStartupRule.serverStarter.cache;
-
-    for (String region : expectedConfig.regions) {
-      assertThat(cache.getRegion(region)).isNotNull();
-    }
-
-    if (!StringUtils.isBlank(expectedConfig.maxLogFileSize)) {
-      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 static void verifyServerJarFilesExistInFileSystem(File workingDir, Set<String> jarNames) {
-    Set<String> expectedJarNames = new HashSet<>();
-    for (String jarName : jarNames) {
-      expectedJarNames.add(getServerJarName(jarName));
-    }
-    Set<String> actualJarNames =
-        Arrays.stream(workingDir.list((dir, filename) -> filename.contains(".jar")))
-            .collect(Collectors.toSet());
-    assertThat(actualJarNames).isEqualTo(expectedJarNames);
-  }
-
-  private static 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 static 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 Set<String> regions = new HashSet<>();
-    public Set<String> jars = new HashSet<>();
-    public String name;
-
-    public ExpectedConfig maxLogFileSize(String maxLogFileSize) {
-      this.maxLogFileSize = maxLogFileSize;
-      return this;
-    }
-
-    public ExpectedConfig regions(String... regions) {
-      this.regions.addAll(Arrays.asList(regions));
-      return this;
-    }
-
-    public ExpectedConfig jars(String... jars) {
-      this.jars.addAll(Arrays.asList(jars));
-      return this;
-    }
-
-    public ExpectedConfig removeJar(String jar) {
-      this.jars.remove(jar);
-      return this;
-    }
-
-    public ExpectedConfig addJar(String jar) {
-      this.jars.add(jar);
-      return this;
-    }
-
-    public ExpectedConfig name(String name) {
-      this.name = name;
-      return this;
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
new file mode 100644
index 0000000..da4fcf9
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.GROUPS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.Locator;
+import org.apache.geode.test.dunit.rules.Server;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
+  private GfshShellConnectionRule gfshConnector;
+
+  @After
+  public void after() throws Exception {
+    if (gfshConnector != null) {
+      gfshConnector.close();
+    }
+  }
+
+  @Test
+  public void testDeployToNoServer() throws Exception {
+    String clusterJarPath = getClass().getResource("cluster.jar").getPath();
+    // set up the locator/servers
+    Locator locator = lsRule.startLocatorVM(0, locatorProps);
+
+    gfshConnector = new GfshShellConnectionRule(locator);
+    gfshConnector.connect();
+    assertThat(gfshConnector.isConnected()).isTrue();
+
+    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJarPath);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+  }
+
+  @Test
+  public void testDeploy() throws Exception {
+    String clusterJar = getClass().getResource("cluster.jar").getPath();
+    String group1Jar = getClass().getResource("group1.jar").getPath();
+    String group2Jar = getClass().getResource("group2.jar").getPath();
+
+    // set up the locator/servers
+    Locator locator = lsRule.startLocatorVM(0, locatorProps);
+    // server1 in no group
+    Server server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    // server2 in group1
+    serverProps.setProperty(GROUPS, "group1");
+    Server server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    // server3 in group1 and group2
+    serverProps.setProperty(GROUPS, "group1,group2");
+    Server server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
+
+    gfshConnector =
+        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
+    gfshConnector.connect();
+    assertThat(gfshConnector.isConnected()).isTrue();
+
+    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJar);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    ConfigGroup cluster = new ConfigGroup("cluster").jars("cluster.jar");
+    ClusterConfig expectedClusterConfig = new ClusterConfig(cluster);
+    expectedClusterConfig.verify(locator);
+    expectedClusterConfig.verify(server1);
+    expectedClusterConfig.verify(server2);
+    expectedClusterConfig.verify(server3);
+
+    result = gfshConnector.executeCommand("deploy --jar=" + group1Jar + " --group=group1");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    ConfigGroup group1 = new ConfigGroup("group1").jars("group1.jar");
+    ClusterConfig expectedGroup1Config = new ClusterConfig(cluster, group1);
+    expectedGroup1Config.verify(locator);
+    expectedClusterConfig.verify(server1);
+    expectedGroup1Config.verify(server2);
+    expectedGroup1Config.verify(server3);
+
+    result = gfshConnector.executeCommand("deploy --jar=" + group2Jar + " --group=group2");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    ConfigGroup group2 = new ConfigGroup("group2").jars("group2.jar");
+    ClusterConfig expectedGroup1and2Config = new ClusterConfig(cluster, group1, group2);
+
+    expectedGroup1and2Config.verify(locator);
+    expectedClusterConfig.verify(server1);
+    expectedGroup1Config.verify(server2);
+    expectedGroup1and2Config.verify(server3);
+  }
+
+
+  @Test
+  public void testUndeployMultiGroup() throws Exception {
+    String clusterJar = getClass().getResource("cluster.jar").getPath();
+    String group1Jar = getClass().getResource("group1.jar").getPath();
+
+    // set up the locator/servers
+    Locator locator = lsRule.startLocatorVM(0, locatorProps);
+    serverProps.setProperty(GROUPS, "group1");
+    Server server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    serverProps.setProperty(GROUPS, "group2");
+    Server server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    serverProps.setProperty(GROUPS, "group1,group2");
+    Server server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
+
+    ConfigGroup cluster = new ConfigGroup("cluster");
+    ConfigGroup group1 = new ConfigGroup("group1");
+    ConfigGroup group2 = new ConfigGroup("group2");
+    ClusterConfig expectedClusterConfig = new ClusterConfig(cluster);
+    ClusterConfig server1Config = new ClusterConfig(cluster, group1);
+    ClusterConfig server2Config = new ClusterConfig(cluster, group2);
+    ClusterConfig server3Config = new ClusterConfig(cluster, group1, group2);
+
+    gfshConnector =
+        new GfshShellConnectionRule(locator.getPort(), GfshShellConnectionRule.PortType.locator);
+    gfshConnector.connect();
+    assertThat(gfshConnector.isConnected()).isTrue();
+
+    CommandResult result = gfshConnector.executeCommand("deploy --jar=" + clusterJar);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    // deploy cluster.jar to the cluster
+    cluster.addJar("cluster.jar");
+    expectedClusterConfig.verify(locator);
+    expectedClusterConfig.verify(server1);
+    expectedClusterConfig.verify(server2);
+    expectedClusterConfig.verify(server3);
+
+    // deploy group1.jar to both group1 and group2
+    result = gfshConnector.executeCommand("deploy --jar=" + group1Jar + " --group=group1,group2");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    group1.addJar("group1.jar");
+    group2.addJar("group1.jar");
+    server3Config.verify(locator);
+    server1Config.verify(server1);
+    server2Config.verify(server2);
+    server3Config.verify(server3);
+
+    // test undeploy cluster
+    result = gfshConnector.executeCommand("undeploy --jar=cluster.jar");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    cluster = cluster.removeJar("cluster.jar");
+    server3Config.verify(locator);
+    server1Config.verify(server1);
+    server2Config.verify(server2);
+    server3Config.verify(server3);
+
+    result = gfshConnector.executeCommand("undeploy --jar=group1.jar --group=group1");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    group1 = group1.removeJar("group1.jar");
+    /*
+     * TODO: This is the current (weird) behavior If you started server4 with group1,group2 after
+     * this undeploy command, it would have group1.jar (brought from
+     * cluster_config/group2/group1.jar on locator) whereas server3 (also in group1,group2) does not
+     * have this jar.
+     */
+    ClusterConfig weirdServer3Config =
+        new ClusterConfig(cluster, group1, new ConfigGroup(group2).removeJar("group1.jar"));
+
+    server3Config.verify(locator);
+    server1Config.verify(server1);
+    server2Config.verify(server2);
+    weirdServer3Config.verify(server3);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
new file mode 100644
index 0000000..c135f3d
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.GROUPS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.Locator;
+import org.apache.geode.test.dunit.rules.Server;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+@Category(DistributedTest.class)
+public class ClusterConfigImportDUnitTest extends ClusterConfigBaseTest {
+  private GfshShellConnectionRule gfshConnector;
+
+  public static final ClusterConfig INITIAL_CONFIG = new ClusterConfig(new ConfigGroup("cluster"));
+
+  private Locator locator;
+
+  @Before
+  public void before() throws Exception {
+    super.before();
+    locator = lsRule.startLocatorVM(0, locatorProps);
+    INITIAL_CONFIG.verify(locator);
+
+    gfshConnector = new GfshShellConnectionRule(locator);
+    gfshConnector.connect();
+    assertThat(gfshConnector.isConnected()).isTrue();
+  }
+
+
+  @After
+  public void after() throws Exception {
+    if (gfshConnector != null) {
+      gfshConnector.close();
+    }
+  }
+
+  @Test
+  public void testImportWithRunningServer() throws Exception {
+    Server server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+
+    CommandResult result = gfshConnector.executeCommand(
+        "import cluster-configuration --zip-file-name=" + EXPORTED_CLUSTER_CONFIG_PATH);
+
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+  }
+
+  @Test
+  public void testImportClusterConfig() throws Exception {
+    CommandResult result = gfshConnector.executeCommand(
+        "import cluster-configuration --zip-file-name=" + EXPORTED_CLUSTER_CONFIG_PATH);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    // verify that the previous folder is backed up to "cluster_configxxxxxx".
+    assertThat(locator.getWorkingDir().listFiles())
+        .filteredOn((File file) -> !file.getName().equals("cluster_config"))
+        .filteredOn((File file) -> file.getName().startsWith("cluster_config")).isNotEmpty();
+    CONFIG_FROM_ZIP.verify(locator);
+
+    // start server1 with no group
+    Server server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    new ClusterConfig(CLUSTER).verify(server1);
+
+    // start server2 in group1
+    serverProps.setProperty(GROUPS, "group1");
+    Server server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    new ClusterConfig(CLUSTER, GROUP1).verify(server2);
+
+    // start server3 in group1 and group2
+    serverProps.setProperty(GROUPS, "group1,group2");
+    Server server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
+    new ClusterConfig(CLUSTER, GROUP1, GROUP2).verify(server3);
+  }
+
+  @Test
+  public void testExportClusterConfig() throws Exception {
+    Server server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+
+    CommandResult result =
+        gfshConnector.executeCommand("create region --name=myRegion --type=REPLICATE");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    ConfigGroup cluster = new ConfigGroup("cluster").regions("myRegion");
+    ClusterConfig expectedClusterConfig = new ClusterConfig(cluster);
+    expectedClusterConfig.verify(server1);
+    expectedClusterConfig.verify(locator);
+
+    Path exportedZipPath = lsRule.getTempFolder().getRoot().toPath().resolve("exportedCC.zip");
+    result = gfshConnector
+        .executeCommand("export cluster-configuration --zip-file-name=" + exportedZipPath);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
+
+    File exportedZip = exportedZipPath.toFile();
+    assertThat(exportedZip).exists();
+
+    Set<String> actualZipEnries =
+        new ZipFile(exportedZip).stream().map(ZipEntry::getName).collect(Collectors.toSet());
+
+    Set<String> expectedZipEntries = new HashSet<>();
+    for (ConfigGroup group : expectedClusterConfig.getGroups()) {
+      String groupDir = group.getName() + "/";
+
+      expectedZipEntries.add(groupDir);
+      for (String jarOrXmlOrPropFile : group.getAllFiles()) {
+        expectedZipEntries.add(groupDir + jarOrXmlOrPropFile);
+      }
+    }
+
+    assertThat(actualZipEnries).isEqualTo(expectedZipEntries);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
new file mode 100644
index 0000000..cfcbeed
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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 org.apache.geode.management.internal.configuration.utils.ZipUtils;
+import org.apache.geode.test.dunit.rules.Locator;
+import org.apache.geode.test.dunit.rules.Server;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.util.Properties;
+
+@Category(DistributedTest.class)
+public class ClusterConfigStartMemberDUnitTest extends ClusterConfigBaseTest {
+  protected Locator locator;
+
+  @Before
+  public void before() throws Exception {
+    super.before();
+    locator = startLocatorWithLoadCCFromDir();
+  }
+
+  @Test
+  public void testStartLocator() throws Exception {
+    locatorProps.setProperty(LOCATORS, "localhost[" + locator.getPort() + "]");
+    Locator secondLocator = lsRule.startLocatorVM(1, locatorProps);
+
+    CONFIG_FROM_ZIP.verify(secondLocator);
+  }
+
+  @Test
+  public void testStartServerWithSingleGroup() throws Exception {
+    ClusterConfig expectedNoGroupConfig = new ClusterConfig(CLUSTER);
+    ClusterConfig expectedGroup1Config = new ClusterConfig(CLUSTER, GROUP1);
+    ClusterConfig expectedGroup2Config = new ClusterConfig(CLUSTER, GROUP2);
+
+    Server serverWithNoGroup = lsRule.startServerVM(1, serverProps, locator.getPort());
+    expectedNoGroupConfig.verify(serverWithNoGroup);
+
+    serverProps.setProperty(GROUPS, "group1");
+    Server serverForGroup1 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    expectedGroup1Config.verify(serverForGroup1);
+
+    serverProps.setProperty(GROUPS, "group2");
+    Server serverForGroup2 = lsRule.startServerVM(3, serverProps, locator.getPort());
+    expectedGroup2Config.verify(serverForGroup2);
+  }
+
+  @Test
+  public void testStartServerWithMultipleGroup() throws Exception {
+    ClusterConfig expectedGroup1And2Config = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
+
+    serverProps.setProperty(GROUPS, "group1,group2");
+    Server server = lsRule.startServerVM(1, serverProps, locator.getPort());
+
+    expectedGroup1And2Config.verify(server);
+  }
+
+  private Locator startLocatorWithLoadCCFromDir() throws Exception {
+    File locatorDir = lsRule.getTempFolder().newFolder("locator-0");
+    File configDir = new File(locatorDir, "cluster_config");
+
+    // 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(EXPORTED_CLUSTER_CONFIG_PATH, configDir.getCanonicalPath());
+
+    Properties properties = new Properties();
+    properties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
+    properties.setProperty(LOAD_CLUSTER_CONFIGURATION_FROM_DIR, "true");
+    properties.setProperty(CLUSTER_CONFIGURATION_DIR, locatorDir.getCanonicalPath());
+
+    Locator locator = lsRule.startLocatorVM(0, properties);
+    CONFIG_FROM_ZIP.verify(locator);
+
+    return locator;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ConfigGroup.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ConfigGroup.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ConfigGroup.java
new file mode 100644
index 0000000..971f667
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ConfigGroup.java
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ *
+ */
+
+/*
+ * 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
+ * thisright 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
+ * this 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 java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class ConfigGroup implements Serializable {
+  public String name;
+  private Set<String> jars = new HashSet<>();
+  private Set<String> configFiles = new HashSet<>();
+  private Set<String> regions = new HashSet<>();
+  private String maxLogFileSize;
+
+  public ConfigGroup(ConfigGroup that) {
+    this.jars.addAll(that.jars);
+    this.configFiles.addAll(that.configFiles);
+    this.regions.addAll(that.regions);
+    this.maxLogFileSize = that.maxLogFileSize;
+    this.name = that.name;
+  }
+
+  public ConfigGroup(String name) {
+    this.name = name;
+    this.configFiles.add(name + ".xml");
+    this.configFiles.add(name + ".properties");
+  }
+
+  public ConfigGroup regions(String... regions) {
+    this.regions.addAll(Arrays.asList(regions));
+    return this;
+  }
+
+  public ConfigGroup jars(String... jars) {
+    this.jars.addAll(Arrays.asList(jars));
+    return this;
+  }
+
+  public ConfigGroup configFiles(String... configFiles) {
+    this.configFiles.addAll(Arrays.asList(configFiles));
+    return this;
+  }
+
+  public ConfigGroup removeJar(String jar) {
+    this.jars.remove(jar);
+    return this;
+  }
+
+  public ConfigGroup addJar(String jar) {
+    this.jars.add(jar);
+    return this;
+  }
+
+  public ConfigGroup maxLogFileSize(String maxLogFileSize) {
+    this.maxLogFileSize = maxLogFileSize;
+    return this;
+  }
+
+  public Set<String> getJars() {
+    return Collections.unmodifiableSet(this.jars);
+  }
+
+  public Set<String> getConfigFiles() {
+    return Collections.unmodifiableSet(this.configFiles);
+  }
+
+  public Set<String> getAllFiles() {
+    return Collections.unmodifiableSet(
+        Stream.concat(this.jars.stream(), this.configFiles.stream()).collect(Collectors.toSet()));
+  }
+
+  public Set<String> getRegions() {
+    return Collections.unmodifiableSet(this.regions);
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public String getMaxLogFileSize() {
+    return this.maxLogFileSize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
index 04807ec..6af9cf2 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.test.dunit.rules;
 
+import com.jayway.awaitility.Awaitility;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.HeadlessGfsh;
@@ -23,6 +24,9 @@ import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.junit.rules.DescribedExternalResource;
 import org.junit.runner.Description;
 
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
 /**
  * Class which eases the connection to the jmxManager {@link ConnectionConfiguration} it allows for
  * the creation of per-test connections with different user/password combinations, or no username
@@ -35,6 +39,10 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
   private HeadlessGfsh gfsh;
   private boolean connected;
 
+  public GfshShellConnectionRule(Locator locator) {
+    this(locator.getPort(), PortType.locator);
+  }
+
   public GfshShellConnectionRule(int port, PortType portType) {
     this.portType = portType;
     this.port = port;
@@ -59,7 +67,6 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
   public void connect(String... options) throws Exception {
     CliUtil.isGfshVM = true;
     final CommandStringBuilder connectCommand = new CommandStringBuilder(CliStrings.CONNECT);
-
     String endpoint;
     if (portType == PortType.locator) {
       // port is the locator port
@@ -85,21 +92,17 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     // javax.naming.CommunicationException [Root exception is java.rmi.NoSuchObjectException: no
     // such object in table]" Exception.
     // Tried to wait on jmx connector server being ready, but it doesn't work.
-    // Add the retry logic here to try at most 10 times for connection.
-    CommandResult result = null;
-    for (int i = 0; i < 50; i++) {
-      System.out.println("trying to connect, attempt " + i);
+    AtomicReference<CommandResult> result = new AtomicReference<>();
+    Awaitility.await().atMost(2, TimeUnit.MINUTES).pollDelay(2, TimeUnit.SECONDS).until(() -> {
       gfsh.executeCommand(connectCommand.toString());
-      result = (CommandResult) gfsh.getResult();
-      System.out.println(gfsh.outputString);
-      if (!gfsh.outputString.contains("no such object in table")) {
-        break;
-      }
-      Thread.currentThread().sleep(2000);
-    }
-    connected = (result.getStatus() == Result.Status.OK);
+      result.set((CommandResult) gfsh.getResult());
+      return !gfsh.outputString.contains("no such object in table");
+    });
+
+    connected = (result.get().getStatus() == Result.Status.OK);
   }
 
+
   /**
    * Override to tear down your specific external resource.
    */
@@ -118,6 +121,14 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     CliUtil.isGfshVM = false;
   }
 
+  public void closeQuietly() {
+    try {
+      this.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
   public HeadlessGfsh getGfsh() {
     return gfsh;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Locator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Locator.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Locator.java
new file mode 100644
index 0000000..0350ffd
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Locator.java
@@ -0,0 +1,28 @@
+/*
+ * 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.VM;
+
+import java.io.File;
+
+public class Locator extends Member {
+
+  public Locator(VM vm, int port, File workingDir) {
+    super(vm, port, workingDir);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/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 3d43b93..04fad5d 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
@@ -30,6 +30,8 @@ import org.junit.rules.TemporaryFolder;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Properties;
 
 
@@ -55,21 +57,21 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
       new DistributedRestoreSystemProperties();
 
   private TemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
-  private Member[] members;
+  private List<Member> members;
 
   @Before
   public void before() throws Throwable {
     restoreSystemProperties.before();
     temporaryFolder.create();
-    Invoke.invokeInEveryVM("Stop each VM", this::stop);
-    members = new Member[4];
+    Invoke.invokeInEveryVM("Stop each VM", this::stopServerOrLocatorInThisVM);
+    members = new ArrayList<>(4);
   }
 
   @After
   public void after() {
     restoreSystemProperties.after();
     temporaryFolder.delete();
-    Invoke.invokeInEveryVM("Stop each VM", this::stop);
+    Invoke.invokeInEveryVM("Stop each VM", this::stopServerOrLocatorInThisVM);
   }
 
   /**
@@ -78,7 +80,7 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
    *
    * @return VM locator vm
    */
-  public Member startLocatorVM(int index, Properties locatorProperties) throws IOException {
+  public Locator startLocatorVM(int index, Properties locatorProperties) throws IOException {
     String name = "locator-" + index;
     locatorProperties.setProperty(NAME, name);
     File workingDir = createWorkingDirForMember(name);
@@ -90,8 +92,9 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
       locatorStarter.startLocator();
       return locatorStarter.locator.getPort();
     });
-    members[index] = new Member(locatorVM, locatorPort, workingDir);
-    return members[index];
+    Locator locator = new Locator(locatorVM, locatorPort, workingDir);
+    members.add(index, locator);
+    return locator;
   }
 
   /**
@@ -99,14 +102,14 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
    * 
    * @return VM node vm
    */
-  public Member startServerVM(int index, Properties properties) throws IOException {
+  public Server 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.
    */
-  public Member startServerVM(int index, Properties properties, int locatorPort)
+  public Server startServerVM(int index, Properties properties, int locatorPort)
       throws IOException {
     String name = "server-" + index;
     properties.setProperty(NAME, name);
@@ -119,22 +122,23 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
       serverStarter.startServer(locatorPort);
       return serverStarter.server.getPort();
     });
-    members[index] = new Member(serverVM, port, workingDir);
-    return members[index];
+    Server server = new Server(serverVM, port, workingDir);
+    members.add(index, server);
+    return server;
   }
 
   /**
    * Returns the {@link Member} running inside the VM with the specified {@code index}
    */
   public Member getMember(int index) {
-    return members[index];
+    return members.get(index);
   }
 
-  public TemporaryFolder getRootFolder() {
+  public TemporaryFolder getTempFolder() {
     return temporaryFolder;
   }
 
-  public final void stop() {
+  private void stopServerOrLocatorInThisVM() {
     if (serverStarter != null) {
       serverStarter.after();
     }
@@ -151,5 +155,4 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
 
     return workingDir;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/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
index 10b6743..129fe68 100644
--- 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
@@ -25,7 +25,7 @@ import java.io.Serializable;
 /**
  * A server or locator inside a DUnit {@link VM}.
  */
-public class Member implements Serializable {
+public abstract class Member implements Serializable {
   private VM vm;
   private int port;
   private File workingDir;

http://git-wip-us.apache.org/repos/asf/geode/blob/9826f5ff/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
new file mode 100644
index 0000000..bc4c5d6
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
@@ -0,0 +1,27 @@
+/*
+ * 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.VM;
+
+import java.io.File;
+
+public class Server extends Member {
+  public Server(VM vm, int port, File workingDir) {
+    super(vm, port, workingDir);
+  }
+}