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 2020/11/28 17:50:46 UTC

[GitHub] [spark] viirya commented on a change in pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

viirya commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532087871



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,95 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
 
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.SparkSubmitUtils
 import org.apache.spark.internal.Logging
-import org.apache.spark.util.{MutableURLClassLoader, Utils}
 
-private[deploy] object DependencyUtils extends Logging {
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): Seq[String] = {
+    Seq(
+      "spark.jars.excludes",
+      "spark.jars.packages",
+      "spark.jars.repositories",
+      "spark.jars.ivy",
+      "spark.jars.ivySettings"
+    ).map(sys.props.get(_).orNull)
+  }
+
+
+  private def parseURLQueryParameter(queryString: String, queryTag: String): Array[String] = {
+    if (queryString == null || queryString.isEmpty) {
+      Array.empty[String]
+    } else {
+      val mapTokens = queryString.split("&")
+      assert(mapTokens.forall(_.split("=").length == 2)
+        , "Invalid URI query string: [ " + queryString + " ]")
+      mapTokens.map(_.split("=")).map(kv => (kv(0), kv(1))).filter(_._1 == queryTag).map(_._2)
+    }
+  }
+
+  /**
+   * Parse excluded list in ivy URL. When download ivy URL jar, Spark won't download transitive jar
+   * in excluded list.
+   *
+   * @param queryString Ivy URI query part string.
+   * @return Exclude list which contains grape parameters of exclude.
+   *         Example: Input:  exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *         Output:  [org.mortbay.jetty:jetty, org.eclipse.jetty:jetty-http]
+   */
+  private def parseExcludeList(queryString: String): String = {
+    parseURLQueryParameter(queryString, "exclude")
+      .flatMap { excludeString =>
+        val excludes: Array[String] = excludeString.split(",")
+        assert(excludes.forall(_.split(":").length == 2),
+          "Invalid exclude string: expected 'org:module,org:module,..'," +
+            " found [ " + excludeString + " ]")
+        excludes
+      }.mkString(":")
+  }
+
+  /**
+   * Parse transitive parameter in ivy URL, default value is false.
+   *
+   * @param queryString Ivy URI query part string.
+   * @return Exclude list which contains grape parameters of transitive.
+   *         Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *         Output:  true
+   */
+  private def parseTransitive(queryString: String): Boolean = {
+    val transitive = parseURLQueryParameter(queryString, "transitive")
+    if (transitive.isEmpty) {
+      false
+    } else {
+      if (transitive.length > 1) {
+        logWarning("It's best to specify `transitive` parameter in ivy URL query only once." +
+          " If there are multiple `transitive` parameter, we will select the last one")
+      }
+      transitive.last.toBoolean
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependent jars.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Comma separated string list of URIs of downloaded jars
+   */
+  def resolveMavenDependencies(uri: URI): String = {
+    val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) =
+      DependencyUtils.getIvyProperties()
+    resolveMavenDependencies(
+      parseTransitive(uri.getQuery),
+      parseExcludeList(uri.getQuery),

Review comment:
       Could you describe the url query format clearly in the method doc? 




----------------------------------------------------------------
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.

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