maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532375495
##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1920,17 +1920,23 @@ class SparkContext(config: SparkConf) extends Logging {
case "file" => addLocalJarFile(new File(uri.getPath))
// A JAR file which exists locally on every worker node
case "local" => "file:" + uri.getPath
+ case "ivy" =>
+ // Since `new Path(path).toUri` will lose query information,
+ // so here we use `URI.create(path)`
+ DependencyUtils.resolveMavenDependencies(URI.create(path))
case _ => checkRemoteJarFile(path)
}
}
- if (key != null) {
+ if (keys != null) {
Review comment:
No chance for the "local"/"file" case to have commas in `keys`?
https://github.com/apache/spark/pull/29966/files#r529246237
##########
File path:
core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
##########
@@ -80,15 +80,10 @@ object DriverWrapper extends Logging {
val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf)
val Seq(packagesExclusions, packages, repositories, ivyRepoPath,
ivySettingsPath) =
- Seq(
- "spark.jars.excludes",
- "spark.jars.packages",
- "spark.jars.repositories",
- "spark.jars.ivy",
- "spark.jars.ivySettings"
- ).map(sys.props.get(_).orNull)
+ DependencyUtils.getIvyProperties()
- val resolvedMavenCoordinates =
DependencyUtils.resolveMavenDependencies(packagesExclusions,
+ val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(
+ true, packagesExclusions,
packages, repositories, ivyRepoPath, Option(ivySettingsPath))
Review comment:
nit:
```
val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(
true, packagesExclusions, packages, repositories, ivyRepoPath,
Option(ivySettingsPath))
```
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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)
+ }
+
Review comment:
nit: redundant blank line.
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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] = {
Review comment:
`parseURLQueryParameter` -> `parseURLQueryParameters` or
`parseURLQueryParams`?
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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")
Review comment:
hive has the same behaviour with this?
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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)
Review comment:
Really `assert`? Rather, `IllegalArgumentException` because the query
string depends on user's input?
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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),
Review comment:
ditto: `IllegalArgumentException`? btw, hive has the same behaviour with
this?
##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,104 @@ 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")
Review comment:
Why do we need to parse `queryString` multiple times (`parseExcludeList
` and `parseTransitive `)? Instead, how about converting `queryString` into
`Map[String, String]`, then checking if keys exist in the parsed map?
##########
File path: docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
##########
@@ -33,15 +33,30 @@ ADD JAR file_name
* **file_name**
- The name of the JAR file to be added. It could be either on a local file
system or a distributed file system.
+ The name of the JAR file to be added. It could be either on a local file
system or a distributed file system or an ivy URL.
+ Apache Ivy is a popular dependency manager focusing on flexibility and
simplicity. Now we support two parameter in URL query string:
+ 1. transitive: whether to download dependent jars related to your ivy
URL.
+ 2. exclude: exclusion list when download ivy URL jar and dependent jars.
Review comment:
When listing something in the doc, `*` seems to be used in most places,
I think. e.g.,
https://github.com/apache/spark/blame/master/docs/sql-ref-syntax-aux-show-tables.md#L45-L48
cc: @huaxingao
##########
File path: docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
##########
@@ -33,15 +33,30 @@ ADD JAR file_name
* **file_name**
- The name of the JAR file to be added. It could be either on a local file
system or a distributed file system.
+ The name of the JAR file to be added. It could be either on a local file
system or a distributed file system or an ivy URL.
+ Apache Ivy is a popular dependency manager focusing on flexibility and
simplicity. Now we support two parameter in URL query string:
Review comment:
nit: redundant space found.
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]