You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by jl...@apache.org on 2014/08/21 23:41:24 UTC

svn commit: r1619605 - in /hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common: ./ dev-support/ src/main/bin/ src/main/java/org/apache/hadoop/util/ src/test/java/org/apache/hadoop/util/

Author: jlowe
Date: Thu Aug 21 21:41:24 2014
New Revision: 1619605

URL: http://svn.apache.org/r1619605
Log:
HADOOP-10893. isolated classloader on the client side. Contributed by Sangjin Lee

Added:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java   (with props)
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java   (with props)
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java   (with props)
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java   (with props)
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java   (with props)
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java   (with props)
Modified:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt Thu Aug 21 21:41:24 2014
@@ -8,6 +8,9 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-10433. Key Management Server based on KeyProvider API. (tucu)
 
+    HADOOP-10893. isolated classloader on the client side (Sangjin Lee via
+    jlowe)
+
   IMPROVEMENTS
 
     HADOOP-10808. Remove unused native code for munlock. (cnauroth)

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml Thu Aug 21 21:41:24 2014
@@ -111,6 +111,11 @@
        <Method name="driver" />
        <Bug pattern="DM_EXIT" />
      </Match>
+     <Match>
+       <Class name="org.apache.hadoop.util.RunJar" />
+       <Method name="run" />
+       <Bug pattern="DM_EXIT" />
+     </Match>
      <!--
        We need to cast objects between old and new api objects
      -->

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd Thu Aug 21 21:41:24 2014
@@ -282,10 +282,12 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR
 @rem
 
 if defined HADOOP_CLASSPATH (
-  if defined HADOOP_USER_CLASSPATH_FIRST (
-    set CLASSPATH=%HADOOP_CLASSPATH%;%CLASSPATH%;
-  ) else (
-    set CLASSPATH=%CLASSPATH%;%HADOOP_CLASSPATH%;
+  if not defined HADOOP_USE_CLIENT_CLASSLOADER (
+    if defined HADOOP_USER_CLASSPATH_FIRST (
+      set CLASSPATH=%HADOOP_CLASSPATH%;%CLASSPATH%;
+    ) else (
+      set CLASSPATH=%CLASSPATH%;%HADOOP_CLASSPATH%;
+    )
   )
 )
 

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh Thu Aug 21 21:41:24 2014
@@ -29,6 +29,23 @@
 #                                    by doing 
 #                                    export HADOOP_USER_CLASSPATH_FIRST=true
 #
+#   HADOOP_USE_CLIENT_CLASSLOADER    When defined, HADOOP_CLASSPATH and the jar
+#                                    as the hadoop jar argument are handled by
+#                                    by a separate isolated client classloader.
+#                                    If it is set, HADOOP_USER_CLASSPATH_FIRST
+#                                    is ignored. Can be defined by doing
+#                                    export HADOOP_USE_CLIENT_CLASSLOADER=true
+#
+#   HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES
+#                                    When defined, it overrides the default
+#                                    definition of system classes for the client
+#                                    classloader when
+#                                    HADOOP_USE_CLIENT_CLASSLOADER is enabled.
+#                                    Names ending in '.' (period) are treated as
+#                                    package names, and names starting with a
+#                                    '-' are treated as negative matches.
+#                                    For example,
+#                                    export HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES="-org.apache.hadoop.UserClass,java.,javax.,org.apache.hadoop."
 
 this="${BASH_SOURCE-$0}"
 common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
@@ -282,7 +299,9 @@ fi
 # Add the user-specified CLASSPATH via HADOOP_CLASSPATH
 # Add it first or last depending on if user has
 # set env-var HADOOP_USER_CLASSPATH_FIRST
-if [ "$HADOOP_CLASSPATH" != "" ]; then
+# if the user set HADOOP_USE_CLIENT_CLASSLOADER, HADOOP_CLASSPATH is not added
+# to the classpath
+if [[ ( "$HADOOP_CLASSPATH" != "" ) && ( "$HADOOP_USE_CLIENT_CLASSLOADER" = "" ) ]]; then
   # Prefix it if its to be preceded
   if [ "$HADOOP_USER_CLASSPATH_FIRST" != "" ]; then
     CLASSPATH=${HADOOP_CLASSPATH}:${CLASSPATH}

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,219 @@
+/**
+ * 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.hadoop.util;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * A {@link URLClassLoader} for application isolation. Classes from the
+ * application JARs are loaded in preference to the parent loader.
+ */
+@Public
+@Unstable
+public class ApplicationClassLoader extends URLClassLoader {
+  /**
+   * Default value of the system classes if the user did not override them.
+   * JDK classes, hadoop classes and resources, and some select third-party
+   * classes are considered system classes, and are not loaded by the
+   * application classloader.
+   */
+  public static final String DEFAULT_SYSTEM_CLASSES =
+        "java.," +
+        "javax.," +
+        "org.w3c.dom.," +
+        "org.xml.sax.," +
+        "org.apache.commons.logging.," +
+        "org.apache.log4j.," +
+        "org.apache.hadoop.," +
+        "core-default.xml," +
+        "hdfs-default.xml," +
+        "mapred-default.xml," +
+        "yarn-default.xml";
+
+  private static final Log LOG =
+    LogFactory.getLog(ApplicationClassLoader.class.getName());
+
+  private static final FilenameFilter JAR_FILENAME_FILTER =
+    new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return name.endsWith(".jar") || name.endsWith(".JAR");
+      }
+  };
+
+  private final ClassLoader parent;
+  private final List<String> systemClasses;
+
+  public ApplicationClassLoader(URL[] urls, ClassLoader parent,
+      List<String> systemClasses) {
+    super(urls, parent);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("urls: " + Arrays.toString(urls));
+      LOG.debug("system classes: " + systemClasses);
+    }
+    this.parent = parent;
+    if (parent == null) {
+      throw new IllegalArgumentException("No parent classloader!");
+    }
+    // if the caller-specified system classes are null or empty, use the default
+    this.systemClasses = (systemClasses == null || systemClasses.isEmpty()) ?
+        Arrays.asList(StringUtils.getTrimmedStrings(DEFAULT_SYSTEM_CLASSES)) :
+        systemClasses;
+    LOG.info("system classes: " + this.systemClasses);
+  }
+
+  public ApplicationClassLoader(String classpath, ClassLoader parent,
+      List<String> systemClasses) throws MalformedURLException {
+    this(constructUrlsFromClasspath(classpath), parent, systemClasses);
+  }
+
+  static URL[] constructUrlsFromClasspath(String classpath)
+      throws MalformedURLException {
+    List<URL> urls = new ArrayList<URL>();
+    for (String element : classpath.split(File.pathSeparator)) {
+      if (element.endsWith("/*")) {
+        String dir = element.substring(0, element.length() - 1);
+        File[] files = new File(dir).listFiles(JAR_FILENAME_FILTER);
+        if (files != null) {
+          for (File file : files) {
+            urls.add(file.toURI().toURL());
+          }
+        }
+      } else {
+        File file = new File(element);
+        if (file.exists()) {
+          urls.add(new File(element).toURI().toURL());
+        }
+      }
+    }
+    return urls.toArray(new URL[urls.size()]);
+  }
+
+  @Override
+  public URL getResource(String name) {
+    URL url = null;
+    
+    if (!isSystemClass(name, systemClasses)) {
+      url= findResource(name);
+      if (url == null && name.startsWith("/")) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Remove leading / off " + name);
+        }
+        url= findResource(name.substring(1));
+      }
+    }
+
+    if (url == null) {
+      url= parent.getResource(name);
+    }
+
+    if (url != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getResource("+name+")=" + url);
+      }
+    }
+    
+    return url;
+  }
+
+  @Override
+  public Class<?> loadClass(String name) throws ClassNotFoundException {
+    return this.loadClass(name, false);
+  }
+
+  @Override
+  protected synchronized Class<?> loadClass(String name, boolean resolve)
+      throws ClassNotFoundException {
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loading class: " + name);
+    }
+
+    Class<?> c = findLoadedClass(name);
+    ClassNotFoundException ex = null;
+
+    if (c == null && !isSystemClass(name, systemClasses)) {
+      // Try to load class from this classloader's URLs. Note that this is like
+      // the servlet spec, not the usual Java 2 behaviour where we ask the
+      // parent to attempt to load first.
+      try {
+        c = findClass(name);
+        if (LOG.isDebugEnabled() && c != null) {
+          LOG.debug("Loaded class: " + name + " ");
+        }
+      } catch (ClassNotFoundException e) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(e);
+        }
+        ex = e;
+      }
+    }
+
+    if (c == null) { // try parent
+      c = parent.loadClass(name);
+      if (LOG.isDebugEnabled() && c != null) {
+        LOG.debug("Loaded class from parent: " + name + " ");
+      }
+    }
+
+    if (c == null) {
+      throw ex != null ? ex : new ClassNotFoundException(name);
+    }
+
+    if (resolve) {
+      resolveClass(c);
+    }
+
+    return c;
+  }
+
+  public static boolean isSystemClass(String name, List<String> systemClasses) {
+    if (systemClasses != null) {
+      String canonicalName = name.replace('/', '.');
+      while (canonicalName.startsWith(".")) {
+        canonicalName=canonicalName.substring(1);
+      }
+      for (String c : systemClasses) {
+        boolean result = true;
+        if (c.startsWith("-")) {
+          c = c.substring(1);
+          result = false;
+        }
+        if (c.endsWith(".") && canonicalName.startsWith(c)) {
+          return result;
+        } else if (canonicalName.equals(c)) {
+          return result;
+        }
+      }
+    }
+    return false;
+  }
+}
\ No newline at end of file

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java Thu Aug 21 21:41:24 2014
@@ -18,23 +18,25 @@
 
 package org.apache.hadoop.util;
 
-import java.lang.reflect.Array;
-import java.lang.reflect.Method;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.net.URLClassLoader;
+import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.File;
-import java.util.regex.Pattern;
-import java.util.Arrays;
+import java.lang.reflect.Array;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Enumeration;
-import java.util.jar.JarFile;
+import java.util.List;
 import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
 import java.util.jar.Manifest;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -56,6 +58,21 @@ public class RunJar {
   public static final int SHUTDOWN_HOOK_PRIORITY = 10;
 
   /**
+   * Environment key for using the client classloader.
+   */
+  public static final String HADOOP_USE_CLIENT_CLASSLOADER =
+      "HADOOP_USE_CLIENT_CLASSLOADER";
+  /**
+   * Environment key for the (user-provided) hadoop classpath.
+   */
+  public static final String HADOOP_CLASSPATH = "HADOOP_CLASSPATH";
+  /**
+   * Environment key for the system classes.
+   */
+  public static final String HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES =
+      "HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES";
+
+  /**
    * Unpack a jar file into a directory.
    *
    * This version unpacks all files inside the jar regardless of filename.
@@ -116,6 +133,10 @@ public class RunJar {
   /** Run a Hadoop job jar.  If the main class is not in the jar's manifest,
    * then it must be provided on the command line. */
   public static void main(String[] args) throws Throwable {
+    new RunJar().run(args);
+  }
+
+  public void run(String[] args) throws Throwable {
     String usage = "RunJar jarFile [mainClass] args...";
 
     if (args.length < 1) {
@@ -187,19 +208,7 @@ public class RunJar {
 
     unJar(file, workDir);
 
-    ArrayList<URL> classPath = new ArrayList<URL>();
-    classPath.add(new File(workDir+"/").toURI().toURL());
-    classPath.add(file.toURI().toURL());
-    classPath.add(new File(workDir, "classes/").toURI().toURL());
-    File[] libs = new File(workDir, "lib").listFiles();
-    if (libs != null) {
-      for (int i = 0; i < libs.length; i++) {
-        classPath.add(libs[i].toURI().toURL());
-      }
-    }
-    
-    ClassLoader loader =
-      new URLClassLoader(classPath.toArray(new URL[0]));
+    ClassLoader loader = createClassLoader(file, workDir);
 
     Thread.currentThread().setContextClassLoader(loader);
     Class<?> mainClass = Class.forName(mainClassName, true, loader);
@@ -214,5 +223,65 @@ public class RunJar {
       throw e.getTargetException();
     }
   }
-  
+
+  /**
+   * Creates a classloader based on the environment that was specified by the
+   * user. If HADOOP_USE_CLIENT_CLASSLOADER is specified, it creates an
+   * application classloader that provides the isolation of the user class space
+   * from the hadoop classes and their dependencies. It forms a class space for
+   * the user jar as well as the HADOOP_CLASSPATH. Otherwise, it creates a
+   * classloader that simply adds the user jar to the classpath.
+   */
+  private ClassLoader createClassLoader(File file, final File workDir)
+      throws MalformedURLException {
+    ClassLoader loader;
+    // see if the client classloader is enabled
+    if (useClientClassLoader()) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(workDir+"/").
+          append(File.pathSeparator).append(file).
+          append(File.pathSeparator).append(workDir+"/classes/").
+          append(File.pathSeparator).append(workDir+"/lib/*");
+      // HADOOP_CLASSPATH is added to the client classpath
+      String hadoopClasspath = getHadoopClasspath();
+      if (hadoopClasspath != null && !hadoopClasspath.isEmpty()) {
+        sb.append(File.pathSeparator).append(hadoopClasspath);
+      }
+      String clientClasspath = sb.toString();
+      // get the system classes
+      String systemClasses = getSystemClasses();
+      List<String> systemClassesList = systemClasses == null ?
+          null :
+          Arrays.asList(StringUtils.getTrimmedStrings(systemClasses));
+      // create an application classloader that isolates the user classes
+      loader = new ApplicationClassLoader(clientClasspath,
+          getClass().getClassLoader(), systemClassesList);
+    } else {
+      List<URL> classPath = new ArrayList<URL>();
+      classPath.add(new File(workDir+"/").toURI().toURL());
+      classPath.add(file.toURI().toURL());
+      classPath.add(new File(workDir, "classes/").toURI().toURL());
+      File[] libs = new File(workDir, "lib").listFiles();
+      if (libs != null) {
+        for (int i = 0; i < libs.length; i++) {
+          classPath.add(libs[i].toURI().toURL());
+        }
+      }
+      // create a normal parent-delegating classloader
+      loader = new URLClassLoader(classPath.toArray(new URL[0]));
+    }
+    return loader;
+  }
+
+  boolean useClientClassLoader() {
+    return Boolean.parseBoolean(System.getenv(HADOOP_USE_CLIENT_CLASSLOADER));
+  }
+
+  String getHadoopClasspath() {
+    return System.getenv(HADOOP_CLASSPATH);
+  }
+
+  String getSystemClasses() {
+    return System.getenv(HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES);
+  }
 }

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.util;
+
+public class ClassLoaderCheck {
+  /**
+   * Verifies the class is loaded by the right classloader.
+   */
+  public static void checkClassLoader(Class cls,
+      boolean shouldBeLoadedByAppClassLoader) {
+    boolean loadedByAppClassLoader =
+        cls.getClassLoader() instanceof ApplicationClassLoader;
+    if ((shouldBeLoadedByAppClassLoader && !loadedByAppClassLoader) ||
+        (!shouldBeLoadedByAppClassLoader && loadedByAppClassLoader)) {
+      throw new RuntimeException("incorrect classloader used");
+    }
+  }
+}

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,34 @@
+/**
+ * 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.hadoop.util;
+
+/**
+ * Test class used by {@link TestRunJar} to verify that it is loaded by the
+ * {@link ApplicationClassLoader}.
+ */
+public class ClassLoaderCheckMain {
+  public static void main(String[] args) {
+    // ClassLoaderCheckMain should be loaded by the application classloader
+    ClassLoaderCheck.checkClassLoader(ClassLoaderCheckMain.class, true);
+    // ClassLoaderCheckSecond should NOT be loaded by the application
+    // classloader
+    ClassLoaderCheck.checkClassLoader(ClassLoaderCheckSecond.class, false);
+    // ClassLoaderCheckThird should be loaded by the application classloader
+    ClassLoaderCheck.checkClassLoader(ClassLoaderCheckThird.class, true);
+  }
+}
\ No newline at end of file

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,24 @@
+/**
+ * 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.hadoop.util;
+
+/**
+ * A class {@link ClassLoaderCheckMain} depends on that should be loaded by the
+ * system classloader.
+ */
+public class ClassLoaderCheckSecond {}
\ No newline at end of file

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,24 @@
+/**
+ * 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.hadoop.util;
+
+/**
+ * A class {@link ClassLoaderCheckMain} depends on that should be loaded by the
+ * application classloader.
+ */
+public class ClassLoaderCheckThird {}
\ No newline at end of file

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java?rev=1619605&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java Thu Aug 21 21:41:24 2014
@@ -0,0 +1,136 @@
+/**
+ * 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.hadoop.util;
+
+import static org.apache.hadoop.util.ApplicationClassLoader.constructUrlsFromClasspath;
+import static org.apache.hadoop.util.ApplicationClassLoader.isSystemClass;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.List;
+import java.util.jar.JarOutputStream;
+import java.util.zip.ZipEntry;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+
+public class TestApplicationClassLoader {
+  
+  private static File testDir = new File(System.getProperty("test.build.data",
+          System.getProperty("java.io.tmpdir")), "appclassloader");
+  
+  @Before
+  public void setUp() {
+    FileUtil.fullyDelete(testDir);
+    testDir.mkdirs();
+  }
+
+  @Test
+  public void testConstructUrlsFromClasspath() throws Exception {
+    File file = new File(testDir, "file");
+    assertTrue("Create file", file.createNewFile());
+
+    File dir = new File(testDir, "dir");
+    assertTrue("Make dir", dir.mkdir());
+
+    File jarsDir = new File(testDir, "jarsdir");
+    assertTrue("Make jarsDir", jarsDir.mkdir());
+    File nonJarFile = new File(jarsDir, "nonjar");
+    assertTrue("Create non-jar file", nonJarFile.createNewFile());
+    File jarFile = new File(jarsDir, "a.jar");
+    assertTrue("Create jar file", jarFile.createNewFile());
+
+    File nofile = new File(testDir, "nofile");
+    // don't create nofile
+
+    StringBuilder cp = new StringBuilder();
+    cp.append(file.getAbsolutePath()).append(File.pathSeparator)
+      .append(dir.getAbsolutePath()).append(File.pathSeparator)
+      .append(jarsDir.getAbsolutePath() + "/*").append(File.pathSeparator)
+      .append(nofile.getAbsolutePath()).append(File.pathSeparator)
+      .append(nofile.getAbsolutePath() + "/*").append(File.pathSeparator);
+    
+    URL[] urls = constructUrlsFromClasspath(cp.toString());
+    
+    assertEquals(3, urls.length);
+    assertEquals(file.toURI().toURL(), urls[0]);
+    assertEquals(dir.toURI().toURL(), urls[1]);
+    assertEquals(jarFile.toURI().toURL(), urls[2]);
+    // nofile should be ignored
+  }
+  
+  @Test
+  public void testIsSystemClass() {
+    assertFalse(isSystemClass("org.example.Foo", null));
+    assertTrue(isSystemClass("org.example.Foo", classes("org.example.Foo")));
+    assertTrue(isSystemClass("/org.example.Foo", classes("org.example.Foo")));
+    assertTrue(isSystemClass("org.example.Foo", classes("org.example.")));
+    assertTrue(isSystemClass("net.example.Foo",
+        classes("org.example.,net.example.")));
+    assertFalse(isSystemClass("org.example.Foo",
+        classes("-org.example.Foo,org.example.")));
+    assertTrue(isSystemClass("org.example.Bar",
+        classes("-org.example.Foo.,org.example.")));
+  }
+  
+  private List<String> classes(String classes) {
+    return Lists.newArrayList(Splitter.on(',').split(classes));
+  }
+  
+  @Test
+  public void testGetResource() throws IOException {
+    URL testJar = makeTestJar().toURI().toURL();
+    
+    ClassLoader currentClassLoader = getClass().getClassLoader();
+    ClassLoader appClassloader = new ApplicationClassLoader(
+        new URL[] { testJar }, currentClassLoader, null);
+
+    assertNull("Resource should be null for current classloader",
+        currentClassLoader.getResourceAsStream("resource.txt"));
+
+    InputStream in = appClassloader.getResourceAsStream("resource.txt");
+    assertNotNull("Resource should not be null for app classloader", in);
+    assertEquals("hello", IOUtils.toString(in));
+  }
+  
+  private File makeTestJar() throws IOException {
+    File jarFile = new File(testDir, "test.jar");
+    JarOutputStream out = new JarOutputStream(new FileOutputStream(jarFile));
+    ZipEntry entry = new ZipEntry("resource.txt");
+    out.putNextEntry(entry);
+    out.write("hello".getBytes());
+    out.closeEntry();
+    out.close();
+    return jarFile;
+  }
+  
+}

Propchange: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java?rev=1619605&r1=1619604&r2=1619605&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java Thu Aug 21 21:41:24 2014
@@ -17,23 +17,30 @@
  */
 package org.apache.hadoop.util;
 
-import junit.framework.TestCase;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.BufferedInputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.jar.JarOutputStream;
 import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.FileUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.hadoop.fs.FileUtil;
 
 public class TestRunJar extends TestCase {
   private File TEST_ROOT_DIR;
 
   private static final String TEST_JAR_NAME="test-runjar.jar";
+  private static final String TEST_JAR_2_NAME = "test-runjar2.jar";
 
   @Override
   @Before
@@ -107,4 +114,59 @@ public class TestRunJar extends TestCase
                new File(unjarDir, "foobaz.txt").exists());
 
   }
+
+  /**
+   * Tests the client classloader to verify the main class and its dependent
+   * class are loaded correctly by the application classloader, and others are
+   * loaded by the system classloader.
+   */
+  @Test
+  public void testClientClassLoader() throws Throwable {
+    RunJar runJar = spy(new RunJar());
+    // enable the client classloader
+    when(runJar.useClientClassLoader()).thenReturn(true);
+    // set the system classes and blacklist the test main class and the test
+    // third class so they can be loaded by the application classloader
+    String mainCls = ClassLoaderCheckMain.class.getName();
+    String thirdCls = ClassLoaderCheckThird.class.getName();
+    String systemClasses = "-" + mainCls + "," +
+        "-" + thirdCls + "," +
+        ApplicationClassLoader.DEFAULT_SYSTEM_CLASSES;
+    when(runJar.getSystemClasses()).thenReturn(systemClasses);
+
+    // create the test jar
+    File testJar = makeClassLoaderTestJar(mainCls, thirdCls);
+    // form the args
+    String[] args = new String[3];
+    args[0] = testJar.getAbsolutePath();
+    args[1] = mainCls;
+
+    // run RunJar
+    runJar.run(args);
+    // it should not throw an exception
+  }
+
+  private File makeClassLoaderTestJar(String... clsNames) throws IOException {
+    File jarFile = new File(TEST_ROOT_DIR, TEST_JAR_2_NAME);
+    JarOutputStream jstream =
+        new JarOutputStream(new FileOutputStream(jarFile));
+    for (String clsName: clsNames) {
+      String name = clsName.replace('.', '/') + ".class";
+      InputStream entryInputStream = this.getClass().getResourceAsStream(
+          "/" + name);
+      ZipEntry entry = new ZipEntry(name);
+      jstream.putNextEntry(entry);
+      BufferedInputStream bufInputStream = new BufferedInputStream(
+          entryInputStream, 2048);
+      int count;
+      byte[] data = new byte[2048];
+      while ((count = bufInputStream.read(data, 0, 2048)) != -1) {
+        jstream.write(data, 0, count);
+      }
+      jstream.closeEntry();
+    }
+    jstream.close();
+
+    return jarFile;
+  }
 }
\ No newline at end of file