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

[6/8] geode git commit: GEODE-2705: Jars undeployed from cluster configuration will not be loaded from disk on member restart

GEODE-2705: Jars undeployed from cluster configuration will not be loaded from disk on member restart


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

Branch: refs/heads/develop
Commit: ee11b0a43b66a42eb10a16b92993436b2298c042
Parents: 6fd2d12
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Apr 6 10:22:26 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Sun Apr 16 09:10:01 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/internal/DeployedJar.java  |   6 +
 .../org/apache/geode/internal/JarDeployer.java  | 148 +++++++++++--------
 .../cache/ClusterConfigurationLoader.java       |  30 +++-
 .../geode/internal/cache/GemFireCacheImpl.java  |   7 +-
 .../ClassPathLoaderIntegrationTest.java         |  10 +-
 .../geode/internal/ClassPathLoaderTest.java     |  58 --------
 .../internal/JarDeployerIntegrationTest.java    |  63 +++++++-
 .../geode/management/DeployJarTestSuite.java    |   4 +-
 .../internal/configuration/ClusterConfig.java   |  23 ++-
 .../ClusterConfigDeployJarDUnitTest.java        |  76 ++++++++++
 10 files changed, 278 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 f4f4028..8adec1f 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
@@ -85,6 +85,9 @@ public class DeployedJar {
     this(versionedJarFile, jarName, Files.readAllBytes(versionedJarFile.toPath()));
   }
 
+  /**
+   * Writes the given jarBytes to versionedJarFile
+   */
   public DeployedJar(File versionedJarFile, final String jarName, byte[] jarBytes)
       throws IOException {
     Assert.assertTrue(jarBytes != null, "jarBytes cannot be null");
@@ -377,6 +380,9 @@ public class DeployedJar {
     return new byte[0];
   }
 
+  /**
+   * @return the unversioned name of this jar file, e.g. myJar.jar
+   */
   public String getJarName() {
     return this.jarName;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 ad5c435..669802c 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
@@ -18,10 +18,6 @@ import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.geode.GemFireException;
-import org.apache.geode.GemFireIOException;
-import org.apache.geode.SystemFailure;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.logging.log4j.Logger;
 
@@ -34,13 +30,9 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -83,11 +75,23 @@ public class JarDeployer implements Serializable {
     return this.deployDirectory;
   }
 
+  /**
+   * Writes the jarBytes for the given jarName to the next version of that jar file (if the bytes do
+   * not match the latest deployed version)
+   * 
+   * @return the DeployedJar that was written from jarBytes, or null if those bytes matched the
+   *         latest deployed version
+   */
   public DeployedJar deployWithoutRegistering(final String jarName, final byte[] jarBytes)
       throws IOException {
     lock.lock();
 
     try {
+      boolean shouldDeployNewVersion = shouldDeployNewVersion(jarName, jarBytes);
+      if (!shouldDeployNewVersion) {
+        return null;
+      }
+
       verifyWritableDeployDirectory();
 
       File newVersionedJarFile = getNextVersionedJarFile(jarName);
@@ -129,6 +133,7 @@ public class JarDeployer implements Serializable {
     lock.unlock();
   }
 
+
   protected File getNextVersionedJarFile(String unversionedJarName) {
     File[] oldVersions = findSortedOldVersionsOfJar(unversionedJarName);
 
@@ -246,21 +251,17 @@ public class JarDeployer implements Serializable {
     }
   }
 
-  protected Set<String> findDistinctDeployedJars() {
+  protected Set<String> findDistinctDeployedJarsOnDisk() {
     // Find all deployed JAR files
-    final File[] oldFiles = this.deployDirectory.listFiles(new FilenameFilter() {
-      @Override
-      public boolean accept(final File file, final String name) {
-        return versionedPattern.matcher(name).matches();
-      }
-    });
+    final File[] oldFiles =
+        this.deployDirectory.listFiles((file, name) -> versionedPattern.matcher(name).matches());
 
     // Now add just the original JAR name to the set
-    final Set<String> jarNames = new HashSet<String>();
+    final Set<String> jarNames = new HashSet<>();
     for (File oldFile : oldFiles) {
       Matcher matcher = versionedPattern.matcher(oldFile.getName());
       matcher.find();
-      jarNames.add(matcher.group(1));
+      jarNames.add(matcher.group(1) + ".jar");
     }
     return jarNames;
   }
@@ -368,60 +369,39 @@ public class JarDeployer implements Serializable {
     String newJarName = unversionedJarNameWithoutExtension + ".v" + jarVersion + ".jar";
 
     File newJar = new File(this.deployDirectory, newJarName);
-    logger.debug("Renaming deployed jar from " + oldJar.getCanonicalPath() + " to "
-        + newJar.getCanonicalPath());
+    logger.debug("Renaming deployed jar from {} to {}", oldJar.getCanonicalPath(),
+        newJar.getCanonicalPath());
 
     FileUtils.moveFile(oldJar, newJar);
-    FileUtils.deleteQuietly(oldJar);
   }
 
   /**
-   * Re-deploy all previously deployed JAR files.
+   * Re-deploy all previously deployed JAR files on disk.
    */
-  public void loadPreviouslyDeployedJars() {
+  public void loadPreviouslyDeployedJarsFromDisk() {
+    logger.info("Loading previously deployed jars");
     lock.lock();
     try {
       verifyWritableDeployDirectory();
       renameJarsWithOldNamingConvention();
 
-      final Set<String> jarNames = findDistinctDeployedJars();
+      final Set<String> jarNames = findDistinctDeployedJarsOnDisk();
       if (jarNames.isEmpty()) {
         return;
       }
 
-      Map<String, DeployedJar> latestVersionOfEachJar = new LinkedHashMap<>();
+      List<DeployedJar> latestVersionOfEachJar = new ArrayList<>();
 
       for (String jarName : jarNames) {
-        final File[] jarFiles = findSortedOldVersionsOfJar(jarName);
-
-        Optional<File> latestValidDeployedJarOptional =
-            Arrays.stream(jarFiles).filter(Objects::nonNull).filter(jarFile -> {
-              try {
-                return DeployedJar.isValidJarContent(FileUtils.readFileToByteArray(jarFile));
-              } catch (IOException e) {
-                return false;
-              }
-            }).findFirst();
-
-        if (!latestValidDeployedJarOptional.isPresent()) {
-          // No valid version of this jar
-          continue;
-        }
+        DeployedJar deployedJar = findLatestValidDeployedJarFromDisk(jarName);
 
-        File latestValidDeployedJar = latestValidDeployedJarOptional.get();
-        latestVersionOfEachJar.put(jarName, new DeployedJar(latestValidDeployedJar, jarName));
-
-        // Remove any old left-behind versions of this JAR file
-        for (File jarFile : jarFiles) {
-          if (!latestValidDeployedJar.equals(jarFile)) {
-            FileUtils.deleteQuietly(jarFile);
-          }
+        if (deployedJar != null) {
+          latestVersionOfEachJar.add(deployedJar);
+          deleteOtherVersionsOfJar(deployedJar);
         }
       }
 
-      registerNewVersions(latestVersionOfEachJar.values().stream().collect(toList()));
-      // ClassPathLoader.getLatest().deploy(latestVersionOfEachJar.keySet().toArray(),
-      // latestVersionOfEachJar.values().toArray())
+      registerNewVersions(latestVersionOfEachJar);
     } catch (Exception e) {
       throw new RuntimeException(e);
     } finally {
@@ -429,6 +409,43 @@ public class JarDeployer implements Serializable {
     }
   }
 
+  /**
+   * Deletes all versions of this jar on disk other than the given version
+   */
+  public void deleteOtherVersionsOfJar(DeployedJar deployedJar) {
+    logger.info("Deleting all versions of " + deployedJar.getJarName() + " other than "
+        + deployedJar.getFileName());
+    final File[] jarFiles = findSortedOldVersionsOfJar(deployedJar.getJarName());
+
+    Stream.of(jarFiles).filter(jarFile -> !jarFile.equals(deployedJar.getFile()))
+        .forEach(jarFile -> {
+          logger.info("Deleting old version of jar: " + jarFile.getAbsolutePath());
+          FileUtils.deleteQuietly(jarFile);
+        });
+  }
+
+  public DeployedJar findLatestValidDeployedJarFromDisk(String unversionedJarName)
+      throws IOException {
+    final File[] jarFiles = findSortedOldVersionsOfJar(unversionedJarName);
+
+    Optional<File> latestValidDeployedJarOptional =
+        Arrays.stream(jarFiles).filter(Objects::nonNull).filter(jarFile -> {
+          try {
+            return DeployedJar.isValidJarContent(FileUtils.readFileToByteArray(jarFile));
+          } catch (IOException e) {
+            return false;
+          }
+        }).findFirst();
+
+    if (!latestValidDeployedJarOptional.isPresent()) {
+      // No valid version of this jar
+      return null;
+    }
+
+    File latestValidDeployedJar = latestValidDeployedJarOptional.get();
+
+    return new DeployedJar(latestValidDeployedJar, unversionedJarName);
+  }
 
   public URL[] getDeployedJarURLs() {
     return this.deployedJars.values().stream().map(DeployedJar::getFileURL).toArray(URL[]::new);
@@ -441,6 +458,7 @@ public class JarDeployer implements Serializable {
     try {
       for (DeployedJar deployedJar : deployedJars) {
         if (deployedJar != null) {
+          logger.info("Registering new version of jar: {}", deployedJar.toString());
           DeployedJar oldJar = this.deployedJars.put(deployedJar.getJarName(), deployedJar);
           if (oldJar != null) {
             oldJar.cleanUp();
@@ -488,13 +506,7 @@ public class JarDeployer implements Serializable {
         String jarName = jarNames[i];
         byte[] newJarBytes = jarBytes[i];
 
-        boolean shouldDeployNewVersion = shouldDeployNewVersion(jarName, newJarBytes);
-
-        if (shouldDeployNewVersion) {
-          deployedJars[i] = deployWithoutRegistering(jarName, newJarBytes);
-        } else {
-          deployedJars[i] = null;
-        }
+        deployedJars[i] = deployWithoutRegistering(jarName, newJarBytes);
       }
 
       return registerNewVersions(Arrays.asList(deployedJars));
@@ -511,7 +523,7 @@ public class JarDeployer implements Serializable {
     }
 
     if (oldDeployedJar.hasSameContentAs(newJarBytes)) {
-      logger.warn("Jar is identical to the latest deployed version: ",
+      logger.warn("Jar is identical to the latest deployed version: {}",
           oldDeployedJar.getFileCanonicalPath());
 
       return false;
@@ -520,6 +532,11 @@ public class JarDeployer implements Serializable {
     return true;
   }
 
+  /**
+   * Returns the latest registered {@link DeployedJar} for the given JarName
+   * 
+   * @param jarName - the unversioned jar name, e.g. myJar.jar
+   */
   public DeployedJar findDeployedJar(String jarName) {
     return this.deployedJars.get(jarName);
   }
@@ -565,9 +582,24 @@ public class JarDeployer implements Serializable {
 
       deployedJar.cleanUp();
 
+      deleteAllVersionsOfJar(jarName);
       return deployedJar.getFileCanonicalPath();
     } finally {
       lock.unlock();
     }
   }
+
+  public void deleteAllVersionsOfJar(String unversionedJarName) {
+    lock.lock();
+    try {
+      File[] jarFiles = findSortedOldVersionsOfJar(unversionedJarName);
+      for (File jarFile : jarFiles) {
+        logger.info("Deleting: {}", jarFile.getAbsolutePath());
+        FileUtils.deleteQuietly(jarFile);
+      }
+    } finally {
+      lock.unlock();
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
index 2b627b2..55e3542 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
@@ -14,6 +14,9 @@
  */
 package org.apache.geode.internal.cache;
 
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toList;
+
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -27,7 +30,9 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
+import java.util.stream.Stream;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.logging.log4j.Logger;
 
@@ -61,21 +66,36 @@ public class ClusterConfigurationLoader {
    */
   public static void deployJarsReceivedFromClusterConfiguration(Cache cache,
       ConfigurationResponse response) throws IOException, ClassNotFoundException {
+    logger.info("Requesting cluster configuration");
     if (response == null) {
       return;
     }
 
     String[] jarFileNames = response.getJarNames();
     byte[][] jarBytes = response.getJars();
+    logger.info("Got response with jars: {}", Stream.of(jarFileNames).collect(joining(",")));
 
     if (jarFileNames != null && jarBytes != null) {
-      List<DeployedJar> deployedJars =
-          ClassPathLoader.getLatest().getJarDeployer().deploy(jarFileNames, jarBytes);
+      JarDeployer jarDeployer = ClassPathLoader.getLatest().getJarDeployer();
+      jarDeployer.suspendAll();
+      try {
+        List<String> extraJarsOnServer =
+            jarDeployer.findDeployedJars().stream().map(DeployedJar::getJarName)
+                .filter(jarName -> !ArrayUtils.contains(jarFileNames, jarName)).collect(toList());
+
+        for (String extraJar : extraJarsOnServer) {
+          logger.info("Removing jar not present in cluster configuration: {}", extraJar);
+          jarDeployer.deleteAllVersionsOfJar(extraJar);
+        }
 
-      deployedJars.stream().filter(Objects::nonNull)
-          .forEach((jar) -> logger.info("Deployed " + (jar.getFile().getAbsolutePath())));
+        List<DeployedJar> deployedJars = jarDeployer.deploy(jarFileNames, jarBytes);
+
+        deployedJars.stream().filter(Objects::nonNull)
+            .forEach((jar) -> logger.info("Deployed: {}", jar.getFile().getAbsolutePath()));
+      } finally {
+        jarDeployer.resumeAll();
+      }
     }
-    // TODO: Jared - Does this need to actually undeploy extra jars like the javadoc says?
   }
 
   /***

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index fb311e7..069b608 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -108,7 +108,6 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.JarDeployer;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
@@ -1238,8 +1237,10 @@ public class GemFireCacheImpl
     initializeServices();
 
     try {
-      // Deploy all the jars from the deploy working dir.
-      ClassPathLoader.getLatest().getJarDeployer().loadPreviouslyDeployedJars();
+      if (configurationResponse == null) {
+        // Deploy all the jars from the deploy working dir.
+        ClassPathLoader.getLatest().getJarDeployer().loadPreviouslyDeployedJarsFromDisk();
+      }
       ClusterConfigurationLoader.applyClusterXmlConfiguration(this, configurationResponse,
           system.getConfig());
       initializeDeclarativeCache();

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 d30feb6..14108c7 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
@@ -91,7 +91,7 @@ public class ClassPathLoaderIntegrationTest {
     fos.write(new byte[TEMP_FILE_BYTES_COUNT]);
     fos.close();
 
-    System.setProperty("user.dir", temporaryFolder.getRoot().getAbsolutePath());
+    // System.setProperty("user.dir", temporaryFolder.getRoot().getAbsolutePath());
     ClassPathLoader.setLatestToDefault(temporaryFolder.getRoot());
   }
 
@@ -226,9 +226,7 @@ public class ClassPathLoaderIntegrationTest {
     outStream.write(jarBytes);
     outStream.close();
 
-    Properties properties = new Properties();
-    properties.setProperty("user.dir", temporaryFolder.getRoot().getAbsolutePath());
-    ServerStarterRule serverStarterRule = new ServerStarterRule();
+    ServerStarterRule serverStarterRule = new ServerStarterRule(temporaryFolder.getRoot());
     serverStarterRule.startServer();
 
     GemFireCacheImpl gemFireCache = GemFireCacheImpl.getInstance();
@@ -248,9 +246,7 @@ public class ClassPathLoaderIntegrationTest {
     File jarVersion1 = createVersionOfJar("Version1", "MyFunction", "MyJar.jar");
     File jarVersion2 = createVersionOfJar("Version2", "MyFunction", "MyJar.jar");
 
-    Properties properties = new Properties();
-    properties.setProperty("user.dir", temporaryFolder.getRoot().getAbsolutePath());
-    ServerStarterRule serverStarterRule = new ServerStarterRule();
+    ServerStarterRule serverStarterRule = new ServerStarterRule(temporaryFolder.getRoot());
     serverStarterRule.startServer();
 
     GemFireCacheImpl gemFireCache = GemFireCacheImpl.getInstance();

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderTest.java b/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderTest.java
index 0d26caf..0e37075 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/ClassPathLoaderTest.java
@@ -494,39 +494,6 @@ public class ClassPathLoaderTest {
     }
   }
 
-  private static void exploreClassLoader(ClassLoader cl, int indent) {
-    String prefix = "";
-    for (int i = 0; i < indent; i++) {
-      prefix += "\t";
-    }
-    System.out.println(prefix + "ClassLoader toString() = " + cl);
-
-    Class<?> clazz = cl.getClass();
-    System.out.println(prefix + "ClassLoader getClass().getName() = " + clazz.getName());
-    exploreClassLoaderSuperClass(prefix, clazz);
-
-    try {
-      URL[] urls = ((URLClassLoader) cl).getURLs();
-      StringBuilder sb = new StringBuilder(prefix).append("ClassLoader getURLs = [");
-      for (int i = 0; i < urls.length; i++) {
-        if (i > 0) {
-          sb.append(", ");
-        }
-        sb.append(urls[i].toString());
-      }
-      sb.append("]");
-      System.out.println(sb.toString());
-    } catch (Exception e) {
-      System.out.println(prefix + "ClassLoader is not a URLClassLoader");
-    }
-
-    ClassLoader parent = cl.getParent();
-    if (parent != null) {
-      System.out.println(prefix + "ClassLoader has parent...");
-      exploreClassLoader(parent, ++indent);
-    }
-  }
-
   private static void exploreClassLoaderSuperClass(String prefix, Class<?> clazz) {
     Class<?> superClazz = clazz.getSuperclass();
     if (superClazz != null) {
@@ -646,31 +613,6 @@ public class ClassPathLoaderTest {
     }
   }
 
-  static class OneClassClassLoader extends ClassLoader {
-
-    private final GeneratingClassLoader genClassLoader = new GeneratingClassLoader();
-    private String className;
-
-    public OneClassClassLoader(final String className) {
-      super(null); // no parent!!
-      this.className = className;
-    }
-
-    @Override
-    public Class<?> findClass(String name) throws ClassNotFoundException {
-      if (!name.equals(className)) {
-        throw new ClassNotFoundException();
-      } else {
-        return this.genClassLoader.findClass(name);
-      }
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-      return (other instanceof OneClassClassLoader);
-    }
-  }
-
   @SuppressWarnings("serial")
   static class BrokenError extends Error {
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 9e42c20..e9af0e7 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
@@ -17,7 +17,6 @@
 package org.apache.geode.internal;
 
 
-import static org.apache.geode.internal.Assert.fail;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
@@ -31,7 +30,6 @@ import org.junit.rules.TemporaryFolder;
 import java.io.File;
 import java.io.IOException;
 import java.util.Set;
-import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -87,7 +85,7 @@ public class JarDeployerIntegrationTest {
     assertThat(sortedOldJars).hasSize(2);
     assertThat(sortedOldJars[0].getName()).contains(".v2.");
     assertThat(sortedOldJars[1].getName()).contains(".v1.");
-    assertThat(jarDeployer.findDistinctDeployedJars()).hasSize(1);
+    assertThat(jarDeployer.findDistinctDeployedJarsOnDisk()).hasSize(1);
   }
 
   @Test
@@ -208,4 +206,63 @@ public class JarDeployerIntegrationTest {
   }
 
 
+  @Test
+  public void testDeleteAllVersionsOfJar() throws Exception {
+    File deployDir = jarDeployer.getDeployDirectory();
+
+    File jarAVersion1 = new File(deployDir, "myJarA.v1.jar");
+    this.classBuilder.writeJarFromName("ClassA", jarAVersion1);
+
+    File jarAVersion2 = new File(deployDir, "myJarA.v2.jar");
+    this.classBuilder.writeJarFromName("ClassA", jarAVersion2);
+
+    File jarBVersion2 = new File(deployDir, "myJarB.v2.jar");
+    this.classBuilder.writeJarFromName("ClassB", jarBVersion2);
+
+    File jarBVersion3 = new File(deployDir, "myJarB.v3.jar");
+    this.classBuilder.writeJarFromName("ClassB", jarBVersion3);
+
+    jarDeployer.deleteAllVersionsOfJar("myJarA.jar");
+
+    assertThat(jarAVersion1).doesNotExist();
+    assertThat(jarAVersion2).doesNotExist();
+    assertThat(jarBVersion2).exists();
+    assertThat(jarBVersion3).exists();
+  }
+
+
+  @Test
+  public void testDeleteOtherVersionsOfJar() throws Exception {
+    File deployDir = jarDeployer.getDeployDirectory();
+
+    File jarAVersion1 = new File(deployDir, "myJarA.v1.jar");
+    this.classBuilder.writeJarFromName("ClassA", jarAVersion1);
+
+    File jarAVersion2 = new File(deployDir, "myJarA.v2.jar");
+    this.classBuilder.writeJarFromName("ClassA", jarAVersion2);
+
+    File jarBVersion2 = new File(deployDir, "myJarB.v2.jar");
+    this.classBuilder.writeJarFromName("ClassB", jarBVersion2);
+
+    File jarBVersion3 = new File(deployDir, "myJarB.v3.jar");
+    this.classBuilder.writeJarFromName("ClassB", jarBVersion3);
+
+    DeployedJar deployedJarBVersion3 = new DeployedJar(jarBVersion3, "myJarB.jar");
+    jarDeployer.deleteOtherVersionsOfJar(deployedJarBVersion3);
+
+    assertThat(jarAVersion1).exists();
+    assertThat(jarAVersion2).exists();
+    assertThat(jarBVersion2).doesNotExist();
+    assertThat(jarBVersion3).exists();
+
+    DeployedJar deployedJarAVersion1 = new DeployedJar(jarAVersion1, "myJarA.jar");
+    jarDeployer.deleteOtherVersionsOfJar(deployedJarAVersion1);
+
+    assertThat(jarAVersion1).exists();
+    assertThat(jarAVersion2).doesNotExist();
+    assertThat(jarBVersion2).doesNotExist();
+    assertThat(jarBVersion3).exists();
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 3149432..ee46cbf 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
@@ -20,12 +20,14 @@ import org.apache.geode.internal.DeployedJarJUnitTest;
 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.runner.RunWith;
 import org.junit.runners.Suite;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses({DeployedJarJUnitTest.class, DeployCommandsDUnitTest.class,
     JarDeployerIntegrationTest.class, ClassPathLoaderIntegrationTest.class,
-    ClassPathLoaderTest.class, DeployCommandRedeployDUnitTest.class})
+    ClassPathLoaderTest.class, DeployCommandRedeployDUnitTest.class,
+    ClusterConfigDeployJarDUnitTest.class})
 public class DeployJarTestSuite {
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
index b111e45..fc920c4 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
@@ -17,6 +17,7 @@
 package org.apache.geode.management.internal.configuration;
 
 
+import static java.util.stream.Collectors.toSet;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE_SIZE_LIMIT;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -47,6 +48,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 public class ClusterConfig implements Serializable {
   private List<ConfigGroup> groups;
@@ -89,7 +91,7 @@ public class ClusterConfig implements Serializable {
 
   public void verifyLocator(MemberVM<Locator> locatorVM) {
     Set<String> expectedGroupConfigs =
-        this.getGroups().stream().map(ConfigGroup::getName).collect(Collectors.toSet());
+        this.getGroups().stream().map(ConfigGroup::getName).collect(toSet());
 
     // verify info exists in memeory
     locatorVM.invoke(() -> {
@@ -135,10 +137,12 @@ public class ClusterConfig implements Serializable {
 
   public void verifyServer(MemberVM<Server> serverVM) throws ClassNotFoundException {
     // verify files exist in filesystem
-    Set<String> expectedJarNames = this.getJarNames().stream().map(ClusterConfig::getServerJarName)
-        .collect(Collectors.toSet());
-    Set<String> actualJarNames = toSetIgnoringHiddenFiles(
-        serverVM.getWorkingDir().list((dir, filename) -> filename.contains(".jar")));
+    Set<String> expectedJarNames = this.getJarNames().stream().collect(toSet());
+
+    String[] actualJarFiles =
+        serverVM.getWorkingDir().list((dir, filename) -> filename.contains(".jar"));
+    Set<String> actualJarNames = Stream.of(actualJarFiles)
+        .map(jar -> jar.replaceAll("\\.v\\d+\\.jar", ".jar")).collect(toSet());
 
     // We will end up with extra jars on disk if they are deployed and then undeployed
     assertThat(expectedJarNames).isSubsetOf(actualJarNames);
@@ -168,6 +172,7 @@ public class ClusterConfig implements Serializable {
       Set<String> undeployedJarNames = new HashSet<>(actualJarNames);
       undeployedJarNames.removeAll(expectedJarNames);
       for (String jar : undeployedJarNames) {
+        System.out.println("Verifying undeployed jar: " + jar);
         DeployedJar undeployedJar =
             ClassPathLoader.getLatest().getJarDeployer().findDeployedJar(jar);
         assertThat(undeployedJar).isNull();
@@ -181,15 +186,9 @@ public class ClusterConfig implements Serializable {
     if (array == null) {
       return new HashSet<>();
     }
-    return Arrays.stream(array).filter((String name) -> !name.startsWith("."))
-        .collect(Collectors.toSet());
-  }
-
-  private static String getServerJarName(String jarName) {
-    return jarName.replace(".jar", "") + ".v1.jar";
+    return Arrays.stream(array).filter((String name) -> !name.startsWith(".")).collect(toSet());
   }
 
-
   private static String nameOfClassContainedInJar(String jarName) {
     switch (jarName) {
       case "cluster.jar":

http://git-wip-us.apache.org/repos/asf/geode/blob/ee11b0a4/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 7cc84d6..980f81f 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
@@ -17,6 +17,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.assertj.core.api.Assertions.assertThat;
 
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
@@ -143,6 +144,7 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigBaseTest {
     serverProps.setProperty(GROUPS, "group2");
     MemberVM server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
     serverProps.setProperty(GROUPS, "group1,group2");
+    serverProps.setProperty(LOG_LEVEL, "info");
     MemberVM server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
 
     ConfigGroup cluster = new ConfigGroup("cluster");
@@ -178,6 +180,80 @@ 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);
+    server2Config.verify(server2);
+    server3Config.verify(server3);
+
+    gfshConnector.executeAndVerifyCommand("undeploy --jar=group1.jar --group=group1");
+
+    group1 = group1.removeJar("group1.jar");
+    /*
+     * TODO: This is the current (weird) behavior If you started server4 with group1,group2 after
+     * this undeploy command, it would have group1.jar (brought from
+     * cluster_config/group2/group1.jar on locator) whereas server3 (also in group1,group2) does not
+     * have this jar.
+     */
+    ClusterConfig weirdServer3Config =
+        new ClusterConfig(cluster, group1, new ConfigGroup(group2).removeJar("group1.jar"));
+
+    server3Config.verify(locator);
+    server1Config.verify(server1);
+    server2Config.verify(server2);
+    weirdServer3Config.verify(server3);
+  }
+
+  @Test
+  public void testUndeployWithServerBounce() throws Exception {
+    // set up the locator/servers
+    MemberVM locator = lsRule.startLocatorVM(0, locatorProps);
+    serverProps.setProperty(GROUPS, "group1");
+    MemberVM server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
+    serverProps.setProperty(GROUPS, "group2");
+    MemberVM server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    serverProps.setProperty(GROUPS, "group1,group2");
+    serverProps.setProperty(LOG_LEVEL, "info");
+    MemberVM server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
+
+    ConfigGroup cluster = new ConfigGroup("cluster");
+    ConfigGroup group1 = new ConfigGroup("group1");
+    ConfigGroup group2 = new ConfigGroup("group2");
+    ClusterConfig expectedClusterConfig = new ClusterConfig(cluster);
+    ClusterConfig server1Config = new ClusterConfig(cluster, group1);
+    ClusterConfig server2Config = new ClusterConfig(cluster, group2);
+    ClusterConfig server3Config = new ClusterConfig(cluster, group1, group2);
+
+    gfshConnector.connect(locator);
+    assertThat(gfshConnector.isConnected()).isTrue();
+
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + clusterJar);
+
+    // deploy cluster.jar to the cluster
+    cluster.addJar("cluster.jar");
+    expectedClusterConfig.verify(locator);
+    expectedClusterConfig.verify(server1);
+    expectedClusterConfig.verify(server2);
+    expectedClusterConfig.verify(server3);
+
+    // deploy group1.jar to both group1 and group2
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + group1Jar + " --group=group1,group2");
+
+    group1.addJar("group1.jar");
+    group2.addJar("group1.jar");
+    server3Config.verify(locator);
+    server1Config.verify(server1);
+    server2Config.verify(server2);
+    server3Config.verify(server3);
+
+    server3.getVM().bounce();
+
+    // test undeploy cluster
+    gfshConnector.executeAndVerifyCommand("undeploy --jar=cluster.jar");
+    server3 = lsRule.startServerVM(3, serverProps, locator.getPort());
+
+
     cluster = cluster.removeJar("cluster.jar");
     server3Config.verify(locator);
     server1Config.verify(server1);