From 2409af9dcf238e1ad87080a389e05a696c41dc72 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 11 Mar 2014 22:39:17 -0700 Subject: SPARK-1064 This reopens PR 649 from incubator-spark against the new repo Author: Sandy Ryza Closes #102 from sryza/sandy-spark-1064 and squashes the following commits: 270e490 [Sandy Ryza] Handle different application classpath variables in different versions 88b04e0 [Sandy Ryza] SPARK-1064. Make it possible to run on YARN without bundling Hadoop jars in Spark assembly --- docs/building-with-maven.md | 6 +++ pom.xml | 46 ++++++++++++++++++++++ .../org/apache/spark/deploy/yarn/ClientBase.scala | 43 +++++++++++++++++++- 3 files changed, 94 insertions(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index d3bc34e68b..730a6e7932 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -88,3 +88,9 @@ Running only java 8 tests and nothing else. Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spite of -DskipTests. For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. + +## Packaging without Hadoop dependencies for deployment on YARN ## + +The assembly jar produced by "mvn package" will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The "hadoop-provided" profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. + + diff --git a/pom.xml b/pom.xml index 986626f029..d2c36dc260 100644 --- a/pom.xml +++ b/pom.xml @@ -807,5 +807,51 @@ + + + + hadoop-provided + + false + + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + org.apache.hadoop + hadoop-yarn-client + provided + + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + + + org.apache.zookeeper + zookeeper + provided + + + + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 4b6c7db836..74de4293d9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -29,8 +29,10 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.mapred.Master +import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ @@ -379,9 +381,48 @@ object ClientBase { // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { - for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + val classpathEntries = Option(conf.getStrings( + YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( + getDefaultYarnApplicationClasspath()) + for (c <- classpathEntries) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) } + + val mrClasspathEntries = Option(conf.getStrings( + "mapreduce.application.classpath")).getOrElse( + getDefaultMRApplicationClasspath()) + if (mrClasspathEntries != null) { + for (c <- mrClasspathEntries) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + } + + def getDefaultYarnApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") + field.get(null).asInstanceOf[Array[String]] + } catch { + case err: NoSuchFieldError => null + } + } + + /** + * In Hadoop 0.23, the MR application classpath comes with the YARN application + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * So we need to use reflection to retrieve it. + */ + def getDefaultMRApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") + if (field.getType == classOf[String]) { + StringUtils.getStrings(field.get(null).asInstanceOf[String]) + } else { + field.get(null).asInstanceOf[Array[String]] + } + } catch { + case err: NoSuchFieldError => null + } } def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { -- cgit v1.2.3