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 ji...@apache.org on 2014/08/27 19:37:06 UTC

[06/29] git commit: HADOOP-10893. isolated classloader on the client side. Contributed by Sangjin Lee

HADOOP-10893. isolated classloader on the client side. Contributed by Sangjin Lee


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1619604 13f79535-47bb-0310-9956-ffa450edef68


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

Branch: refs/heads/HDFS-6584
Commit: da4ba50269254456650c08c739f2b394d1182ee4
Parents: cbbb899
Author: Jason Darrell Lowe <jl...@apache.org>
Authored: Thu Aug 21 21:38:16 2014 +0000
Committer: Jason Darrell Lowe <jl...@apache.org>
Committed: Thu Aug 21 21:38:16 2014 +0000

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../dev-support/findbugsExcludeFile.xml         |   5 +
 .../src/main/bin/hadoop-config.cmd              |  10 +-
 .../src/main/bin/hadoop-functions.sh            |  21 +-
 .../hadoop-common/src/main/bin/hadoop.cmd       |  20 ++
 .../hadoop-common/src/main/conf/hadoop-env.sh   |  11 +
 .../hadoop/util/ApplicationClassLoader.java     | 219 +++++++++++++++++++
 .../java/org/apache/hadoop/util/RunJar.java     | 115 ++++++++--
 .../apache/hadoop/util/ClassLoaderCheck.java    |  33 +++
 .../hadoop/util/ClassLoaderCheckMain.java       |  34 +++
 .../hadoop/util/ClassLoaderCheckSecond.java     |  24 ++
 .../hadoop/util/ClassLoaderCheckThird.java      |  24 ++
 .../hadoop/util/TestApplicationClassLoader.java | 136 ++++++++++++
 .../java/org/apache/hadoop/util/TestRunJar.java |  66 +++++-
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |   3 +-
 .../hadoop/mapreduce/v2/util/TestMRApps.java    |   5 +-
 .../src/main/resources/mapred-default.xml       |  14 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |   5 +-
 .../dev-support/findbugs-exclude.xml            |   7 +
 .../yarn/util/ApplicationClassLoader.java       | 170 +-------------
 .../yarn/util/TestApplicationClassLoader.java   | 136 ------------
 21 files changed, 714 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 6c20271..c880e11 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -383,6 +383,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)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index e0b2171..1469034 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -108,6 +108,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
      -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
index 3ea576c..d8da5b1 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.cmd
@@ -282,10 +282,12 @@ if not "%HADOOP_MAPRED_HOME%\%MAPRED_DIR%" == "%HADOOP_YARN_HOME%\%YARN_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%;
+    )
   )
 )
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index 646c11e..f2437fa 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -450,7 +450,8 @@ function hadoop_add_to_classpath_mapred
 function hadoop_add_to_classpath_userpath
 {
   # Add the user-specified HADOOP_CLASSPATH to the
-  # official CLASSPATH env var.
+  # official CLASSPATH env var if HADOOP_USE_CLIENT_CLASSLOADER
+  # is not set.
   # Add it first or last depending on if user has
   # set env-var HADOOP_USER_CLASSPATH_FIRST
   # we'll also dedupe it, because we're cool like that.
@@ -469,14 +470,16 @@ function hadoop_add_to_classpath_userpath
     done
     let j=c-1
     
-    if [[ -z "${HADOOP_USER_CLASSPATH_FIRST}" ]]; then
-      for ((i=j; i>=0; i--)); do
-        hadoop_add_classpath "${array[$i]}" before
-      done
-    else
-      for ((i=0; i<=j; i++)); do
-        hadoop_add_classpath "${array[$i]}" after
-      done
+    if [[ -z "${HADOOP_USE_CLIENT_CLASSLOADER}" ]]; then
+      if [[ -z "${HADOOP_USER_CLASSPATH_FIRST}" ]]; then
+        for ((i=j; i>=0; i--)); do
+          hadoop_add_classpath "${array[$i]}" before
+        done
+      else
+        for ((i=0; i<=j; i++)); do
+          hadoop_add_classpath "${array[$i]}" after
+        done
+      fi
     fi
   fi
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd b/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
index 04a302c..f9cfe14 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
@@ -29,6 +29,26 @@
 @rem                                    by doing
 @rem                                    export HADOOP_USER_CLASSPATH_FIRST=true
 @rem
+@rem   HADOOP_USE_CLIENT_CLASSLOADER    When defined, HADOOP_CLASSPATH and the
+@rem                                    jar as the hadoop jar argument are
+@rem                                    handled by a separate isolated client
+@rem                                    classloader. If it is set,
+@rem                                    HADOOP_USER_CLASSPATH_FIRST is
+@rem                                    ignored. Can be defined by doing
+@rem                                    export HADOOP_USE_CLIENT_CLASSLOADER=true
+@rem
+@rem   HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES
+@rem                                    When defined, it overrides the default
+@rem                                    definition of system classes for the
+@rem                                    client classloader when
+@rem                                    HADOOP_USE_CLIENT_CLASSLOADER is
+@rem                                    enabled. Names ending in '.' (period)
+@rem                                    are treated as package names, and names
+@rem                                    starting with a '-' are treated as
+@rem                                    negative matches. For example,
+@rem                                    export HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES="-org.apache.hadoop.UserClass,java.,javax.,org.apache.hadoop."
+
+@rem
 @rem   HADOOP_HEAPSIZE  The maximum amount of heap to use, in MB.
 @rem                    Default is 1000.
 @rem

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
index f50e412..eda47c9 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
@@ -111,6 +111,17 @@ esac
 # Should HADOOP_USER_CLASSPATH be first in the official CLASSPATH?
 # export HADOOP_USER_CLASSPATH_FIRST="yes"
 
+# If HADOOP_USE_CLIENT_CLASSLOADER is set, HADOOP_CLASSPATH along with the main
+# jar are handled 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 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."
+
 ###
 # Options for remote shell connectivity
 ###

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
new file mode 100644
index 0000000..5dda10f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index 08b4fd1..75b43b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -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);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java
new file mode 100644
index 0000000..aa2cc0e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheck.java
@@ -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");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java
new file mode 100644
index 0000000..bb14ac9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckMain.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java
new file mode 100644
index 0000000..45601bd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckSecond.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java
new file mode 100644
index 0000000..dd4c0c4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/ClassLoaderCheckThird.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java
new file mode 100644
index 0000000..5d0e131
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestApplicationClassLoader.java
@@ -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;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
index 8903fca..9e27968 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
index 423b842..3bd8414 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
@@ -34,6 +34,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -56,6 +57,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.util.ApplicationClassLoader;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
@@ -67,7 +69,6 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.util.ApplicationClassLoader;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.RollingFileAppender;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
index 2e0423f..02a59e7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.util.ApplicationClassLoader;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -58,7 +59,6 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.util.ApplicationClassLoader;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -514,7 +514,8 @@ public class TestMRApps {
   @Test
   public void testSystemClasses() {
     final List<String> systemClasses =
-        Arrays.asList(MRApps.getSystemClasses(new Configuration()));
+        Arrays.asList(StringUtils.getTrimmedStrings(
+        ApplicationClassLoader.DEFAULT_SYSTEM_CLASSES));
     for (String defaultXml : DEFAULT_XMLS) {
       assertTrue(defaultXml + " must be system resource",
           ApplicationClassLoader.isSystemClass(defaultXml, systemClasses));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index b2503c7..802ffa1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1227,13 +1227,13 @@
 
 <property>
    <name>mapreduce.job.classloader.system.classes</name>
-   <value>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</value>
-  <description>A comma-separated list of classes that should be loaded from the
-    system classpath, not the user-supplied JARs, when mapreduce.job.classloader
-    is enabled. Names ending in '.' (period) are treated as package names,
-    and names starting with a '-' are treated as negative matches.
+   <value></value>
+  <description>Used to override the default definition of the system classes for
+    the job classloader. The system classes are a comma-separated list of
+    classes that should be loaded from the system classpath, not the
+    user-supplied JARs, when mapreduce.job.classloader is enabled. Names ending
+    in '.' (period) are treated as package names, and names starting with a '-'
+    are treated as negative matches.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index 6b47554..3215399 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -84,13 +84,13 @@ import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ApplicationClassLoader;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.util.ApplicationClassLoader;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
@@ -242,8 +242,7 @@ public class TestMRJobs {
       // to test AM loading user classes such as output format class, we want
       // to blacklist them from the system classes (they need to be prepended
       // as the first match wins)
-      String systemClasses =
-          sleepConf.get(MRJobConfig.MAPREDUCE_JOB_CLASSLOADER_SYSTEM_CLASSES);
+      String systemClasses = ApplicationClassLoader.DEFAULT_SYSTEM_CLASSES;
       // exclude the custom classes from system classes
       systemClasses = "-" + CustomOutputFormat.class.getName() + ",-" +
           CustomSpeculator.class.getName() + "," +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 6609a26..b1dfb1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -344,4 +344,11 @@
       <Class name="org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider"/>
       <Bug pattern="DC_DOUBLECHECK" />
   </Match>
+
+  <!-- ApplicationClassLoader is deprecated and moved to hadoop-common; ignore
+       warning on the identical name as it should be removed later -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.util.ApplicationClassLoader"/>
+    <Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS"/>
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ApplicationClassLoader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ApplicationClassLoader.java
index 63dc5b7..ee9ad4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ApplicationClassLoader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ApplicationClassLoader.java
@@ -18,180 +18,30 @@
 
 package org.apache.hadoop.yarn.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.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;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Splitter;
-
 /**
- * A {@link URLClassLoader} for YARN application isolation. Classes from
- * the application JARs are loaded in preference to the parent loader.
+ * This type has been deprecated in favor of
+ * {@link org.apache.hadoop.util.ApplicationClassLoader}. All new uses of
+ * ApplicationClassLoader should use that type instead.
  */
 @Public
 @Unstable
-public class ApplicationClassLoader extends URLClassLoader {
-
-  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 ClassLoader parent;
-  private List<String> systemClasses;
-
+@Deprecated
+public class ApplicationClassLoader extends
+    org.apache.hadoop.util.ApplicationClassLoader {
   public ApplicationClassLoader(URL[] urls, ClassLoader parent,
       List<String> systemClasses) {
-    super(urls, parent);
-    this.parent = parent;
-    if (parent == null) {
-      throw new IllegalArgumentException("No parent classloader!");
-    }
-    this.systemClasses = systemClasses;
+    super(urls, parent, systemClasses);
   }
-  
+
   public ApplicationClassLoader(String classpath, ClassLoader parent,
       List<String> systemClasses) throws MalformedURLException {
-    this(constructUrlsFromClasspath(classpath), parent, systemClasses);
-  }
-  
-  @VisibleForTesting
-  static URL[] constructUrlsFromClasspath(String classpath)
-      throws MalformedURLException {
-    List<URL> urls = new ArrayList<URL>();
-    for (String element : Splitter.on(File.pathSeparator).split(classpath)) {
-      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;
-  }
-
-  @VisibleForTesting
-  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;
+    super(classpath, parent, systemClasses);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ba502/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApplicationClassLoader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApplicationClassLoader.java
deleted file mode 100644
index bb4b28c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApplicationClassLoader.java
+++ /dev/null
@@ -1,136 +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.hadoop.yarn.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.apache.hadoop.yarn.util.ApplicationClassLoader.constructUrlsFromClasspath;
-import static org.apache.hadoop.yarn.util.ApplicationClassLoader.isSystemClass;
-
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
-
-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;
-
-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;
-  }
-  
-}