You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/10/11 14:50:03 UTC

[GitHub] [spark] attilapiros commented on a change in pull request #34120: [SPARK-35672][CORE][YARN] Pass user classpath entries to executors using config instead of command line.

attilapiros commented on a change in pull request #34120:
URL: https://github.com/apache/spark/pull/34120#discussion_r726185164



##########
File path: resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
##########
@@ -583,6 +584,79 @@ class ClientSuite extends SparkFunSuite with Matchers {
     }
   }
 
+  test("SPARK-35672: test Client.getUserClasspathUrls") {
+    val gatewayRootPath = "/local/matching/replace"
+    val replacementRootPath = "/replaced/path"
+    val conf = new SparkConf()
+        .set(SECONDARY_JARS, Seq(
+          s"local:$gatewayRootPath/foo.jar",
+          "local:/local/not/matching/replace/foo.jar",
+          "file:/absolute/file/path/foo.jar",
+          s"$gatewayRootPath/but-not-actually-local/foo.jar",
+          "/absolute/path/foo.jar",
+          "relative/path/foo.jar"
+        ))
+        .set(GATEWAY_ROOT_PATH, gatewayRootPath)
+        .set(REPLACEMENT_ROOT_PATH, replacementRootPath)
+
+    def assertUserClasspathUrls(cluster: Boolean, expectedReplacementPath: String): Unit = {
+      val expectedUrls = Seq(
+        Paths.get(APP_JAR_NAME).toAbsolutePath.toUri.toString,
+        s"file:$expectedReplacementPath/foo.jar",
+        "file:/local/not/matching/replace/foo.jar",
+        "file:/absolute/file/path/foo.jar",
+        // since this path wasn't a local URI, it should never be replaced
+        s"file:$gatewayRootPath/but-not-actually-local/foo.jar",
+        "file:/absolute/path/foo.jar",
+        Paths.get("relative/path/foo.jar").toAbsolutePath.toUri.toString
+      ).map(URI.create(_).toURL).toArray
+      assert(Client.getUserClasspathUrls(conf, cluster) === expectedUrls)
+    }
+    // assert that no replacement happens when cluster = false by expecting the replacement
+    // path to be the same as the original path
+    assertUserClasspathUrls(cluster = false, gatewayRootPath)
+    assertUserClasspathUrls(cluster = true, replacementRootPath)
+  }
+
+  test("SPARK-35672: test replaceEnvVars in Unix mode") {

Review comment:
       If you managed to move the `replaceEnvVars` method into `YarnSparkHadoopUtil` then do not forget to move these two tests to `YarnSparkHadoopUtilSuite`.

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1470,6 +1471,90 @@ private object Client extends Logging {
     (mainUri ++ secondaryUris).toArray
   }
 
+  /**
+   * Returns a list of local, absolute file URLs representing the user classpath. Note that this
+   * must be executed on the same host which will access the URLs, as it will resolve relative
+   * paths based on the current working directory, as well as environment variables.
+   * See SPARK-35672 for discussion of why it is necessary to do environment variable substitution.
+   *
+   * @param conf Spark configuration.
+   * @param useClusterPath Whether to use the 'cluster' path when resolving paths with the
+   *                       `local` scheme. This should be used when running on the cluster, but
+   *                       not when running on the gateway (i.e. for the driver in `client` mode).
+   * @return Array of local URLs ready to be passed to a [[java.net.URLClassLoader]].
+   */
+  def getUserClasspathUrls(conf: SparkConf, useClusterPath: Boolean): Array[URL] = {
+    Client.getUserClasspath(conf).map { uri =>
+      val inputPath = uri.getPath
+      val replacedFilePath = if (Utils.isLocalUri(uri.toString) && useClusterPath) {
+        Client.getClusterPath(conf, inputPath)
+      } else {
+        // Any other URI schemes should have been resolved by this point
+        assert(uri.getScheme == null || uri.getScheme == "file" || Utils.isLocalUri(uri.toString),
+          "getUserClasspath should only return 'file' or 'local' URIs but found: " + uri)
+        inputPath
+      }
+      val envVarResolvedFilePath = replaceEnvVars(replacedFilePath, sys.env)
+      Paths.get(envVarResolvedFilePath).toAbsolutePath.toUri.toURL
+    }
+  }
+
+  // scalastyle:off line.size.limit
+  /**
+   * Replace environment variables in a string according to the same rules as [[Environment]]:
+   * `$VAR_NAME` for Unix, `%VAR_NAME%` for Windows, and `{{VAR_NAME}}` for all OS.
+   * This support escapes for `$` and `%` characters, e.g.
+   * `\$FOO`, `^%FOO^%`, and `%%FOO%%` will be resolved to `$FOO`, `%FOO%`, and `%FOO%`,
+   * respectively, instead of being treated as variable names. Note that Unix variable naming
+   * conventions (alphanumeric plus underscore, case-sensitive, can't start with a digit) are
+   * used for both Unix and Windows, following the convention of Hadoop's [[Shell]]
+   * (see specifically [[Shell.getEnvironmentVariableRegex]]).
+   *
+   * @param unresolvedString The unresolved string which may contain variable references.
+   * @param env The System environment
+   * @param isWindows True iff running in a Windows environment
+   * @return The input string with variables replaced with their values from `env`
+   * @see [[https://pubs.opengroup.org/onlinepubs/000095399/basedefs/xbd_chap08.html Environment Variables (IEEE Std 1003.1-2017)]]
+   * @see [[https://en.wikibooks.org/wiki/Windows_Batch_Scripting#Quoting_and_escaping Windows Batch Scripting | Quoting and Escaping]]
+   */
+  // scalastyle:on line.size.limit
+  def replaceEnvVars(

Review comment:
       Please try to move this method into the `YarnSparkHadoopUtil`  object where we already deal with some environment variable transformation (build env vars from string).
   
   It has higher chance for finding it there when somebody will need something similar (+ it has no state so an object fits better for this helper method).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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