You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2017/04/21 23:42:18 UTC

[22/51] [abbrv] geode git commit: GEODE-2290: Limit scanning of deployed jars

GEODE-2290: Limit scanning of deployed jars

 - Uses fast-classpath-scanner to scan jars for classes containing Functions without eagerly loading all classes in the jar.


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

Branch: refs/heads/feature/GEODE-2097
Commit: 6f7f943998153de1f00b1446eb80fa6d304ac40f
Parents: 8f9624d
Author: Jared Stewart <js...@pivotal.io>
Authored: Wed Apr 12 10:53:13 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Sun Apr 16 09:10:02 2017 -0700

----------------------------------------------------------------------
 geode-assembly/build.gradle                     |   1 +
 .../src/test/resources/expected_jars.txt        |   1 +
 geode-core/build.gradle                         |  15 +-
 .../org/apache/geode/internal/DeployedJar.java  |  97 +++---
 .../org/apache/geode/internal/JarDeployer.java  |   7 +-
 .../ClassPathLoaderIntegrationTest.java         |   4 -
 .../geode/internal/DeployedJarJUnitTest.java    | 310 ++++++-------------
 .../internal/JarDeployerIntegrationTest.java    |  18 +-
 .../cache/IncrementalBackupDUnitTest.java       |   6 +-
 .../geode/management/DeployJarTestSuite.java    |   3 +
 .../DeployCommandRedeployDUnitTest.java         |  95 +++---
 .../cli/commands/DeployCommandsDUnitTest.java   |  40 +--
 .../configuration/ClusterConfigBaseTest.java    | 127 --------
 .../ClusterConfigDeployJarDUnitTest.java        |  10 +-
 .../ClusterConfigImportDUnitTest.java           |   3 +-
 .../ClusterConfigStartMemberDUnitTest.java      |   3 +-
 .../configuration/ClusterConfigTestBase.java    | 128 ++++++++
 .../dunit/rules/LocatorServerStartupRule.java   |  12 +
 ...oyCommandRedeployDUnitTest_FunctionATemplate |  30 ++
 ...oyCommandRedeployDUnitTest_FunctionBTemplate |  26 ++
 gradle/dependency-versions.properties           |   1 +
 21 files changed, 439 insertions(+), 498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index 304a1c4..a4f0c69 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -139,6 +139,7 @@ def cp = {
         it.contains('commons-io') ||
         it.contains('commons-lang') ||
         it.contains('commons-logging') ||
+        it.contains('fast-classpath-scanner') ||
         it.contains('fastutil') ||
         it.contains('jackson-annotations') ||
         it.contains('jackson-core') ||

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-assembly/src/test/resources/expected_jars.txt
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/resources/expected_jars.txt b/geode-assembly/src/test/resources/expected_jars.txt
index b7d1dc2..6260167 100644
--- a/geode-assembly/src/test/resources/expected_jars.txt
+++ b/geode-assembly/src/test/resources/expected_jars.txt
@@ -10,6 +10,7 @@ commons-io
 commons-lang
 commons-logging
 commons-modeler
+fast-classpath-scanner
 fastutil
 findbugs-annotations
 gfsh-dependencies.jar

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 757599a..f07444a 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -32,7 +32,7 @@ configurations {
 
 dependencies {
    // Source Dependencies
-  // External 
+  // External
   provided files("${System.getProperty('java.home')}/../lib/tools.jar")
   compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
   compile 'org.jgroups:jgroups:' + project.'jgroups.version'
@@ -104,17 +104,22 @@ dependencies {
   }
   compile ('org.iq80.snappy:snappy:' + project.'snappy-java.version') {
     ext.optional = true
-  } 
+  }
 
   compile 'org.apache.shiro:shiro-core:' + project.'shiro.version'
   // This is only added since shiro is using an old version of beanutils and we want
   // to use a standard version. Once shiro deps are updated, remove this explicit dependency
   // in favor of a transitive dependency on beanutils.
   compile 'commons-beanutils:commons-beanutils:' + project.'commons-beanutils.version'
-  
+
+  // https://mvnrepository.com/artifact/io.github.lukehutch/fast-classpath-scanner
+    compile 'io.github.lukehutch:fast-classpath-scanner:' + project.'fast-classpath-scanner.version'
+
+
+
   compile project(':geode-common')
   compile project(':geode-json')
-  
+
   jcaCompile sourceSets.main.output
 
   testCompile project(':geode-junit')
@@ -181,7 +186,7 @@ jar {
 
   from sourceSets.main.output
   from sourceSets.jca.output
-  
+
   exclude 'org/apache/geode/management/internal/web/**'
   exclude 'org/apache/geode/internal/i18n/StringIdResourceBundle_ja.txt'
   exclude 'org/apache/geode/admin/doc-files/ds4_0.dtd'

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
index 8adec1f..f96863f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
@@ -14,6 +14,9 @@
  */
 package org.apache.geode.internal;
 
+import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
+import io.github.lukehutch.fastclasspathscanner.scanner.ScanResult;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -25,7 +28,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.net.URL;
-import java.nio.channels.FileLock;
+import java.net.URLClassLoader;
 import java.nio.file.Files;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -37,7 +40,6 @@ import java.util.Properties;
 import java.util.jar.JarEntry;
 import java.util.jar.JarInputStream;
 
-import org.apache.geode.cache.Cache;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.cache.CacheClosedException;
@@ -48,7 +50,6 @@ import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.pdx.internal.TypeRegistry;
-import sun.nio.ch.ChannelInputStream;
 
 /**
  * ClassLoader for a single JAR file.
@@ -103,7 +104,7 @@ public class DeployedJar {
           + ", was modified prior to obtaining a lock: " + jarName);
     }
 
-    if (!isValidJarContent(getJarContent())) {
+    if (!hasValidJarContent(getJarContent())) {
       throw new IllegalArgumentException(
           "File does not contain valid JAR content: " + versionedJarFile.getAbsolutePath());
     }
@@ -147,23 +148,10 @@ public class DeployedJar {
    * @param jarBytes Bytes of data to be validated.
    * @return True if the data has JAR content, false otherwise
    */
-  public static boolean isValidJarContent(final byte[] jarBytes) {
+  public static boolean hasValidJarContent(final byte[] jarBytes) {
     return hasValidJarContent(new ByteArrayInputStream(jarBytes));
   }
 
-  /**
-   * Peek into the JAR data and make sure that it is valid JAR content.
-   * 
-   * @param jarFile File whose contents should be validated.
-   * @return True if the data has JAR content, false otherwise
-   */
-  public static boolean hasValidJarContent(final File jarFile) {
-    try {
-      return hasValidJarContent(new FileInputStream(jarFile));
-    } catch (IOException ioex) {
-      return false;
-    }
-  }
 
   /**
    * Scan the JAR file and attempt to load all classes and register any function classes found.
@@ -184,37 +172,42 @@ public class DeployedJar {
 
     JarInputStream jarInputStream = null;
     try {
+      List<String> functionClasses = findFunctionsInThisJar();
+
       jarInputStream = new JarInputStream(byteArrayInputStream);
       JarEntry jarEntry = jarInputStream.getNextJarEntry();
 
       while (jarEntry != null) {
         if (jarEntry.getName().endsWith(".class")) {
-          if (isDebugEnabled) {
-            logger.debug("Attempting to load class: {}, from JAR file: {}", jarEntry.getName(),
-                this.file.getAbsolutePath());
-          }
-
           final String className = jarEntry.getName().replaceAll("/", "\\.").substring(0,
               (jarEntry.getName().length() - 6));
-          try {
-            Class<?> clazz = ClassPathLoader.getLatest().forName(className);
-            Collection<Function> registerableFunctions = getRegisterableFunctionsFromClass(clazz);
-            for (Function function : registerableFunctions) {
-              FunctionService.registerFunction(function);
-              if (isDebugEnabled) {
-                logger.debug("Registering function class: {}, from JAR file: {}", className,
-                    this.file.getAbsolutePath());
+
+          if (functionClasses.contains(className)) {
+            if (isDebugEnabled) {
+              logger.debug("Attempting to load class: {}, from JAR file: {}", jarEntry.getName(),
+                  this.file.getAbsolutePath());
+            }
+            try {
+              Class<?> clazz = ClassPathLoader.getLatest().forName(className);
+              Collection<Function> registerableFunctions = getRegisterableFunctionsFromClass(clazz);
+              for (Function function : registerableFunctions) {
+                FunctionService.registerFunction(function);
+                if (isDebugEnabled) {
+                  logger.debug("Registering function class: {}, from JAR file: {}", className,
+                      this.file.getAbsolutePath());
+                }
+                this.registeredFunctions.add(function);
               }
-              this.registeredFunctions.add(function);
+            } catch (ClassNotFoundException | NoClassDefFoundError cnfex) {
+              logger.error("Unable to load all classes from JAR file: {}",
+                  this.file.getAbsolutePath(), cnfex);
+              throw cnfex;
+            }
+          } else {
+            if (isDebugEnabled) {
+              logger.debug("No functions found in class: {}, from JAR file: {}", jarEntry.getName(),
+                  this.file.getAbsolutePath());
             }
-          } catch (ClassNotFoundException cnfex) {
-            logger.error("Unable to load all classes from JAR file: {}",
-                this.file.getAbsolutePath(), cnfex);
-            throw cnfex;
-          } catch (NoClassDefFoundError ncdfex) {
-            logger.error("Unable to load all classes from JAR file: {}",
-                this.file.getAbsolutePath(), ncdfex);
-            throw ncdfex;
           }
         }
         jarEntry = jarInputStream.getNextJarEntry();
@@ -327,6 +320,15 @@ public class DeployedJar {
     return registerableFunctions;
   }
 
+  private List<String> findFunctionsInThisJar() throws IOException {
+    URLClassLoader urlClassLoader =
+        new URLClassLoader(new URL[] {this.getFile().getCanonicalFile().toURL()});
+    FastClasspathScanner fastClasspathScanner = new FastClasspathScanner()
+        .removeTemporaryFilesAfterScan(true).overrideClassLoaders(urlClassLoader);
+    ScanResult scanResult = fastClasspathScanner.scan();
+    return scanResult.getNamesOfClassesImplementing(Function.class);
+  }
+
   private Function newFunction(final Class<Function> clazz, final boolean errorOnNoSuchMethod) {
     try {
       final Constructor<Function> constructor = clazz.getConstructor();
@@ -342,20 +344,11 @@ public class DeployedJar {
               clazz.getName());
         }
       }
-    } catch (SecurityException sex) {
-      logger.error("Zero-arg constructor of function not accessible for class: {}", clazz.getName(),
-          sex);
-    } catch (IllegalAccessException iae) {
-      logger.error("Zero-arg constructor of function not accessible for class: {}", clazz.getName(),
-          iae);
-    } catch (InvocationTargetException ite) {
+    } catch (Exception ex) {
       logger.error("Error when attempting constructor for function for class: {}", clazz.getName(),
-          ite);
-    } catch (InstantiationException ie) {
-      logger.error("Unable to instantiate function for class: {}", clazz.getName(), ie);
-    } catch (ExceptionInInitializerError eiiex) {
-      logger.error("Error during function initialization for class: {}", clazz.getName(), eiiex);
+          ex);
     }
+
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java b/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
index da4c136..a65cd0f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
@@ -26,7 +26,6 @@ import java.io.BufferedInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.Serializable;
@@ -35,7 +34,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -51,7 +49,6 @@ import java.util.stream.Stream;
 public class JarDeployer implements Serializable {
   private static final long serialVersionUID = 1L;
   private static final Logger logger = LogService.getLogger();
-  public static final String JAR_PREFIX = "";
   public static final String JAR_PREFIX_FOR_REGEX = "";
   private static final Lock lock = new ReentrantLock();
 
@@ -438,7 +435,7 @@ public class JarDeployer implements Serializable {
     Optional<File> latestValidDeployedJarOptional =
         Arrays.stream(jarFiles).filter(Objects::nonNull).filter(jarFile -> {
           try {
-            return DeployedJar.isValidJarContent(FileUtils.readFileToByteArray(jarFile));
+            return DeployedJar.hasValidJarContent(FileUtils.readFileToByteArray(jarFile));
           } catch (IOException e) {
             return false;
           }
@@ -501,7 +498,7 @@ public class JarDeployer implements Serializable {
     DeployedJar[] deployedJars = new DeployedJar[jarNames.length];
 
     for (int i = 0; i < jarNames.length; i++) {
-      if (!DeployedJar.isValidJarContent(jarBytes[i])) {
+      if (!DeployedJar.hasValidJarContent(jarBytes[i])) {
         throw new IllegalArgumentException(
             "File does not contain valid JAR content: " + jarNames[i]);
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderIntegrationTest.java
index 14108c7..c783318 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderIntegrationTest.java
@@ -66,7 +66,6 @@ public class ClassPathLoaderIntegrationTest {
 
   private File tempFile;
   private File tempFile2;
-  private File extLibsDir;
 
   @Rule
   public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
@@ -78,9 +77,6 @@ public class ClassPathLoaderIntegrationTest {
   public void setUp() throws Exception {
     System.setProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, "false");
 
-    extLibsDir = new File(this.temporaryFolder.getRoot(), "ext");
-    extLibsDir.mkdirs();
-
     this.tempFile = this.temporaryFolder.newFile("tempFile1.tmp");
     FileOutputStream fos = new FileOutputStream(this.tempFile);
     fos.write(new byte[TEMP_FILE_BYTES_COUNT]);

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/internal/DeployedJarJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/DeployedJarJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/DeployedJarJUnitTest.java
index 7216463..5e7c40f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/DeployedJarJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/DeployedJarJUnitTest.java
@@ -14,12 +14,9 @@
  */
 package org.apache.geode.internal;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
@@ -27,6 +24,7 @@ import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultSender;
 import org.apache.geode.internal.cache.execute.FunctionContextImpl;
 import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.awaitility.Awaitility;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -43,17 +41,12 @@ import java.io.OutputStream;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.Random;
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
-/**
- * TODO: Need to fix this testDeclarableFunctionsWithParms and testClassOnClasspath on Windows:
- */
 @Category(IntegrationTest.class)
 public class DeployedJarJUnitTest {
   @Rule
@@ -62,13 +55,13 @@ public class DeployedJarJUnitTest {
   @Rule
   public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
-  private final ClassBuilder classBuilder = new ClassBuilder();
+  private ClassBuilder classBuilder;
 
   @Before
   public void setup() throws Exception {
     File workingDir = temporaryFolder.newFolder();
-
     ClassPathLoader.setLatestToDefault(workingDir);
+    classBuilder = new ClassBuilder();
   }
 
   @After
@@ -82,13 +75,14 @@ public class DeployedJarJUnitTest {
 
   @Test
   public void testIsValidJarContent() throws IOException {
-    assertTrue(
-        DeployedJar.isValidJarContent(this.classBuilder.createJarFromName("JarClassLoaderJUnitA")));
+    assertThat(
+        DeployedJar.hasValidJarContent(this.classBuilder.createJarFromName("JarClassLoaderJUnitA")))
+            .isTrue();
   }
 
   @Test
   public void testIsInvalidJarContent() {
-    assertFalse(DeployedJar.isValidJarContent("INVALID JAR CONTENT".getBytes()));
+    assertThat(DeployedJar.hasValidJarContent("INVALID JAR CONTENT".getBytes())).isFalse();
   }
 
   @Test
@@ -99,11 +93,7 @@ public class DeployedJarJUnitTest {
             "package com.jcljunit; public class JarClassLoaderJUnitA {}");
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnit.jar", jarBytes);
 
-    try {
-      ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitA");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
+    ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitA");
 
     // Update the JAR file and make sure the first class is no longer on the Classpath
     // and the second one is.
@@ -111,54 +101,36 @@ public class DeployedJarJUnitTest {
         "package com.jcljunit; public class JarClassLoaderJUnitB {}");
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnit.jar", jarBytes);
 
-    try {
-      ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitB");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
-
-    try {
-      ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitA");
-      fail("Class should not be found on Classpath");
-    } catch (ClassNotFoundException expected) { // expected
-    }
-
+    ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitB");
+    assertThatThrownBy(
+        () -> ClassPathLoader.getLatest().forName("com.jcljunit.JarClassLoaderJUnitA"))
+            .isInstanceOf(ClassNotFoundException.class);
   }
 
   @Test
   public void testFailingCompilation() throws Exception {
-    StringBuffer stringBuffer = new StringBuffer();
-    stringBuffer.append("import org.apache.geode.cache.Declarable;");
-    stringBuffer.append("import org.apache.geode.cache.execute.Function;");
-    stringBuffer.append("import org.apache.geode.cache.execute.FunctionContext;");
-    stringBuffer.append("public class JarClassLoaderJUnitFunction implements Function {}");
-    String functionString = stringBuffer.toString();
-
-    try {
-      this.classBuilder.createJarFromClassContent("JarClassLoaderJUnitFunction", functionString);
-      fail("This code should have failed to compile and thrown an exception");
-    } catch (Exception ex) {
-      // All good
-    }
+    String functionString = "import org.apache.geode.cache.Declarable;"
+        + "import org.apache.geode.cache.execute.Function;"
+        + "import org.apache.geode.cache.execute.FunctionContext;"
+        + "public class JarClassLoaderJUnitFunction implements Function {}";
+
+    assertThatThrownBy(() -> this.classBuilder
+        .createJarFromClassContent("JarClassLoaderJUnitFunction", functionString)).isNotNull();
   }
 
   @Test
-  public void testFunctions() throws IOException, ClassNotFoundException {
+  public void testFunctions() throws Exception {
     // Test creating a JAR file with a function
-    StringBuffer stringBuffer = new StringBuffer();
-    stringBuffer.append("import java.util.Properties;");
-    stringBuffer.append("import org.apache.geode.cache.Declarable;");
-    stringBuffer.append("import org.apache.geode.cache.execute.Function;");
-    stringBuffer.append("import org.apache.geode.cache.execute.FunctionContext;");
-    stringBuffer.append("public class JarClassLoaderJUnitFunction implements Function {");
-    stringBuffer.append("public void init(Properties props) {}");
-    stringBuffer.append("public boolean hasResult() {return true;}");
-    stringBuffer.append(
-        "public void execute(FunctionContext context) {context.getResultSender().lastResult(\"GOODv1\");}");
-    stringBuffer.append("public String getId() {return \"JarClassLoaderJUnitFunction\";}");
-    stringBuffer.append("public boolean optimizeForWrite() {return false;}");
-    stringBuffer.append("public boolean isHA() {return false;}}");
-    String functionString = stringBuffer.toString();
+    String functionString =
+        "import java.util.Properties;" + "import org.apache.geode.cache.Declarable;"
+            + "import org.apache.geode.cache.execute.Function;"
+            + "import org.apache.geode.cache.execute.FunctionContext;"
+            + "public class JarClassLoaderJUnitFunction implements Function {"
+            + "public void init(Properties props) {}" + "public boolean hasResult() {return true;}"
+            + "public void execute(FunctionContext context) {context.getResultSender().lastResult(\"GOODv1\");}"
+            + "public String getId() {return \"JarClassLoaderJUnitFunction\";}"
+            + "public boolean optimizeForWrite() {return false;}"
+            + "public boolean isHA() {return false;}}";
 
     byte[] jarBytes =
         this.classBuilder.createJarFromClassContent("JarClassLoaderJUnitFunction", functionString);
@@ -166,11 +138,11 @@ public class DeployedJarJUnitTest {
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnit.jar", jarBytes);
 
     Function function = FunctionService.getFunction("JarClassLoaderJUnitFunction");
-    assertNotNull(function);
+    assertThat(function).isNotNull();
     TestResultSender resultSender = new TestResultSender();
     FunctionContext functionContext = new FunctionContextImpl(function.getId(), null, resultSender);
     function.execute(functionContext);
-    assertEquals("GOODv1", (String) resultSender.getResults());
+    assertThat(resultSender.getResults()).isEqualTo("GOODv1");
 
     // Test updating the function with a new JAR file
     functionString = functionString.replace("v1", "v2");
@@ -179,11 +151,11 @@ public class DeployedJarJUnitTest {
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnit.jar", jarBytes);
 
     function = FunctionService.getFunction("JarClassLoaderJUnitFunction");
-    assertNotNull(function);
+    assertThat(function).isNotNull();
     resultSender = new TestResultSender();
     functionContext = new FunctionContextImpl(function.getId(), null, resultSender);
     function.execute(functionContext);
-    assertEquals("GOODv2", (String) resultSender.getResults());
+    assertThat(resultSender.getResults()).isEqualTo("GOODv2");
 
     // Test returning null for the Id
     String functionNullIdString =
@@ -192,88 +164,71 @@ public class DeployedJarJUnitTest {
         functionNullIdString);
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnit.jar", jarBytes);
 
-    assertNull(FunctionService.getFunction("JarClassLoaderJUnitFunction"));
+    assertThat(FunctionService.getFunction("JarClassLoaderJUnitFunction")).isNull();
 
     // Test removing the JAR
     ClassPathLoader.getLatest().getJarDeployer().undeploy("JarClassLoaderJUnit.jar");
-    assertNull(FunctionService.getFunction("JarClassLoaderJUnitFunction"));
+    assertThat(FunctionService.getFunction("JarClassLoaderJUnitFunction")).isNull();
   }
 
   /**
    * Ensure that abstract functions aren't added to the Function Service.
    */
   @Test
-  public void testAbstractFunction() throws IOException, ClassNotFoundException {
+  public void testAbstractFunction() throws Exception {
     // Add an abstract Function to the Classpath
-    StringBuffer stringBuffer = new StringBuffer();
-    stringBuffer.append("import org.apache.geode.cache.execute.Function;");
-    stringBuffer.append("public abstract class JarClassLoaderJUnitFunction implements Function {");
-    stringBuffer.append("public String getId() {return \"JarClassLoaderJUnitFunction\";}}");
-    String functionString = stringBuffer.toString();
+    String functionString = "import org.apache.geode.cache.execute.Function;"
+        + "public abstract class JarClassLoaderJUnitFunction implements Function {"
+        + "public String getId() {return \"JarClassLoaderJUnitFunction\";}}";
 
     byte[] jarBytes =
         this.classBuilder.createJarFromClassContent("JarClassLoaderJUnitFunction", functionString);
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnitFunction.jar",
         jarBytes);
 
-    try {
-      ClassPathLoader.getLatest().forName("JarClassLoaderJUnitFunction");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
+    ClassPathLoader.getLatest().forName("JarClassLoaderJUnitFunction");
 
     Function function = FunctionService.getFunction("JarClassLoaderJUnitFunction");
-    assertNull(function);
+    assertThat(function).isNull();
   }
 
   @Test
   public void testDeclarableFunctionsWithNoCacheXml() throws Exception {
-
     final String jarName = "JarClassLoaderJUnitNoXml.jar";
 
     // Add a Declarable Function without parameters for the class to the Classpath
-    StringBuffer stringBuffer = new StringBuffer();
-    stringBuffer.append("import java.util.Properties;");
-    stringBuffer.append("import org.apache.geode.cache.Declarable;");
-    stringBuffer.append("import org.apache.geode.cache.execute.Function;");
-    stringBuffer.append("import org.apache.geode.cache.execute.FunctionContext;");
-    stringBuffer
-        .append("public class JarClassLoaderJUnitFunctionNoXml implements Function, Declarable {");
-    stringBuffer.append("public String getId() {return \"JarClassLoaderJUnitFunctionNoXml\";}");
-    stringBuffer.append("public void init(Properties props) {}");
-    stringBuffer.append(
-        "public void execute(FunctionContext context) {context.getResultSender().lastResult(\"NOPARMSv1\");}");
-    stringBuffer.append("public boolean hasResult() {return true;}");
-    stringBuffer.append("public boolean optimizeForWrite() {return false;}");
-    stringBuffer.append("public boolean isHA() {return false;}}");
-    String functionString = stringBuffer.toString();
+    String functionString =
+        "import java.util.Properties;" + "import org.apache.geode.cache.Declarable;"
+            + "import org.apache.geode.cache.execute.Function;"
+            + "import org.apache.geode.cache.execute.FunctionContext;"
+            + "public class JarClassLoaderJUnitFunctionNoXml implements Function, Declarable {"
+            + "public String getId() {return \"JarClassLoaderJUnitFunctionNoXml\";}"
+            + "public void init(Properties props) {}"
+            + "public void execute(FunctionContext context) {context.getResultSender().lastResult(\"NOPARMSv1\");}"
+            + "public boolean hasResult() {return true;}"
+            + "public boolean optimizeForWrite() {return false;}"
+            + "public boolean isHA() {return false;}}";
 
     byte[] jarBytes = this.classBuilder
         .createJarFromClassContent("JarClassLoaderJUnitFunctionNoXml", functionString);
 
     ClassPathLoader.getLatest().getJarDeployer().deploy(jarName, jarBytes);
 
-    try {
-      ClassPathLoader.getLatest().forName("JarClassLoaderJUnitFunctionNoXml");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
+    ClassPathLoader.getLatest().forName("JarClassLoaderJUnitFunctionNoXml");
 
     // Check to see if the function without parameters executes correctly
     Function function = FunctionService.getFunction("JarClassLoaderJUnitFunctionNoXml");
-    assertNotNull(function);
+    assertThat(function).isNotNull();
     TestResultSender resultSender = new TestResultSender();
     function.execute(new FunctionContextImpl(function.getId(), null, resultSender));
-    assertEquals("NOPARMSv1", (String) resultSender.getResults());
+    assertThat((String) resultSender.getResults()).isEqualTo("NOPARMSv1");
   }
 
   @Test
-  public void testDependencyBetweenJars() throws IOException, ClassNotFoundException {
+  public void testDependencyBetweenJars() throws Exception {
     final File parentJarFile = temporaryFolder.newFile("JarClassLoaderJUnitParent.jar");
     final File usesJarFile = temporaryFolder.newFile("JarClassLoaderJUnitUses.jar");
 
-    JarDeployer jarDeployer = ClassPathLoader.getLatest().getJarDeployer();
-
     // Write out a JAR files.
     StringBuffer stringBuffer = new StringBuffer();
     stringBuffer.append("package jcljunit.parent;");
@@ -319,17 +274,15 @@ public class DeployedJarJUnitTest {
     jarBytes = functionClassBuilder.createJarFromClassContent(
         "jcljunit/function/JarClassLoaderJUnitFunction", stringBuffer.toString());
 
-
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnitFunction.jar",
         jarBytes);
 
-
     Function function = FunctionService.getFunction("JarClassLoaderJUnitFunction");
-    assertNotNull(function);
+    assertThat(function).isNotNull();
     TestResultSender resultSender = new TestResultSender();
     FunctionContext functionContext = new FunctionContextImpl(function.getId(), null, resultSender);
     function.execute(functionContext);
-    assertEquals("PARENT:USES", (String) resultSender.getResults());
+    assertThat((String) resultSender.getResults()).isEqualTo("PARENT:USES");
   }
 
   @Test
@@ -342,39 +295,26 @@ public class DeployedJarJUnitTest {
         jarBytes);
 
     InputStream inputStream = ClassPathLoader.getLatest().getResourceAsStream(fileName);
-    assertNotNull(inputStream);
+    assertThat(inputStream).isNotNull();
 
     final byte[] fileBytes = new byte[fileContent.length()];
     inputStream.read(fileBytes);
     inputStream.close();
-    assertTrue(fileContent.equals(new String(fileBytes)));
+    assertThat(fileContent).isEqualTo(new String(fileBytes));
   }
 
   @Test
-  public void testUpdateClassInJar() throws IOException, ClassNotFoundException {
+  public void testUpdateClassInJar() throws Exception {
     // First use of the JAR file
     byte[] jarBytes = this.classBuilder.createJarFromClassContent("JarClassLoaderJUnitTestClass",
         "public class JarClassLoaderJUnitTestClass { public Integer getValue5() { return new Integer(5); } }");
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnitUpdate.jar", jarBytes);
 
-    try {
-      Class<?> clazz = ClassPathLoader.getLatest().forName("JarClassLoaderJUnitTestClass");
-      Object object = clazz.newInstance();
-      Method getValue5Method = clazz.getMethod("getValue5", new Class[] {});
-      Integer value = (Integer) getValue5Method.invoke(object, new Object[] {});
-      assertEquals(value.intValue(), 5);
-
-    } catch (InvocationTargetException itex) {
-      fail("JAR file not correctly added to Classpath" + itex);
-    } catch (NoSuchMethodException nsmex) {
-      fail("JAR file not correctly added to Classpath" + nsmex);
-    } catch (InstantiationException iex) {
-      fail("JAR file not correctly added to Classpath" + iex);
-    } catch (IllegalAccessException iaex) {
-      fail("JAR file not correctly added to Classpath" + iaex);
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath" + cnfex);
-    }
+    Class<?> clazz = ClassPathLoader.getLatest().forName("JarClassLoaderJUnitTestClass");
+    Object object = clazz.newInstance();
+    Method getValue5Method = clazz.getMethod("getValue5");
+    Integer value = (Integer) getValue5Method.invoke(object);
+    assertThat(value).isEqualTo(5);
 
     // Now create an updated JAR file and make sure that the method from the new
     // class is available.
@@ -382,29 +322,15 @@ public class DeployedJarJUnitTest {
         "public class JarClassLoaderJUnitTestClass { public Integer getValue10() { return new Integer(10); } }");
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnitUpdate.jar", jarBytes);
 
-
-    try {
-      Class<?> clazz = ClassPathLoader.getLatest().forName("JarClassLoaderJUnitTestClass");
-      Object object = clazz.newInstance();
-      Method getValue10Method = clazz.getMethod("getValue10", new Class[] {});
-      Integer value = (Integer) getValue10Method.invoke(object, new Object[] {});
-      assertEquals(value.intValue(), 10);
-
-    } catch (InvocationTargetException itex) {
-      fail("JAR file not correctly added to Classpath" + itex);
-    } catch (NoSuchMethodException nsmex) {
-      fail("JAR file not correctly added to Classpath" + nsmex);
-    } catch (InstantiationException iex) {
-      fail("JAR file not correctly added to Classpath" + iex);
-    } catch (IllegalAccessException iaex) {
-      fail("JAR file not correctly added to Classpath" + iaex);
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath" + cnfex);
-    }
+    clazz = ClassPathLoader.getLatest().forName("JarClassLoaderJUnitTestClass");
+    object = clazz.newInstance();
+    Method getValue10Method = clazz.getMethod("getValue10");
+    value = (Integer) getValue10Method.invoke(object);
+    assertThat(value).isEqualTo(10);
   }
 
   @Test
-  public void testMultiThread() throws IOException, ClassNotFoundException {
+  public void testMultiThreadingDoesNotCauseDeadlock() throws Exception {
     // Add two JARs to the classpath
     byte[] jarBytes = this.classBuilder.createJarFromName("JarClassLoaderJUnitA");
     ClassPathLoader.getLatest().getJarDeployer().deploy("JarClassLoaderJUnitA.jar", jarBytes);
@@ -416,54 +342,32 @@ public class DeployedJarJUnitTest {
     String[] classNames = new String[] {"JarClassLoaderJUnitA", "com.jcljunit.JarClassLoaderJUnitB",
         "NON-EXISTENT CLASS"};
 
-    // Spawn some threads which try to instantiate these classes
     final int threadCount = 10;
-    final int numLoops = 1000;
-    final CyclicBarrier cyclicBarrier = new CyclicBarrier(threadCount + 1);
+    ExecutorService executorService = Executors.newFixedThreadPool(threadCount);
     for (int i = 0; i < threadCount; i++) {
-      new ForNameExerciser(cyclicBarrier, numLoops, classNames).start();
+      executorService.submit(new ForNameExerciser(classNames));
     }
 
-    // Wait for all of the threads to be ready
-    try {
-      cyclicBarrier.await();
-    } catch (InterruptedException iex) {
-      fail("Interrupted while waiting for barrier");
-    } catch (BrokenBarrierException bbex) {
-      fail("Broken barrier while waiting");
-    }
+    executorService.shutdown();
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(executorService::isTerminated);
 
-    // Loop while each thread tries N times to instantiate a non-existent class
-    for (int i = 0; i < numLoops; i++) {
-      try {
-        cyclicBarrier.await(5, TimeUnit.SECONDS);
-      } catch (InterruptedException iex) {
-        fail("Interrupted while waiting for barrier");
-      } catch (TimeoutException tex) {
-        ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
-        long[] threadIds = threadMXBean.findDeadlockedThreads();
-
-        if (threadIds != null) {
-          StringBuffer deadLockTrace = new StringBuffer();
-          for (long threadId : threadIds) {
-            ThreadInfo threadInfo = threadMXBean.getThreadInfo(threadId, 100);
-            deadLockTrace.append(threadInfo.getThreadName()).append("\n");
-            for (StackTraceElement stackTraceElem : threadInfo.getStackTrace()) {
-              deadLockTrace.append("\t").append(stackTraceElem).append("\n");
-            }
-          }
-
-          fail("Deadlock with trace:\n" + deadLockTrace.toString());
-        }
+    ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
+    long[] threadIds = threadMXBean.findDeadlockedThreads();
 
-        fail("Timeout while waiting for barrier - no deadlock detected");
-      } catch (BrokenBarrierException bbex) {
-        fail("Broken barrier while waiting");
+    if (threadIds != null) {
+      StringBuilder deadLockTrace = new StringBuilder();
+      for (long threadId : threadIds) {
+        ThreadInfo threadInfo = threadMXBean.getThreadInfo(threadId, 100);
+        deadLockTrace.append(threadInfo.getThreadName()).append("\n");
+        for (StackTraceElement stackTraceElem : threadInfo.getStackTrace()) {
+          deadLockTrace.append("\t").append(stackTraceElem).append("\n");
+        }
       }
+      System.out.println(deadLockTrace);
     }
+    assertThat(threadIds).isNull();
   }
 
-
   private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
     final OutputStream outStream = new FileOutputStream(jarFile);
     outStream.write(jarBytes);
@@ -497,40 +401,24 @@ public class DeployedJarJUnitTest {
 
   static final Random random = new Random();
 
-  private class ForNameExerciser extends Thread {
-    private final CyclicBarrier cyclicBarrier;
-    private final int numLoops;
+  private class ForNameExerciser implements Runnable {
+    private final int numLoops = 1000;
     private final String[] classNames;
 
-    ForNameExerciser(final CyclicBarrier cyclicBarrier, final int numLoops,
-        final String[] classNames) {
-      this.cyclicBarrier = cyclicBarrier;
-      this.numLoops = numLoops;
+    ForNameExerciser(final String[] classNames) {
       this.classNames = classNames;
     }
 
     @Override
     public void run() {
-      try {
-        this.cyclicBarrier.await();
-      } catch (InterruptedException iex) {
-        fail("Interrupted while waiting for latch");
-      } catch (BrokenBarrierException bbex) {
-        fail("Broken barrier while waiting");
-      }
       for (int i = 0; i < this.numLoops; i++) {
         try {
           // Random select a name from the list of class names and try to load it
           String className = this.classNames[random.nextInt(this.classNames.length)];
           ClassPathLoader.getLatest().forName(className);
         } catch (ClassNotFoundException expected) { // expected
-        }
-        try {
-          this.cyclicBarrier.await();
-        } catch (InterruptedException iex) {
-          fail("Interrupted while waiting for barrrier");
-        } catch (BrokenBarrierException bbex) {
-          fail("Broken barrier while waiting");
+        } catch (Exception e) {
+          throw new RuntimeException(e);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/internal/JarDeployerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/JarDeployerIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/JarDeployerIntegrationTest.java
index e9af0e7..b81e3e9 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/JarDeployerIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/JarDeployerIntegrationTest.java
@@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.awaitility.Awaitility;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -34,6 +35,7 @@ import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
@@ -94,8 +96,6 @@ public class JarDeployerIntegrationTest {
     alternateDir.delete();
 
     final JarDeployer jarDeployer = new JarDeployer(alternateDir);
-
-    final CyclicBarrier barrier = new CyclicBarrier(2);
     final byte[] jarBytes = this.classBuilder.createJarFromName("JarDeployerDUnitDTID");
 
     // Test to verify that deployment fails if the directory doesn't exist.
@@ -106,16 +106,20 @@ public class JarDeployerIntegrationTest {
     // Test to verify that deployment succeeds if the directory doesn't
     // initially exist, but is then created while the JarDeployer is looping
     // looking for a valid directory.
-    Future<Boolean> done = Executors.newSingleThreadExecutor().submit(() -> {
+    final AtomicBoolean isDeployed = new AtomicBoolean(false);
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+
+    Executors.newSingleThreadExecutor().submit(() -> {
       barrier.await();
       jarDeployer.deployWithoutRegistering("JarDeployerIntegrationTest.jar", jarBytes);
+      isDeployed.set(true);
       return true;
     });
 
     barrier.await();
-    Thread.sleep(500);
-    alternateDir.mkdir();
-    assertThat(done.get(2, TimeUnit.MINUTES)).isTrue();
+    alternateDir.mkdirs();
+    Awaitility.await().atMost(1, TimeUnit.MINUTES)
+        .until(() -> assertThat(isDeployed.get()).isTrue());
   }
 
   @Test
@@ -137,7 +141,7 @@ public class JarDeployerIntegrationTest {
   @Test
   public void testVersionNumberMatcher() throws IOException {
     int version =
-        jarDeployer.extractVersionFromFilename(temporaryFolder.newFile("MyJar.v1.jar").getName());
+        JarDeployer.extractVersionFromFilename(temporaryFolder.newFile("MyJar.v1.jar").getName());
 
     assertThat(version).isEqualTo(1);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
index dcbbeb0..0dbe75f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
@@ -1103,7 +1103,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
       }
     });
 
-    assert (deployedJarFile.exists());
+    assertTrue(deployedJarFile.exists());
     /*
      * Perform backup. Make sure it is successful.
      */
@@ -1138,7 +1138,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
     /*
      * Cleanup "dummy" jar from file system.
      */
-    Pattern pattern = Pattern.compile("^" + JarDeployer.JAR_PREFIX + jarName + ".*#\\d++$");
+    Pattern pattern = Pattern.compile("^" + jarName + ".*#\\d++$");
     deleteMatching(new File("."), pattern);
 
     // Execute the restore
@@ -1179,7 +1179,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
     /*
      * Cleanup "dummy" jar from file system.
      */
-    pattern = Pattern.compile("^" + JarDeployer.JAR_PREFIX + jarName + ".*#\\d++$");
+    pattern = Pattern.compile("^" + jarName + ".*#\\d++$");
     deleteMatching(new File(vmDir), pattern);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/management/DeployJarTestSuite.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/DeployJarTestSuite.java b/geode-core/src/test/java/org/apache/geode/management/DeployJarTestSuite.java
index ee46cbf..6dfab66 100644
--- a/geode-core/src/test/java/org/apache/geode/management/DeployJarTestSuite.java
+++ b/geode-core/src/test/java/org/apache/geode/management/DeployJarTestSuite.java
@@ -21,9 +21,12 @@ import org.apache.geode.internal.JarDeployerIntegrationTest;
 import org.apache.geode.management.internal.cli.commands.DeployCommandRedeployDUnitTest;
 import org.apache.geode.management.internal.cli.commands.DeployCommandsDUnitTest;
 import org.apache.geode.management.internal.configuration.ClusterConfigDeployJarDUnitTest;
+import org.junit.Ignore;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
+
+@Ignore
 @RunWith(Suite.class)
 @Suite.SuiteClasses({DeployedJarJUnitTest.class, DeployCommandsDUnitTest.class,
     JarDeployerIntegrationTest.class, ClassPathLoaderIntegrationTest.class,

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest.java
index 8280f5d..7780c0e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.management.internal.cli.commands;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedSystem;
@@ -23,10 +24,8 @@ import org.apache.geode.internal.ClassBuilder;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.Locator;
 import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.dunit.rules.Server;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.junit.Before;
 import org.junit.Rule;
@@ -34,8 +33,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.io.File;
-import java.io.IOException;
 import java.io.Serializable;
+import java.net.URL;
 import java.util.List;
 
 @Category(DistributedTest.class)
@@ -43,15 +42,15 @@ public class DeployCommandRedeployDUnitTest implements Serializable {
   private static final String VERSION1 = "Version1";
   private static final String VERSION2 = "Version2";
 
-  private static final String jarNameA = "DeployCommandRedeployDUnitTestA.jar";
-  private static final String functionA = "DeployCommandRedeployDUnitFunctionA";
+  private static final String JAR_NAME_A = "DeployCommandRedeployDUnitTestA.jar";
+  private static final String FUNCTION_A = "DeployCommandRedeployDUnitFunctionA";
   private File jarAVersion1;
   private File jarAVersion2;
 
-  private static final String jarNameB = "DeployCommandRedeployDUnitTestB.jar";
-  private static final String functionB = "DeployCommandRedeployDUnitFunctionB";
-  private static final String packageB = "jddunit.function";
-  private static final String fullyQualifiedFunctionB = packageB + "." + functionB;
+  private static final String JAR_NAME_B = "DeployCommandRedeployDUnitTestB.jar";
+  private static final String FUNCTION_B = "DeployCommandRedeployDUnitFunctionB";
+  private static final String PACKAGE_B = "jddunit.function";
+  private static final String FULLY_QUALIFIED_FUNCTION_B = PACKAGE_B + "." + FUNCTION_B;
   private File jarBVersion1;
   private File jarBVersion2;
 
@@ -59,7 +58,7 @@ public class DeployCommandRedeployDUnitTest implements Serializable {
   private MemberVM server;
 
   @Rule
-  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule(true);
 
   @Rule
   public transient GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
@@ -81,64 +80,60 @@ public class DeployCommandRedeployDUnitTest implements Serializable {
   @Test
   public void redeployJarsWithNewVersionsOfFunctions() throws Exception {
     gfshConnector.executeAndVerifyCommand("deploy --jar=" + jarAVersion1.getCanonicalPath());
-    server.invoke(() -> assertThatCanLoad(jarNameA, functionA));
-    server.invoke(() -> assertThatFunctionHasVersion(functionA, VERSION1));
-
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_A, FUNCTION_A));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_A, VERSION1));
 
     gfshConnector.executeAndVerifyCommand("deploy --jar=" + jarBVersion1.getCanonicalPath());
-    server.invoke(() -> assertThatCanLoad(jarNameA, functionA));
-    server.invoke(() -> assertThatCanLoad(jarNameB, fullyQualifiedFunctionB));
-    server.invoke(() -> assertThatFunctionHasVersion(functionA, VERSION1));
-    server.invoke(() -> assertThatFunctionHasVersion(functionB, VERSION1));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_A, FUNCTION_A));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_B, FULLY_QUALIFIED_FUNCTION_B));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_A, VERSION1));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_B, VERSION1));
 
     gfshConnector.executeAndVerifyCommand("deploy --jar=" + jarBVersion2.getCanonicalPath());
-    server.invoke(() -> assertThatCanLoad(jarNameA, functionA));
-    server.invoke(() -> assertThatCanLoad(jarNameB, fullyQualifiedFunctionB));
-    server.invoke(() -> assertThatFunctionHasVersion(functionA, VERSION1));
-    server.invoke(() -> assertThatFunctionHasVersion(functionB, VERSION2));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_A, FUNCTION_A));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_B, FULLY_QUALIFIED_FUNCTION_B));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_A, VERSION1));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_B, VERSION2));
 
     gfshConnector.executeAndVerifyCommand("deploy --jar=" + jarAVersion2.getCanonicalPath());
-    server.invoke(() -> assertThatCanLoad(jarNameA, functionA));
-    server.invoke(() -> assertThatCanLoad(jarNameB, fullyQualifiedFunctionB));
-    server.invoke(() -> assertThatFunctionHasVersion(functionA, VERSION2));
-    server.invoke(() -> assertThatFunctionHasVersion(functionB, VERSION2));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_A, FUNCTION_A));
+    server.invoke(() -> assertThatCanLoad(JAR_NAME_B, FULLY_QUALIFIED_FUNCTION_B));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_A, VERSION2));
+    server.invoke(() -> assertThatFunctionHasVersion(FUNCTION_B, VERSION2));
   }
 
   // Note that jar A is a Declarable Function, while jar B is only a Function.
   // Also, the function for jar A resides in the default package, whereas jar B specifies a package.
   // This ensures that this test has identical coverage to some tests that it replaced.
   private File createJarWithFunctionA(String version) throws Exception {
-    String classContents =
-        "import java.util.Properties;" + "import org.apache.geode.cache.Declarable;"
-            + "import org.apache.geode.cache.execute.Function;"
-            + "import org.apache.geode.cache.execute.FunctionContext;" + "public class " + functionA
-            + " implements Function, Declarable {" + "public String getId() {return \"" + functionA
-            + "\";}" + "public void init(Properties props) {}"
-            + "public void execute(FunctionContext context) {context.getResultSender().lastResult(\""
-            + version + "\");}" + "public boolean hasResult() {return true;}"
-            + "public boolean optimizeForWrite() {return false;}"
-            + "public boolean isHA() {return false;}}";
-
-    File jar = new File(lsRule.getTempFolder().newFolder(jarNameA + version), this.jarNameA);
+    URL classTemplateUrl = DeployCommandRedeployDUnitTest.class
+        .getResource("DeployCommandRedeployDUnitTest_FunctionATemplate");
+    assertThat(classTemplateUrl).isNotNull();
+
+    String classContents = FileUtils.readFileToString(new File(classTemplateUrl.toURI()), "UTF-8");
+    classContents = classContents.replaceAll("FUNCTION_A", FUNCTION_A);
+    classContents = classContents.replaceAll("VERSION", version);
+
+    File jar = new File(lsRule.getTempFolder().newFolder(JAR_NAME_A + version), this.JAR_NAME_A);
     ClassBuilder functionClassBuilder = new ClassBuilder();
-    functionClassBuilder.writeJarFromContent(functionA, classContents, jar);
+    functionClassBuilder.writeJarFromContent(FUNCTION_A, classContents, jar);
 
     return jar;
   }
 
-  private File createJarWithFunctionB(String version) throws IOException {
-    String classContents =
-        "package " + packageB + ";" + "import org.apache.geode.cache.execute.Function;"
-            + "import org.apache.geode.cache.execute.FunctionContext;" + "public class " + functionB
-            + " implements Function {" + "public boolean hasResult() {return true;}"
-            + "public void execute(FunctionContext context) {context.getResultSender().lastResult(\""
-            + version + "\");}" + "public String getId() {return \"" + functionB + "\";}"
-            + "public boolean optimizeForWrite() {return false;}"
-            + "public boolean isHA() {return false;}}";
-
-    File jar = new File(lsRule.getTempFolder().newFolder(jarNameB + version), this.jarNameB);
+  private File createJarWithFunctionB(String version) throws Exception {
+    URL classTemplateUrl = DeployCommandRedeployDUnitTest.class
+        .getResource("DeployCommandRedeployDUnitTest_FunctionATemplate");
+    assertThat(classTemplateUrl).isNotNull();
+
+    String classContents = FileUtils.readFileToString(new File(classTemplateUrl.toURI()), "UTF-8");
+    classContents = classContents.replaceAll("PACKAGE_B", PACKAGE_B);
+    classContents = classContents.replaceAll("FUNCTION_B", FUNCTION_B);
+    classContents = classContents.replaceAll("VERSION", version);
+
+    File jar = new File(lsRule.getTempFolder().newFolder(JAR_NAME_B + version), this.JAR_NAME_B);
     ClassBuilder functionClassBuilder = new ClassBuilder();
-    functionClassBuilder.writeJarFromContent("jddunit/function/" + functionB, classContents, jar);
+    functionClassBuilder.writeJarFromContent("jddunit/function/" + FUNCTION_B, classContents, jar);
 
     return jar;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
index 6df2572..9ed5bed 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.management.internal.cli.commands;
 
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
+import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.junit.Assert.assertNotNull;
@@ -39,7 +40,7 @@ import java.util.Properties;
 
 /**
  * Unit tests for the DeployCommands class
- *
+ * 
  * @since GemFire 7.0
  */
 @SuppressWarnings("serial")
@@ -76,6 +77,9 @@ public class DeployCommandsDUnitTest implements Serializable {
 
   @Before
   public void setup() throws Exception {
+    getHost(0).getVM(1).bounce();
+    getHost(0).getVM(2).bounce();
+
     ClassBuilder classBuilder = new ClassBuilder();
     File jarsDir = lsRule.getTempFolder().newFolder();
     jar1 = new File(jarsDir, jarName1);
@@ -108,7 +112,7 @@ public class DeployCommandsDUnitTest implements Serializable {
     // Deploy a jar to a single group
     CommandResult cmdResult =
         gfshConnector.executeAndVerifyCommand("deploy --jar=" + jar2 + " --group=" + GROUP1);
-    String resultString = commandResultToString(cmdResult);
+    String resultString = gfshConnector.getGfshOutput();
 
     assertThat(resultString).contains(server1.getName());
     assertThat(resultString).doesNotContain(server2.getName());
@@ -121,9 +125,9 @@ public class DeployCommandsDUnitTest implements Serializable {
   @Test
   public void deployMultipleJarsToOneGroup() throws Exception {
     // Deploy of multiple JARs to a single group
-    CommandResult cmdResult = gfshConnector.executeAndVerifyCommand(
+    gfshConnector.executeAndVerifyCommand(
         "deploy --group=" + GROUP1 + " --dir=" + subdirWithJars3and4.getCanonicalPath());
-    String resultString = commandResultToString(cmdResult);
+    String resultString = gfshConnector.getGfshOutput();
 
     assertThat(resultString).describedAs(resultString).contains(server1.getName());
     assertThat(resultString).doesNotContain(server2.getName());
@@ -139,7 +143,6 @@ public class DeployCommandsDUnitTest implements Serializable {
       assertThatCannotLoad(jarName4, class4);
     });
 
-
     // Undeploy of multiple jars by specifying group
     gfshConnector.executeAndVerifyCommand("undeploy --group=" + GROUP1);
     server1.invoke(() -> {
@@ -155,9 +158,9 @@ public class DeployCommandsDUnitTest implements Serializable {
   @Test
   public void deployJarToAllServers() throws Exception {
     // Deploy a jar to all servers
-    CommandResult cmdResult = gfshConnector.executeAndVerifyCommand("deploy --jar=" + jar1);
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + jar1);
+    String resultString = gfshConnector.getGfshOutput();
 
-    String resultString = commandResultToString(cmdResult);
     assertThat(resultString).contains(server1.getName());
     assertThat(resultString).contains(server2.getName());
     assertThat(resultString).contains(jarName1);
@@ -242,16 +245,16 @@ public class DeployCommandsDUnitTest implements Serializable {
         "deploy jar --group=" + GROUP2 + " --jar=" + jar2.getCanonicalPath());
 
     // List for all members
-    CommandResult commandResult = gfshConnector.executeAndVerifyCommand("list deployed");
-    String resultString = commandResultToString(commandResult);
+    gfshConnector.executeAndVerifyCommand("list deployed");
+    String resultString = gfshConnector.getGfshOutput();
     assertThat(resultString).contains(server1.getName());
     assertThat(resultString).contains(server2.getName());
     assertThat(resultString).contains(jarName1);
     assertThat(resultString).contains(jarName2);
 
     // List for members in Group1
-    commandResult = gfshConnector.executeAndVerifyCommand("list deployed --group=" + GROUP1);
-    resultString = commandResultToString(commandResult);
+    gfshConnector.executeAndVerifyCommand("list deployed --group=" + GROUP1);
+    resultString = gfshConnector.getGfshOutput();
     assertThat(resultString).contains(server1.getName());
     assertThat(resultString).doesNotContain(server2.getName());
 
@@ -259,23 +262,12 @@ public class DeployCommandsDUnitTest implements Serializable {
     assertThat(resultString).doesNotContain(jarName2);
 
     // List for members in Group2
-    commandResult = gfshConnector.executeAndVerifyCommand("list deployed --group=" + GROUP2);
-    resultString = commandResultToString(commandResult);
+    gfshConnector.executeAndVerifyCommand("list deployed --group=" + GROUP2);
+    resultString = gfshConnector.getGfshOutput();
     assertThat(resultString).doesNotContain(server1.getName());
     assertThat(resultString).contains(server2.getName());
 
     assertThat(resultString).doesNotContain(jarName1);
     assertThat(resultString).contains(jarName2);
   }
-
-  protected static String commandResultToString(final CommandResult commandResult) {
-    assertNotNull(commandResult);
-    commandResult.resetToFirstLine();
-    StringBuilder buffer = new StringBuilder(commandResult.getHeader());
-    while (commandResult.hasNextLine()) {
-      buffer.append(commandResult.nextLine());
-    }
-    buffer.append(commandResult.getFooter());
-    return buffer.toString();
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/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
deleted file mode 100644
index cecc8cf..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigBaseTest.java
+++ /dev/null
@@ -1,127 +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 org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
-import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.geode.internal.ClassBuilder;
-import org.apache.geode.management.internal.configuration.utils.ZipUtils;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.junit.Before;
-import org.junit.Rule;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Properties;
-
-public class ClusterConfigBaseTest {
-  public String clusterConfigZipPath;
-
-  public static final ConfigGroup CLUSTER = new ConfigGroup("cluster").regions("regionForCluster")
-      .jars("cluster.jar").maxLogFileSize("5000").configFiles("cluster.properties", "cluster.xml");
-  public static final ConfigGroup GROUP1 = new ConfigGroup("group1").regions("regionForGroup1")
-      .jars("group1.jar").maxLogFileSize("6000").configFiles("group1.properties", "group1.xml");
-  public static final ConfigGroup GROUP2 = new ConfigGroup("group2").regions("regionForGroup2")
-      .jars("group2.jar").maxLogFileSize("7000").configFiles("group2.properties", "group2.xml");
-
-  public static final ClusterConfig CONFIG_FROM_ZIP = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
-
-  public static final ClusterConfig REPLICATED_CONFIG_FROM_ZIP = new ClusterConfig(
-      new ConfigGroup("cluster").maxLogFileSize("5000").jars("cluster.jar")
-          .regions("regionForCluster"),
-      new ConfigGroup("group1").maxLogFileSize("6000").jars("group1.jar")
-          .regions("regionForGroup1"),
-      new ConfigGroup("group2").maxLogFileSize("7000").jars("group2.jar")
-          .regions("regionForGroup2"));
-
-  @Rule
-  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
-
-  protected Properties locatorProps;
-  protected Properties serverProps;
-
-  @Before
-  public void before() throws Exception {
-    clusterConfigZipPath = buildClusterZipFile();
-    locatorProps = new Properties();
-    serverProps = new Properties();
-
-    // the following are default values, we don't need to set them. We do it for clarity purpose
-    locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
-    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
-  }
-
-  private String buildClusterZipFile() throws Exception {
-    ClassBuilder classBuilder = new ClassBuilder();
-    File clusterConfigDir = this.lsRule.getTempFolder().newFolder("cluster_config");
-
-    File clusterDir = new File(clusterConfigDir, "cluster");
-    String clusterXml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
-        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
-        + "<region name=\"regionForCluster\">\n"
-        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
-        + "  </region>\n" + "</cache>\n";
-    writeFile(clusterDir, "cluster.xml", clusterXml);
-    writeFile(clusterDir, "cluster.properties", "log-file-size-limit=5000");
-    createJarFileWithClass("Cluster", "cluster.jar", clusterDir);
-
-    File group1Dir = new File(clusterConfigDir, "group1");
-    String group1Xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
-        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
-        + "<region name=\"regionForGroup1\">\n"
-        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
-        + "  </region>\n" + "</cache>\n";
-    writeFile(group1Dir, "group1.xml", group1Xml);
-    writeFile(group1Dir, "group1.properties", "log-file-size-limit=6000");
-    createJarFileWithClass("Group1", "group1.jar", group1Dir);
-
-
-    File group2Dir = new File(clusterConfigDir, "group2");
-    String group2Xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
-        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
-        + "<region name=\"regionForGroup2\">\n"
-        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
-        + "  </region>\n" + "</cache>\n";
-    writeFile(group2Dir, "group2.xml", group2Xml);
-    writeFile(group2Dir, "group2.properties", "log-file-size-limit=7000");
-    createJarFileWithClass("Group2", "group2.jar", group2Dir);
-
-
-    File clusterConfigZip = new File(lsRule.getTempFolder().newFolder(), "cluster_config.zip");
-    ZipUtils.zipDirectory(clusterConfigDir.getCanonicalPath(), clusterConfigZip.getCanonicalPath());
-
-    FileUtils.deleteDirectory(clusterConfigDir);
-    return clusterConfigZip.getCanonicalPath();
-  }
-
-  private File writeFile(File dir, String fileName, String content) throws IOException {
-    dir.mkdirs();
-    File file = new File(dir, fileName);
-    FileUtils.writeStringToFile(file, content);
-
-    return file;
-  }
-
-  protected String createJarFileWithClass(String className, String jarName, File dir)
-      throws IOException {
-    File jarFile = new File(dir, jarName);
-    new ClassBuilder().writeJarFromName(className, jarFile);
-    return jarFile.getCanonicalPath();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/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
index 980f81f..3781c98 100644
--- 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
@@ -18,6 +18,7 @@ package org.apache.geode.management.internal.configuration;
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
@@ -29,7 +30,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category(DistributedTest.class)
-public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
+public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
   @Rule
   public GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
 
@@ -37,7 +38,6 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
 
   @Before
   public void before() throws Exception {
-    super.before();
     clusterJar = createJarFileWithClass("Cluster", "cluster.jar", lsRule.getTempFolder().getRoot());
     group1Jar = createJarFileWithClass("Group1", "group1.jar", lsRule.getTempFolder().getRoot());
     group2Jar = createJarFileWithClass("Group2", "group2.jar", lsRule.getTempFolder().getRoot());
@@ -180,7 +180,6 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
     // test undeploy cluster
     gfshConnector.executeAndVerifyCommand("undeploy --jar=cluster.jar");
 
-
     cluster = cluster.removeJar("cluster.jar");
     server3Config.verify(locator);
     server1Config.verify(server1);
@@ -253,7 +252,6 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
     gfshConnector.executeAndVerifyCommand("undeploy --jar=cluster.jar");
     server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
 
-
     cluster = cluster.removeJar("cluster.jar");
     server3Config.verify(locator);
     server1Config.verify(server1);
@@ -264,8 +262,8 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
 
     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
+     * TODO: GEODE-2779 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.
      */

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/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
index 696d22c..521e084 100644
--- 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
@@ -42,7 +42,7 @@ import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
 @Category(DistributedTest.class)
-public class ClusterConfigImportDUnitTest extends ClusterConfigBaseTest {
+public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   @Rule
   public GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
 
@@ -52,7 +52,6 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigBaseTest {
 
   @Before
   public void before() throws Exception {
-    super.before();
     locatorVM = lsRule.startLocatorVM(0, locatorProps);
     INITIAL_CONFIG.verify(locatorVM);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/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
index 652ec60..1cdda4c 100644
--- 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
@@ -33,12 +33,11 @@ import java.io.File;
 import java.util.Properties;
 
 @Category(DistributedTest.class)
-public class ClusterConfigStartMemberDUnitTest extends ClusterConfigBaseTest {
+public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
   protected MemberVM locator;
 
   @Before
   public void before() throws Exception {
-    super.before();
     locator = startLocatorWithLoadCCFromDir();
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
new file mode 100644
index 0000000..c5aaa74
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
@@ -0,0 +1,128 @@
+/*
+ * 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 static org.apache.geode.test.dunit.Host.getHost;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.geode.internal.ClassBuilder;
+import org.apache.geode.management.internal.configuration.utils.ZipUtils;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.junit.Before;
+import org.junit.Rule;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+public abstract class ClusterConfigTestBase {
+  public String clusterConfigZipPath;
+
+  public static final ConfigGroup CLUSTER = new ConfigGroup("cluster").regions("regionForCluster")
+      .jars("cluster.jar").maxLogFileSize("5000").configFiles("cluster.properties", "cluster.xml");
+  public static final ConfigGroup GROUP1 = new ConfigGroup("group1").regions("regionForGroup1")
+      .jars("group1.jar").maxLogFileSize("6000").configFiles("group1.properties", "group1.xml");
+  public static final ConfigGroup GROUP2 = new ConfigGroup("group2").regions("regionForGroup2")
+      .jars("group2.jar").maxLogFileSize("7000").configFiles("group2.properties", "group2.xml");
+
+  public static final ClusterConfig CONFIG_FROM_ZIP = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
+
+  public static final ClusterConfig REPLICATED_CONFIG_FROM_ZIP = new ClusterConfig(
+      new ConfigGroup("cluster").maxLogFileSize("5000").jars("cluster.jar")
+          .regions("regionForCluster"),
+      new ConfigGroup("group1").maxLogFileSize("6000").jars("group1.jar")
+          .regions("regionForGroup1"),
+      new ConfigGroup("group2").maxLogFileSize("7000").jars("group2.jar")
+          .regions("regionForGroup2"));
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule(true);
+
+  protected Properties locatorProps;
+  protected Properties serverProps;
+
+  @Before
+  public void beforeClusterConfigTestBase() throws Exception {
+    clusterConfigZipPath = buildClusterZipFile();
+    locatorProps = new Properties();
+    serverProps = new Properties();
+
+    // the following are default values, we don't need to set them. We do it for clarity purpose
+    locatorProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
+    serverProps.setProperty(USE_CLUSTER_CONFIGURATION, "true");
+  }
+
+  private String buildClusterZipFile() throws Exception {
+    File clusterConfigDir = this.lsRule.getTempFolder().newFolder("cluster_config");
+
+    File clusterDir = new File(clusterConfigDir, "cluster");
+    String clusterXml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
+        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
+        + "<region name=\"regionForCluster\">\n"
+        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
+        + "  </region>\n" + "</cache>\n";
+    writeFile(clusterDir, "cluster.xml", clusterXml);
+    writeFile(clusterDir, "cluster.properties", "log-file-size-limit=5000");
+    createJarFileWithClass("Cluster", "cluster.jar", clusterDir);
+
+    File group1Dir = new File(clusterConfigDir, "group1");
+    String group1Xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
+        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
+        + "<region name=\"regionForGroup1\">\n"
+        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
+        + "  </region>\n" + "</cache>\n";
+    writeFile(group1Dir, "group1.xml", group1Xml);
+    writeFile(group1Dir, "group1.properties", "log-file-size-limit=6000");
+    createJarFileWithClass("Group1", "group1.jar", group1Dir);
+
+
+    File group2Dir = new File(clusterConfigDir, "group2");
+    String group2Xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
+        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" xsi:schemaLocation=\"http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd\">\n"
+        + "<region name=\"regionForGroup2\">\n"
+        + "    <region-attributes data-policy=\"replicate\" scope=\"distributed-ack\"/>\n"
+        + "  </region>\n" + "</cache>\n";
+    writeFile(group2Dir, "group2.xml", group2Xml);
+    writeFile(group2Dir, "group2.properties", "log-file-size-limit=7000");
+    createJarFileWithClass("Group2", "group2.jar", group2Dir);
+
+
+    File clusterConfigZip = new File(lsRule.getTempFolder().newFolder(), "cluster_config.zip");
+    ZipUtils.zipDirectory(clusterConfigDir.getCanonicalPath(), clusterConfigZip.getCanonicalPath());
+
+    FileUtils.deleteDirectory(clusterConfigDir);
+    return clusterConfigZip.getCanonicalPath();
+  }
+
+  private File writeFile(File dir, String fileName, String content) throws IOException {
+    dir.mkdirs();
+    File file = new File(dir, fileName);
+    FileUtils.writeStringToFile(file, content);
+
+    return file;
+  }
+
+  protected String createJarFileWithClass(String className, String jarName, File dir)
+      throws IOException {
+    File jarFile = new File(dir, jarName);
+    new ClassBuilder().writeJarFromName(className, jarFile);
+    return jarFile.getCanonicalPath();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/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 97c636b..34506c4 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
@@ -57,9 +57,18 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
 
   private TemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
   private MemberVM[] members;
+  private final boolean bounceVms;
 
   public LocatorServerStartupRule() {
+    this(false);
+  }
+
+  /**
+   * If your DUnit tests fail due to insufficient cleanup, try setting bounceVms=true.
+   */
+  public LocatorServerStartupRule(boolean bounceVms) {
     DUnitLauncher.launchIfNeeded();
+    this.bounceVms = bounceVms;
   }
 
   @Override
@@ -67,6 +76,9 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     restoreSystemProperties.before();
     temporaryFolder.create();
     Invoke.invokeInEveryVM("Stop each VM", this::cleanupVm);
+    if (bounceVms) {
+      getHost(0).getAllVMs().forEach(VM::bounce);
+    }
     members = new MemberVM[4];
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionATemplate
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionATemplate b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionATemplate
new file mode 100644
index 0000000..766963a
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionATemplate
@@ -0,0 +1,30 @@
+import java.util.Properties;
+
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class FUNCTION_A implements Function, Declarable {
+  public String getId() {
+    return "FUNCTION_A";
+  }
+
+  public void init(Properties props) {
+  }
+
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("VERSION");
+  }
+
+  public boolean hasResult() {
+    return true;
+  }
+
+  public boolean optimizeForWrite() {
+    return false;
+  }
+
+  public boolean isHA() {
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionBTemplate
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionBTemplate b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionBTemplate
new file mode 100644
index 0000000..b37a6cf
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/DeployCommandRedeployDUnitTest_FunctionBTemplate
@@ -0,0 +1,26 @@
+package PACKAGE_B;
+
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class FUNCTION_B implements Function {
+  public boolean hasResult() {
+    return true;
+  }
+
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("VERSION");
+  }
+
+  public String getId() {
+    return "FUNCTION_B";
+  }
+
+  public boolean optimizeForWrite() {
+    return false;
+  }
+
+  public boolean isHA() {
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/6f7f9439/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index da8bdf2..a9e3fdf 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -33,6 +33,7 @@ commons-digester.version=2.1
 commons-exec.version=1.3
 derby.version = 10.13.1.1
 dom4j.version = 1.6.1
+fast-classpath-scanner.version=2.0.11
 fastutil.version = 7.1.0
 google-gson.version=2.8.0
 guava.version = 21.0