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/05/06 21:40:49 UTC

[18/37] incubator-geode git commit: GEODE-1255: Add test coverage for using a custom log4j2.xml with Geode

GEODE-1255: Add test coverage for using a custom log4j2.xml with Geode

* refactor launcher integration tests (rename files, use TemporaryFolder)
* refactor cluster config tests (use TemporaryFolder)
* change several additional tests to use TemporaryFolder
* add better support for TemporaryFolder usage to HeadlessGfsh
* alter HeadlessGfsh tests to use TemporaryFolder
* rescope variables and methods to private where possible
* fixup file formatting
* prevent eating of exceptions
* remove incomplete or useless javadocs
* rename LocatorJUnitTest test methods from TRAC #s to meaningful names


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

Branch: refs/heads/feature/GEODE-1276
Commit: 566fce96aa8759d14b9825577116293c1578147e
Parents: 024cd22
Author: Kirk Lund <kl...@apache.org>
Authored: Thu May 5 15:46:26 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu May 5 15:48:30 2016 -0700

----------------------------------------------------------------------
 .../LocatorLauncherAssemblyIntegrationTest.java |  155 ++
 .../LocatorLauncherAssemblyJUnitTest.java       |  156 --
 .../SharedConfigurationEndToEndDUnitTest.java   |  311 ++--
 .../cache/operations/OperationContext.java      |    4 -
 .../internal/SharedConfiguration.java           |  532 +++----
 .../AbstractLauncherIntegrationJUnitTest.java   |   71 -
 .../AbstractLauncherIntegrationTest.java        |   71 +
 .../AbstractLauncherIntegrationTestCase.java    |  254 ++++
 .../distributed/AbstractLauncherJUnitTest.java  |  298 ----
 .../AbstractLauncherJUnitTestCase.java          |  254 ----
 .../AbstractLauncherServiceStatusJUnitTest.java |  264 ----
 .../AbstractLauncherServiceStatusTest.java      |  264 ++++
 .../distributed/AbstractLauncherTest.java       |  298 ++++
 ...tractLocatorLauncherIntegrationTestCase.java |  117 ++
 .../AbstractLocatorLauncherJUnitTestCase.java   |  105 --
 ...ocatorLauncherRemoteIntegrationTestCase.java |   67 +
 ...stractServerLauncherIntegrationTestCase.java |   95 ++
 .../AbstractServerLauncherJUnitTestCase.java    |   93 --
 ...ServerLauncherRemoteIntegrationTestCase.java |   95 ++
 .../distributed/DistributedTestSuite.java       |   35 -
 .../distributed/HostedLocatorsDUnitTest.java    |   27 +-
 .../LauncherMemberMXBeanIntegrationTest.java    |  151 ++
 .../LauncherMemberMXBeanJUnitTest.java          |  152 --
 .../gemfire/distributed/LauncherTestSuite.java  |   47 -
 .../gemfire/distributed/LocatorJUnitTest.java   |   67 +-
 .../LocatorLauncherIntegrationJUnitTest.java    |  248 ---
 .../LocatorLauncherIntegrationTest.java         |  248 +++
 .../distributed/LocatorLauncherJUnitTest.java   |  341 -----
 ...LocatorLauncherLocalFileIntegrationTest.java |   54 +
 .../LocatorLauncherLocalFileJUnitTest.java      |   51 -
 .../LocatorLauncherLocalIntegrationTest.java    |  827 ++++++++++
 .../LocatorLauncherLocalJUnitTest.java          |  842 -----------
 ...ocatorLauncherRemoteFileIntegrationTest.java |  218 +++
 .../LocatorLauncherRemoteFileJUnitTest.java     |  218 ---
 .../LocatorLauncherRemoteIntegrationTest.java   |  977 ++++++++++++
 .../LocatorLauncherRemoteJUnitTest.java         | 1011 -------------
 ...rRemoteWithCustomLoggingIntegrationTest.java |  124 ++
 .../distributed/LocatorLauncherTest.java        |  340 +++++
 .../distributed/LocatorStateJUnitTest.java      |  208 ---
 .../gemfire/distributed/LocatorStateTest.java   |  208 +++
 .../MockServerLauncherCacheProvider.java        |    8 +-
 .../ServerLauncherIntegrationJUnitTest.java     |  312 ----
 .../ServerLauncherIntegrationTest.java          |  311 ++++
 .../distributed/ServerLauncherJUnitTest.java    |  903 -----------
 .../ServerLauncherLocalFileIntegrationTest.java |   54 +
 .../ServerLauncherLocalFileJUnitTest.java       |   54 -
 .../ServerLauncherLocalIntegrationTest.java     | 1073 +++++++++++++
 .../ServerLauncherLocalJUnitTest.java           | 1073 -------------
 ...ServerLauncherRemoteFileIntegrationTest.java |  222 +++
 .../ServerLauncherRemoteFileJUnitTest.java      |  222 ---
 .../ServerLauncherRemoteIntegrationTest.java    | 1380 +++++++++++++++++
 .../ServerLauncherRemoteJUnitTest.java          | 1430 ------------------
 ...rRemoteWithCustomLoggingIntegrationTest.java |  122 ++
 .../gemfire/distributed/ServerLauncherTest.java |  898 +++++++++++
 ...rverLauncherWithProviderIntegrationTest.java |   89 ++
 .../ServerLauncherWithProviderJUnitTest.java    |   90 --
 .../gemstone/gemfire/internal/ClassBuilder.java |    2 +-
 .../logging/log4j/custom/BasicAppender.java     |   76 +
 .../CustomConfigWithCacheIntegrationTest.java   |  148 ++
 ...stomConfigWithLogServiceIntegrationTest.java |  119 ++
 .../log4j/custom/CustomConfiguration.java       |   64 +
 ...leProcessControllerIntegrationJUnitTest.java |   13 +-
 .../management/internal/cli/HeadlessGfsh.java   |   23 +-
 .../cli/HeadlessGfshIntegrationTest.java        |   90 ++
 .../internal/cli/HeadlessGfshJUnitTest.java     |   85 --
 .../cli/commands/CliCommandTestBase.java        |   31 +-
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  |  305 ++++
 .../cli/commands/DeployCommandsDUnitTest.java   |   52 +-
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  159 +-
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  117 +-
 .../cli/commands/ListIndexCommandDUnitTest.java |  121 +-
 .../cli/commands/ShellCommandsDUnitTest.java    |    2 +-
 .../cli/commands/UserCommandsDUnitTest.java     |    2 +-
 .../SharedConfigurationDUnitTest.java           |  227 ++-
 .../SharedConfigurationTestUtils.java           |   40 +
 .../SharedConfigurationUsingDirDUnitTest.java   |  128 +-
 .../security/GfshShellConnectionRule.java       |    8 +-
 .../logging/log4j/custom/log4j2-custom.xml      |   27 +
 .../pulse/internal/data/JMXDataUpdater.java     |    3 -
 .../wan/wancommand/WANCommandTestBase.java      |    2 +-
 .../ClusterConfigurationDUnitTest.java          | 1236 ++++++++-------
 gradle/dependency-versions.properties           |   20 +-
 82 files changed, 11076 insertions(+), 10163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
new file mode 100644
index 0000000..332cbf4
--- /dev/null
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.process.ProcessType;
+import com.gemstone.gemfire.internal.process.ProcessUtils;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.ManagerMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+
+import static org.junit.Assert.*;
+
+/**
+ * These tests are part of assembly as they require the REST war file to be present.
+ */
+@Category(IntegrationTest.class)
+public class LocatorLauncherAssemblyIntegrationTest extends AbstractLocatorLauncherIntegrationTestCase {
+
+  @Before
+  public final void setUpLocatorLauncherLocalTest() throws Exception {
+    disconnectFromDS();
+    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName() + "-");
+  }
+
+  @After
+  public final void tearDownLocatorLauncherLocalTest() throws Exception {
+    disconnectFromDS();
+  }
+
+  /*
+   * This test addresses GEODE-528
+   */
+  @Test
+  public void testLocatorStopsWhenJmxPortIsZero() throws Throwable {
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+
+    final Builder builder = new Builder()
+        .setMemberName(getUniqueName())
+        .setPort(this.locatorPort)
+        .setRedirectOutput(false)
+        .setWorkingDirectory(rootFolder)
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
+        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
+
+    performTest(builder);
+  }
+
+  /*
+   * This test addresses GEODE-528
+   */
+  @Test
+  public void testLocatorStopsWhenJmxPortIsNonZero() throws Throwable {
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    final int jmxPort = AvailablePortHelper.getRandomAvailableTCPPorts(1)[0];
+
+    final Builder builder = new Builder().setMemberName(getUniqueName())
+        .setPort(this.locatorPort)
+        .setRedirectOutput(false)
+        .setWorkingDirectory(rootFolder)
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
+        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, Integer.toString(jmxPort));
+
+    performTest(builder);
+  }
+
+  private void performTest(Builder builder) {
+    assertFalse(builder.getForce());
+    this.launcher = builder.build();
+    assertFalse(this.launcher.isForcing());
+
+    LocatorLauncher dirLauncher = null;
+    int initialThreadCount = Thread.activeCount();
+
+    try {
+      this.launcher.start();
+      waitForLocatorToStart(this.launcher);
+
+      // validate the pid file and its contents
+      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
+      assertTrue("Pid file " + this.pidFile.getCanonicalPath().toString() + " should exist", this.pidFile.exists());
+      final int pid = readPid(this.pidFile);
+      assertTrue(pid > 0);
+      assertEquals(ProcessUtils.identifyPid(), pid);
+
+      dirLauncher = new Builder().setWorkingDirectory(builder.getWorkingDirectory()).build();
+      assertNotNull(dirLauncher);
+      assertFalse(dirLauncher.isRunning());
+
+      // Stop the manager
+      Cache cache = CacheFactory.getAnyInstance();
+      ManagerMXBean managerBean = ManagementService.getManagementService(cache).getManagerMXBean();
+      managerBean.stop();
+
+      // stop the locator
+      final LocatorLauncher.LocatorState locatorState = dirLauncher.stop();
+      assertNotNull(locatorState);
+      assertEquals(Status.STOPPED, locatorState.getStatus());
+    } catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+
+    try {
+      // verify the PID file was deleted
+      waitForFileToDelete(this.pidFile);
+    } catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+
+    int finalThreadCount = Integer.MAX_VALUE;
+
+    // Spin for up to 5 seconds waiting for threads to finish
+    for (int i = 0; i < 50 && finalThreadCount > initialThreadCount; i++) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ex) {
+        // ignored
+      }
+      finalThreadCount = Thread.activeCount();
+    }
+
+    assertEquals(initialThreadCount, finalThreadCount);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
deleted file mode 100644
index 0984508..0000000
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
-import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.process.ProcessType;
-import com.gemstone.gemfire.internal.process.ProcessUtils;
-import com.gemstone.gemfire.management.ManagementService;
-import com.gemstone.gemfire.management.ManagerMXBean;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.File;
-
-import static org.junit.Assert.*;
-
-/**
- * These tests are part of assembly as they require the REST war file to be present.
- *
- */
-@Category(IntegrationTest.class)
-public class LocatorLauncherAssemblyJUnitTest extends AbstractLocatorLauncherJUnitTestCase {
-
-  @Before
-  public final void setUpLocatorLauncherLocalTest() throws Exception {
-    disconnectFromDS();
-    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName() + "-");
-  }
-
-  @After
-  public final void tearDownLocatorLauncherLocalTest() throws Exception {
-    disconnectFromDS();
-  }
-
-  /*
-   * This test addresses GEODE-528
-   */
-  @Test
-  public void testLocatorStopsWhenJmxPortIsZero() throws Throwable {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
-
-    final Builder builder = new Builder()
-        .setMemberName(getUniqueName())
-        .setPort(this.locatorPort)
-        .setRedirectOutput(false)
-        .setWorkingDirectory(rootFolder)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
-        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
-        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
-        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
-        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
-
-    performTest(builder);
-  }
-
-  /*
-   * This test addresses GEODE-528
-   */
-  @Test
-  public void testLocatorStopsWhenJmxPortIsNonZero() throws Throwable {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
-    final int jmxPort = AvailablePortHelper.getRandomAvailableTCPPorts(1)[0];
-
-    final Builder builder = new Builder().setMemberName(getUniqueName())
-        .setPort(this.locatorPort)
-        .setRedirectOutput(false)
-        .setWorkingDirectory(rootFolder)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
-        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
-        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
-        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
-        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, Integer.toString(jmxPort));
-
-    performTest(builder);
-  }
-
-  private void performTest(Builder builder) {
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher dirLauncher = null;
-    int initialThreadCount = Thread.activeCount();
-
-    try {
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-
-      // validate the pid file and its contents
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue("Pid file " + this.pidFile.getCanonicalPath().toString() + " should exist", this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertEquals(ProcessUtils.identifyPid(), pid);
-
-      dirLauncher = new Builder().setWorkingDirectory(builder.getWorkingDirectory()).build();
-      assertNotNull(dirLauncher);
-      assertFalse(dirLauncher.isRunning());
-
-      // Stop the manager
-      Cache cache = CacheFactory.getAnyInstance();
-      ManagerMXBean managerBean = ManagementService.getManagementService(cache).getManagerMXBean();
-      managerBean.stop();
-
-      // stop the locator
-      final LocatorLauncher.LocatorState locatorState = dirLauncher.stop();
-      assertNotNull(locatorState);
-      assertEquals(Status.STOPPED, locatorState.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      // verify the PID file was deleted
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    int finalThreadCount = Integer.MAX_VALUE;
-
-    // Spin for up to 5 seconds waiting for threads to finish
-    for (int i = 0; i < 50 && finalThreadCount > initialThreadCount; i++) {
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException ex) {
-        // ignored
-      }
-      finalThreadCount = Thread.activeCount();
-    }
-
-    assertEquals(initialThreadCount, finalThreadCount);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index 653bbdc..caee9ea 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -16,117 +16,143 @@
  */
 package com.gemstone.gemfire.management.internal.configuration;
 
+import static com.gemstone.gemfire.cache.RegionShortcut.*;
+import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
+import static com.gemstone.gemfire.internal.AvailablePortHelper.*;
+import static com.gemstone.gemfire.internal.FileUtil.*;
+import static com.gemstone.gemfire.internal.lang.StringUtils.*;
+import static com.gemstone.gemfire.management.internal.cli.CliUtil.*;
 import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Host.*;
+import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
 import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 import static com.gemstone.gemfire.test.dunit.Wait.*;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
 import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.ClassBuilder;
-import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.JarDeployer;
 import com.gemstone.gemfire.internal.admin.remote.ShutdownAllRequest;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.management.cli.Result.Status;
-import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
 import com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-import org.apache.commons.io.FileUtils;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
 @Category(DistributedTest.class)
 public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
+
   private static final int TIMEOUT = 10000;
   private static final int INTERVAL = 500;
+
   private static final String REGION1 = "R1";
   private static final String REGION2 = "R2";
   private static final String INDEX1 = "ID1";
-  private transient ClassBuilder classBuilder = new ClassBuilder();
-  public static Set<String> serverNames = new HashSet<String>();
-  public static Set<String> jarFileNames = new HashSet<String>();
 
-  private static final long serialVersionUID = -2276690105585944041L;
+  private static Set<String> serverNames;
+  private static Set<String> jarFileNames;
 
-  public Set<String> startServers(HeadlessGfsh gfsh, String locatorString, int numServers, String serverNamePrefix, int startNum) throws ClassNotFoundException, IOException {
-    Set<String> serverNames = new HashSet<String>();
+  private transient ClassBuilder classBuilder;
+  private transient String jmxHost;
+  private transient int jmxPort;
+  private transient int httpPort;
+  private transient String locatorString;
 
-    final int[] serverPorts = AvailablePortHelper.getRandomAvailableTCPPorts(numServers);
-    for (int i=0; i<numServers; i++) {
-      int port = serverPorts[i];
-      String serverName = serverNamePrefix+ Integer.toString(i+startNum) + "-" + port;
-      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.START_SERVER);
-      csb.addOption(CliStrings.START_SERVER__NAME, serverName);
-      csb.addOption(CliStrings.START_SERVER__LOCATORS, locatorString);
-      csb.addOption(CliStrings.START_SERVER__SERVER_PORT, Integer.toString(port));
-      CommandResult cmdResult = executeCommand(gfsh, csb.getCommandString());
-      assertEquals(Status.OK, cmdResult.getStatus());
-    }
-    return serverNames;
-  }
+  @Override
+  public final void postSetUpCliCommandTestBase() throws Exception {
+    disconnectAllFromDS();
+
+    addIgnoredException("EntryDestroyedException");
+
+    serverNames = new HashSet<>();
+    jarFileNames = new HashSet<>();
+
+    this.classBuilder = new ClassBuilder();
 
-  @Test
-  public void testStartServerAndExecuteCommands() throws InterruptedException, ClassNotFoundException, IOException, ExecutionException {
-    IgnoredException.addIgnoredException("EntryDestroyedException");
     Object[] result = setup();
-    final int locatorPort = (Integer) result[0];
-    final String jmxHost = (String) result[1];
-    final int jmxPort = (Integer) result[2];
-    final int httpPort = (Integer) result[3];
-    final String locatorString = "localHost[" + locatorPort + "]";
+    int locatorPort = (Integer) result[0];
+
+    this.jmxHost = (String) result[1];
+    this.jmxPort = (Integer) result[2];
+    this.httpPort = (Integer) result[3];
+    this.locatorString = "localHost[" + locatorPort + "]";
+  }
 
-    final HeadlessGfsh gfsh = new HeadlessGfsh("gfsh2", 300);
+  @Override
+  public final void preTearDownCliCommandTestBase() throws Exception {
+    //shutdown everything
+    shutdownAll();
+
+    serverNames.clear();
+    jarFileNames.clear();
+
+    serverNames = null;
+    jarFileNames = null;
+  }
+
+  @Test
+  public void testStartServerAndExecuteCommands() throws Exception {
+    final HeadlessGfsh gfsh = new HeadlessGfsh("gfsh2", 300, this.gfshDir);
     assertNotNull(gfsh);
     shellConnect(jmxHost, jmxPort, httpPort, gfsh);
 
     serverNames.addAll(startServers(gfsh, locatorString, 2, "Server", 1));
     doCreateCommands();
     serverNames.addAll(startServers(gfsh, locatorString, 1, "NewMember", 4));
+
     verifyRegionCreateOnAllMembers(REGION1);
     verifyRegionCreateOnAllMembers(REGION2);
     verifyIndexCreationOnAllMembers(INDEX1);
     verifyAsyncEventQueueCreation();
-   
+  }
 
+  private Set<String> startServers(final HeadlessGfsh gfsh, final String locatorString, final int numServers, final String serverNamePrefix, final int startNum) throws ClassNotFoundException, IOException {
+    Set<String> serverNames = new HashSet<>();
 
-    //shutdown everything
-    getLogWriter().info("Shutting down all the members");
-    shutdownAll();
-    deleteSavedJarFiles();
-  }
+    final int[] serverPorts = getRandomAvailableTCPPorts(numServers);
+    for (int i=0; i<numServers; i++) {
+      int port = serverPorts[i];
+      String serverName = serverNamePrefix+ Integer.toString(i+startNum) + "-" + port;
 
+      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.START_SERVER);
+      csb.addOption(CliStrings.START_SERVER__NAME, serverName);
+      csb.addOption(CliStrings.START_SERVER__LOCATORS, locatorString);
+      csb.addOption(CliStrings.START_SERVER__SERVER_PORT, Integer.toString(port));
+
+      CommandResult cmdResult = executeCommand(gfsh, csb.getCommandString());
+
+      assertEquals(Status.OK, cmdResult.getStatus());
+    }
+    return serverNames;
+  }
 
-  private void doCreateCommands() {
-    createRegion(REGION1, RegionShortcut.REPLICATE, null);
-    createRegion(REGION2, RegionShortcut.PARTITION, null);
+  private void doCreateCommands() throws IOException {
+    createRegion(REGION1, REPLICATE, null);
+    createRegion(REGION2, PARTITION, null);
     createIndex(INDEX1 , "AAPL", REGION1, null);
-    createAndDeployJar("Deploy1.jar");
+    createAndDeployJar(this.temporaryFolder.getRoot().getCanonicalPath() + File.separator + "Deploy1.jar");
     createAsyncEventQueue("q1");
     final String autoCompact = "true";
     final String allowForceCompaction = "true";
@@ -143,34 +169,33 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     createDiskStore(diskStoreName, diskDirs, autoCompact, allowForceCompaction, compactionThreshold, duCritical, duWarning, maxOplogSize, queueSize, timeInterval, writeBufferSize);
   }
 
-
-  protected void executeAndVerifyCommand(String commandString) {
+  private void executeAndVerifyCommand(final String commandString) {
     CommandResult cmdResult = executeCommand(commandString);
     getLogWriter().info("Command Result : \n" + commandResultToString(cmdResult));
     assertEquals(Status.OK, cmdResult.getStatus());
     assertFalse(cmdResult.failedToPersist());
   }
 
-  private void createRegion(String regionName, RegionShortcut regionShortCut, String group) {
+  private void createRegion(final String regionName, final RegionShortcut regionShortCut, final String group) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_REGION);
     csb.addOption(CliStrings.CREATE_REGION__REGION, regionName);
     csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, regionShortCut.name());
     executeAndVerifyCommand(csb.getCommandString());
   }
 
-  private void destroyRegion(String regionName) {
+  private void destroyRegion(final String regionName) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_REGION);
     csb.addOption(CliStrings.DESTROY_REGION__REGION, regionName);
     executeAndVerifyCommand(csb.getCommandString());
   }
 
-  private void stopServer(String serverName) {
+  private void stopServer(final String serverName) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.STOP_SERVER);
     csb.addOption(CliStrings.STOP_SERVER__MEMBER, serverName);
     executeAndVerifyCommand(csb.getCommandString());
   }
 
-  public void createAsyncEventQueue(String queueName) {
+  private void createAsyncEventQueue(final String queueName) throws IOException {
     String queueCommandsJarName = "testEndToEndSC-QueueCommands.jar";
     final File jarFile = new File(queueCommandsJarName);
 
@@ -208,23 +233,22 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
       
       executeAndVerifyCommand(csb.getCommandString());
 
-    } catch (IOException e) {
-      e.printStackTrace();
     } finally {
       FileUtils.deleteQuietly(jarFile);
     }
   }
-  private void createDiskStore(String diskStoreName, 
-      String diskDirs, 
-      String autoCompact, 
-      String allowForceCompaction, 
-      String compactionThreshold, 
-      String duCritical, 
-      String duWarning,
-      String maxOplogSize,
-      String queueSize,
-      String timeInterval,
-      String writeBufferSize) {
+
+  private void createDiskStore(final String diskStoreName,
+                               final String diskDirs,
+                               final String autoCompact,
+                               final String allowForceCompaction,
+                               final String compactionThreshold,
+                               final String duCritical,
+                               final String duWarning,
+                               final String maxOplogSize,
+                               final String queueSize,
+                               final String timeInterval,
+                               final String writeBufferSize) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
     csb.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStoreName);
     csb.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskDirs);
@@ -240,13 +264,14 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     executeAndVerifyCommand(csb.getCommandString());
   }
   
-  private void destroyDiskStore(String diskStoreName, String group) {
+  private void destroyDiskStore(final String diskStoreName, final String group) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
     csb.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStoreName);
     csb.addOptionWithValueCheck(CliStrings.DESTROY_DISK_STORE__GROUP, group);
     executeAndVerifyCommand(csb.toString());
   }
-  public void createIndex(String indexName, String expression, String regionName, String group) {
+
+  private void createIndex(final String indexName, final String expression, final String regionName, final String group) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
     csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
     csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, expression);
@@ -254,57 +279,50 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     executeAndVerifyCommand(csb.getCommandString());
   }
 
-  public void destoyIndex(String indexName, String regionName, String group) {
-    if (StringUtils.isBlank(indexName) && StringUtils.isBlank(regionName) && StringUtils.isBlank(group)) {
+  private void destroyIndex(final String indexName, final String regionName, final String group) {
+    if (isBlank(indexName) && isBlank(regionName) && isBlank(group)) {
       return;
     }
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
-    if (!StringUtils.isBlank(indexName)) {
+    if (!isBlank(indexName)) {
       csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
     }
 
-    if (!StringUtils.isBlank(regionName)) {
+    if (!isBlank(regionName)) {
       csb.addOption(CliStrings.DESTROY_INDEX__REGION, regionName);
     }
 
-    if (!StringUtils.isBlank(group)) {
+    if (!isBlank(group)) {
       csb.addOption(CliStrings.DESTROY_INDEX__GROUP, group);
     }
     executeAndVerifyCommand(csb.getCommandString());
   }
 
-  public void createAndDeployJar(String jarName) {
+  private void createAndDeployJar(final String jarName) throws IOException {
     File newDeployableJarFile = new File(jarName);
-    try {
-      this.classBuilder.writeJarFromName("ShareConfigClass", newDeployableJarFile);
-      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEPLOY);
-      csb.addOption(CliStrings.DEPLOY__JAR, jarName);
-      executeAndVerifyCommand(csb.getCommandString());
-      jarFileNames.add(jarName);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
+    this.classBuilder.writeJarFromName("ShareConfigClass", newDeployableJarFile);
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEPLOY);
+    csb.addOption(CliStrings.DEPLOY__JAR, jarName);
+    executeAndVerifyCommand(csb.getCommandString());
+    jarFileNames.add(jarName);
   }
 
-  public void deleteSavedJarFiles() {
-    try {
-      FileUtil.deleteMatching(new File("."), "^" + JarDeployer.JAR_PREFIX + "Deploy1.*#\\d++$");
-      FileUtil.delete(new File("Deploy1.jar"));
-    } catch (IOException ioe) {
-      ioe.printStackTrace();
-    }
+  private void deleteSavedJarFiles() throws IOException {
+    deleteMatching(new File("."), "^" + JarDeployer.JAR_PREFIX + "Deploy1.*#\\d++$");
+    delete(new File("Deploy1.jar"));
   }
 
-  public Object[] setup() {
-    disconnectAllFromDS();
-    final int [] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+  private Object[] setup() throws IOException {
+    final int [] ports = getRandomAvailableTCPPorts(3);
     final int locator1Port = ports[0];
+
     final String locator1Name = "locator1-" + locator1Port;
-    VM locatorAndMgr = Host.getHost(0).getVM(3);
+    final String locatorLogPath = this.temporaryFolder.getRoot().getCanonicalPath() + File.separator + "locator-" + locator1Port + ".log";
 
+    VM locatorAndMgr = getHost(0).getVM(3);
     Object[] result = (Object[]) locatorAndMgr.invoke(new SerializableCallable() {
       @Override
-      public Object call() {
+      public Object call() throws IOException {
         int httpPort;
         int jmxPort;
         String jmxHost;
@@ -316,42 +334,37 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
           jmxHost = "localhost";
         }
 
-        final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+        final int[] ports = getRandomAvailableTCPPorts(2);
 
         jmxPort = ports[0];
         httpPort = ports[1];
 
-        final File locatorLogFile = new File("locator-" + locator1Port + ".log");
+        final File locatorLogFile = new File(locatorLogPath);
 
         final Properties locatorProps = new Properties();
-        locatorProps.setProperty(DistributionConfig.NAME_NAME, locator1Name);
-        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "config");
-        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
-        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
-        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
-        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_BIND_ADDRESS_NAME, String.valueOf(jmxHost));
-        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
-        locatorProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
-
-        try {
-          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locator1Port, locatorLogFile, null,
-              locatorProps);
-          WaitCriterion wc = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return locator.isSharedConfigurationRunning();
-            }
-
-            @Override
-            public String description() {
-              return "Waiting for shared configuration to be started";
-            }
-          };
-          waitForCriterion(wc, TIMEOUT, INTERVAL, true);
-        } catch (IOException ioex) {
-          fail("Unable to create a locator with a shared configuration");
-        }
+        locatorProps.setProperty(NAME_NAME, locator1Name);
+        locatorProps.setProperty(MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(LOG_LEVEL_NAME, "config");
+        locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        locatorProps.setProperty(JMX_MANAGER_NAME, "true");
+        locatorProps.setProperty(JMX_MANAGER_START_NAME, "true");
+        locatorProps.setProperty(JMX_MANAGER_BIND_ADDRESS_NAME, String.valueOf(jmxHost));
+        locatorProps.setProperty(JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
+        locatorProps.setProperty(HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
+
+        final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locator1Port, locatorLogFile, null, locatorProps);
+
+        WaitCriterion wc = new WaitCriterion() {
+          @Override
+          public boolean done() {
+            return locator.isSharedConfigurationRunning();
+          }
+          @Override
+          public String description() {
+            return "Waiting for shared configuration to be started";
+          }
+        };
+        waitForCriterion(wc, TIMEOUT, INTERVAL, true);
 
         final Object[] result = new Object[4];
         result[0] = locator1Port;
@@ -368,32 +381,28 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     int httpPort = (Integer)result[3];
 
     shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+
     // Create a cache in VM 1
-    VM dataMember = Host.getHost(0).getVM(1);
+    VM dataMember = getHost(0).getVM(1);
     dataMember.invoke(new SerializableCallable() {
       @Override
       public Object call() {
         Properties localProps = new Properties();
-        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locator1Port);
-        localProps.setProperty(DistributionConfig.NAME_NAME, "DataMember");
+        localProps.setProperty(MCAST_PORT_NAME, "0");
+        localProps.setProperty(LOCATORS_NAME, "localhost:" + locator1Port);
+        localProps.setProperty(NAME_NAME, "DataMember");
         getSystem(localProps);
         Cache cache = getCache();
         assertNotNull(cache);
-        return CliUtil.getAllNormalMembers(cache);
+        return getAllNormalMembers(cache);
       }
     });
     return result;
   }
 
   private void shutdownAll() throws IOException {
-    VM locatorAndMgr = Host.getHost(0).getVM(3);
+    VM locatorAndMgr = getHost(0).getVM(3);
     locatorAndMgr.invoke(new SerializableCallable() {
-      /**
-       * 
-       */
-      private static final long serialVersionUID = 1L;
-
       @Override
       public Object call() throws Exception {
         GemFireCacheImpl cache = (GemFireCacheImpl)CacheFactory.getAnyInstance();
@@ -402,7 +411,8 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
       }
     });
 
-    locatorAndMgr.invoke(SharedConfigurationDUnitTest.locatorCleanup);
+    locatorAndMgr.invoke(SharedConfigurationTestUtils.cleanupLocator);
+
     //Clean up the directories
     if (!serverNames.isEmpty()) {
       for (String serverName : serverNames) {
@@ -412,10 +422,9 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
       }
     }
     serverNames.clear();
-    serverNames = null;
   }
 
-  private void verifyRegionCreateOnAllMembers(String regionName) {
+  private void verifyRegionCreateOnAllMembers(final String regionName) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
     csb.addOption(CliStrings.DESCRIBE_REGION__NAME, regionName);
     CommandResult cmdResult = executeCommand(csb.getCommandString());
@@ -426,7 +435,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     }
   }     
 
-  private void verifyIndexCreationOnAllMembers(String indexName) {
+  private void verifyIndexCreationOnAllMembers(final String indexName) {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
     CommandResult cmdResult = executeCommand(csb.getCommandString());
     String resultAsString = commandResultToString(cmdResult);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
index b632edb..dec716c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
@@ -303,10 +303,6 @@ public abstract class OperationContext extends WildcardPermission{
     return Resource.NULL;
   }
 
-  /**
-   *
-   * @return
-   */
   public String getRegionName(){
     return "NULL";
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
index 8f190eb..0f349e9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
@@ -83,50 +83,38 @@ import com.gemstone.gemfire.management.internal.configuration.messages.SharedCon
 import com.gemstone.gemfire.management.internal.configuration.utils.XmlUtils;
 import com.gemstone.gemfire.management.internal.configuration.utils.ZipUtils;
 
-/*********
- * 
- *
- */
-@SuppressWarnings("deprecation")
+@SuppressWarnings({ "deprecation", "unchecked" })
 public class SharedConfiguration {
 
   private static final Logger logger = LogService.getLogger();
   
-  static class JarFileFilter implements FilenameFilter {
-    @Override
-    public boolean accept(File dir, String name) {
-      return name.endsWith(".jar");
-    }
-  }
-  /****
+  /**
    * Name of the directory where the shared configuration artifacts are stored
    */
   public static final String CLUSTER_CONFIG_ARTIFACTS_DIR_NAME = "cluster_config";
-  public static final String CLUSTER_CONFIG_DISK_STORE_NAME = "cluster_config";
-  public static String CONFIG_DIR_PATH;//FilenameUtils.concat(System.getProperty("user.dir"), CONFIG_ARTIFACTS_DIR_NAME);
+  private static final String CLUSTER_CONFIG_DISK_STORE_NAME = "cluster_config";
   public static final String CLUSTER_CONFIG_DISK_DIR_PREFIX = "ConfigDiskDir_";
 
   public static final String CLUSTER_CONFIG = "cluster";
 
-
-  /***
+  /**
    * Name of the lock service used for shared configuration
    */
-  public static final String SHARED_CONFIG_LOCK_SERVICE_NAME = "__CLUSTER_CONFIG_LS";
+  private static final String SHARED_CONFIG_LOCK_SERVICE_NAME = "__CLUSTER_CONFIG_LS";
 
-  /***
+  /**
    * Name of the lock for locking the shared configuration
    */
   public static final String SHARED_CONFIG_LOCK_NAME = "__CLUSTER_CONFIG_LOCK";
 
-  /***
+  /**
    * Name of the region which is used to store the configuration information
    */
-  public static final String CONFIG_REGION_NAME = "_ConfigurationRegion";
+  private static final String CONFIG_REGION_NAME = "_ConfigurationRegion";
 
-
-  public String CONFIG_DISK_DIR_NAME;
-  public String CONFIG_DISK_DIR_PATH;;
+  private final String configDirPath;
+  private final String configDiskDirName;
+  private final String configDiskDirPath;;
 
   private final Set<PersistentMemberPattern> newerSharedConfigurationLocatorInfo = new HashSet<PersistentMemberPattern>();
   private final AtomicReference<SharedConfigurationStatus> status = new AtomicReference<SharedConfigurationStatus>();
@@ -136,12 +124,9 @@ public class SharedConfiguration {
   private  GemFireCacheImpl cache;
   private final DistributedLockService sharedConfigLockingService;
 
-
-  /****
+  /**
    * Gets or creates (if not created) shared configuration lock service 
-   * @return DistributedLockService 
    */
-
   public static DistributedLockService getSharedConfigLockService(DistributedSystem ds) {
     DistributedLockService sharedConfigDls = DLockService.getServiceNamed(SHARED_CONFIG_LOCK_SERVICE_NAME);
     try {
@@ -154,19 +139,9 @@ public class SharedConfiguration {
     return sharedConfigDls;
   }
 
-  /**
-   * Returns an array containing the names of the subdirectories in a given directory
-   * @param path Path of the directory whose subdirectories are listed
-   * @return String[] names of first level subdirectories, null if no subdirectories are found or if the path is incorrect
-   */
-  private static String[] getSubdirectories(String path) {
-    File directory = new File(path);
-    return directory.list(DirectoryFileFilter.INSTANCE);
-  }
-
   public SharedConfiguration(Cache cache) throws IOException {
     this.cache = (GemFireCacheImpl)cache;
-    CONFIG_DISK_DIR_NAME = CLUSTER_CONFIG_DISK_DIR_PREFIX + cache.getDistributedSystem().getName();
+    this.configDiskDirName = CLUSTER_CONFIG_DISK_DIR_PREFIX + cache.getDistributedSystem().getName();
     String clusterConfigDir = cache.getDistributedSystem().getProperties().getProperty(DistributionConfig.CLUSTER_CONFIGURATION_DIR);
     if (StringUtils.isBlank(clusterConfigDir)) {
       clusterConfigDir = System.getProperty("user.dir");
@@ -177,17 +152,14 @@ public class SharedConfiguration {
       }
       clusterConfigDir = diskDir.getCanonicalPath();
     }
-    CONFIG_DISK_DIR_PATH = FilenameUtils.concat(clusterConfigDir, CONFIG_DISK_DIR_NAME);
-    CONFIG_DIR_PATH = FilenameUtils.concat(clusterConfigDir, CLUSTER_CONFIG_ARTIFACTS_DIR_NAME);
+    this.configDiskDirPath = FilenameUtils.concat(clusterConfigDir, this.configDiskDirName);
+    configDirPath = FilenameUtils.concat(clusterConfigDir, CLUSTER_CONFIG_ARTIFACTS_DIR_NAME);
     sharedConfigLockingService = getSharedConfigLockService(cache.getDistributedSystem());
     status.set(SharedConfigurationStatus.NOT_STARTED);
   }
 
-  /*****
+  /**
    * Add jar information into the shared configuration and save the jars in the file system
-   * @param jarNames
-   * @param jarBytes
-   * @param groups
    * @return true on success
    */
   public boolean addJars(String []jarNames, byte[][]jarBytes, String[]groups)  {
@@ -198,7 +170,7 @@ public class SharedConfiguration {
       }
       Region<String, Configuration> configRegion = getConfigurationRegion();
       for (String group : groups) {
-        Configuration configuration = (Configuration) configRegion.get(group);
+        Configuration configuration = configRegion.get(group);
 
         if (configuration == null) {
           configuration = new Configuration(group);
@@ -206,7 +178,7 @@ public class SharedConfiguration {
         }
         configuration.addJarNames(jarNames);
         configRegion.put(group, configuration);
-        String groupDir = FilenameUtils.concat(CONFIG_DIR_PATH, group);
+        String groupDir = FilenameUtils.concat(configDirPath, group);
         writeJarFiles(groupDir, jarNames, jarBytes);
       }
     } catch (Exception e) {
@@ -216,12 +188,8 @@ public class SharedConfiguration {
     return success;
   }
 
-
-  /***
+  /**
    * Adds/replaces the xml entity in the shared configuration
-   * @param xmlEntity
-   * @param groups
-   * @throws Exception
    */
   public void addXmlEntity(XmlEntity xmlEntity, String[] groups) throws Exception {
     Region<String, Configuration> configRegion = getConfigurationRegion();
@@ -248,30 +216,6 @@ public class SharedConfiguration {
     }
   }
 
-  /**
-   * Create a {@link Document} using
-   * {@link XmlUtils#createDocumentFromXml(String)} and if the version attribute
-   * is not equal to the current version then update the XML to the current
-   * schema and return the document.
-   * 
-   * @param xmlContent
-   *          XML content to load and upgrade.
-   * @return {@link Document} from xmlContent.
-   * @throws IOException 
-   * @throws ParserConfigurationException 
-   * @throws SAXException 
-   * @throws XPathExpressionException 
-   * @since 8.1
-   */
-  // UnitTest SharedConfigurationJUnitTest.testCreateAndUpgradeDocumentFromXml
-  static Document createAndUpgradeDocumentFromXml(final String xmlContent) throws SAXException, ParserConfigurationException, IOException, XPathExpressionException {
-    Document doc = XmlUtils.createDocumentFromXml(xmlContent);
-    if (!CacheXml.VERSION_LATEST.equals(XmlUtils.getAttribute(doc.getDocumentElement(), CacheXml.VERSION, CacheXml.GEODE_NAMESPACE))) {
-      doc = XmlUtils.upgradeSchema(doc, CacheXml.GEODE_NAMESPACE, CacheXml.LATEST_SCHEMA_LOCATION, CacheXml.VERSION_LATEST);
-    }
-    return doc;
-  }
-
   public void clearSharedConfiguration() throws Exception {
     Region<String, Configuration> configRegion = getConfigurationRegion();
     if (configRegion != null) {
@@ -279,11 +223,9 @@ public class SharedConfiguration {
     }
   }
 
-
-  /*****
+  /**
    * Creates the shared configuration service
    * @param loadSharedConfigFromDir when set to true, loads the configuration from the share_config directory
-   * @throws Exception 
    */
   public void initSharedConfiguration(boolean loadSharedConfigFromDir) throws Exception {
     status.set(SharedConfigurationStatus.STARTED);
@@ -349,23 +291,11 @@ public class SharedConfiguration {
     status.set(SharedConfigurationStatus.RUNNING);
   }
 
-
-  public boolean lockSharedConfiguration() {
-    return sharedConfigLockingService.lock(SHARED_CONFIG_LOCK_NAME, -1, -1);
-  }
-
-  public void unlockSharedConfiguration() {
-    sharedConfigLockingService.unlock(SHARED_CONFIG_LOCK_NAME);
-  }
-
-  /****
+  /**
    * Creates a ConfigurationResponse based on the configRequest, configuration response contains the requested shared configuration
    * This method locks the SharedConfiguration
-   * @param configRequest
-   * @return ConfigurationResponse
-   * @throws Exception 
    */
-  public ConfigurationResponse createConfigurationReponse(ConfigurationRequest configRequest) throws Exception {
+  public ConfigurationResponse createConfigurationReponse(final ConfigurationRequest configRequest) throws Exception {
 
     ConfigurationResponse configResponse = new ConfigurationResponse();
 
@@ -401,7 +331,7 @@ public class SharedConfiguration {
     return configResponse;
   }
 
-  /***
+  /**
    * Create a response containing the status of the Shared configuration and information about other locators containing newer
    * shared configuration data (if at all)
    * @return {@link SharedConfigurationStatusResponse} containing the {@link SharedConfigurationStatus}
@@ -413,13 +343,10 @@ public class SharedConfiguration {
     return response;
   }
 
-  /*****
+  /**
    * Deletes the xml entity from the shared configuration.
-   * @param xmlEntity 
-   * @param groups
-   * @throws Exception
    */
-  public void deleteXmlEntity (XmlEntity xmlEntity, String[] groups) throws Exception {
+  public void deleteXmlEntity(final XmlEntity xmlEntity, String[] groups) throws Exception {
     Region<String, Configuration> configRegion = getConfigurationRegion();
     //No group is specified, so delete in every single group if it exists.
     if (groups == null) {
@@ -441,7 +368,7 @@ public class SharedConfiguration {
     }
   }
 
-  public void modifyCacheAttributes(XmlEntity xmlEntity, String [] groups) throws Exception {
+  public void modifyCacheAttributes(final XmlEntity xmlEntity, String [] groups) throws Exception {
     Region<String, Configuration> configRegion = getConfigurationRegion();
     //No group is specified, so modify the cache attributes for a in every single group if it exists.
     if (groups == null) {
@@ -474,27 +401,26 @@ public class SharedConfiguration {
     }
   }
 
-  /***
-   * Only to be used for clean up in DUnits. 
+  /**
+   * For tests only. TODO: clean this up and remove from production code
+   * <p/>
+   * Throws {@code AssertionError} wrapping any exception thrown by operation.
    */
-  public void destroySharedConfiguration() {
-    Region<String, Configuration> configRegion;
+  public void destroySharedConfiguration_forTestsOnly() {
     try {
-      configRegion = getConfigurationRegion();
+      Region<String, Configuration> configRegion = getConfigurationRegion();
       if (configRegion != null) {
         configRegion.destroyRegion();
       }
       DiskStore configDiskStore = this.cache.findDiskStore(CLUSTER_CONFIG_ARTIFACTS_DIR_NAME);
       if (configDiskStore != null) {
         configDiskStore.destroy();
-        File file = new File(CONFIG_DISK_DIR_PATH);
+        File file = new File(configDiskDirPath);
         FileUtils.deleteDirectory(file);
       }
-      FileUtils.deleteDirectory(new File(CONFIG_DIR_PATH));
-    } catch (IOException e) {
-      e.printStackTrace();
-    } catch (Exception e1) {
-      e1.printStackTrace();
+      FileUtils.deleteDirectory(new File(configDirPath));
+    } catch (Exception exception) {
+      throw new AssertionError(exception);
     }
   }
 
@@ -521,7 +447,7 @@ public class SharedConfiguration {
         if (configuration != null) {
           Set<String> jarNameSet = configuration.getJarNames();
           for (String jarName : jarNameSet) {
-            String groupDirPath = FilenameUtils.concat(CONFIG_DIR_PATH, group);
+            String groupDirPath = FilenameUtils.concat(configDirPath, group);
             if (!jarsAdded.contains(jarName)) {
               String jarFilePath = FilenameUtils.concat(groupDirPath, jarName);
               jarNames[ctr]=jarName;
@@ -531,18 +457,163 @@ public class SharedConfiguration {
           }
         }
       }
+
       jars[0] = jarNames;
       jars[1] = jarBytes;
     }
     return jars;
   }
 
+  public Configuration getConfiguration(String groupName) throws Exception {
+    Configuration configuration = getConfigurationRegion().get(groupName);
+    return configuration;
+  }
+
+  public Map<String, Configuration> getEntireConfiguration() throws Exception {
+    Set<String> keys = getConfigurationRegion().keySet();
+    return getConfigurationRegion().getAll(keys);
+  }
+
+  /**
+   * Returns the path of Shared configuration directory
+   * @return {@link String}  path of the shared configuration directory
+   */
+  public String getSharedConfigurationDirPath() {
+    return configDirPath;
+  }
+
+  /**
+   * Gets the current status of the SharedConfiguration
+   * If the status is started , it determines if the shared configuration is waiting for new configuration on
+   * other locators
+   * @return {@link SharedConfigurationStatus}
+   */
+  public SharedConfigurationStatus getStatus() {
+    SharedConfigurationStatus scStatus = this.status.get();
+    if (scStatus == SharedConfigurationStatus.STARTED) {
+      PersistentMemberManager pmm = cache.getPersistentMemberManager();
+      Map<String, Set<PersistentMemberID>> waitingRegions = pmm.getWaitingRegions();
+      if (!waitingRegions.isEmpty()) {
+        this.status.compareAndSet(SharedConfigurationStatus.STARTED, SharedConfigurationStatus.WAITING);
+        Set<PersistentMemberID> persMemIds =  waitingRegions.get(Region.SEPARATOR_CHAR + CONFIG_REGION_NAME);
+        for (PersistentMemberID persMemId : persMemIds) {
+          newerSharedConfigurationLocatorInfo.add(new PersistentMemberPattern(persMemId));
+        }
+      }
+    }
+    return this.status.get();
+  }
+
+  /**
+   * Loads the
+   * @throws Exception
+   */
+  public void loadSharedConfigurationFromDisk() throws Exception {
+    Map<String, Configuration> sharedConfigurationMap = readSharedConfigurationFromDisk();
+    getConfigurationRegion().clear();
+    getConfigurationRegion().putAll(sharedConfigurationMap);
+  }
+
+  public void modifyProperties(final Properties properties, String[] groups) throws Exception {
+    if (groups == null) {
+      groups = new String[] {SharedConfiguration.CLUSTER_CONFIG};
+    }
+    Region<String, Configuration> configRegion = getConfigurationRegion();
+    for (String group : groups) {
+      Configuration configuration = configRegion.get(group);
+      if (configuration == null) {
+        configuration = new Configuration(group);
+      }
+      configuration.getGemfireProperties().putAll(properties);
+      configRegion.put(group, configuration);
+      writeConfig(configuration);
+    }
+  }
+
+  /**
+   * Removes the jar files from the shared configuration.
+   * @param jarNames Names of the jar files.
+   * @param groups Names of the groups which had the jar file deployed.
+   * @return true on success.
+   */
+  public boolean removeJars(final String[] jarNames, String[] groups){
+    boolean success = true;
+    try {
+      Region<String, Configuration> configRegion = getConfigurationRegion();
+      if (groups == null) {
+        Set<String> groupSet = configRegion.keySet();
+        groups = groupSet.toArray(new String[groupSet.size()]);
+      }
+      for (String group : groups) {
+        Configuration configuration = (Configuration) configRegion.get(group);
+        if (configuration != null) {
+          String dirPath = FilenameUtils.concat(getSharedConfigurationDirPath(), configuration.getConfigName());
+          removeJarFiles(dirPath, jarNames);
+        }
+      }
+      for (String group : groups) {
+        Configuration configuration = (Configuration) configRegion.get(group);
+        if (configuration != null) {
+          if (!configuration.getJarNames().isEmpty()) {
+            configuration.removeJarNames(jarNames);
+            configRegion.put(group, configuration);
+          }
+        }
+      }
+    } catch (Exception e) {
+      logger.info("Exception occurred while deleting the jar files", e);
+      success = false;
+    }
+    return success;
+  }
+
+  public void renameExistingSharedConfigDirectory() {
+    File configDirFile = new File(configDirPath);
+    if (configDirFile.exists()) {
+      String configDirFileName2 = CLUSTER_CONFIG_ARTIFACTS_DIR_NAME + new SimpleDateFormat("yyyyMMddhhmm").format(new Date()) + "." + System.nanoTime();
+      File configDirFile2 = new File(FilenameUtils.concat(configDirFileName2, configDirFileName2));
+      try {
+        FileUtils.moveDirectoryToDirectory(configDirFile, configDirFile2, true);
+      } catch (IOException e) {
+        logger.info(e);
+      }
+    }
+  }
+
+  /**
+   * Writes the contents of the {@link Configuration} to the file system
+   */
+  public void writeConfig(final Configuration configuration) throws Exception {
+    File configDir = new File(getSharedConfigurationDirPath());
+    if (!configDir.exists()) {
+      if (!configDir.mkdirs()) {
+        throw new IOException("Cannot create directory : " + getSharedConfigurationDirPath());
+      }
+    }
+    String dirPath = FilenameUtils.concat(getSharedConfigurationDirPath(), configuration.getConfigName());
+    File file = new File(dirPath);
+    if (!file.exists()) {
+      if (!file.mkdir()) {
+        throw new IOException("Cannot create directory : " + dirPath);
+      }
+    }
+
+    writeProperties(dirPath, configuration);
+    writeCacheXml(dirPath, configuration);
+  }
+
+  private boolean lockSharedConfiguration() {
+    return sharedConfigLockingService.lock(SHARED_CONFIG_LOCK_NAME, -1, -1);
+  }
+
+  private void unlockSharedConfiguration() {
+    sharedConfigLockingService.unlock(SHARED_CONFIG_LOCK_NAME);
+  }
 
-  /***
+  /**
    * Gets the Jar from existing locators in the system
-   * @throws Exception 
    */
-  public void getAllJarsFromOtherLocators() throws Exception {
+  private void getAllJarsFromOtherLocators() throws Exception {
     logger.info("Getting Jar files from other locators");
     DM dm = cache.getDistributionManager();
     DistributedMember me = cache.getMyId();
@@ -555,7 +626,6 @@ public class SharedConfiguration {
       logger.info("No other locators present");
       return;
     }
-    @SuppressWarnings("unchecked")
     ResultCollector<?, List<Object>> rc = (ResultCollector<?, List<Object>>) CliUtil.executeFunction(getAllJarsFunction, null , locators);
 
     List<Object> results = rc.getResult();
@@ -584,7 +654,7 @@ public class SharedConfiguration {
       for (String group : groups) {
         Configuration config = entireConfiguration.get(group);
         Set<String> groupJarNames = config.getJarNames();
-        String groupDirPath = FilenameUtils.concat(CONFIG_DIR_PATH, group);
+        String groupDirPath = FilenameUtils.concat(configDirPath, group);
 
         for (String groupJarName : groupJarNames) {
           Integer index = jarIndex.get(groupJarName);
@@ -609,29 +679,22 @@ public class SharedConfiguration {
     }
   }
 
-  public Configuration getConfiguration(String groupName) throws Exception {
-    Configuration configuration = (Configuration)getConfigurationRegion().get(groupName);
-    return configuration;
-  }
-
-  /*****
+  /**
    * Gets the region containing the shared configuration data.
    * The region is created , if it does not exist already.
    * Note : this could block if this locator contains stale persistent configuration data.
-   * @return  {@link Region} ConfigurationRegion
-   * @throws Exception 
+   * @return {@link Region} ConfigurationRegion
    */
   private Region<String, Configuration> getConfigurationRegion() throws Exception {
-    @SuppressWarnings("unchecked")
     Region<String, Configuration> configRegion = cache.getRegion(CONFIG_REGION_NAME);
 
     try {
       if (configRegion == null) {
-        File diskDir = new File(CONFIG_DISK_DIR_PATH);
+        File diskDir = new File(configDiskDirPath);
 
         if (!diskDir.exists()) {
           if (!diskDir.mkdirs()) {
-            throw new IOException("Cannot create directory at " + CONFIG_DISK_DIR_PATH);
+            throw new IOException("Cannot create directory at " + configDiskDirPath);
           }
         }
 
@@ -653,83 +716,24 @@ public class SharedConfiguration {
 
         configRegion = cache.createVMRegion(CONFIG_REGION_NAME, regionAttrsFactory.create(), internalArgs);
       }
+
     } catch (CancelException e) {
       if (configRegion == null) {
         this.status.set(SharedConfigurationStatus.STOPPED);
       }
       throw e; // CONFIG: don't rethrow as Exception, keep it a subclass of CancelException
+
     } catch (Exception e) {
       if (configRegion == null) {
         this.status.set(SharedConfigurationStatus.STOPPED);
       }
       throw new Exception("Error occurred while initializing cluster configuration", e);
     }
-    return configRegion;
-  }
 
-  public Map<String, Configuration> getEntireConfiguration() throws Exception {
-    Set<String> keys = getConfigurationRegion().keySet();
-    return getConfigurationRegion().getAll(keys);
-  }
-
-
-  /****
-   * Returns the path of Shared configuration directory
-   * @return {@link String}  path of the shared configuration directory
-   */
-  public String getSharedConfigurationDirPath() {
-    return CONFIG_DIR_PATH;
-  }
-
-
-  /*****
-   * Gets the current status of the SharedConfiguration
-   * If the status is started , it determines if the shared configuration is waiting for new configuration on 
-   * other locators
-   * @return {@link SharedConfigurationStatus}
-   */
-  public SharedConfigurationStatus getStatus() {
-    SharedConfigurationStatus scStatus = this.status.get();
-    if (scStatus == SharedConfigurationStatus.STARTED) {
-      PersistentMemberManager pmm = cache.getPersistentMemberManager();
-      Map<String, Set<PersistentMemberID>> waitingRegions = pmm.getWaitingRegions();
-      if (!waitingRegions.isEmpty()) {
-        this.status.compareAndSet(SharedConfigurationStatus.STARTED, SharedConfigurationStatus.WAITING);
-        Set<PersistentMemberID> persMemIds =  waitingRegions.get(Region.SEPARATOR_CHAR + CONFIG_REGION_NAME);
-        for (PersistentMemberID persMemId : persMemIds) {
-          newerSharedConfigurationLocatorInfo.add(new PersistentMemberPattern(persMemId));
-        }
-      }
-    }
-    return this.status.get();
-  }
-  /****
-   * Loads the 
-   * @throws Exception 
-   */
-  public void loadSharedConfigurationFromDisk() throws Exception {
-    Map<String, Configuration> sharedConfigurationMap = readSharedConfigurationFromDisk();
-    getConfigurationRegion().clear();
-    getConfigurationRegion().putAll(sharedConfigurationMap);
-  }
-
-  public void modifyProperties(Properties properties, String[] groups) throws Exception {
-    if (groups == null) {
-      groups = new String[] {SharedConfiguration.CLUSTER_CONFIG};
-    }
-    Region<String, Configuration> configRegion = getConfigurationRegion();
-    for (String group : groups) {
-      Configuration configuration = (Configuration) configRegion.get(group);
-      if (configuration == null) {
-        configuration = new Configuration(group);
-      } 
-      configuration.getGemfireProperties().putAll(properties);
-      configRegion.put(group, configuration);
-      writeConfig(configuration);
-    }
+    return configRegion;
   }
 
-  /*****
+  /**
    * Reads the configuration information from the shared configuration directory and returns a {@link Configuration} object
    * @param configName
    * @param configDirectory
@@ -739,7 +743,7 @@ public class SharedConfiguration {
    * @throws ParserConfigurationException 
    * @throws SAXException 
    */
-  private Configuration readConfiguration(String configName, String configDirectory) throws SAXException, ParserConfigurationException, TransformerFactoryConfigurationError, TransformerException {
+  private Configuration readConfiguration(final String configName, final String configDirectory) throws SAXException, ParserConfigurationException, TransformerFactoryConfigurationError, TransformerException {
     Configuration configuration = new Configuration(configName);
     String cacheXmlFullPath = FilenameUtils.concat(configDirectory, configuration.getCacheXmlFileName());
     String propertiesFullPath = FilenameUtils.concat(configDirectory, configuration.getPropertiesFileName());
@@ -760,14 +764,13 @@ public class SharedConfiguration {
     return configuration;
   }
 
-  /*****
+  /**
    * Reads the properties from the properties file.
    * @param propertiesFilePath
    * @return {@link Properties}
    * @throws IOException
    */
-  public Properties readProperties(String propertiesFilePath) throws IOException{
-
+  private Properties readProperties(final String propertiesFilePath) throws IOException {
     Properties properties = new Properties();
     File propsFile = new File(propertiesFilePath);
     FileInputStream fis = null;
@@ -784,9 +787,8 @@ public class SharedConfiguration {
     return properties;
   }
 
-
-  /****
-   * Reads the "shared_config" directory and loads all the cache.xml , gemfire.properties and deployd jars information
+  /**
+   * Reads the "shared_config" directory and loads all the cache.xml, gemfire.properties and deployed jars information
    * @return {@link Map} 
    * @throws TransformerException 
    * @throws TransformerFactoryConfigurationError 
@@ -794,12 +796,12 @@ public class SharedConfiguration {
    * @throws SAXException 
    */
   private Map<String, Configuration> readSharedConfigurationFromDisk() throws SAXException, ParserConfigurationException, TransformerFactoryConfigurationError, TransformerException {
-    String []subdirectoryNames = getSubdirectories(CONFIG_DIR_PATH);
+    String[] subdirectoryNames = getSubdirectories(configDirPath);
     Map<String, Configuration> sharedConfiguration = new HashMap<String, Configuration>();
 
     if (subdirectoryNames != null) {
       for (String subdirectoryName : subdirectoryNames) {
-        String fullpath = FilenameUtils.concat(CONFIG_DIR_PATH, subdirectoryName);
+        String fullpath = FilenameUtils.concat(configDirPath, subdirectoryName);
         Configuration configuration = readConfiguration(subdirectoryName, fullpath);
         sharedConfiguration.put(subdirectoryName, configuration);
       }
@@ -807,16 +809,13 @@ public class SharedConfiguration {
     return sharedConfiguration;
   }
 
-
-
-  /****
+  /**
    * Removes the jar files from the given directory     
    * @param dirPath Path of the configuration directory
    * @param jarNames Names of the jar files
    * @throws IOException
    */
-
-  public void removeJarFiles (String dirPath, String[] jarNames) throws IOException {
+  private void removeJarFiles(final String dirPath, final String[] jarNames) throws IOException {
     if (jarNames != null) {
       for (int i=0; i<jarNames.length; i++) {
         File jarFile = new File(FilenameUtils.concat(dirPath, jarNames[i]));
@@ -838,96 +837,21 @@ public class SharedConfiguration {
     }
   }
 
-  /****
-   * Removes the jar files from the shared configuration. 
-   * @param jarNames Names of the jar files.
-   * @param groups Names of the groups which had the jar file deployed.
-   * @return true on success.
-   */
-  public boolean removeJars(String []jarNames, String[] groups){
-    boolean success = true;
-    try {
-      Region<String, Configuration> configRegion = getConfigurationRegion();
-      if (groups == null) {
-        Set<String> groupSet = configRegion.keySet();
-        groups = groupSet.toArray(new String[groupSet.size()]);
-      }
-      for (String group : groups) {
-        Configuration configuration = (Configuration) configRegion.get(group);
-        if (configuration != null) {
-          String dirPath = FilenameUtils.concat(getSharedConfigurationDirPath(), configuration.getConfigName());
-          removeJarFiles(dirPath, jarNames);
-        }
-      }
-      for (String group : groups) {
-        Configuration configuration = (Configuration) configRegion.get(group);
-        if (configuration != null) {
-          if (!configuration.getJarNames().isEmpty()) {
-            configuration.removeJarNames(jarNames);
-            configRegion.put(group, configuration);
-          }
-        } 
-      }
-    } catch (Exception e) {
-      logger.info("Exception occurred while deleting the jar files", e);
-      success = false;
-    }
-    return success;
-  }
-
-  public void renameExistingSharedConfigDirectory() {
-    File configDirFile = new File(CONFIG_DIR_PATH);
-    if (configDirFile.exists()) {
-      String configDirFileName2 = CLUSTER_CONFIG_ARTIFACTS_DIR_NAME + new SimpleDateFormat("yyyyMMddhhmm").format(new Date()) + "." + System.nanoTime(); 
-      File configDirFile2 = new File(FilenameUtils.concat(configDirFileName2, configDirFileName2));
-      try {
-        FileUtils.moveDirectoryToDirectory(configDirFile, configDirFile2, true);
-      } catch (IOException e) {
-        logger.info(e);
-      }
-    } 
-  }
-  /***
+  /**
    * Writes the cache.xml to the file , based on Configuration
-   * @param dirPath Path of the directory in which the configuration is written
-   * @param configuration 
-   * @throws IOException 
    */
-  private void writeCacheXml(String dirPath, Configuration configuration) throws IOException {
+  private void writeCacheXml(final String dirPath, final Configuration configuration) throws IOException {
     String fullPath = FilenameUtils.concat(dirPath,configuration.getCacheXmlFileName());
     FileUtils.writeStringToFile(new File(fullPath), configuration.getCacheXmlContent(), "UTF-8") ;
   }
-  /***
-   * Writes the contents of the {@link Configuration} to the file system
-   * @param configuration
-   * @throws Exception
-   */
-  public void writeConfig(Configuration configuration) throws Exception {
-    File configDir = new File(getSharedConfigurationDirPath());
-    if (!configDir.exists()) {
-      if (!configDir.mkdirs()) {
-        throw new IOException("Cannot create directory : " + getSharedConfigurationDirPath());
-      }
-    }
-    String dirPath = FilenameUtils.concat(getSharedConfigurationDirPath(), configuration.getConfigName());
-    File file = new File(dirPath);
-    if (!file.exists()) {
-      if (!file.mkdir()) {
-        throw new IOException("Cannot create directory : " + dirPath);
-      }
-    }
-
-    writeProperties(dirPath, configuration);
-    writeCacheXml(dirPath, configuration);
-  }
 
-  /*****
+  /**
    * Writes the 
    * @param dirPath target directory , where the jar files are to be written
    * @param jarNames Array containing the name of the jar files.
    * @param jarBytes Array of byte arrays for the jar files.
    */
-  private void writeJarFiles(String dirPath , String[] jarNames, byte[][] jarBytes) {
+  private void writeJarFiles(final String dirPath, final String[] jarNames, final byte[][] jarBytes) {
     for (int i=0; i<jarNames.length; i++) {
       String filePath = FilenameUtils.concat(dirPath, jarNames[i]);
       File jarFile = new File(filePath);
@@ -939,17 +863,49 @@ public class SharedConfiguration {
     }
   }
 
-  /****
+  /**
    * Writes the properties to the file based on the {@link Configuration}
-   * @param dirPath
-   * @param configuration
-   * @throws IOException
    */
-  private void writeProperties(String dirPath, Configuration configuration) throws IOException {
+  private void writeProperties(final String dirPath, final Configuration configuration) throws IOException {
     String fullPath = FilenameUtils.concat(dirPath,configuration.getPropertiesFileName());
     BufferedWriter bw = new BufferedWriter(new FileWriter(fullPath));
     configuration.getGemfireProperties().store(bw, "");
     bw.close();
   }
 
+  /**
+   * Create a {@link Document} using
+   * {@link XmlUtils#createDocumentFromXml(String)} and if the version attribute
+   * is not equal to the current version then update the XML to the current
+   * schema and return the document.
+   *
+   * @param xmlContent XML content to load and upgrade.
+   * @return {@link Document} from xmlContent.
+   * @since 8.1
+   */
+  // UnitTest SharedConfigurationJUnitTest.testCreateAndUpgradeDocumentFromXml
+  static Document createAndUpgradeDocumentFromXml(final String xmlContent) throws SAXException, ParserConfigurationException, IOException, XPathExpressionException {
+    Document doc = XmlUtils.createDocumentFromXml(xmlContent);
+    if (!CacheXml.VERSION_LATEST.equals(XmlUtils.getAttribute(doc.getDocumentElement(), CacheXml.VERSION, CacheXml.GEODE_NAMESPACE))) {
+      doc = XmlUtils.upgradeSchema(doc, CacheXml.GEODE_NAMESPACE, CacheXml.LATEST_SCHEMA_LOCATION, CacheXml.VERSION_LATEST);
+    }
+    return doc;
+  }
+
+  /**
+   * Returns an array containing the names of the subdirectories in a given directory
+   * @param path Path of the directory whose subdirectories are listed
+   * @return String[] names of first level subdirectories, null if no subdirectories are found or if the path is incorrect
+   */
+  private static String[] getSubdirectories(String path) {
+    File directory = new File(path);
+    return directory.list(DirectoryFileFilter.INSTANCE);
+  }
+
+  private static class JarFileFilter implements FilenameFilter {
+    @Override
+    public boolean accept(File dir, String name) {
+      return name.endsWith(".jar");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
deleted file mode 100755
index 745090d..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed;
-
-import static org.assertj.core.api.Assertions.*;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.util.Properties;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Integration tests for AbstractLauncher class. These tests require file system I/O.
- */
-@Category(IntegrationTest.class)
-public class AbstractLauncherIntegrationJUnitTest {
-
-  @Rule
-  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
-  
-  @Rule
-  public final TestName testName = new TestName();
-  
-  private File gemfirePropertiesFile;
-  private Properties expectedGemfireProperties;
-  
-  @Before
-  public void setUp() throws Exception {
-    this.gemfirePropertiesFile = this.temporaryFolder.newFile("gemfire.properties");
-    
-    this.expectedGemfireProperties = new Properties();
-    this.expectedGemfireProperties.setProperty(DistributionConfig.NAME_NAME, "memberOne");
-    this.expectedGemfireProperties.setProperty(DistributionConfig.GROUPS_NAME, "groupOne, groupTwo");
-    this.expectedGemfireProperties.store(new FileWriter(this.gemfirePropertiesFile, false), this.testName.getMethodName());
-
-    assertThat(this.gemfirePropertiesFile).isNotNull();
-    assertThat(this.gemfirePropertiesFile.exists()).isTrue();
-    assertThat(this.gemfirePropertiesFile.isFile()).isTrue();
-  }
-  
-  @Test
-  public void testLoadGemFirePropertiesFromFile() throws Exception {
-    final Properties actualGemFireProperties = AbstractLauncher.loadGemFireProperties(this.gemfirePropertiesFile.toURI().toURL());
-
-    assertThat(actualGemFireProperties).isNotNull();
-    assertThat(actualGemFireProperties).isEqualTo(this.expectedGemfireProperties);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/566fce96/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationTest.java
new file mode 100755
index 0000000..040d15f
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for AbstractLauncher class. These tests require file system I/O.
+ */
+@Category(IntegrationTest.class)
+public class AbstractLauncherIntegrationTest {
+
+  private File gemfirePropertiesFile;
+  private Properties expectedGemfireProperties;
+
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
+  @Before
+  public void setUp() throws Exception {
+    this.gemfirePropertiesFile = this.temporaryFolder.newFile("gemfire.properties");
+    
+    this.expectedGemfireProperties = new Properties();
+    this.expectedGemfireProperties.setProperty(DistributionConfig.NAME_NAME, "memberOne");
+    this.expectedGemfireProperties.setProperty(DistributionConfig.GROUPS_NAME, "groupOne, groupTwo");
+    this.expectedGemfireProperties.store(new FileWriter(this.gemfirePropertiesFile, false), this.testName.getMethodName());
+
+    assertThat(this.gemfirePropertiesFile).isNotNull();
+    assertThat(this.gemfirePropertiesFile.exists()).isTrue();
+    assertThat(this.gemfirePropertiesFile.isFile()).isTrue();
+  }
+  
+  @Test
+  public void testLoadGemFirePropertiesFromFile() throws Exception {
+    final Properties actualGemFireProperties = AbstractLauncher.loadGemFireProperties(this.gemfirePropertiesFile.toURI().toURL());
+
+    assertThat(actualGemFireProperties).isNotNull();
+    assertThat(actualGemFireProperties).isEqualTo(this.expectedGemfireProperties);
+  }
+}