aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorSandy Ryza <sandy@cloudera.com>2014-03-11 22:39:17 -0700
committerPatrick Wendell <pwendell@gmail.com>2014-03-11 22:39:17 -0700
commit2409af9dcf238e1ad87080a389e05a696c41dc72 (patch)
tree2c65a789a99b7e95389cf766140eecf55af99f31
parent16788a654246067fd966033b5dc9bc0d4c759b70 (diff)
downloadspark-2409af9dcf238e1ad87080a389e05a696c41dc72.tar.gz
spark-2409af9dcf238e1ad87080a389e05a696c41dc72.tar.bz2
spark-2409af9dcf238e1ad87080a389e05a696c41dc72.zip
SPARK-1064
This reopens PR 649 from incubator-spark against the new repo Author: Sandy Ryza <sandy@cloudera.com> 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
-rw-r--r--docs/building-with-maven.md6
-rw-r--r--pom.xml46
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala43
3 files changed, 94 insertions, 1 deletions
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 @@
</modules>
</profile>
+
+ <!-- Build without Hadoop dependencies that are included in some runtime environments. -->
+ <profile>
+ <id>hadoop-provided</id>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ </activation>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro-ipc</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+
</profiles>
</project>
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]) {