You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by va...@apache.org on 2017/08/29 19:51:57 UTC

spark git commit: Revert "[SPARK-21714][CORE][BACKPORT-2.2] Avoiding re-uploading remote resources in yarn client mode"

Repository: spark
Updated Branches:
  refs/heads/branch-2.2 59529b21a -> 917fe6635


Revert "[SPARK-21714][CORE][BACKPORT-2.2] Avoiding re-uploading remote resources in yarn client mode"

This reverts commit 59529b21a99f3c4db16b31da9dc7fce62349cf11.


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

Branch: refs/heads/branch-2.2
Commit: 917fe6635891ea76b22a3bcba282040afd14651d
Parents: 59529b2
Author: Marcelo Vanzin <va...@cloudera.com>
Authored: Tue Aug 29 12:51:27 2017 -0700
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Tue Aug 29 12:51:27 2017 -0700

----------------------------------------------------------------------
 .../org/apache/spark/deploy/SparkSubmit.scala   | 66 +++++++------------
 .../apache/spark/internal/config/package.scala  |  2 +-
 .../scala/org/apache/spark/util/Utils.scala     | 25 +++----
 .../apache/spark/deploy/SparkSubmitSuite.scala  | 68 ++++----------------
 .../main/scala/org/apache/spark/repl/Main.scala |  2 +-
 5 files changed, 48 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/917fe663/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 86d578e..c60a2a1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -208,20 +208,14 @@ object SparkSubmit extends CommandLineUtils {
 
   /**
    * Prepare the environment for submitting an application.
-   *
-   * @param args the parsed SparkSubmitArguments used for environment preparation.
-   * @param conf the Hadoop Configuration, this argument will only be set in unit test.
-   * @return a 4-tuple:
-   *        (1) the arguments for the child process,
-   *        (2) a list of classpath entries for the child,
-   *        (3) a map of system properties, and
-   *        (4) the main class for the child
-   *
+   * This returns a 4-tuple:
+   *   (1) the arguments for the child process,
+   *   (2) a list of classpath entries for the child,
+   *   (3) a map of system properties, and
+   *   (4) the main class for the child
    * Exposed for testing.
    */
-  private[deploy] def prepareSubmitEnvironment(
-      args: SparkSubmitArguments,
-      conf: Option[HadoopConfiguration] = None)
+  private[deploy] def prepareSubmitEnvironment(args: SparkSubmitArguments)
       : (Seq[String], Seq[String], Map[String, String], String) = {
     // Return values
     val childArgs = new ArrayBuffer[String]()
@@ -317,16 +311,12 @@ object SparkSubmit extends CommandLineUtils {
     }
 
     // In client mode, download remote files.
-    var localPrimaryResource: String = null
-    var localJars: String = null
-    var localPyFiles: String = null
-    var localFiles: String = null
     if (deployMode == CLIENT) {
-      val hadoopConf = conf.getOrElse(new HadoopConfiguration())
-      localPrimaryResource = Option(args.primaryResource).map(downloadFile(_, hadoopConf)).orNull
-      localJars = Option(args.jars).map(downloadFileList(_, hadoopConf)).orNull
-      localPyFiles = Option(args.pyFiles).map(downloadFileList(_, hadoopConf)).orNull
-      localFiles = Option(args.files).map(downloadFileList(_, hadoopConf)).orNull
+      val hadoopConf = new HadoopConfiguration()
+      args.primaryResource = Option(args.primaryResource).map(downloadFile(_, hadoopConf)).orNull
+      args.jars = Option(args.jars).map(downloadFileList(_, hadoopConf)).orNull
+      args.pyFiles = Option(args.pyFiles).map(downloadFileList(_, hadoopConf)).orNull
+      args.files = Option(args.files).map(downloadFileList(_, hadoopConf)).orNull
     }
 
     // Require all python files to be local, so we can add them to the PYTHONPATH
@@ -376,7 +366,7 @@ object SparkSubmit extends CommandLineUtils {
         // If a python file is provided, add it to the child arguments and list of files to deploy.
         // Usage: PythonAppRunner <main python file> <extra python files> [app arguments]
         args.mainClass = "org.apache.spark.deploy.PythonRunner"
-        args.childArgs = ArrayBuffer(localPrimaryResource, localPyFiles) ++ args.childArgs
+        args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs
         if (clusterManager != YARN) {
           // The YARN backend distributes the primary file differently, so don't merge it.
           args.files = mergeFileLists(args.files, args.primaryResource)
@@ -386,8 +376,8 @@ object SparkSubmit extends CommandLineUtils {
         // The YARN backend handles python files differently, so don't merge the lists.
         args.files = mergeFileLists(args.files, args.pyFiles)
       }
-      if (localPyFiles != null) {
-        sysProps("spark.submit.pyFiles") = localPyFiles
+      if (args.pyFiles != null) {
+        sysProps("spark.submit.pyFiles") = args.pyFiles
       }
     }
 
@@ -441,7 +431,7 @@ object SparkSubmit extends CommandLineUtils {
         // If an R file is provided, add it to the child arguments and list of files to deploy.
         // Usage: RRunner <main R file> [app arguments]
         args.mainClass = "org.apache.spark.deploy.RRunner"
-        args.childArgs = ArrayBuffer(localPrimaryResource) ++ args.childArgs
+        args.childArgs = ArrayBuffer(args.primaryResource) ++ args.childArgs
         args.files = mergeFileLists(args.files, args.primaryResource)
       }
     }
@@ -478,7 +468,6 @@ object SparkSubmit extends CommandLineUtils {
       OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.queue"),
       OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES,
         sysProp = "spark.executor.instances"),
-      OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.pyFiles"),
       OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.jars"),
       OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"),
       OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.archives"),
@@ -502,28 +491,15 @@ object SparkSubmit extends CommandLineUtils {
         sysProp = "spark.driver.cores"),
       OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER,
         sysProp = "spark.driver.supervise"),
-      OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"),
-
-      // An internal option used only for spark-shell to add user jars to repl's classloader,
-      // previously it uses "spark.jars" or "spark.yarn.dist.jars" which now may be pointed to
-      // remote jars, so adding a new option to only specify local jars for spark-shell internally.
-      OptionAssigner(localJars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.repl.local.jars")
+      OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy")
     )
 
     // In client mode, launch the application main class directly
     // In addition, add the main application jar and any added jars (if any) to the classpath
-    if (deployMode == CLIENT) {
-      childMainClass = args.mainClass
-      if (localPrimaryResource != null && isUserJar(localPrimaryResource)) {
-        childClasspath += localPrimaryResource
-      }
-      if (localJars != null) { childClasspath ++= localJars.split(",") }
-    }
-    // Add the main application jar and any added jars to classpath in case YARN client
+    // Also add the main application jar and any added jars to classpath in case YARN client
     // requires these jars.
-    // This assumes both primaryResource and user jars are local jars, otherwise it will not be
-    // added to the classpath of YARN client.
-    if (isYarnCluster) {
+    if (deployMode == CLIENT || isYarnCluster) {
+      childMainClass = args.mainClass
       if (isUserJar(args.primaryResource)) {
         childClasspath += args.primaryResource
       }
@@ -580,6 +556,10 @@ object SparkSubmit extends CommandLineUtils {
       if (args.isPython) {
         sysProps.put("spark.yarn.isPython", "true")
       }
+
+      if (args.pyFiles != null) {
+        sysProps("spark.submit.pyFiles") = args.pyFiles
+      }
     }
 
     // assure a keytab is available from any place in a JVM

http://git-wip-us.apache.org/repos/asf/spark/blob/917fe663/core/src/main/scala/org/apache/spark/internal/config/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 1588dfe..c0fcf99 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -87,7 +87,7 @@ package object config {
     .intConf
     .createOptional
 
-  private[spark] val PY_FILES = ConfigBuilder("spark.yarn.dist.pyFiles")
+  private[spark] val PY_FILES = ConfigBuilder("spark.submit.pyFiles")
     .internal()
     .stringConf
     .toSequence

http://git-wip-us.apache.org/repos/asf/spark/blob/917fe663/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 69c6c33..999486c 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -2580,23 +2580,18 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Return the jar files pointed by the "spark.jars" property. Spark internally will distribute
-   * these jars through file server. In the YARN mode, it will return an empty list, since YARN
-   * has its own mechanism to distribute jars.
+   * In YARN mode this method returns a union of the jar files pointed by "spark.jars" and the
+   * "spark.yarn.dist.jars" properties, while in other modes it returns the jar files pointed by
+   * only the "spark.jars" property.
    */
-  def getUserJars(conf: SparkConf): Seq[String] = {
+  def getUserJars(conf: SparkConf, isShell: Boolean = false): Seq[String] = {
     val sparkJars = conf.getOption("spark.jars")
-    sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten
-  }
-
-  /**
-   * Return the local jar files which will be added to REPL's classpath. These jar files are
-   * specified by --jars (spark.jars) or --packages, remote jars will be downloaded to local by
-   * SparkSubmit at first.
-   */
-  def getLocalUserJarsForShell(conf: SparkConf): Seq[String] = {
-    val localJars = conf.getOption("spark.repl.local.jars")
-    localJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten
+    if (conf.get("spark.master") == "yarn" && isShell) {
+      val yarnJars = conf.getOption("spark.yarn.dist.jars")
+      unionFileLists(sparkJars, yarnJars).toSeq
+    } else {
+      sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten
+    }
   }
 
   private[spark] val REDACTION_REPLACEMENT_TEXT = "*********(redacted)"

http://git-wip-us.apache.org/repos/asf/spark/blob/917fe663/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 3c49b1f..6fa3a09 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -27,7 +27,7 @@ import scala.io.Source
 import com.google.common.io.ByteStreams
 import org.apache.commons.io.{FilenameUtils, FileUtils}
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
+import org.apache.hadoop.fs.Path
 import org.scalatest.{BeforeAndAfterEach, Matchers}
 import org.scalatest.concurrent.Timeouts
 import org.scalatest.time.SpanSugar._
@@ -738,7 +738,10 @@ class SparkSubmitSuite
 
   test("downloadFile - file doesn't exist") {
     val hadoopConf = new Configuration()
-    updateConfWithFakeS3Fs(hadoopConf)
+    // Set s3a implementation to local file system for testing.
+    hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem")
+    // Disable file system impl cache to make sure the test file system is picked up.
+    hadoopConf.set("fs.s3a.impl.disable.cache", "true")
     intercept[FileNotFoundException] {
       SparkSubmit.downloadFile("s3a:/no/such/file", hadoopConf)
     }
@@ -756,7 +759,10 @@ class SparkSubmitSuite
     val content = "hello, world"
     FileUtils.write(jarFile, content)
     val hadoopConf = new Configuration()
-    updateConfWithFakeS3Fs(hadoopConf)
+    // Set s3a implementation to local file system for testing.
+    hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem")
+    // Disable file system impl cache to make sure the test file system is picked up.
+    hadoopConf.set("fs.s3a.impl.disable.cache", "true")
     val sourcePath = s"s3a://${jarFile.getAbsolutePath}"
     val outputPath = SparkSubmit.downloadFile(sourcePath, hadoopConf)
     checkDownloadedFile(sourcePath, outputPath)
@@ -769,7 +775,10 @@ class SparkSubmitSuite
     val content = "hello, world"
     FileUtils.write(jarFile, content)
     val hadoopConf = new Configuration()
-    updateConfWithFakeS3Fs(hadoopConf)
+    // Set s3a implementation to local file system for testing.
+    hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem")
+    // Disable file system impl cache to make sure the test file system is picked up.
+    hadoopConf.set("fs.s3a.impl.disable.cache", "true")
     val sourcePaths = Seq("/local/file", s"s3a://${jarFile.getAbsolutePath}")
     val outputPaths = SparkSubmit.downloadFileList(sourcePaths.mkString(","), hadoopConf).split(",")
 
@@ -780,43 +789,6 @@ class SparkSubmitSuite
     }
   }
 
-  test("Avoid re-upload remote resources in yarn client mode") {
-    val hadoopConf = new Configuration()
-    updateConfWithFakeS3Fs(hadoopConf)
-
-    val tmpDir = Utils.createTempDir()
-    val file = File.createTempFile("tmpFile", "", tmpDir)
-    val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir)
-    val mainResource = File.createTempFile("tmpPy", ".py", tmpDir)
-    val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir)
-    val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}"
-
-    val args = Seq(
-      "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"),
-      "--name", "testApp",
-      "--master", "yarn",
-      "--deploy-mode", "client",
-      "--jars", tmpJarPath,
-      "--files", s"s3a://${file.getAbsolutePath}",
-      "--py-files", s"s3a://${pyFile.getAbsolutePath}",
-      s"s3a://$mainResource"
-      )
-
-    val appArgs = new SparkSubmitArguments(args)
-    val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))._3
-
-    // All the resources should still be remote paths, so that YARN client will not upload again.
-    sysProps("spark.yarn.dist.jars") should be (tmpJarPath)
-    sysProps("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}")
-    sysProps("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}")
-
-    // Local repl jars should be a local path.
-    sysProps("spark.repl.local.jars") should (startWith("file:"))
-
-    // local py files should not be a URI format.
-    sysProps("spark.submit.pyFiles") should (startWith("/"))
-  }
-
   // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
   private def runSparkSubmit(args: Seq[String]): Unit = {
     val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
@@ -856,11 +828,6 @@ class SparkSubmitSuite
       Utils.deleteRecursively(tmpDir)
     }
   }
-
-  private def updateConfWithFakeS3Fs(conf: Configuration): Unit = {
-    conf.set("fs.s3a.impl", classOf[TestFileSystem].getCanonicalName)
-    conf.set("fs.s3a.impl.disable.cache", "true")
-  }
 }
 
 object JarCreationTest extends Logging {
@@ -930,13 +897,4 @@ class TestFileSystem extends org.apache.hadoop.fs.LocalFileSystem {
     // Ignore the scheme for testing.
     super.copyToLocalFile(new Path(src.toUri.getPath), dst)
   }
-
-  override def globStatus(pathPattern: Path): Array[FileStatus] = {
-    val newPath = new Path(pathPattern.toUri.getPath)
-    super.globStatus(newPath).map { status =>
-      val path = s"s3a://${status.getPath.toUri.getPath}"
-      status.setPath(new Path(path))
-      status
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/917fe663/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
----------------------------------------------------------------------
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
index 0b16e1b..9702a1e 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
@@ -57,7 +57,7 @@ object Main extends Logging {
   // Visible for testing
   private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = {
     interp = _interp
-    val jars = Utils.getLocalUserJarsForShell(conf)
+    val jars = Utils.getUserJars(conf, isShell = true)
       // Remove file:///, file:// or file:/ scheme if exists for each jar
       .map { x => if (x.startsWith("file:")) new File(new URI(x)).getPath else x }
       .mkString(File.pathSeparator)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org