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:15 UTC

[19/51] [abbrv] geode git commit: GEODE-2686: Remove JarClassLoader

GEODE-2686: Remove JarClassLoader

 - Remove JarClassLoader
 - Replace ClassPathLoader's collection of JarClassLoaders with a single URLClassLoader
 - Change naming scheme for deployed jars from 'vf.gf#myJar.jar#1' to 'myJar.v1.jar'


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

Branch: refs/heads/feature/GEODE-2097
Commit: 6fd2d123a6c4dbb0c7af0bbf135314bda0c6d5e9
Parents: f272762
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Jan 19 12:00:04 2017 -0800
Committer: Jared Stewart <js...@pivotal.io>
Committed: Sun Apr 16 09:10:00 2017 -0700

----------------------------------------------------------------------
 .../apache/geode/internal/ClassPathLoader.java  | 515 ++---------
 .../org/apache/geode/internal/DeployedJar.java  | 442 ++++++++++
 .../geode/internal/InternalDataSerializer.java  |   2 +-
 .../apache/geode/internal/JarClassLoader.java   | 721 ----------------
 .../org/apache/geode/internal/JarDeployer.java  | 699 +++++++--------
 .../cache/ClusterConfigurationLoader.java       |  40 +-
 .../geode/internal/cache/GemFireCacheImpl.java  |   4 +-
 .../cache/persistence/BackupManager.java        |  10 +-
 .../internal/cli/functions/DeployFunction.java  |  11 +-
 .../cli/functions/ListDeployedFunction.java     |  10 +-
 .../cli/functions/UndeployFunction.java         |  15 +-
 .../ClassPathLoaderIntegrationTest.java         | 454 +++++-----
 .../geode/internal/ClassPathLoaderTest.java     | 402 ++-------
 .../geode/internal/DeployedJarJUnitTest.java    | 538 ++++++++++++
 .../geode/internal/JarClassLoaderJUnitTest.java | 851 -------------------
 .../geode/internal/JarDeployerDUnitTest.java    | 574 -------------
 .../internal/JarDeployerIntegrationTest.java    | 214 +++--
 .../cache/IncrementalBackupDUnitTest.java       |  33 +-
 .../geode/management/DeployJarTestSuite.java    |  31 +
 .../DeployCommandRedeployDUnitTest.java         | 159 ++++
 .../cli/commands/DeployCommandsDUnitTest.java   | 662 +++++----------
 .../cli/commands/UserCommandsDUnitTest.java     | 164 ----
 .../internal/configuration/ClusterConfig.java   |  35 +-
 .../dunit/rules/GfshShellConnectionRule.java    |  10 +
 .../cli/commands/CommandOverHttpDUnitTest.java  |   2 +-
 25 files changed, 2323 insertions(+), 4275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/6fd2d123/geode-core/src/main/java/org/apache/geode/internal/ClassPathLoader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/ClassPathLoader.java b/geode-core/src/main/java/org/apache/geode/internal/ClassPathLoader.java
index 9ab7c30..41cce05 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/ClassPathLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/ClassPathLoader.java
@@ -14,6 +14,9 @@
  */
 package org.apache.geode.internal;
 
+import static java.util.stream.Collectors.joining;
+
+import org.apache.commons.io.FileUtils;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.util.CollectionUtils;
@@ -27,6 +30,7 @@ import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
@@ -62,271 +66,54 @@ public final class ClassPathLoader {
    * 
    * See also http://docs.oracle.com/javase/specs/jvms/se7/html/jvms-5.html
    */
-
-  public static final String ENABLE_TRACE_PROPERTY =
-      DistributionConfig.GEMFIRE_PREFIX + "ClassPathLoader.enableTrace";
-  public static final String ENABLE_TRACE_DEFAULT_VALUE = "false";
-  private final boolean ENABLE_TRACE = false;
-
   private static final Logger logger = LogService.getLogger();
 
   public static final String EXCLUDE_TCCL_PROPERTY =
       DistributionConfig.GEMFIRE_PREFIX + "excludeThreadContextClassLoader";
   public static final boolean EXCLUDE_TCCL_DEFAULT_VALUE = false;
-  private boolean excludeTCCL;
-
-  // This calculates the location of the extlib directory relative to the
-  // location of the gemfire jar file. If for some reason the ClassPathLoader
-  // class is found in a directory instead of a JAR file (as when testing),
-  // then it will be relative to the location of the root of the package and
-  // class.
-  public static final String EXT_LIB_DIR_PARENT_PROPERTY =
-      DistributionConfig.GEMFIRE_PREFIX + "ClassPathLoader.EXT_LIB_DIR";
-  public static final String EXT_LIB_DIR_PARENT_DEFAULT =
-      ClassPathLoader.class.getProtectionDomain().getCodeSource().getLocation().getPath();
-
-  static final File defineEXT_LIB_DIR() {
-    return new File(
-        (new File(System.getProperty(EXT_LIB_DIR_PARENT_PROPERTY, EXT_LIB_DIR_PARENT_DEFAULT)))
-            .getParent(),
-        "ext");
-  }
-
-  // This token is placed into the list of class loaders to determine where
-  // to insert the TCCL when in forName(...), getResource(...), etc.
-  private static final ClassLoader TCCL_PLACEHOLDER = new ClassLoader() { // This is never used for
-                                                                          // class loading
-  };
-
-  private static final AtomicReference<ClassPathLoader> latest =
-      new AtomicReference<ClassPathLoader>();
-
-  private final List<ClassLoader> classLoaders;
-
-  private static final Set<ClassLoader> defaultLoaders;
-  static {
-    defaultLoaders = new HashSet<ClassLoader>();
-    try {
-      ClassLoader classLoader = ClassPathLoader.class.getClassLoader();
-      if (classLoader != null) {
-        defaultLoaders.add(classLoader);
-      }
-    } catch (SecurityException sex) {
-      // Nothing to do, just don't add it
-    }
 
-    try {
-      ClassLoader classLoader = ClassLoader.getSystemClassLoader();
-      if (classLoader != null) {
-        defaultLoaders.add(classLoader);
-      }
-    } catch (SecurityException sex) {
-      // Nothing to do, just don't add it
-    }
+  private static volatile ClassPathLoader latest;
 
-    setLatestToDefault();
-  }
+  private volatile URLClassLoader classLoaderForDeployedJars;
+  private final JarDeployer jarDeployer;
 
-  /**
-   * Starting at the files or directories identified by 'files', search for valid JAR files and
-   * return a list of their URLs. Sub-directories will also be searched.
-   * 
-   * @param files Files or directories to search for valid JAR content.
-   * 
-   * @return A list of URLs for all JAR files found.
-   */
-  private static List<URL> getJarURLsFromFiles(final File... files) {
-    final List<URL> urls = new ArrayList<URL>();
-
-    Assert.assertTrue(files != null, "file list cannot be null");
-
-    for (File file : files) {
-      if (file.exists()) {
-        if (file.isDirectory()) {
-          urls.addAll(getJarURLsFromFiles(file.listFiles()));
-        } else {
-          if (!JarClassLoader.hasValidJarContent(file)) {
-            logger.warn("Invalid JAR content when attempting to create ClassLoader for file: {}",
-                file.getAbsolutePath());
-            continue;
-          }
+  private boolean excludeTCCL;
 
-          try {
-            urls.add(file.toURI().toURL());
-          } catch (MalformedURLException muex) {
-            logger.warn(
-                "Encountered invalid URL when attempting to create ClassLoader for file: {}:{}",
-                file.getAbsolutePath(), muex.getMessage());
-            continue;
-          }
-        }
-      }
-    }
+  public void rebuildClassLoaderForDeployedJars() {
+    ClassLoader parent = ClassPathLoader.class.getClassLoader();
 
-    return urls;
+    this.classLoaderForDeployedJars = new URLClassLoader(jarDeployer.getDeployedJarURLs(), parent);
   }
 
-  private ClassPathLoader(final List<ClassLoader> classLoaders, final boolean excludeTCCL) {
-
-    Assert.assertTrue(classLoaders != null, "custom loaders must not be null");
-    for (ClassLoader classLoader : classLoaders) {
-      Assert.assertTrue(classLoader != null, "null classloaders not allowed");
-    }
-
-    this.classLoaders = new ArrayList<ClassLoader>(classLoaders);
+  public ClassPathLoader(boolean excludeTCCL) {
     this.excludeTCCL = excludeTCCL;
+    this.jarDeployer = new JarDeployer();
+    rebuildClassLoaderForDeployedJars();
   }
 
-  /**
-   * Get a copy of the collection of ClassLoaders currently in use.
-   * 
-   * @return Collection of ClassLoaders currently in use.
-   */
-  public Collection<ClassLoader> getClassLoaders() {
-    List<ClassLoader> classLoadersCopy = new ArrayList<ClassLoader>(this.classLoaders);
-
-    for (int i = 0; i < classLoadersCopy.size(); i++) {
-      if (classLoadersCopy.get(i).equals(TCCL_PLACEHOLDER)) {
-        if (excludeTCCL) {
-          classLoadersCopy.remove(i);
-        } else {
-          classLoadersCopy.set(i, Thread.currentThread().getContextClassLoader());
-        }
-        break;
-      }
-    }
-
-    return classLoadersCopy;
-  }
-
-  // This is exposed for testing.
-  static ClassPathLoader createWithDefaults(final boolean excludeTCCL) {
-    List<ClassLoader> classLoaders = new LinkedList<ClassLoader>();
-
-    classLoaders.add(TCCL_PLACEHOLDER);
-
-    for (final ClassLoader classLoader : defaultLoaders) {
-      classLoaders.add(classLoader);
-    }
-
-    // Add user JAR files from the EXT_LIB_DIR directory using a single ClassLoader
-    try {
-      File EXT_LIB_DIR = defineEXT_LIB_DIR();
-      if (EXT_LIB_DIR.exists()) {
-        if (!EXT_LIB_DIR.isDirectory() || !EXT_LIB_DIR.canRead()) {
-          logger.warn("Cannot read from directory when attempting to load JAR files: {}",
-              EXT_LIB_DIR.getAbsolutePath());
-        } else {
-          List<URL> extLibJarURLs = getJarURLsFromFiles(EXT_LIB_DIR);
-          ClassLoader classLoader =
-              new URLClassLoader(extLibJarURLs.toArray(new URL[extLibJarURLs.size()]));
-          classLoaders.add(classLoader);
-        }
-      }
-    } catch (SecurityException sex) {
-      // Nothing to do, just don't add it
-    }
-
-    return new ClassPathLoader(classLoaders, excludeTCCL);
+  public ClassPathLoader(boolean excludeTCCL, File workingDir) {
+    this.excludeTCCL = excludeTCCL;
+    this.jarDeployer = new JarDeployer(workingDir);
+    rebuildClassLoaderForDeployedJars();
   }
 
   public static ClassPathLoader setLatestToDefault() {
-    return setLatestToDefault(Boolean.getBoolean(EXCLUDE_TCCL_PROPERTY));
+    latest = new ClassPathLoader(Boolean.getBoolean(EXCLUDE_TCCL_PROPERTY));
+    return latest;
   }
 
-  public static ClassPathLoader setLatestToDefault(final boolean excludeTCCL) {
-    ClassPathLoader classPathLoader = createWithDefaults(excludeTCCL);
-
-    // Clean up JarClassLoaders that attached to the previous ClassPathLoader
-    ClassPathLoader oldClassPathLoader = latest.getAndSet(classPathLoader);
-    if (oldClassPathLoader != null) {
-      for (ClassLoader classLoader : oldClassPathLoader.classLoaders) {
-        if (classLoader instanceof JarClassLoader) {
-          ((JarClassLoader) classLoader).cleanUp();
-        }
-      }
-    }
-
-    return classPathLoader;
+  public static ClassPathLoader setLatestToDefault(File workingDir) {
+    latest = new ClassPathLoader(Boolean.getBoolean(EXCLUDE_TCCL_PROPERTY), workingDir);
+    return latest;
   }
 
-  // This is exposed for testing.
-  ClassPathLoader addOrReplace(final ClassLoader classLoader) {
-    final boolean isDebugEnabled = logger.isTraceEnabled();
-    if (isDebugEnabled) {
-      logger.trace("adding classLoader: {}", classLoader);
-    }
-
-    List<ClassLoader> classLoadersCopy = new ArrayList<ClassLoader>(this.classLoaders);
-    classLoadersCopy.add(0, classLoader);
-
-    // Ensure there is only one instance of this class loader in the list
-    ClassLoader removingClassLoader = null;
-    int index = classLoadersCopy.lastIndexOf(classLoader);
-    if (index != 0) {
-      removingClassLoader = classLoadersCopy.get(index);
-      if (isDebugEnabled) {
-        logger.trace("removing previous classLoader: {}", removingClassLoader);
-      }
-      classLoadersCopy.remove(index);
-    }
-
-    if (removingClassLoader != null && removingClassLoader instanceof JarClassLoader) {
-      ((JarClassLoader) removingClassLoader).cleanUp();
-    }
-
-    return new ClassPathLoader(classLoadersCopy, this.excludeTCCL);
-  }
-
-  /**
-   * Add or replace the provided {@link ClassLoader} to the list held by this ClassPathLoader. Then
-   * use the resulting list to create a new ClassPathLoader and set it as the latest.
-   * 
-   * @param classLoader {@link ClassLoader} to add
-   */
-  public ClassPathLoader addOrReplaceAndSetLatest(final ClassLoader classLoader) {
-    ClassPathLoader classPathLoader = addOrReplace(classLoader);
-    latest.set(classPathLoader);
-    return classPathLoader;
+  public JarDeployer getJarDeployer() {
+    return this.jarDeployer;
   }
 
   // This is exposed for testing.
-  ClassPathLoader remove(final ClassLoader classLoader) {
-    final boolean isDebugEnabled = logger.isTraceEnabled();
-    if (isDebugEnabled) {
-      logger.trace("removing classLoader: {}", classLoader);
-    }
-
-    List<ClassLoader> classLoadersCopy = new ArrayList<ClassLoader>();
-    classLoadersCopy.addAll(this.classLoaders);
-
-    if (!classLoadersCopy.contains(classLoader)) {
-      if (isDebugEnabled) {
-        logger.trace("cannot remove classLoader since it doesn't exist: {}", classLoader);
-      }
-      return this;
-    }
-
-    classLoadersCopy.remove(classLoader);
-
-    if (classLoader instanceof JarClassLoader) {
-      ((JarClassLoader) classLoader).cleanUp();
-    }
-
-    return new ClassPathLoader(classLoadersCopy, this.excludeTCCL);
-  }
-
-  /**
-   * Remove the provided {@link ClassLoader} from the list held by this ClassPathLoader. Then use
-   * the resulting list to create a new ClassPathLoader and set it as the latest. Silently ignores
-   * requests to remove non-existent ClassLoaders.
-   * 
-   * @param classLoader {@link ClassLoader} to remove
-   */
-  public ClassPathLoader removeAndSetLatest(final ClassLoader classLoader) {
-    ClassPathLoader classPathLoader = remove(classLoader);
-    latest.set(classPathLoader);
-    return classPathLoader;
+  static ClassPathLoader createWithDefaults(final boolean excludeTCCL) {
+    return new ClassPathLoader(excludeTCCL);
   }
 
   public URL getResource(final String name) {
@@ -334,52 +121,26 @@ public final class ClassPathLoader {
     if (isDebugEnabled) {
       logger.trace("getResource({})", name);
     }
-    URL url = null;
-    ClassLoader tccl = null;
-    if (!excludeTCCL) {
-      tccl = Thread.currentThread().getContextClassLoader();
-    }
 
-    for (ClassLoader classLoader : this.classLoaders) {
-      if (classLoader == TCCL_PLACEHOLDER) {
-        try {
-          if (tccl != null) {
-            if (isDebugEnabled) {
-              logger.trace("getResource trying TCCL: {}", tccl);
-            }
-            url = tccl.getResource(name);
-            if (url != null) {
-              if (isDebugEnabled) {
-                logger.trace("getResource found by TCCL");
-              }
-              return url;
-            }
-          } else {
-            if (isDebugEnabled) {
-              logger.trace("getResource skipping TCCL because it's null");
-            }
-          }
-        } catch (SecurityException sex) {
-          // Continue to next ClassLoader
-        }
-      } else if (excludeTCCL || !classLoader.equals(tccl)) {
-        if (isDebugEnabled) {
-          logger.trace("getResource trying classLoader: {}", classLoader);
-        }
-        url = classLoader.getResource(name);
+    for (ClassLoader classLoader : getClassLoaders()) {
+      if (isDebugEnabled) {
+        logger.trace("getResource trying: {}", classLoader);
+      }
+      try {
+        URL url = classLoader.getResource(name);
+
         if (url != null) {
           if (isDebugEnabled) {
-            logger.trace("getResource found by classLoader: {}", classLoader);
+            logger.trace("getResource found by: {}", classLoader);
           }
           return url;
         }
+      } catch (SecurityException e) {
+        // try next classLoader
       }
     }
 
-    if (isDebugEnabled) {
-      logger.trace("getResource returning null");
-    }
-    return url;
+    return null;
   }
 
   public Class<?> forName(final String name) throws ClassNotFoundException {
@@ -387,53 +148,25 @@ public final class ClassPathLoader {
     if (isDebugEnabled) {
       logger.trace("forName({})", name);
     }
-    Class<?> clazz = null;
-    ClassLoader tccl = null;
-    if (!excludeTCCL) {
-      tccl = Thread.currentThread().getContextClassLoader();
-    }
 
-    for (ClassLoader classLoader : this.classLoaders) {
+    for (ClassLoader classLoader : this.getClassLoaders()) {
+      if (isDebugEnabled) {
+        logger.trace("forName trying: {}", classLoader);
+      }
       try {
-        if (classLoader == TCCL_PLACEHOLDER) {
-          if (tccl != null) {
-            if (isDebugEnabled) {
-              logger.trace("forName trying TCCL: {}", tccl);
-            }
-            clazz = Class.forName(name, true, tccl);
-            if (clazz != null) {
-              if (isDebugEnabled) {
-                logger.trace("forName found by TCCL");
-              }
-              return clazz;
-            } else {
-              if (isDebugEnabled) {
-                logger.trace("forName skipping TCCL because it's null");
-              }
-            }
-          }
-        } else if (excludeTCCL || !classLoader.equals(tccl)) {
+        Class<?> clazz = Class.forName(name, true, classLoader);
+
+        if (clazz != null) {
           if (isDebugEnabled) {
-            logger.trace("forName trying classLoader: {}", classLoader);
-          }
-          clazz = Class.forName(name, true, classLoader);
-          if (clazz != null) {
-            if (isDebugEnabled) {
-              logger.trace("forName found by classLoader: {}", classLoader);
-            }
-            return clazz;
+            logger.trace("forName found by: {}", classLoader);
           }
+          return clazz;
         }
-      } catch (SecurityException sex) {
-        // Continue to next ClassLoader
-      } catch (ClassNotFoundException cnfex) {
-        // Continue to next ClassLoader
+      } catch (SecurityException | ClassNotFoundException e) {
+        // try next classLoader
       }
     }
 
-    if (isDebugEnabled) {
-      logger.trace("forName throwing ClassNotFoundException");
-    }
     throw new ClassNotFoundException(name);
   }
 
@@ -442,20 +175,10 @@ public final class ClassPathLoader {
    */
   public Class<?> getProxyClass(final Class<?>[] classObjs) {
     IllegalArgumentException ex = null;
-    ClassLoader tccl = null;
-    if (!excludeTCCL) {
-      tccl = Thread.currentThread().getContextClassLoader();
-    }
 
-    for (ClassLoader classLoader : this.classLoaders) {
+    for (ClassLoader classLoader : this.getClassLoaders()) {
       try {
-        if (classLoader == TCCL_PLACEHOLDER) {
-          if (tccl != null) {
-            return Proxy.getProxyClass(tccl, classObjs);
-          }
-        } else if (excludeTCCL || !classLoader.equals(tccl)) {
-          return Proxy.getProxyClass(classLoader, classObjs);
-        }
+        return Proxy.getProxyClass(classLoader, classObjs);
       } catch (SecurityException sex) {
         // Continue to next classloader
       } catch (IllegalArgumentException iaex) {
@@ -464,7 +187,6 @@ public final class ClassPathLoader {
       }
     }
 
-    assert ex != null;
     if (ex != null) {
       throw ex;
     }
@@ -475,19 +197,9 @@ public final class ClassPathLoader {
   public String toString() {
     final StringBuilder sb = new StringBuilder(getClass().getName());
     sb.append("@").append(System.identityHashCode(this)).append("{");
-    sb.append("isLatest=").append(getLatest() == this);
     sb.append(", excludeTCCL=").append(this.excludeTCCL);
     sb.append(", classLoaders=[");
-    for (int i = 0; i < this.classLoaders.size(); i++) {
-      if (i > 0) {
-        sb.append(", ");
-      }
-      sb.append(this.classLoaders.get(i).toString());
-    }
-    sb.append("]");
-    if (!this.excludeTCCL) {
-      sb.append(", TCCL=").append(Thread.currentThread().getContextClassLoader());
-    }
+    sb.append(this.getClassLoaders().stream().map(ClassLoader::toString).collect(joining(", ")));
     sb.append("]}");
     return sb.toString();
   }
@@ -514,13 +226,12 @@ public final class ClassPathLoader {
 
   /**
    * Returns an input stream for reading the specified resource.
-   * 
+   *
    * <p>
    * The search order is described in the documentation for {@link #getResource(String)}.
    * </p>
    * 
    * @param name The resource name
-   * 
    * @return An input stream for reading the resource, or <tt>null</tt> if the resource could not be
    *         found
    */
@@ -553,122 +264,51 @@ public final class ClassPathLoader {
     return getResourceAsStream(name);
   }
 
+
   /**
    * Finds all the resources with the given name. This method will first search the class loader of
    * the context class for the resource before searching all other {@link ClassLoader}s.
    * 
    * @param contextClass The class whose class loader will first be searched
-   * 
    * @param name The resource name
-   *
    * @return An enumeration of {@link java.net.URL <tt>URL</tt>} objects for the resource. If no
    *         resources could be found, the enumeration will be empty. Resources that the class
    *         loader doesn't have access to will not be in the enumeration.
-   *
    * @throws IOException If I/O errors occur
-   * 
    * @see ClassLoader#getResources(String)
    */
   public Enumeration<URL> getResources(final Class<?> contextClass, final String name)
       throws IOException {
-    final boolean isDebugEnabled = logger.isTraceEnabled();
+    final LinkedHashSet<URL> urls = new LinkedHashSet<URL>();
 
-    if (isDebugEnabled) {
-      logger.trace(new StringBuilder("getResources(").append(name).append(")"));
+    if (contextClass != null) {
+      CollectionUtils.addAll(urls, contextClass.getClassLoader().getResources(name));
     }
 
-    final LinkedHashSet<URL> urls = new LinkedHashSet<URL>();
-
-    try {
-      if (contextClass != null) {
-        CollectionUtils.addAll(urls, contextClass.getClassLoader().getResources(name));
+    for (ClassLoader classLoader : getClassLoaders()) {
+      Enumeration<URL> resources = classLoader.getResources(name);
+      if (resources != null && resources.hasMoreElements()) {
+        CollectionUtils.addAll(urls, resources);
       }
-    } catch (IOException ignore) {
-      // ignore and search others
     }
 
-    Enumeration<URL> resources = null;
-    ClassLoader tccl = null;
-    if (!excludeTCCL) {
-      tccl = Thread.currentThread().getContextClassLoader();
-    }
+    return Collections.enumeration(urls);
+  }
 
-    IOException ioException = null;
-    for (ClassLoader classLoader : this.classLoaders) {
-      ioException = null; // reset to null for next ClassLoader
-      if (classLoader == TCCL_PLACEHOLDER) {
-        try {
-          if (tccl != null) {
-            if (isDebugEnabled) {
-              logger.trace("getResources trying TCCL: {}", tccl);
-            }
-            resources = tccl.getResources(name);
-            if (resources != null && resources.hasMoreElements()) {
-              if (isDebugEnabled) {
-                logger.trace("getResources found by TCCL");
-              }
-              CollectionUtils.addAll(urls, resources);
-            }
-          } else {
-            if (isDebugEnabled) {
-              logger.trace("getResources skipping TCCL because it's null");
-            }
-          }
-        } catch (SecurityException ignore) {
-          // Continue to next ClassLoader
-        } catch (IOException ignore) {
-          ioException = ignore;
-          // Continue to next ClassLoader
-        }
-      } else if (excludeTCCL || !classLoader.equals(tccl)) {
-        try {
-          if (isDebugEnabled) {
-            logger.trace("getResources trying classLoader: {}", classLoader);
-          }
-          resources = classLoader.getResources(name);
-          if (resources != null && resources.hasMoreElements()) {
-            if (logger.isTraceEnabled()) {
-              logger.trace(
-                  new StringBuilder("getResources found by classLoader: ").append(classLoader));
-            }
-            CollectionUtils.addAll(urls, resources);
-          }
-        } catch (IOException ignore) {
-          ioException = ignore;
-          // Continue to next ClassLoader
-        }
-      }
-    }
+  public Enumeration<URL> getResources(final String name) throws IOException {
+    return getResources(null, name);
+  }
 
-    if (ioException != null) {
-      if (isDebugEnabled) {
-        logger.trace("getResources throwing IOException");
-      }
-      throw ioException;
-    }
+  private List<ClassLoader> getClassLoaders() {
+    ArrayList<ClassLoader> classLoaders = new ArrayList<>();
 
-    if (isDebugEnabled) {
-      logger.trace("getResources returning empty enumeration");
+    if (!excludeTCCL) {
+      classLoaders.add(Thread.currentThread().getContextClassLoader());
     }
 
-    return Collections.enumeration(urls);
-  }
+    classLoaders.add(classLoaderForDeployedJars);
 
-  /**
-   * Finds all the resources with the given name.
-   * 
-   * @param name The resource name
-   *
-   * @return An enumeration of {@link java.net.URL <tt>URL</tt>} objects for the resource. If no
-   *         resources could be found, the enumeration will be empty. Resources that the class
-   *         loader doesn't have access to will not be in the enumeration.
-   *
-   * @throws IOException If I/O errors occur
-   * 
-   * @see ClassLoader#getResources(String)
-   */
-  public Enumeration<URL> getResources(String name) throws IOException {
-    return getResources(null, name);
+    return classLoaders;
   }
 
   /**
@@ -707,7 +347,14 @@ public final class ClassPathLoader {
   }
 
   public static ClassPathLoader getLatest() {
-    return latest.get();
+    if (latest == null) {
+      synchronized (ClassPathLoader.class) {
+        if (latest == null)
+          setLatestToDefault();
+      }
+    }
+
+    return latest;
   }
 
   /**
@@ -717,7 +364,7 @@ public final class ClassPathLoader {
    * @since GemFire 8.1
    */
   public static final ClassLoader getLatestAsClassLoader() {
-    return latest.get().asClassLoader();
+    return latest.asClassLoader();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/6fd2d123/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
new file mode 100644
index 0000000..f4f4028
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
@@ -0,0 +1,442 @@
+/*
+ * 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.internal;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+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.nio.file.Files;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+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;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.execute.Function;
+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.
+ * 
+ * @since GemFire 7.0
+ */
+public class DeployedJar {
+  private final static Logger logger = LogService.getLogger();
+  private final static MessageDigest messageDigest = getMessageDigest();
+
+  private final String jarName;
+  private final File file;
+  private final byte[] md5hash;
+  private final Collection<Function> registeredFunctions = new ArrayList<Function>();
+
+  private static MessageDigest getMessageDigest() {
+    try {
+      return MessageDigest.getInstance("MD5");
+    } catch (NoSuchAlgorithmException nsaex) {
+      // Failure just means we can't do a simple compare for content equality
+    }
+    return null;
+  }
+
+  public File getFile() {
+    return this.file;
+  }
+
+  public int getVersion() {
+    return JarDeployer.extractVersionFromFilename(file.getName());
+  }
+
+  public DeployedJar(File versionedJarFile, String jarName) throws IOException {
+    this(versionedJarFile, jarName, Files.readAllBytes(versionedJarFile.toPath()));
+  }
+
+  public DeployedJar(File versionedJarFile, final String jarName, byte[] jarBytes)
+      throws IOException {
+    Assert.assertTrue(jarBytes != null, "jarBytes cannot be null");
+    Assert.assertTrue(jarName != null, "jarName cannot be null");
+    Assert.assertTrue(versionedJarFile != null, "versionedJarFile cannot be null");
+
+    this.file = versionedJarFile;
+    this.jarName = jarName;
+
+    final byte[] fileContent = getJarContent();
+    if (!Arrays.equals(fileContent, jarBytes)) {
+      throw new FileNotFoundException("JAR file: " + versionedJarFile.getAbsolutePath()
+          + ", was modified prior to obtaining a lock: " + jarName);
+    }
+
+    if (!isValidJarContent(getJarContent())) {
+      throw new IllegalArgumentException(
+          "File does not contain valid JAR content: " + versionedJarFile.getAbsolutePath());
+    }
+
+    if (messageDigest != null) {
+      this.md5hash = messageDigest.digest(jarBytes);
+    } else {
+      this.md5hash = null;
+    }
+  }
+
+  /**
+   * Peek into the JAR data and make sure that it is valid JAR content.
+   * 
+   * @param inputStream InputStream containing data to be validated.
+   * @return True if the data has JAR content, false otherwise
+   */
+  private static boolean hasValidJarContent(final InputStream inputStream) {
+    JarInputStream jarInputStream = null;
+    boolean valid = false;
+
+    try {
+      jarInputStream = new JarInputStream(inputStream);
+      valid = (jarInputStream.getNextJarEntry() != null);
+    } catch (IOException ignore) {
+      // Ignore this exception and just return false
+    } finally {
+      try {
+        jarInputStream.close();
+      } catch (IOException ioex) {
+        // Ignore this exception and just return result
+      }
+    }
+
+    return valid;
+  }
+
+  /**
+   * Peek into the JAR data and make sure that it is valid JAR content.
+   * 
+   * @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) {
+    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.
+   */
+  // This method will process the contents of the JAR file as stored in this.jarByteContent
+  // instead of reading from the original JAR file. This is done because we can't open up
+  // the original file and then close it without releasing the shared lock that was obtained
+  // in the constructor. Once this method is finished, all classes will have been loaded and
+  // there will no longer be a need to hang on to the original contents so they will be
+  // discarded.
+  public synchronized void loadClassesAndRegisterFunctions() throws ClassNotFoundException {
+    final boolean isDebugEnabled = logger.isDebugEnabled();
+    if (isDebugEnabled) {
+      logger.debug("Registering functions with DeployedJar: {}", this);
+    }
+
+    ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(this.getJarContent());
+
+    JarInputStream jarInputStream = null;
+    try {
+      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());
+              }
+              this.registeredFunctions.add(function);
+            }
+          } 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();
+      }
+    } catch (IOException ioex) {
+      logger.error("Exception when trying to read class from ByteArrayInputStream", ioex);
+    } finally {
+      if (jarInputStream != null) {
+        try {
+          jarInputStream.close();
+        } catch (IOException ioex) {
+          logger.error("Exception attempting to close JAR input stream", ioex);
+        }
+      }
+    }
+  }
+
+  synchronized void cleanUp() {
+    for (Function function : this.registeredFunctions) {
+      FunctionService.unregisterFunction(function.getId());
+    }
+    this.registeredFunctions.clear();
+
+    try {
+      TypeRegistry typeRegistry =
+          ((GemFireCacheImpl) CacheFactory.getAnyInstance()).getPdxRegistry();
+      if (typeRegistry != null) {
+        typeRegistry.flushCache();
+      }
+    } catch (CacheClosedException ccex) {
+      // That's okay, it just means there was nothing to flush to begin with
+    }
+  }
+
+  /**
+   * Uses MD5 hashes to determine if the original byte content of this DeployedJar is the same as
+   * that past in.
+   * 
+   * @param compareToBytes Bytes to compare the original content to
+   * @return True of the MD5 hash is the same o
+   */
+  public boolean hasSameContentAs(final byte[] compareToBytes) {
+    // If the MD5 hash can't be calculated then silently return no match
+    if (messageDigest == null || this.md5hash == null) {
+      return Arrays.equals(compareToBytes, getJarContent());
+    }
+
+    byte[] compareToMd5 = messageDigest.digest(compareToBytes);
+    if (logger.isDebugEnabled()) {
+      logger.debug("For JAR file: {}, Comparing MD5 hash {} to {}", this.file.getAbsolutePath(),
+          new String(this.md5hash), new String(compareToMd5));
+    }
+    return Arrays.equals(this.md5hash, compareToMd5);
+  }
+
+  /**
+   * Check to see if the class implements the Function interface. If so, it will be registered with
+   * FunctionService. Also, if the functions's class was originally declared in a cache.xml file
+   * then any properties specified at that time will be reused when re-registering the function.
+   * 
+   * @param clazz Class to check for implementation of the Function class
+   * @return A collection of Objects that implement the Function interface.
+   */
+  private Collection<Function> getRegisterableFunctionsFromClass(Class<?> clazz) {
+    final List<Function> registerableFunctions = new ArrayList<Function>();
+
+    try {
+      if (Function.class.isAssignableFrom(clazz) && !Modifier.isAbstract(clazz.getModifiers())) {
+        boolean registerUninitializedFunction = true;
+        if (Declarable.class.isAssignableFrom(clazz)) {
+          try {
+            final List<Properties> propertiesList =
+                ((GemFireCacheImpl) CacheFactory.getAnyInstance())
+                    .getDeclarableProperties(clazz.getName());
+
+            if (!propertiesList.isEmpty()) {
+              registerUninitializedFunction = false;
+              // It's possible that the same function was declared multiple times in cache.xml
+              // with different properties. So, register the function using each set of
+              // properties.
+              for (Properties properties : propertiesList) {
+                @SuppressWarnings("unchecked")
+                Function function = newFunction((Class<Function>) clazz, true);
+                if (function != null) {
+                  ((Declarable) function).init(properties);
+                  if (function.getId() != null) {
+                    registerableFunctions.add(function);
+                  }
+                }
+              }
+            }
+          } catch (CacheClosedException ccex) {
+            // That's okay, it just means there were no properties to init the function with
+          }
+        }
+
+        if (registerUninitializedFunction) {
+          @SuppressWarnings("unchecked")
+          Function function = newFunction((Class<Function>) clazz, false);
+          if (function != null && function.getId() != null) {
+            registerableFunctions.add(function);
+          }
+        }
+      }
+    } catch (Exception ex) {
+      logger.error("Attempting to register function from JAR file: " + this.file.getAbsolutePath(),
+          ex);
+    }
+
+    return registerableFunctions;
+  }
+
+  private Function newFunction(final Class<Function> clazz, final boolean errorOnNoSuchMethod) {
+    try {
+      final Constructor<Function> constructor = clazz.getConstructor();
+      return constructor.newInstance();
+    } catch (NoSuchMethodException nsmex) {
+      if (errorOnNoSuchMethod) {
+        logger.error("Zero-arg constructor is required, but not found for class: {}",
+            clazz.getName(), nsmex);
+      } else {
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "Not registering function because it doesn't have a zero-arg constructor: {}",
+              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) {
+      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);
+    }
+    return null;
+  }
+
+  private byte[] getJarContent() {
+    InputStream channelInputStream = null;
+    try {
+      channelInputStream = new FileInputStream(this.file);
+
+      final ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
+      final byte[] bytes = new byte[4096];
+
+      int bytesRead;
+      while (((bytesRead = channelInputStream.read(bytes)) != -1)) {
+        byteOutStream.write(bytes, 0, bytesRead);
+      }
+      channelInputStream.close();
+      return byteOutStream.toByteArray();
+    } catch (IOException e) {
+      logger.error("Error when attempting to read jar contents: ", e);
+    }
+
+    return new byte[0];
+  }
+
+  public String getJarName() {
+    return this.jarName;
+  }
+
+  public String getFileName() {
+    return this.file.getName();
+  }
+
+  public String getFileCanonicalPath() throws IOException {
+    return this.file.getCanonicalPath();
+  }
+
+  public URL getFileURL() {
+    try {
+      return this.file.toURL();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    return null;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((this.jarName == null) ? 0 : this.jarName.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    DeployedJar other = (DeployedJar) obj;
+    if (this.jarName == null) {
+      if (other.jarName != null) {
+        return false;
+      }
+    } else if (!this.jarName.equals(other.jarName)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(getClass().getName());
+    sb.append("@").append(System.identityHashCode(this)).append("{");
+    sb.append("jarName=").append(this.jarName);
+    sb.append(",file=").append(this.file.getAbsolutePath());
+    sb.append(",md5hash=").append(Arrays.toString(this.md5hash));
+    sb.append(",version=").append(this.getVersion());
+    sb.append("}");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/6fd2d123/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
index f4f4069..51212c6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
@@ -4033,7 +4033,7 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
 
   public static void flushClassCache() {
     if (classCache != null) {
-      // Not locking classCache during clear as doing so causes a deadlock in the JarClassLoader
+      // Not locking classCache during clear as doing so causes a deadlock in the DeployedJar
       classCache.clear();
     }
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/6fd2d123/geode-core/src/main/java/org/apache/geode/internal/JarClassLoader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/JarClassLoader.java b/geode-core/src/main/java/org/apache/geode/internal/JarClassLoader.java
deleted file mode 100644
index 9cd0589..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/JarClassLoader.java
+++ /dev/null
@@ -1,721 +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.internal;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Modifier;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.Properties;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.jar.JarEntry;
-import java.util.jar.JarInputStream;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.Declarable;
-import org.apache.geode.cache.execute.Function;
-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;
-
-/**
- * ClassLoader for a single JAR file.
- * 
- * @since GemFire 7.0
- */
-public class JarClassLoader extends ClassLoader {
-  private final static Logger logger = LogService.getLogger();
-  private final static MessageDigest messageDigest;
-
-  private final String jarName;
-  private final File file;
-  private final byte[] md5hash;
-  private FileLock fileLock;
-  private final Collection<Function> registeredFunctions = new ArrayList<Function>();
-
-  private final ThreadLocal<Boolean> alreadyScanned = new ThreadLocal<Boolean>();
-
-  // Lock used by ChannelInputStream (inner class) to prevent multiple threads from
-  // trying to use the channel simultaneously.
-  static final ReentrantLock channelLock = new ReentrantLock();
-
-  private byte[] jarByteContent;
-
-  static {
-    MessageDigest md = null;
-    try {
-      md = MessageDigest.getInstance("MD5");
-    } catch (NoSuchAlgorithmException nsaex) {
-      // Failure just means we can't do a simple compare for content equality
-    }
-    messageDigest = md;
-  }
-
-  public JarClassLoader(final File file, final String jarName, byte[] jarBytes) throws IOException {
-    Assert.assertTrue(file != null, "file cannot be null");
-    Assert.assertTrue(jarName != null, "jarName cannot be null");
-
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    try {
-      @SuppressWarnings("resource")
-      FileInputStream fileInputStream = new FileInputStream(file);
-      this.fileLock = fileInputStream.getChannel().lock(0, file.length(), true);
-
-      if (isDebugEnabled) {
-        logger.debug("Acquired shared file lock w/ channel: {}, for JAR: {}",
-            this.fileLock.channel(), file.getAbsolutePath());
-      }
-
-      if (file.length() == 0) {
-        throw new FileNotFoundException(
-            "JAR file was truncated prior to obtaining a lock: " + jarName);
-      }
-
-      final byte[] fileContent = getJarContent();
-      if (!Arrays.equals(fileContent, jarBytes)) {
-        throw new FileNotFoundException("JAR file: " + file.getAbsolutePath()
-            + ", was modified prior to obtaining a lock: " + jarName);
-      }
-
-      if (!isValidJarContent(jarBytes)) {
-        if (this.fileLock != null) {
-          this.fileLock.release();
-          this.fileLock.channel().close();
-          if (isDebugEnabled) {
-            logger.debug(
-                "Prematurely releasing shared file lock due to bad content for JAR file: {}, w/ channel: {}",
-                file.getAbsolutePath(), this.fileLock.channel());
-          }
-        }
-        throw new IllegalArgumentException(
-            "File does not contain valid JAR content: " + file.getAbsolutePath());
-      }
-
-      Assert.assertTrue(jarBytes != null, "jarBytes cannot be null");
-
-      // Temporarily save the contents of the JAR file until they can be processed by the
-      // loadClassesandRegisterFunctions() method.
-      this.jarByteContent = jarBytes;
-
-      if (messageDigest != null) {
-        this.md5hash = messageDigest.digest(this.jarByteContent);
-      } else {
-        this.md5hash = null;
-      }
-
-      this.file = file;
-      this.jarName = jarName;
-
-    } catch (FileNotFoundException fnfex) {
-      if (this.fileLock != null) {
-        this.fileLock.release();
-        this.fileLock.channel().close();
-        if (isDebugEnabled) {
-          logger.debug(
-              "Prematurely releasing shared file lock due to file not found for JAR file: {}, w/ channel: {}",
-              file.getAbsolutePath(), this.fileLock.channel());
-        }
-      }
-      throw fnfex;
-    }
-  }
-
-  /**
-   * Peek into the JAR data and make sure that it is valid JAR content.
-   * 
-   * @param inputStream InputStream containing data to be validated.
-   * 
-   * @return True if the data has JAR content, false otherwise
-   */
-  private static boolean hasValidJarContent(final InputStream inputStream) {
-    JarInputStream jarInputStream = null;
-    boolean valid = false;
-
-    try {
-      jarInputStream = new JarInputStream(inputStream);
-      valid = (jarInputStream.getNextJarEntry() != null);
-    } catch (IOException ignore) {
-      // Ignore this exception and just return false
-    } finally {
-      try {
-        jarInputStream.close();
-      } catch (IOException ioex) {
-        // Ignore this exception and just return result
-      }
-    }
-
-    return valid;
-  }
-
-  /**
-   * Peek into the JAR data and make sure that it is valid JAR content.
-   * 
-   * @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) {
-    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.
-   */
-  // This method will process the contents of the JAR file as stored in this.jarByteContent
-  // instead of reading from the original JAR file. This is done because we can't open up
-  // the original file and then close it without releasing the shared lock that was obtained
-  // in the constructor. Once this method is finished, all classes will have been loaded and
-  // there will no longer be a need to hang on to the original contents so they will be
-  // discarded.
-  public synchronized void loadClassesAndRegisterFunctions() throws ClassNotFoundException {
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (isDebugEnabled) {
-      logger.debug("Registering functions with JarClassLoader: {}", this);
-    }
-
-    ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(this.jarByteContent);
-
-    JarInputStream jarInputStream = null;
-    try {
-      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 = loadClass(className, true, false);
-            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);
-            }
-          } 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();
-      }
-    } catch (IOException ioex) {
-      logger.error("Exception when trying to read class from ByteArrayInputStream", ioex);
-    } finally {
-      if (jarInputStream != null) {
-        try {
-          jarInputStream.close();
-        } catch (IOException ioex) {
-          logger.error("Exception attempting to close JAR input stream", ioex);
-        }
-      }
-    }
-    this.jarByteContent = new byte[0];
-  }
-
-  synchronized void cleanUp() {
-    for (Function function : this.registeredFunctions) {
-      FunctionService.unregisterFunction(function.getId());
-    }
-    this.registeredFunctions.clear();
-
-    try {
-      TypeRegistry typeRegistry =
-          ((GemFireCacheImpl) CacheFactory.getAnyInstance()).getPdxRegistry();
-      if (typeRegistry != null) {
-        typeRegistry.flushCache();
-      }
-    } catch (CacheClosedException ccex) {
-      // That's okay, it just means there was nothing to flush to begin with
-    }
-
-    try {
-      this.fileLock.release();
-      this.fileLock.channel().close();
-      if (logger.isDebugEnabled()) {
-        logger.debug("Released shared file lock on JAR file: {}, w/ channel: {}",
-            this.file.getAbsolutePath(), this.fileLock.channel());
-      }
-    } catch (IOException ioex) {
-      logger.error("Could not release the shared lock for JAR file: {}",
-          this.file.getAbsolutePath(), ioex);
-    }
-  }
-
-  /**
-   * Uses MD5 hashes to determine if the original byte content of this JarClassLoader is the same as
-   * that past in.
-   * 
-   * @param compareToBytes Bytes to compare the original content to
-   * @return True of the MD5 hash is the same o
-   */
-  public boolean hasSameContent(final byte[] compareToBytes) {
-    // If the MD5 hash can't be calculated then silently return no match
-    if (messageDigest == null || this.md5hash == null) {
-      return false;
-    }
-
-    byte[] compareToMd5 = messageDigest.digest(compareToBytes);
-    if (logger.isDebugEnabled()) {
-      logger.debug("For JAR file: {}, Comparing MD5 hash {} to {}", this.file.getAbsolutePath(),
-          new String(this.md5hash), new String(compareToMd5));
-    }
-    return Arrays.equals(this.md5hash, compareToMd5);
-  }
-
-  private boolean alreadyScanned() {
-    if (this.alreadyScanned.get() == null) {
-      this.alreadyScanned.set(Boolean.FALSE);
-    }
-    return this.alreadyScanned.get();
-  }
-
-  @Override
-  protected URL findResource(String resourceName) {
-    URL returnURL = null;
-    JarInputStream jarInputStream = null;
-
-    try {
-      ChannelInputStream channelInputStream = new ChannelInputStream(this.fileLock.channel());
-      jarInputStream = new JarInputStream(channelInputStream);
-
-      JarEntry jarEntry = jarInputStream.getNextJarEntry();
-      while (jarEntry != null && !jarEntry.getName().equals(resourceName)) {
-        jarEntry = jarInputStream.getNextJarEntry();
-      }
-      if (jarEntry != null) {
-        try {
-          returnURL = new URL("jar", "", this.file.toURI().toURL() + "!/" + jarEntry.getName());
-        } catch (MalformedURLException muex) {
-          logger.error("Could not create resource URL from file URL", muex);
-        }
-      }
-    } catch (IOException ioex) {
-      logger.error("Exception when trying to read class from ByteArrayInputStream", ioex);
-    } finally {
-      if (jarInputStream != null) {
-        try {
-          jarInputStream.close();
-        } catch (IOException ioex) {
-          logger.error("Unable to close JAR input stream when finding resource", ioex);
-        }
-      }
-    }
-
-    return returnURL;
-  }
-
-  @Override
-  protected Enumeration<URL> findResources(final String resourceName) {
-    return new Enumeration<URL>() {
-      private URL element = findResource(resourceName);
-
-      @Override
-      public boolean hasMoreElements() {
-        return this.element != null;
-      }
-
-      @Override
-      public URL nextElement() {
-        if (this.element != null) {
-          URL element = this.element;
-          this.element = null;
-          return element;
-        }
-        throw new NoSuchElementException();
-      }
-    };
-  }
-
-  @Override
-  public Class<?> loadClass(final String className) throws ClassNotFoundException {
-    return (loadClass(className, true));
-  }
-
-  @Override
-  public Class<?> loadClass(final String className, final boolean resolveIt)
-      throws ClassNotFoundException {
-    return loadClass(className, resolveIt, true);
-  }
-
-  Class<?> loadClass(final String className, final boolean resolveIt,
-      final boolean useClassPathLoader) throws ClassNotFoundException {
-    Class<?> clazz = findLoadedClass(className);
-    if (clazz != null) {
-      return clazz;
-    }
-
-    try {
-      clazz = findClass(className);
-      if (resolveIt) {
-        resolveClass(clazz);
-      }
-    } catch (ClassNotFoundException cnfex) {
-      if (!useClassPathLoader) {
-        throw cnfex;
-      }
-    }
-
-    if (clazz == null) {
-      try {
-        this.alreadyScanned.set(true);
-        return forName(className, ClassPathLoader.getLatest().getClassLoaders());
-      } finally {
-        this.alreadyScanned.set(false);
-      }
-    }
-
-    return clazz;
-  }
-
-  // When loadClassesAndRegisterFunctions() is called and it starts to load classes, this method
-  // may be called multiple times to resolve dependencies on other classes in the same or
-  // another JAR file. During this stage, this.jarByteContent will contain the complete contents of
-  // the JAR file and will be used when attempting to resolve these dependencies. Once
-  // loadClassesAndRegisterFunctions() is complete it discards the data in this.jarByteContent.
-  // However, at that point all of the classes available in the JAR file will already have been
-  // loaded. Future calls to loadClass(...) will return the cached Class object for any
-  // classes available in this JAR and findClass(...) will no longer be needed to find them.
-  @Override
-  protected Class<?> findClass(String className) throws ClassNotFoundException {
-    String formattedClassName = className.replaceAll("\\.", "/") + ".class";
-
-    JarInputStream jarInputStream = null;
-    if (this.jarByteContent.length == 0) {
-      throw new ClassNotFoundException(className);
-    }
-
-    try {
-      jarInputStream = new JarInputStream(new ByteArrayInputStream(this.jarByteContent));
-      JarEntry jarEntry = jarInputStream.getNextJarEntry();
-
-      while (jarEntry != null && !jarEntry.getName().equals(formattedClassName)) {
-        jarEntry = jarInputStream.getNextJarEntry();
-      }
-
-      if (jarEntry != null) {
-        byte[] buffer = new byte[1024];
-        ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(buffer.length);
-
-        int bytesRead = -1;
-        while ((bytesRead = jarInputStream.read(buffer)) != -1) {
-          byteOutStream.write(buffer, 0, bytesRead);
-        }
-
-        // Add the package first if it doesn't already exist
-        int lastDotIndex = className.lastIndexOf('.');
-        if (lastDotIndex != -1) {
-          String pkgName = className.substring(0, lastDotIndex);
-          Package pkg = getPackage(pkgName);
-          if (pkg == null) {
-            definePackage(pkgName, null, null, null, null, null, null, null);
-          }
-        }
-
-        byte[] classBytes = byteOutStream.toByteArray();
-
-        synchronized (this.file) {
-          Class<?> clazz = findLoadedClass(className);
-          if (clazz == null) {
-            clazz = defineClass(className, classBytes, 0, classBytes.length, null);
-          }
-          return clazz;
-        }
-      }
-    } catch (IOException ioex) {
-      logger.error("Exception when trying to read class from ByteArrayInputStream", ioex);
-    } finally {
-      if (jarInputStream != null) {
-        try {
-          jarInputStream.close();
-        } catch (IOException ioex) {
-          logger.error("Exception attempting to close JAR input stream", ioex);
-        }
-      }
-    }
-
-    throw new ClassNotFoundException(className);
-  }
-
-  /**
-   * Check to see if the class implements the Function interface. If so, it will be registered with
-   * FunctionService. Also, if the functions's class was originally declared in a cache.xml file
-   * then any properties specified at that time will be reused when re-registering the function.
-   * 
-   * @param clazz Class to check for implementation of the Function class
-   * @return A collection of Objects that implement the Function interface.
-   */
-  private Collection<Function> getRegisterableFunctionsFromClass(Class<?> clazz) {
-    final List<Function> registerableFunctions = new ArrayList<Function>();
-
-    try {
-      if (Function.class.isAssignableFrom(clazz) && !Modifier.isAbstract(clazz.getModifiers())) {
-        boolean registerUninitializedFunction = true;
-        if (Declarable.class.isAssignableFrom(clazz)) {
-          try {
-            final List<Properties> propertiesList =
-                ((GemFireCacheImpl) CacheFactory.getAnyInstance())
-                    .getDeclarableProperties(clazz.getName());
-
-            if (!propertiesList.isEmpty()) {
-              registerUninitializedFunction = false;
-              // It's possible that the same function was declared multiple times in cache.xml
-              // with different properties. So, register the function using each set of
-              // properties.
-              for (Properties properties : propertiesList) {
-                @SuppressWarnings("unchecked")
-                Function function = newFunction((Class<Function>) clazz, true);
-                if (function != null) {
-                  ((Declarable) function).init(properties);
-                  if (function.getId() != null) {
-                    registerableFunctions.add(function);
-                  }
-                }
-              }
-            }
-          } catch (CacheClosedException ccex) {
-            // That's okay, it just means there were no properties to init the function with
-          }
-        }
-
-        if (registerUninitializedFunction) {
-          @SuppressWarnings("unchecked")
-          Function function = newFunction((Class<Function>) clazz, false);
-          if (function != null && function.getId() != null) {
-            registerableFunctions.add(function);
-          }
-        }
-      }
-    } catch (Exception ex) {
-      logger.error("Attempting to register function from JAR file: " + this.file.getAbsolutePath(),
-          ex);
-    }
-
-    return registerableFunctions;
-  }
-
-  private Class<?> forName(final String className, final Collection<ClassLoader> classLoaders)
-      throws ClassNotFoundException {
-    Class<?> clazz = null;
-
-    for (ClassLoader classLoader : classLoaders) {
-      try {
-        if (classLoader instanceof JarClassLoader) {
-          if (!((JarClassLoader) classLoader).alreadyScanned()) {
-            clazz = ((JarClassLoader) classLoader).loadClass(className, true, false);
-          }
-        } else {
-          clazz = Class.forName(className, true, classLoader);
-        }
-        if (clazz != null) {
-          return clazz;
-        }
-      } catch (SecurityException sex) {
-        // Continue to next ClassLoader
-      } catch (ClassNotFoundException cnfex) {
-        // Continue to next ClassLoader
-      }
-    }
-    throw new ClassNotFoundException(className);
-  }
-
-  private Function newFunction(final Class<Function> clazz, final boolean errorOnNoSuchMethod) {
-    try {
-      final Constructor<Function> constructor = clazz.getConstructor();
-      return constructor.newInstance();
-    } catch (NoSuchMethodException nsmex) {
-      if (errorOnNoSuchMethod) {
-        logger.error("Zero-arg constructor is required, but not found for class: {}",
-            clazz.getName(), nsmex);
-      } else {
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "Not registering function because it doesn't have a zero-arg constructor: {}",
-              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) {
-      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);
-    }
-    return null;
-  }
-
-  private byte[] getJarContent() throws IOException {
-    ChannelInputStream channelInputStream = new ChannelInputStream(this.fileLock.channel());
-    final ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
-    final byte[] bytes = new byte[4096];
-
-    int bytesRead;
-    while (((bytesRead = channelInputStream.read(bytes)) != -1)) {
-      byteOutStream.write(bytes, 0, bytesRead);
-    }
-    channelInputStream.close();
-    return byteOutStream.toByteArray();
-  }
-
-  public String getJarName() {
-    return this.jarName;
-  }
-
-  public String getFileName() {
-    return this.file.getName();
-  }
-
-  public String getFileCanonicalPath() throws IOException {
-    return this.file.getCanonicalPath();
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((this.jarName == null) ? 0 : this.jarName.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    JarClassLoader other = (JarClassLoader) obj;
-    if (this.jarName == null) {
-      if (other.jarName != null)
-        return false;
-    } else if (!this.jarName.equals(other.jarName))
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder(getClass().getName());
-    sb.append("@").append(System.identityHashCode(this)).append("{");
-    sb.append("jarName=").append(this.jarName);
-    sb.append(",file=").append(this.file.getAbsolutePath());
-    sb.append(",md5hash=").append(Arrays.toString(this.md5hash));
-    sb.append(",fileLock=").append(this.fileLock);
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * When a lock is acquired it is done so through an open file (FileInputStream, etc.). If for any
-   * reason that same file is used to open another input stream, when the second input stream is
-   * closed the file lock will not be held (although an OverlappingFileLock exception will be thrown
-   * if an attempt is made to acquire the lock again). To get around this problem, this class is
-   * used to wrap the original file channel used by the lock with an InputStream. When this class is
-   * instantiated a lock is obtained to prevent other threads from attempting to use the file
-   * channel at the same time. The file channel can then be read as with any other InputStream. When
-   * the input stream is closed, instead of closing the file channel, the lock is released instead.
-   * 
-   * This class is thread safe. However, multiple instances cannot be created by the same thread.
-   * The reason for this is that the lock will be obtained in all cases (it's reentrant), and then
-   * the channel position will be modified by both instances causing arbitrary values being returned
-   * from the read() method.
-   * 
-   * @since GemFire 7.0
-   */
-  private class ChannelInputStream extends InputStream {
-    private final FileChannel fileChannel;
-    private final ByteBuffer byteBuffer = ByteBuffer.allocateDirect(1);
-
-    ChannelInputStream(final FileChannel fileChannel) throws IOException {
-      channelLock.lock();
-      this.fileChannel = fileChannel;
-      this.fileChannel.position(0);
-    }
-
-    @Override
-    public int read() throws IOException {
-      this.byteBuffer.rewind();
-      if (this.fileChannel.read(this.byteBuffer) <= 0) {
-        return -1;
-      }
-      this.byteBuffer.rewind();
-      return (this.byteBuffer.get() & 255);
-    }
-
-    @Override
-    public void close() {
-      channelLock.unlock();
-    }
-  }
-}