aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--docs/running-on-yarn.md28
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala248
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala11
-rw-r--r--yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala90
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala1
5 files changed, 274 insertions, 104 deletions
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index fecd8f2cc2..43b06df99d 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -95,10 +95,19 @@ Most of the configs are the same for Spark on YARN as for other deployment modes
The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc.
</td>
</tr>
+<tr>
+ <td><code>spark.yarn.jar</code></td>
+ <td>(none)</td>
+ <td>
+ The location of the Spark jar file, in case overriding the default location is desired.
+ By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be
+ in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't
+ need to be distributed each time an application runs. To point to a jar on HDFS, for example,
+ set this configuration to "hdfs:///some/path".
+ </td>
+</tr>
</table>
-By default, Spark on YARN will use a Spark jar installed locally, but the Spark JAR can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an application runs. To point to a JAR on HDFS, `export SPARK_JAR=hdfs:///some/path`.
-
# Launching Spark on YARN
Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster.
@@ -156,7 +165,20 @@ all environment variables used for launching each container. This process is use
classpath problems in particular. (Note that enabling this requires admin privileges on cluster
settings and a restart of all node managers. Thus, this is not applicable to hosted clusters).
-# Important Notes
+To use a custom log4j configuration for the application master or executors, there are two options:
+
+- upload a custom log4j.properties using spark-submit, by adding it to the "--files" list of files
+ to be uploaded with the application.
+- add "-Dlog4j.configuration=<location of configuration file>" to "spark.driver.extraJavaOptions"
+ (for the driver) or "spark.executor.extraJavaOptions" (for executors). Note that if using a file,
+ the "file:" protocol should be explicitly provided, and the file needs to exist locally on all
+ the nodes.
+
+Note that for the first option, both executors and the application master will share the same
+log4j configuration, which may cause issues when they run on the same node (e.g. trying to write
+to the same log file).
+
+# Important notes
- Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured.
- The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored.
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 8f22675999..556f493429 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
@@ -154,7 +154,7 @@ trait ClientBase extends Logging {
}
/** Copy the file into HDFS if needed. */
- private def copyRemoteFile(
+ private[yarn] def copyRemoteFile(
dstDir: Path,
originalPath: Path,
replication: Short,
@@ -213,10 +213,19 @@ trait ClientBase extends Logging {
val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
- Map(
- ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar,
- ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
- ).foreach { case(destName, _localPath) =>
+ val oldLog4jConf = Option(System.getenv("SPARK_LOG4J_CONF"))
+ if (oldLog4jConf.isDefined) {
+ logWarning(
+ "SPARK_LOG4J_CONF detected in the system environment. This variable has been " +
+ "deprecated. Please refer to the \"Launching Spark on YARN\" documentation " +
+ "for alternatives.")
+ }
+
+ List(
+ (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR),
+ (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR),
+ ("log4j.properties", oldLog4jConf.getOrElse(null), null)
+ ).foreach { case(destName, _localPath, confKey) =>
val localPath: String = if (_localPath != null) _localPath.trim() else ""
if (! localPath.isEmpty()) {
val localURI = new URI(localPath)
@@ -225,6 +234,8 @@ trait ClientBase extends Logging {
val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions)
distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
destName, statCache)
+ } else if (confKey != null) {
+ sparkConf.set(confKey, localPath)
}
}
}
@@ -246,6 +257,8 @@ trait ClientBase extends Logging {
if (addToClasspath) {
cachedSecondaryJarLinks += linkname
}
+ } else if (addToClasspath) {
+ cachedSecondaryJarLinks += file.trim()
}
}
}
@@ -265,14 +278,10 @@ trait ClientBase extends Logging {
val env = new HashMap[String, String]()
val extraCp = sparkConf.getOption("spark.driver.extraClassPath")
- val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
- ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp)
+ ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp)
env("SPARK_YARN_MODE") = "true"
env("SPARK_YARN_STAGING_DIR") = stagingDir
env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
- if (log4jConf != null) {
- env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf
- }
// Set the environment variables to be passed on to the executors.
distCacheMgr.setDistFilesEnv(env)
@@ -285,7 +294,6 @@ trait ClientBase extends Logging {
// Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
env("SPARK_YARN_USER_ENV") = userEnvs
}
-
env
}
@@ -310,6 +318,37 @@ trait ClientBase extends Logging {
logInfo("Setting up container launch context")
val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
amContainer.setLocalResources(localResources)
+
+ // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to
+ // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's
+ // SparkContext will not let that set spark* system properties, which is expected behavior for
+ // Yarn clients. So propagate it through the environment.
+ //
+ // Note that to warn the user about the deprecation in cluster mode, some code from
+ // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition
+ // described above).
+ if (args.amClass == classOf[ApplicationMaster].getName) {
+ sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
+ val warning =
+ s"""
+ |SPARK_JAVA_OPTS was detected (set to '$value').
+ |This is deprecated in Spark 1.0+.
+ |
+ |Please instead use:
+ | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
+ | - ./spark-submit with --driver-java-options to set -X options for a driver
+ | - spark.executor.extraJavaOptions to set -X options for executors
+ """.stripMargin
+ logWarning(warning)
+ for (proc <- Seq("driver", "executor")) {
+ val key = s"spark.$proc.extraJavaOptions"
+ if (sparkConf.contains(key)) {
+ throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.")
+ }
+ }
+ env("SPARK_JAVA_OPTS") = value
+ }
+ }
amContainer.setEnvironment(env)
val amMemory = calculateAMMemory(newApp)
@@ -341,30 +380,20 @@ trait ClientBase extends Logging {
javaOpts += "-XX:CMSIncrementalDutyCycle=10"
}
- // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility:
- sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
- sparkConf.set("spark.executor.extraJavaOptions", opts)
- sparkConf.set("spark.driver.extraJavaOptions", opts)
- }
-
+ // Forward the Spark configuration to the application master / executors.
// TODO: it might be nicer to pass these as an internal environment variable rather than
// as Java options, due to complications with string parsing of nested quotes.
- if (args.amClass == classOf[ExecutorLauncher].getName) {
- // If we are being launched in client mode, forward the spark-conf options
- // onto the executor launcher
- for ((k, v) <- sparkConf.getAll) {
- javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\""
- }
- } else {
- // If we are being launched in standalone mode, capture and forward any spark
- // system properties (e.g. set by spark-class).
- for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) {
- javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\""
- }
- sys.props.get("spark.driver.extraJavaOptions").foreach(opts => javaOpts += opts)
- sys.props.get("spark.driver.libraryPath").foreach(p => javaOpts += s"-Djava.library.path=$p")
+ for ((k, v) <- sparkConf.getAll) {
+ javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\""
+ }
+
+ if (args.amClass == classOf[ApplicationMaster].getName) {
+ sparkConf.getOption("spark.driver.extraJavaOptions")
+ .orElse(sys.env.get("SPARK_JAVA_OPTS"))
+ .foreach(opts => javaOpts += opts)
+ sparkConf.getOption("spark.driver.libraryPath")
+ .foreach(p => javaOpts += s"-Djava.library.path=$p")
}
- javaOpts += ClientBase.getLog4jConfiguration(localResources)
// Command for the ApplicationMaster
val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++
@@ -377,7 +406,10 @@ trait ClientBase extends Logging {
"1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout",
"2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
- logInfo("Command for starting the Spark ApplicationMaster: " + commands)
+ logInfo("Yarn AM launch context:")
+ logInfo(s" class: ${args.amClass}")
+ logInfo(s" env: $env")
+ logInfo(s" command: ${commands.mkString(" ")}")
// TODO: it would be nicer to just make sure there are no null commands here
val printableCommands = commands.map(s => if (s == null) "null" else s).toList
@@ -391,12 +423,39 @@ trait ClientBase extends Logging {
object ClientBase extends Logging {
val SPARK_JAR: String = "__spark__.jar"
val APP_JAR: String = "__app__.jar"
- val LOG4J_PROP: String = "log4j.properties"
- val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF"
val LOCAL_SCHEME = "local"
+ val CONF_SPARK_JAR = "spark.yarn.jar"
+ /**
+ * This is an internal config used to propagate the location of the user's jar file to the
+ * driver/executors.
+ */
+ val CONF_SPARK_USER_JAR = "spark.yarn.user.jar"
+ /**
+ * This is an internal config used to propagate the list of extra jars to add to the classpath
+ * of executors.
+ */
val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars"
+ val ENV_SPARK_JAR = "SPARK_JAR"
- def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head)
+ /**
+ * Find the user-defined Spark jar if configured, or return the jar containing this
+ * class if not.
+ *
+ * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the
+ * user environment if that is not found (for backwards compatibility).
+ */
+ def sparkJar(conf: SparkConf) = {
+ if (conf.contains(CONF_SPARK_JAR)) {
+ conf.get(CONF_SPARK_JAR)
+ } else if (System.getenv(ENV_SPARK_JAR) != null) {
+ logWarning(
+ s"$ENV_SPARK_JAR detected in the system environment. This variable has been deprecated " +
+ s"in favor of the $CONF_SPARK_JAR configuration variable.")
+ System.getenv(ENV_SPARK_JAR)
+ } else {
+ SparkContext.jarOfClass(this.getClass).head
+ }
+ }
def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = {
val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf)
@@ -469,71 +528,74 @@ object ClientBase extends Logging {
triedDefault.toOption
}
+ def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf,
+ env: HashMap[String, String], extraClassPath: Option[String] = None) {
+ extraClassPath.foreach(addClasspathEntry(_, env))
+ addClasspathEntry(Environment.PWD.$(), env)
+
+ // Normally the users app.jar is last in case conflicts with spark jars
+ if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) {
+ addUserClasspath(args, sparkConf, env)
+ addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env)
+ ClientBase.populateHadoopClasspath(conf, env)
+ } else {
+ addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env)
+ ClientBase.populateHadoopClasspath(conf, env)
+ addUserClasspath(args, sparkConf, env)
+ }
+
+ // Append all jar files under the working directory to the classpath.
+ addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env);
+ }
/**
- * Returns the java command line argument for setting up log4j. If there is a log4j.properties
- * in the given local resources, it is used, otherwise the SPARK_LOG4J_CONF environment variable
- * is checked.
+ * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly
+ * to the classpath.
*/
- def getLog4jConfiguration(localResources: HashMap[String, LocalResource]): String = {
- var log4jConf = LOG4J_PROP
- if (!localResources.contains(log4jConf)) {
- log4jConf = System.getenv(LOG4J_CONF_ENV_KEY) match {
- case conf: String =>
- val confUri = new URI(conf)
- if (ClientBase.LOCAL_SCHEME.equals(confUri.getScheme())) {
- "file://" + confUri.getPath()
- } else {
- ClientBase.LOG4J_PROP
- }
- case null => "log4j-spark-container.properties"
+ private def addUserClasspath(args: ClientArguments, conf: SparkConf,
+ env: HashMap[String, String]) = {
+ if (args != null) {
+ addFileToClasspath(args.userJar, APP_JAR, env)
+ if (args.addJars != null) {
+ args.addJars.split(",").foreach { case file: String =>
+ addFileToClasspath(file, null, env)
+ }
}
+ } else {
+ val userJar = conf.get(CONF_SPARK_USER_JAR, null)
+ addFileToClasspath(userJar, APP_JAR, env)
+
+ val cachedSecondaryJarLinks = conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "").split(",")
+ cachedSecondaryJarLinks.foreach(jar => addFileToClasspath(jar, null, env))
}
- " -Dlog4j.configuration=" + log4jConf
}
- def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: String,
- env: HashMap[String, String], extraClassPath: Option[String] = None) {
-
- if (log4jConf != null) {
- // If a custom log4j config file is provided as a local: URI, add its parent directory to the
- // classpath. Note that this only works if the custom config's file name is
- // "log4j.properties".
- val localPath = getLocalPath(log4jConf)
- if (localPath != null) {
- val parentPath = new File(localPath).getParent()
- YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, parentPath,
- File.pathSeparator)
+ /**
+ * Adds the given path to the classpath, handling "local:" URIs correctly.
+ *
+ * If an alternate name for the file is given, and it's not a "local:" file, the alternate
+ * name will be added to the classpath (relative to the job's work directory).
+ *
+ * If not a "local:" file and no alternate name, the environment is not modified.
+ *
+ * @param path Path to add to classpath (optional).
+ * @param fileName Alternate name for the file (optional).
+ * @param env Map holding the environment variables.
+ */
+ private def addFileToClasspath(path: String, fileName: String,
+ env: HashMap[String, String]) : Unit = {
+ if (path != null) {
+ scala.util.control.Exception.ignoring(classOf[URISyntaxException]) {
+ val localPath = getLocalPath(path)
+ if (localPath != null) {
+ addClasspathEntry(localPath, env)
+ return
+ }
}
}
-
- /** Add entry to the classpath. */
- def addClasspathEntry(path: String) = YarnSparkHadoopUtil.addToEnvironment(env,
- Environment.CLASSPATH.name, path, File.pathSeparator)
- /** Add entry to the classpath. Interpreted as a path relative to the working directory. */
- def addPwdClasspathEntry(entry: String) =
- addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry)
-
- extraClassPath.foreach(addClasspathEntry)
-
- val cachedSecondaryJarLinks =
- sparkConf.getOption(CONF_SPARK_YARN_SECONDARY_JARS).getOrElse("").split(",")
- .filter(_.nonEmpty)
- // Normally the users app.jar is last in case conflicts with spark jars
- if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) {
- addPwdClasspathEntry(APP_JAR)
- cachedSecondaryJarLinks.foreach(addPwdClasspathEntry)
- addPwdClasspathEntry(SPARK_JAR)
- ClientBase.populateHadoopClasspath(conf, env)
- } else {
- addPwdClasspathEntry(SPARK_JAR)
- ClientBase.populateHadoopClasspath(conf, env)
- addPwdClasspathEntry(APP_JAR)
- cachedSecondaryJarLinks.foreach(addPwdClasspathEntry)
+ if (fileName != null) {
+ addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env);
}
- // Append all class files and jar files under the working directory to the classpath.
- addClasspathEntry(Environment.PWD.$())
- addPwdClasspathEntry("*")
}
/**
@@ -547,4 +609,8 @@ object ClientBase extends Logging {
null
}
+ private def addClasspathEntry(path: String, env: HashMap[String, String]) =
+ YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path,
+ File.pathSeparator)
+
}
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
index 43dbb2464f..4ba7133a95 100644
--- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
@@ -55,10 +55,12 @@ trait ExecutorRunnableUtil extends Logging {
sys.props.get("spark.executor.extraJavaOptions").foreach { opts =>
javaOpts += opts
}
+ sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
+ javaOpts += opts
+ }
javaOpts += "-Djava.io.tmpdir=" +
new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
- javaOpts += ClientBase.getLog4jConfiguration(localResources)
// Certain configs need to be passed here because they are needed before the Executor
// registers with the Scheduler and transfers the spark configs. Since the Executor backend
@@ -166,13 +168,8 @@ trait ExecutorRunnableUtil extends Logging {
def prepareEnvironment: HashMap[String, String] = {
val env = new HashMap[String, String]()
-
val extraCp = sparkConf.getOption("spark.executor.extraClassPath")
- val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
- ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp)
- if (log4jConf != null) {
- env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf
- }
+ ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp)
// Allow users to specify some environment variables
YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"),
diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala
index 608c6e9262..686714dc36 100644
--- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala
+++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala
@@ -17,22 +17,31 @@
package org.apache.spark.deploy.yarn
+import java.io.File
import java.net.URI
+import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.MRJobConfig
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.mockito.Matchers._
+import org.mockito.Mockito._
import org.scalatest.FunSuite
-import org.scalatest.matchers.ShouldMatchers._
+import org.scalatest.Matchers
import scala.collection.JavaConversions._
import scala.collection.mutable.{ HashMap => MutableHashMap }
import scala.util.Try
+import org.apache.spark.SparkConf
+import org.apache.spark.util.Utils
-class ClientBaseSuite extends FunSuite {
+class ClientBaseSuite extends FunSuite with Matchers {
test("default Yarn application classpath") {
ClientBase.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP))
@@ -68,6 +77,67 @@ class ClientBaseSuite extends FunSuite {
}
}
+ private val SPARK = "local:/sparkJar"
+ private val USER = "local:/userJar"
+ private val ADDED = "local:/addJar1,local:/addJar2,/addJar3"
+
+ test("Local jar URIs") {
+ val conf = new Configuration()
+ val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK)
+ val env = new MutableHashMap[String, String]()
+ val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf)
+
+ ClientBase.populateClasspath(args, conf, sparkConf, env, None)
+
+ val cp = env("CLASSPATH").split(File.pathSeparator)
+ s"$SPARK,$USER,$ADDED".split(",").foreach({ entry =>
+ val uri = new URI(entry)
+ if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) {
+ cp should contain (uri.getPath())
+ } else {
+ cp should not contain (uri.getPath())
+ }
+ })
+ cp should contain (Environment.PWD.$())
+ cp should contain (s"${Environment.PWD.$()}${File.separator}*")
+ cp should not contain (ClientBase.SPARK_JAR)
+ cp should not contain (ClientBase.APP_JAR)
+ }
+
+ test("Jar path propagation through SparkConf") {
+ val conf = new Configuration()
+ val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK)
+ val yarnConf = new YarnConfiguration()
+ val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf)
+
+ val client = spy(new DummyClient(args, conf, sparkConf, yarnConf))
+ doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]),
+ any(classOf[Path]), anyShort(), anyBoolean())
+
+ var tempDir = Files.createTempDir();
+ try {
+ client.prepareLocalResources(tempDir.getAbsolutePath())
+ sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER))
+
+ // The non-local path should be propagated by name only, since it will end up in the app's
+ // staging dir.
+ val expected = ADDED.split(",")
+ .map(p => {
+ val uri = new URI(p)
+ if (ClientBase.LOCAL_SCHEME == uri.getScheme()) {
+ p
+ } else {
+ Option(uri.getFragment()).getOrElse(new File(p).getName())
+ }
+ })
+ .mkString(",")
+
+ sparkConf.getOption(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS) should be (Some(expected))
+ } finally {
+ Utils.deleteRecursively(tempDir)
+ }
+ }
+
object Fixtures {
val knownDefYarnAppCP: Seq[String] =
@@ -109,4 +179,18 @@ class ClientBaseSuite extends FunSuite {
def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B =
Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults)
+ private class DummyClient(
+ val args: ClientArguments,
+ val conf: Configuration,
+ val sparkConf: SparkConf,
+ val yarnConf: YarnConfiguration) extends ClientBase {
+
+ override def calculateAMMemory(newApp: GetNewApplicationResponse): Int =
+ throw new UnsupportedOperationException()
+
+ override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit =
+ throw new UnsupportedOperationException()
+
+ }
+
}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
index 117b33f466..07ba0a4b30 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
@@ -81,6 +81,7 @@ class ExecutorRunnable(
val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores,
localResources)
+ logInfo(s"Setting up executor with environment: $env")
logInfo("Setting up executor with commands: " + commands)
ctx.setCommands(commands)