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/10/07 11:39:04 UTC

[GitHub] [spark] AngersZhuuuu opened a new pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

AngersZhuuuu opened a new pull request #29966:
URL: https://github.com/apache/spark/pull/29966


   ### What changes were proposed in this pull request?
   Support add jar with ivy path
   
   ### Why are the changes needed?
   Since submit app can support ivy, add jar we can also support ivy now.
   
   ### Does this PR introduce _any_ user-facing change?
   User can add jar with sql like 
   ```
   add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true
   add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false
   ```
   
   core api 
   ```
   sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true")
   sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false")
   ```
   
   
   ### How was this patch tested?
   Added UT


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533868828



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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] = {

Review comment:
       add case class is good and then we don't need docs since parameter name is clear enough.
   But change to Option[String] will change a lot related function, I think we can refactor this later like  https://github.com/apache/spark/pull/29966#discussion_r531917374




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735337600






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739248159


   retest this please


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547813363



##########
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:
 
+     * transitive: whether to download dependent jars related to your ivy URL. It is case-sensitive and only take last one if multiple transitive parameters are specified.
+     * exclude: exclusion list when download ivy URL jar and dependent jars.

Review comment:
       Done




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749450739


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133198/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749412306


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37797/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r554656682



##########
File path: core/src/test/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+
+import org.apache.spark.SparkFunSuite
+
+class DependencyUtilsSuite extends SparkFunSuite {
+
+  test("SPARK-33084: Add jar support Ivy URI -- test invalid ivy uri") {
+    val e1 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(URI.create("ivy://"))
+    }.getMessage
+    assert(e1.contains("Expected authority at index 6: ivy://"))
+
+    val e2 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(URI.create("ivy://org.apache.hive:hive-contrib"))
+    }.getMessage
+    assert(e2.contains("Invalid Ivy URI authority in uri ivy://org.apache.hive:hive-contrib:" +
+      " Expected 'org:module:version', found org.apache.hive:hive-contrib."))
+
+    val e3 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(
+        URI.create("ivy://org.apache.hive:hive-contrib:2.3.7?foo="))

Review comment:
       > Do we need to keep `2.3.7` consistent with the built-in Hive version?
   
   Emmm, is there any concern about this?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735786330


   **[Test build #131989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131989/testReport)** for PR 29966 at commit [`8c5cb7c`](https://github.com/apache/spark/commit/8c5cb7ccb21cf3b9fbf4d87da1c5f9991448d035).


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r529305517



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2980,6 +2980,75 @@ private[spark] object Utils extends Logging {
     metadata.toString
   }
 
+  /**
+   * 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) =
+      Seq(
+        "spark.jars.repositories",
+        "spark.jars.ivy",
+        "spark.jars.ivySettings"
+      ).map(sys.props.get(_).orNull)
+    // Create the IvySettings, either load from file or build defaults
+    val ivySettings = Option(ivySettingsPath) match {
+      case Some(path) =>
+        SparkSubmitUtils.loadIvySettings(path, Option(repositories), Option(ivyRepoPath))

Review comment:
       Or remove this part? just use default ivy setting?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705128990






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-752830306


   > please note this breaks scala 2.13
   > where can i find source code for [SPARK-33084](https://issues.apache.org/jira/browse/SPARK-33084).jar?
   > 
   > ```
   > [info]   Cause: java.lang.NoSuchMethodError: org.apache.spark.sql.types.StructType$.apply(Lscala/collection/Seq;)Lorg/apache/spark/sql/types/StructType;
   > [info]   at org.apache.spark.examples.sql.Spark33084.inputSchema(Spark33084.scala:9)
   > [info]   at org.apache.spark.sql.execution.aggregate.ScalaUDAF.<init>(udaf.scala:347)
   > ```
   
   Defined it by my self. I will fix this issue.


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735857103






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r531921325



##########
File path: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
##########
@@ -1348,6 +1348,7 @@ private[spark] object SparkSubmitUtils {
       coordinates: String,
       ivySettings: IvySettings,
       exclusions: Seq[String] = Nil,
+      transitive: Boolean = true,

Review comment:
       Done and `transitive` moved to front of `exclusions`




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r529239846



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2980,6 +2980,75 @@ private[spark] object Utils extends Logging {
     metadata.toString
   }
 
+  /**
+   * 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) =
+      Seq(
+        "spark.jars.repositories",
+        "spark.jars.ivy",
+        "spark.jars.ivySettings"
+      ).map(sys.props.get(_).orNull)
+    // Create the IvySettings, either load from file or build defaults
+    val ivySettings = Option(ivySettingsPath) match {
+      case Some(path) =>
+        SparkSubmitUtils.loadIvySettings(path, Option(repositories), Option(ivyRepoPath))

Review comment:
       Could you add tests for this path?

##########
File path: docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
##########
@@ -33,7 +33,7 @@ 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 a ivy URL.

Review comment:
       nit: `a` -> `an`




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r534596650



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false

Review comment:
       Should do like this:
   ```
       val transitiveParams = groupedParams.get("transitive")
         if (transitiveParams.map(_.size).getOrElse(0) > 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")
         }
         val transitive = transitiveParams.flatMap(_.takeRight(1).map { case (_, value) =>
           value match {
             case "true" => true
             case _ => false
           }
         }.headOption).getOrElse(false)
   ```
   
   is equal with current logic.




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744459334


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37370/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731900415






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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735337600






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533844227



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.flatMap { case (_, excludeString) =>
+          val excludes = excludeString.split(",")
+          if (excludes.exists(_.split(":").length != 2)) {
+            throw new URISyntaxException(excludeString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of URIs of downloaded jars
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) =
+      DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found null")

Review comment:
       Yea




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533382150



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,115 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  def parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach(value => {
+          if (value == "true") {
+            transitive = true
+          } else if (value == "false") {
+            transitive = false
+          }

Review comment:
       nit format:
   ```
           params.map(_._2).foreach {
             case "true" => transitive = true
             case "false" => transitive = false
             case _ =>
           }
   ```
   Also, could you add more tests for a duplicated `transitive` case like `transitive=true&transitive=false&transitive=invalidValue`,  `transitive=false&transitive=invalidValue&transitive=true`, ...? 




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745299668


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37433/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r529245629



##########
File path: docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
##########
@@ -33,7 +33,7 @@ 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 a ivy URL.

Review comment:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750042784


   **[Test build #133291 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133291/testReport)** for PR 29966 at commit [`90491d5`](https://github.com/apache/spark/commit/90491d5fd822658222d9d8156a10c6554c6e1427).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737306243


   **[Test build #132057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132057/testReport)** for PR 29966 at commit [`b69a62e`](https://github.com/apache/spark/commit/b69a62ea633653e71269f3875c1a4a8af8f1665b).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749508760


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37808/
   


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


[GitHub] [spark] dongjoon-hyun commented on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-751125152


   BTW, Merry Christmas and Happy New Year, all!


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745429482


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132831/
   


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547767161



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -445,6 +445,29 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("SPARK-33084: add jar when path contains comma") {

Review comment:
       Thanks. Yes. Please remove this.




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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705407342






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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-738553046


   **[Test build #132170 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132170/testReport)** for PR 29966 at commit [`ebe1c9c`](https://github.com/apache/spark/commit/ebe1c9ccb75898b5c68a570d43d89444d4c92970).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r534601930



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -955,6 +978,121 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
         .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop()
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url -- default transitive = false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- invalid transitive use default false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- transitive=true will download dependency jars") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0" +
+      "?exclude=commons-lang:commons-lang&transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test different version") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.6.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.6.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test invalid param") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?invalidParam=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" +
+      "transitive=true&transitive=false&transitive=invalidValue")

Review comment:
       > > Could you add tests for ?transitive=true&transitive=invalidValue, too?
   > > #29966 (comment)
   > 
   > Where?
   
    In `test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") `




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731900415






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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732681739


   **[Test build #131617 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131617/testReport)** for PR 29966 at commit [`0e3c1ec`](https://github.com/apache/spark/commit/0e3c1ec7476804beec25eb47e674d6ed4d8dcb7a).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750115759


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133276/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533099768



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  /**
+   * 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(excludes: Array[String]): String = {

Review comment:
       Yea, thank s for your advise




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749568587


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133211/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533090454



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1860,7 +1860,7 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   private def addJar(path: String, addedOnSubmit: Boolean): Unit = {
-    def addLocalJarFile(file: File): String = {
+    def addLocalJarFile(file: File): Array[String] = {

Review comment:
       Yea

##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1869,15 +1869,15 @@ class SparkContext(config: SparkConf) extends Logging {
           throw new IllegalArgumentException(
             s"Directory ${file.getAbsoluteFile} is not allowed for addJar")
         }
-        env.rpcEnv.fileServer.addJar(file)
+        Array(env.rpcEnv.fileServer.addJar(file))
       } catch {
         case NonFatal(e) =>
           logError(s"Failed to add $path to Spark environment", e)
-          null
+          Array.empty
       }
     }
 
-    def checkRemoteJarFile(path: String): String = {
+    def checkRemoteJarFile(path: String): Array[String] = {

Review comment:
       Yea




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749517944


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37808/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532395160



##########
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:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750108501


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37887/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533839043



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.flatMap { case (_, excludeString) =>
+          val excludes = excludeString.split(",")
+          if (excludes.exists(_.split(":").length != 2)) {

Review comment:
       Yea




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745198407


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37421/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547813502



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3771,6 +3792,40 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       }
     })
   }
+
+  test("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") {
+    val sumFuncClass = "org.apache.spark.examples.sql.Spark33084"
+    val functionName = "test_udf"
+    withTempDir { dir => {

Review comment:
       DOne




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


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

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-740028815


   > I'm neutral on this, but we don't basically merge a new feature into the release branch (v3.1) after the branch-cut.
   
   Yeah, I was thinking we had one more day before the branch cut -- had my dates a little off. Looks like we missed it. Thanks for checking @maropu 
   
   > Wait..seems we all ignore how can we pass these Jars to executor side....
   
   Good catch! Approach LGTM, one minor comment.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547822836



##########
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:
 
+     * transitive: whether to download dependent jars related to your ivy URL. It is case-sensitive and only take last one if multiple transitive parameters are specified.
+     * exclude: exclusion list when download ivy URL jar and dependent jars.
+
+    User can write ivy URL such as:
+
+      ivy://group:module:version
+      ivy://group:module:version?transitive=[true|false]
+      ivy://group:module:version?exclude=group:module,group:module

Review comment:
       > `exclude` seems to be misleading here because `transitive` is false by default. Maybe, shall we revise like the following?
   > 
   > ```
   > ivy://group:module:version
   > ivy://group:module:version?transitive=[true|false]
   > ivy://group:module:version?transitive=[true|false]&exclude=group:module,group:module
   > ```
   
   Done




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731941024






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533115969



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -366,6 +366,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("add jar local path with comma") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+    sc.addJar("file://Test,UDTF.jar")
+    assert(!sc.listJars().exists(_.contains("UDTF.jar")))

Review comment:
       > What does this test mean? Why didn't you write it like ` assert(sc.listJars().exists(_.contains("Test,UDTF.jar")))`?
   
   Mess in brain...

##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -366,6 +366,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("add jar local path with comma") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))

Review comment:
       > Cold you add tests for other other schemas?
   
   Updated




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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705436622


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34157/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737016158






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532426860



##########
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:
       > If so, what's the hive behaviour? it will throw an exception instead selecting the last one?
   
   Select the last one
   https://github.com/apache/hive/blob/aed7c86cdd59f0b2a4979633fbd191d451f2fd75/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java#L121-L127




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533087617



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  /**
+   * 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(excludes: Array[String]): String = {
+    excludes.flatMap { excludeString =>
+      val excludes: Array[String] = excludeString.split(",")
+      if (excludes.exists(_.split(":").length != 2)) {
+        throw new URISyntaxException(excludeString,
+          "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(transitives: Array[String]): Boolean = {
+    if (transitives.isEmpty) {
+      false
+    } else {
+      if (transitives.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")
+      }
+      transitives.last.toBoolean
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of URIs of downloaded jars
+   */
+  def resolveMavenDependencies(uri: URI): String = {
+    val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) =
+      DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found null")
+    }
+    if (authority.split(":").length != 3) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found " + authority)
+    }
+
+    val uriQuery = uri.getQuery
+    val queryParams: Array[(String, String)] = if (uriQuery == null) {
+      Array.empty[(String, String)]
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      mapTokens.map(kv => (kv(0), kv(1)))
+    }
+
+    resolveMavenDependencies(
+      parseTransitive(queryParams.filter(_._1.equals("transitive")).map(_._2)),

Review comment:
       We need to respect case-sensitivity for params?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750725173






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750039765


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37874/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749498158


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37808/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745282519


   **[Test build #132831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132831/testReport)** for PR 29966 at commit [`57c351d`](https://github.com/apache/spark/commit/57c351d076da51fd8c41323f16ff8d8fbc4b964b).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735219679


   **[Test build #131901 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131901/testReport)** for PR 29966 at commit [`2200076`](https://github.com/apache/spark/commit/220007647275c294c9a31c0d0a3f0a52507a585c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735049807


   gentle ping @maropu  @xkrogen Sorry for my late reply since busy work, have updated to merge code using DependencyUtils, since it's origin access privilege is  `deploy` package, but now it's. not only. used in `deploy`  so remove to `utils` package


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533370716



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,115 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  def parseQueryParams(uriQuery: String): (Boolean, String) = {

Review comment:
       private?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731898421


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36125/
   


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533837188



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -955,6 +978,121 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
         .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop()
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url -- default transitive = false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- invalid transitive use default false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- transitive=true will download dependency jars") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0" +
+      "?exclude=commons-lang:commons-lang&transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test different version") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.6.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.6.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test invalid param") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?invalidParam=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" +
+      "transitive=true&transitive=false&transitive=invalidValue")

Review comment:
       Could you add tests for `?transitive=true&transitive=invalidValue`, too?
   https://github.com/apache/spark/pull/29966#discussion_r533434365




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739450061


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36880/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745225820


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132819/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737348862






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744974124


   retest this please


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739258237


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36874/
   


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547730366



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1959,47 +1959,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {

Review comment:
       If we want to use `plural`, it's `schemes`.




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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731887480


   **[Test build #131524 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131524/testReport)** for PR 29966 at commit [`9161340`](https://github.com/apache/spark/commit/9161340abd46471252e2e933fd019cd72446828b).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744227011


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132744/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547731808



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1959,47 +1959,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {
         // For local paths with backslashes on Windows, URI throws an exception
-        addLocalJarFile(new File(path))
+        (addLocalJarFile(new File(path)), "local")
       } else {
         val uri = new Path(path).toUri
         // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies
         Utils.validateURL(uri)
-        uri.getScheme match {
+        val uriSchema = uri.getScheme

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1959,47 +1959,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {

Review comment:
       > `schema` -> `scheme` because `scheme` is different from `schema`.
   
   Very embarrassed, updated




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736066611


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132004/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735091489


   **[Test build #131897 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131897/testReport)** for PR 29966 at commit [`d2c1950`](https://github.com/apache/spark/commit/d2c1950451e5d9b1a27e9b6d8a6d3540b9a60b4d).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-733035516






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705073412






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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750039765


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37874/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732734928






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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735337456


   **[Test build #131921 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131921/testReport)** for PR 29966 at commit [`875d8a7`](https://github.com/apache/spark/commit/875d8a7b640b8112d280e49d827677215e3fb1b8).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737348862






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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547037172



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1959,47 +1959,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {
         // For local paths with backslashes on Windows, URI throws an exception
-        addLocalJarFile(new File(path))
+        (addLocalJarFile(new File(path)), "local")
       } else {
         val uri = new Path(path).toUri
         // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies
         Utils.validateURL(uri)
-        uri.getScheme match {
+        val uriSchema = uri.getScheme
+        val jarPaths = uriSchema match {
           // A JAR file which exists only on the driver node
           case null =>
             // SPARK-22585 path without schema is not url encoded
             addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Seq("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))
+              .flatMap(jar => addLocalJarFile(new File(jar)))
           case _ => checkRemoteJarFile(path)
         }
+        (jarPaths, uriSchema)
       }
-      if (key != null) {
+      if (keys.nonEmpty) {
         val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis
-        if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
-          logInfo(s"Added JAR $path at $key with timestamp $timestamp")
+        val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty)
+        if (added.nonEmpty) {
+          val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri"
+          logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp")
           postEnvironmentUpdate()
-        } else {
-          logWarning(s"The jar $path has been added already. Overwriting of added jars " +
-            "is not supported in the current version.")
+        }
+        if (existed.nonEmpty) {
+          val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri"
+          logInfo(s"The $jarMessage $path at ${existed.mkString(",")} has been added already." +
+            s" Overwriting of added jar is not supported in the current version.")

Review comment:
       nit: remove `s`

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted

Review comment:
       nit: could you move `.sorted` inside the `if` block below?

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)

Review comment:
       nit: remove `s`

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " +
+          s"in ivy uri query `$uriQuery`.")
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    val ivyProperties = DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy url authority in uri ${uri.toString}:" +
+          s" Expected 'org:module:version', found null.")

Review comment:
       nit: remove `s`.

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)
+        |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
+    sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
+    sql("select * from src join t1 on src.key = t1.a")

Review comment:
       nit: please use uppercases for SQL keywords where possible.

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)
+        |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
+    sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
+    sql("select * from src join t1 on src.key = t1.a")
+    sql("DROP TABLE t1")

Review comment:
       We cannot use `withTable` here?

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {

Review comment:
       Could you add tests for the same purpose in `sql/core`, too?

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)

Review comment:
       nit: `t1` -> `t`

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " +
+          s"in ivy uri query `$uriQuery`.")
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    val ivyProperties = DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy url authority in uri ${uri.toString}:" +
+          s" Expected 'org:module:version', found null.")
+    }
+    if (authority.split(":").length != 3) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy uri authority in uri ${uri.toString}:" +

Review comment:
       nti: `ivy uri` -> `Ivy URI`? I have the same question for the other error messages.




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739596092


   > @maropu are there any more blockers from your side? I'd love to get this into the 3.1 release if possible.
   
   I'm neutral on this, but we don't basically merge a new feature into the release branch (v3.1) after the branch-cut. cc: @HyukjinKwon @dongjoon-hyun @gatorsmile 


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750113613


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37887/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750516258


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37915/
   


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533923652



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1890,47 +1890,66 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      var schema = ""
+      val keys = if (path.contains("\\") && Utils.isWindows) {
         // For local paths with backslashes on Windows, URI throws an exception
         addLocalJarFile(new File(path))
       } else {
         val uri = new Path(path).toUri
         // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies
         Utils.validateURL(uri)
-        uri.getScheme match {
+        schema = uri.getScheme
+        schema match {
           // A JAR file which exists only on the driver node
           case null =>
             // SPARK-22585 path without schema is not url encoded
             addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Seq("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.nonEmpty) {
         val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis
-        if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
-          logInfo(s"Added JAR $path at $key with timestamp $timestamp")
+        val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty)
+        if (added.nonEmpty) {
+          if (schema != "ivy") {
+            logInfo(s"Added JAR $path at ${added.mkString(",")} with timestamp $timestamp")
+          } else {
+            logInfo(s"Added dependency jars of ivy uri $path at ${added.mkString(",")}" +
+              s" with timestamp $timestamp")

Review comment:
       nit: 
   ```
             val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri"
             logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp")
   ```




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735050580






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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547752608



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy URI need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of Ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in Ivy URI ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an Ivy URI, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 1) {
+        logWarning("It's best to specify `transitive` parameter in ivy URI query only once." +
+          " If there are multiple `transitive` parameter, we will select the last one")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)

Review comment:
       Oh, never mind. I found the test cases.
   ```
   test("SPARK-33084: Add jar support Ivy URI -- test param case sensitive") {
   test("SPARK-33084: Add jar support Ivy URI -- test transitive value case sensitive") {
   ```

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy URI need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of Ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in Ivy URI ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an Ivy URI, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 1) {
+        logWarning("It's best to specify `transitive` parameter in ivy URI query only once." +
+          " If there are multiple `transitive` parameter, we will select the last one")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)

Review comment:
       ~This `true` looks like case sensitive. Shall we do this in a case-insensitive way?~




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739685603


   retest this please


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-704880265


   **[Test build #129507 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129507/testReport)** for PR 29966 at commit [`51daf9a`](https://github.com/apache/spark/commit/51daf9aa19e425e317eef9991a31f52261cb303f).


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533845204



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -955,6 +978,121 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
         .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop()
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url -- default transitive = false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- invalid transitive use default false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- transitive=true will download dependency jars") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0" +
+      "?exclude=commons-lang:commons-lang&transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test different version") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.6.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.6.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test invalid param") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?invalidParam=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" +
+      "transitive=true&transitive=false&transitive=invalidValue")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" +
+      "transitive=false&transitive=invalidValue&transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test param case sensitive") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?TRANSITIVE=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test transitive value case sensitive") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=TRUE")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+

Review comment:
       yea




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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547763450



##########
File path: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
##########
@@ -1348,6 +1348,7 @@ private[spark] object SparkSubmitUtils {
       coordinates: String,
       ivySettings: IvySettings,
       exclusions: Seq[String] = Nil,
+      transitive: Boolean = true,

Review comment:
       What happens if we remove this default value? If it doesn't make many changes, let's remove this default value.




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750711742


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133328/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739450063


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36880/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749450128


   **[Test build #133198 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133198/testReport)** for PR 29966 at commit [`8c53b83`](https://github.com/apache/spark/commit/8c53b83d1650a69b4225cdbca4fd26d1d5537d94).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744379994


   Any more update?


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-738555064


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132170/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739255490


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36874/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532575369



##########
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:
       > It seems you don't push the latest commit.
   
   Yea... back network push. failed...

##########
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:
       > It seems you don't push the latest commit.
   
   Yea... back network push. failed...pushed now




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739679742


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132321/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750516314


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37915/
   


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547747201



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -445,6 +445,29 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("SPARK-33084: add jar when path contains comma") {

Review comment:
       Do we really need to support this `comma`?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750712178


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37920/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735346176






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739110962


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132254/
   


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


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

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r534310229



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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] = {

Review comment:
       Sounds good.




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735174280






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r531971599



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2980,6 +2980,77 @@ private[spark] object Utils extends Logging {
     metadata.toString
   }
 
+  /**
+   * 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) =
+      Seq(
+        "spark.jars.repositories",
+        "spark.jars.ivy",
+        "spark.jars.ivySettings"
+      ).map(sys.props.get(_).orNull)
+    // Create the IvySettings, either load from file or build defaults
+    val ivySettings = Option(ivySettingsPath) match {
+      case Some(path) =>
+        SparkSubmitUtils.loadIvySettings(path, Option(repositories), Option(ivyRepoPath))
+
+      case None =>
+        SparkSubmitUtils.buildIvySettings(Option(repositories), Option(ivyRepoPath))
+    }
+    SparkSubmitUtils.resolveMavenCoordinates(uri.getAuthority, ivySettings,
+      parseExcludeList(uri.getQuery), parseTransitive(uri.getQuery))
+  }
+
+  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 query string: " + queryString)

Review comment:
       Added




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744227011


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132744/
   


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739596538


   Thanks for this work, @AngersZhuuuu . Btw, are you planning to support a `ivy` schema in `--packages`, too?


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533920784



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1890,47 +1890,66 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      var schema = ""

Review comment:
       `val schema = uri.getScheme`? Please avoid to use `var` where possible.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735856350


   **[Test build #131989 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131989/testReport)** for PR 29966 at commit [`8c5cb7c`](https://github.com/apache/spark/commit/8c5cb7ccb21cf3b9fbf4d87da1c5f9991448d035).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533845770



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -366,6 +366,29 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("SPARK-33084: add jar when path contains comma") {
+    withTempDir { tmpDir =>
+      val tmpJar = File.createTempFile("Test,UDTF", ".jar", tmpDir)
+      sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+      sc.addJar(tmpJar.getAbsolutePath)
+      assert(sc.listJars().exists(_.contains("Test,UDTF")))
+
+      val jarPath = "hdfs:///no/path/to/Test,UDTF1.jar"
+      sc.addJar(jarPath)
+      // Add jar failed since file not exists
+      assert(!sc.listJars().exists(_.contains("/no/path/to/Test,UDTF.jar")))
+
+      Seq("http", "https", "ftp").foreach { scheme =>
+        val badURL = s"$scheme://user:pwd/path/Test,UDTF_${scheme}.jar"
+        val e1 = intercept[MalformedURLException] {

Review comment:
       DOne




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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-738498715


   **[Test build #132170 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132170/testReport)** for PR 29966 at commit [`ebe1c9c`](https://github.com/apache/spark/commit/ebe1c9ccb75898b5c68a570d43d89444d4c92970).


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533931068



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,158 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(token =>
+        token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+        throw new URISyntaxException(uri.toString, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive: Boolean = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(token =>
+            token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+            throw new URISyntaxException(uri.toString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val invalidParams = groupedParams
+        .filter(entry => !Seq("transitive", "exclude").contains(entry._1))
+        .keys.toArray.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(
+          s"Invalid parameters `${invalidParams.mkString(",")}` found in URI query `$uriQuery`.")
+      }
+
+      groupedParams.foreach { case (key: String, values: Array[(String, String)]) =>
+        if (key != "transitive" || key != "exclude") {
+          logWarning("Invalid parameter")
+        }
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    try {
+      val ivyProperties = DependencyUtils.getIvyProperties()
+      val authority = uri.getAuthority
+      if (authority == null) {
+        throw new URISyntaxException(

Review comment:
       Why don't you use `IllegalArgumentException ` directly?




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735050901






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532399083



##########
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?
   
   https://github.com/apache/hive/blob/aed7c86cdd59f0b2a4979633fbd191d451f2fd75/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java#L143-L146
   

##########
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?
   
   Hive logical https://github.com/apache/hive/blob/aed7c86cdd59f0b2a4979633fbd191d451f2fd75/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java#L116-L119
   




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739795578


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132337/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736441080


   > Thanks! Could you move the screenshot into the PR description?
   
   Updated


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750711742


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133328/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749495766


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133210/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744226156


   **[Test build #132744 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132744/testReport)** for PR 29966 at commit [`d53f302`](https://github.com/apache/spark/commit/d53f3021de54d4072b450059dde91d6cf79c00f5).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750495269


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37915/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745429482


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132831/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739642660


   **[Test build #132321 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132321/testReport)** for PR 29966 at commit [`afea73f`](https://github.com/apache/spark/commit/afea73fcadc1d6641c6feb1b07b7fc874b836b29).


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


[GitHub] [spark] AngersZhuuuu removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732532614


   > Ah, I see. This looks useful. I think we need to document it. Otherwise, user cannot notice it. https://spark.apache.org/docs/3.0.1/sql-ref-syntax-aux-resource-mgmt-add-jar.html
   
   Yea, will update the guide toody


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731900404


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36127/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745282519


   **[Test build #132831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132831/testReport)** for PR 29966 at commit [`57c351d`](https://github.com/apache/spark/commit/57c351d076da51fd8c41323f16ff8d8fbc4b964b).


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r536459776



##########
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:
 
+     * transitive: whether to download dependent jars related to your ivy URL.

Review comment:
       > "It is case-sensitive and only take last one if multiple transitive parameters are specified. Invalid value will be treated as false."
   
   Done and update snapshot in desc.




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


[GitHub] [spark] dongjoon-hyun commented on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-706272109


   Got it. Thank you, @AngersZhuuuu 


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547732346



##########
File path: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
##########
@@ -304,8 +304,8 @@ private[spark] class SparkSubmit extends Logging {
       // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files
       // too for packages that include Python code
       val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(
-        args.packagesExclusions, args.packages, args.repositories, args.ivyRepoPath,
-        args.ivySettingsPath)
+        true, args.packagesExclusions, args.packages,

Review comment:
       nit. `true` -> `transitive = true`.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750712468


   **[Test build #133329 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133329/testReport)** for PR 29966 at commit [`4c44dae`](https://github.com/apache/spark/commit/4c44daecc7c77527e150d30051170f7ee8667f70).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739737919


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36937/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731885897


   **[Test build #131522 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131522/testReport)** for PR 29966 at commit [`b3e3211`](https://github.com/apache/spark/commit/b3e3211cd0d3d96bcf7a021d41553619dc894125).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532399380



##########
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?
   
   Hive logical https://github.com/apache/hive/blob/aed7c86cdd59f0b2a4979633fbd191d451f2fd75/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java#L116-L119
   
   IMO, return a `URISyntaxException ` too is better




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-704920515


   Merged build finished. Test FAILed.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739457605


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132279/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750712775


   **[Test build #133335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133335/testReport)** for PR 29966 at commit [`4c44dae`](https://github.com/apache/spark/commit/4c44daecc7c77527e150d30051170f7ee8667f70).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731885901


   Merged build finished. Test FAILed.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745225351


   **[Test build #132819 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132819/testReport)** for PR 29966 at commit [`57c351d`](https://github.com/apache/spark/commit/57c351d076da51fd8c41323f16ff8d8fbc4b964b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533495197



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1919,18 +1919,24 @@ class SparkContext(config: SparkConf) extends Logging {
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Seq("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))

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,115 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  def parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach(value => {
+          if (value == "true") {
+            transitive = true
+          } else if (value == "false") {
+            transitive = false
+          }
+        })
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.flatMap { case (_, excludeString) =>
+          val excludes = excludeString.split(",")
+          if (excludes.exists(_.split(":").length != 2)) {
+            throw new URISyntaxException(excludeString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      (transitive, exclusionList)

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  /**
+   * 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(excludes: Array[String]): String = {
+    excludes.flatMap { excludeString =>
+      val excludes: Array[String] = excludeString.split(",")
+      if (excludes.exists(_.split(":").length != 2)) {
+        throw new URISyntaxException(excludeString,
+          "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(transitives: Array[String]): Boolean = {
+    if (transitives.isEmpty) {
+      false
+    } else {
+      if (transitives.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")
+      }
+      transitives.last.toBoolean
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of URIs of downloaded jars
+   */
+  def resolveMavenDependencies(uri: URI): String = {
+    val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) =
+      DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found null")
+    }
+    if (authority.split(":").length != 3) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found " + authority)
+    }
+
+    val uriQuery = uri.getQuery
+    val queryParams: Array[(String, String)] = if (uriQuery == null) {
+      Array.empty[(String, String)]
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      mapTokens.map(kv => (kv(0), kv(1)))
+    }
+
+    resolveMavenDependencies(
+      parseTransitive(queryParams.filter(_._1.equals("transitive")).map(_._2)),

Review comment:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533429295



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,115 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  def parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach(value => {
+          if (value == "true") {
+            transitive = true
+          } else if (value == "false") {
+            transitive = false
+          }

Review comment:
       Since hive is  https://github.com/apache/hive/blob/cb2ac3dcc6af276c6f64ee00f034f082fe75222b/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java#L122-L126
   change to 
   ```
   params.map(_._2).foreach {
             case "true" => transitive = true
             case _ => transitive = false
           }
   ```




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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705127815


   **[Test build #129520 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129520/testReport)** for PR 29966 at commit [`d6e8caf`](https://github.com/apache/spark/commit/d6e8caf69d33d41b55b1d71768582886dae493ad).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737021222






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


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

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r537648126



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1890,47 +1890,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {
         // For local paths with backslashes on Windows, URI throws an exception
-        addLocalJarFile(new File(path))
+        (addLocalJarFile(new File(path)), "local")
       } else {
         val uri = new Path(path).toUri
         // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies
         Utils.validateURL(uri)
-        uri.getScheme match {
+        val uriSchema = uri.getScheme
+        val jarPaths = uriSchema match {
           // A JAR file which exists only on the driver node
           case null =>
             // SPARK-22585 path without schema is not url encoded
             addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Seq("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))
+              .map(jar => env.rpcEnv.fileServer.addJar(new File(jar)))

Review comment:
       Should we use the existing method for the sake of consistency:
   ```
   .map(jar => addLocalJarFile(new File(jar))
   ```
   Core logic will be the same, but there are some additional checks (verifies that `DependencyUtils` did its job correctly) and consolidates the logic.




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736236902


   For other reviewers, could you put the screenshot of the updated doc in the PR description?


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739631644


   Wait..seems we all ignore how can we pass these Jars to executor side....


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739110962


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132254/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739795578


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132337/
   


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547745481



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -445,6 +445,29 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("SPARK-33084: add jar when path contains comma") {

Review comment:
       For a negative test case, could you be more specific? Otherwise, this test case name can be misleading.
   ```scala
   - test("SPARK-33084: add jar when path contains comma") {
   + test("SPARK-33084: add jar should fail when path contains comma") {
   ```




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547823920



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
##########
@@ -159,6 +161,13 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade
     }
   }
 
+  protected def resolveJars(path: URI): Seq[String] = {

Review comment:
       > Although this class is `SessionResourceLoader `, `resolveJars` looks like more a utility function. Do we need to add here as `protected def`?
   
   remove `protect` since user or other feature can use this 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.

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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533844056



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false

Review comment:
       > To be a bit more functional / Scala-idiomatic, I think we can do something like `groupedParams.get("transitive").takeRight(1).headOption.getOrElse(false)` instead of the `foreach` call
   
   In this way will lose warning message?

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {

Review comment:
       Good suggestion




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533845979



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -955,6 +978,121 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
         .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop()
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url -- default transitive = false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- invalid transitive use default false") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=foo")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(!sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- transitive=true will download dependency jars") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0" +
+      "?exclude=commons-lang:commons-lang&transitive=true")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    assert(sc.listJars().exists(_.contains("org.slf4j_slf4j-api-1.7.10.jar")))
+    assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
+  }
+
+  test("SPARK-33084: Add jar support ivy url -- test different version") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]"))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar")))
+    sc.addJar("ivy://org.apache.hive:hive-storage-api:2.6.0")
+    assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.6.0.jar")))

Review comment:
       > Could you add the jars first, then check if the dependent jars co-exist in `sc.listJars`?
   
   Done




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533088945



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  /**
+   * 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(excludes: Array[String]): String = {
+    excludes.flatMap { excludeString =>
+      val excludes: Array[String] = excludeString.split(",")
+      if (excludes.exists(_.split(":").length != 2)) {
+        throw new URISyntaxException(excludeString,
+          "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(transitives: Array[String]): Boolean = {
+    if (transitives.isEmpty) {
+      false
+    } else {
+      if (transitives.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")
+      }
+      transitives.last.toBoolean
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of URIs of downloaded jars
+   */
+  def resolveMavenDependencies(uri: URI): String = {
+    val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) =
+      DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found null")
+    }
+    if (authority.split(":").length != 3) {
+      throw new URISyntaxException(
+        authority, "Invalid url: Expected 'org:module:version', found " + authority)
+    }
+
+    val uriQuery = uri.getQuery
+    val queryParams: Array[(String, String)] = if (uriQuery == null) {
+      Array.empty[(String, String)]
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      mapTokens.map(kv => (kv(0), kv(1)))
+    }
+
+    resolveMavenDependencies(
+      parseTransitive(queryParams.filter(_._1.equals("transitive")).map(_._2)),

Review comment:
       > We need to respect case-sensitivity for params?
   
   Normally we use lowercase, respect may be good, and hive respect too.




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r534256199



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,158 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(token =>
+        token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+        throw new URISyntaxException(uri.toString, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive: Boolean = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(token =>
+            token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {

Review comment:
       Good suggestion




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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736996100


   **[Test build #132021 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132021/testReport)** for PR 29966 at commit [`9c22882`](https://github.com/apache/spark/commit/9c228823e0be56a87ebc498c254c627babc9db45).


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736624034


   Split UT to make it clear and add more detail case and use ivy path with simply dependency to speed up test.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r531943937



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2980,6 +2980,77 @@ private[spark] object Utils extends Logging {
     metadata.toString
   }
 
+  /**
+   * 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) =
+      Seq(
+        "spark.jars.repositories",
+        "spark.jars.ivy",
+        "spark.jars.ivySettings"
+      ).map(sys.props.get(_).orNull)
+    // Create the IvySettings, either load from file or build defaults
+    val ivySettings = Option(ivySettingsPath) match {
+      case Some(path) =>
+        SparkSubmitUtils.loadIvySettings(path, Option(repositories), Option(ivyRepoPath))
+
+      case None =>
+        SparkSubmitUtils.buildIvySettings(Option(repositories), Option(ivyRepoPath))
+    }
+    SparkSubmitUtils.resolveMavenCoordinates(uri.getAuthority, ivySettings,
+      parseExcludeList(uri.getQuery), parseTransitive(uri.getQuery))
+  }
+
+  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 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): Array[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
+      }
+  }
+
+  /**
+   * 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 {
+      transitive.last.toBoolean

Review comment:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737405115






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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739660067


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36921/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-751144475


   Merry Christmas! Thanks all for your patient reviews. 


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749445314


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37797/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732518519


   **[Test build #131604 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131604/testReport)** for PR 29966 at commit [`9161340`](https://github.com/apache/spark/commit/9161340abd46471252e2e933fd019cd72446828b).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744151300


   **[Test build #132740 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132740/testReport)** for PR 29966 at commit [`13000f2`](https://github.com/apache/spark/commit/13000f28afe425785602d36edd0c5875bef49753).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731884733


   **[Test build #131522 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131522/testReport)** for PR 29966 at commit [`b3e3211`](https://github.com/apache/spark/commit/b3e3211cd0d3d96bcf7a021d41553619dc894125).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-737016158






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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735337596


   **[Test build #131921 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131921/testReport)** for PR 29966 at commit [`875d8a7`](https://github.com/apache/spark/commit/875d8a7b640b8112d280e49d827677215e3fb1b8).
    * This patch **fails Scala style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739294071


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132273/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739444952


   retest this please


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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-704880265


   **[Test build #129507 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129507/testReport)** for PR 29966 at commit [`51daf9a`](https://github.com/apache/spark/commit/51daf9aa19e425e317eef9991a31f52261cb303f).


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


[GitHub] [spark] AngersZhuuuu edited a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu edited a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735049807


   gentle ping @maropu  @xkrogen Sorry for my later reply since busy work, have updated to merge code using DependencyUtils, since it's origin access privilege is  `deploy` package, but now it's. not only. used in `deploy`  so remove to `utils` package. Also add some UT about query format


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


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

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r554514508



##########
File path: core/src/test/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+
+import org.apache.spark.SparkFunSuite
+
+class DependencyUtilsSuite extends SparkFunSuite {
+
+  test("SPARK-33084: Add jar support Ivy URI -- test invalid ivy uri") {
+    val e1 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(URI.create("ivy://"))
+    }.getMessage
+    assert(e1.contains("Expected authority at index 6: ivy://"))
+
+    val e2 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(URI.create("ivy://org.apache.hive:hive-contrib"))
+    }.getMessage
+    assert(e2.contains("Invalid Ivy URI authority in uri ivy://org.apache.hive:hive-contrib:" +
+      " Expected 'org:module:version', found org.apache.hive:hive-contrib."))
+
+    val e3 = intercept[IllegalArgumentException] {
+      DependencyUtils.resolveMavenDependencies(
+        URI.create("ivy://org.apache.hive:hive-contrib:2.3.7?foo="))

Review comment:
       Do we need to keep `2.3.7` consistent with the built-in Hive version?




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745217476


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37421/
   


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


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

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745387320


   @maropu or @mridulm , would you be able to help merge this if there are no more outstanding concerns?


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744168300


   **[Test build #132744 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132744/testReport)** for PR 29966 at commit [`d53f302`](https://github.com/apache/spark/commit/d53f3021de54d4072b450059dde91d6cf79c00f5).


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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547761656



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
##########
@@ -159,6 +161,13 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade
     }
   }
 
+  protected def resolveJars(path: URI): Seq[String] = {

Review comment:
       Although this class is `SessionResourceLoader `, `resolveJars` looks like more a utility function. Do we need to add here as `protected def`?




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750701449


   > Could you answer this question?
   > 
   > * https://github.com/apache/spark/pull/29966/files#r547763450
   > 
   > > What happens if we remove this default value? If it doesn't make many changes, let's remove this default value.
   
   Sorry for missing this one.  Replied  and changed.


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532533545



##########
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:
       It seems you don't push the latest commit.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750702317


   **[Test build #133328 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133328/testReport)** for PR 29966 at commit [`4c44dae`](https://github.com/apache/spark/commit/4c44daecc7c77527e150d30051170f7ee8667f70).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750113479


   **[Test build #133276 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133276/testReport)** for PR 29966 at commit [`6bd41cd`](https://github.com/apache/spark/commit/6bd41cd85462a6bef0e8be61e9a61c2d628bc8fe).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749495941


   **[Test build #133211 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133211/testReport)** for PR 29966 at commit [`2ffb431`](https://github.com/apache/spark/commit/2ffb4319bcd651f0d63d513f4f9c72e116b37318).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731898429


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36125/
   Test FAILed.


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732725669


   **[Test build #131632 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131632/testReport)** for PR 29966 at commit [`63e877b`](https://github.com/apache/spark/commit/63e877b64f7a61888ea59a2c632ef5e9373e087e).


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533928805



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,158 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(token =>
+        token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+        throw new URISyntaxException(uri.toString, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive: Boolean = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(token =>
+            token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+            throw new URISyntaxException(uri.toString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val invalidParams = groupedParams
+        .filter(entry => !Seq("transitive", "exclude").contains(entry._1))
+        .keys.toArray.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(
+          s"Invalid parameters `${invalidParams.mkString(",")}` found in URI query `$uriQuery`.")
+      }
+
+      groupedParams.foreach { case (key: String, values: Array[(String, String)]) =>
+        if (key != "transitive" || key != "exclude") {
+          logWarning("Invalid parameter")
+        }
+      }

Review comment:
       What's this?




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r531917374



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2980,6 +2980,77 @@ private[spark] object Utils extends Logging {
     metadata.toString
   }
 
+  /**
+   * Download Ivy URIs dependent jars.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Comma separated string list of URIs of downloaded jars

Review comment:
       > Should we be returning a `List[String]` instead of `String` (here and in `SparkSubmitUtils`)? It seems odd to have `SparkSubmitUtils` do a `mkString` to convert a list to string, then re-convert back to a list later.
   
   It's a nice suggestion。
   Here return `String` since `SparkSubmit.resolveMavenCoordinates` return `String` if we change these method will change a lot and it's not related to current pr, how about start a new pr after or before current pr to refactor these code about return `Array[String]`?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749445314


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37797/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739660054


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36921/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547752259



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy URI need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of Ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in Ivy URI ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an Ivy URI, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 1) {
+        logWarning("It's best to specify `transitive` parameter in ivy URI query only once." +
+          " If there are multiple `transitive` parameter, we will select the last one")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)

Review comment:
       > This `true` looks like case sensitive. Shall we do this in a case-insensitive way?
   
   We discuss this before, since hive is case-sensitive and we always write ivy uri in lowercase so make it case-sensitive




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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547110204



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)
+        |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
+    sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
+    sql("select * from src join t1 on src.key = t1.a")
+    sql("DROP TABLE t1")

Review comment:
       Done

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)

Review comment:
       DONE

##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {
+    val testData = TestHive.getHiveFile("data/files/sample.json").toURI
+    sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7")
+    sql(
+      """CREATE TABLE t1(a string, b string)
+        |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
+    sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
+    sql("select * from src join t1 on src.key = t1.a")

Review comment:
       DONE

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " +
+          s"in ivy uri query `$uriQuery`.")
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    val ivyProperties = DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy url authority in uri ${uri.toString}:" +
+          s" Expected 'org:module:version', found null.")
+    }
+    if (authority.split(":").length != 3) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy uri authority in uri ${uri.toString}:" +

Review comment:
       Done, all place change to Ivy URI

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " +
+          s"in ivy uri query `$uriQuery`.")
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    val ivyProperties = DependencyUtils.getIvyProperties()
+    val authority = uri.getAuthority
+    if (authority == null) {
+      throw new IllegalArgumentException(
+        s"Invalid ivy url authority in uri ${uri.toString}:" +
+          s" Expected 'org:module:version', found null.")

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val validParams = Set("transitive", "exclude")
+      val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -25,12 +25,140 @@ 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  private def isInvalidQueryString(tokens: Array[String]): Boolean = {
+    tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1))
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(isInvalidQueryString)) {
+        throw new IllegalArgumentException(
+          s"Invalid query string in ivy uri ${uri.toString}: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      val transitiveParams = groupedParams.get("transitive")
+      if (transitiveParams.map(_.size).getOrElse(0) > 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")
+      }
+      val transitive =
+        transitiveParams.flatMap(_.takeRight(1).map(_._2 == "true").headOption).getOrElse(false)
+
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(isInvalidQueryString)) {
+            throw new IllegalArgumentException(
+              s"Invalid exclude string in ivy uri ${uri.toString}:" +
+                s" expected 'org:module,org:module,..', found " + excludeString)

Review comment:
       Done

##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1959,47 +1959,58 @@ class SparkContext(config: SparkConf) extends Logging {
             throw new IllegalArgumentException(
               s"Directory ${path} is not allowed for addJar")
           }
-          path
+          Seq(path)
         } catch {
           case NonFatal(e) =>
             logError(s"Failed to add $path to Spark environment", e)
-            null
+            Nil
         }
       } else {
-        path
+        Seq(path)
       }
     }
 
     if (path == null || path.isEmpty) {
       logWarning("null or empty path specified as parameter to addJar")
     } else {
-      val key = if (path.contains("\\") && Utils.isWindows) {
+      val (keys, schema) = if (path.contains("\\") && Utils.isWindows) {
         // For local paths with backslashes on Windows, URI throws an exception
-        addLocalJarFile(new File(path))
+        (addLocalJarFile(new File(path)), "local")
       } else {
         val uri = new Path(path).toUri
         // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies
         Utils.validateURL(uri)
-        uri.getScheme match {
+        val uriSchema = uri.getScheme
+        val jarPaths = uriSchema match {
           // A JAR file which exists only on the driver node
           case null =>
             // SPARK-22585 path without schema is not url encoded
             addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Seq("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))
+              .flatMap(jar => addLocalJarFile(new File(jar)))
           case _ => checkRemoteJarFile(path)
         }
+        (jarPaths, uriSchema)
       }
-      if (key != null) {
+      if (keys.nonEmpty) {
         val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis
-        if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
-          logInfo(s"Added JAR $path at $key with timestamp $timestamp")
+        val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty)
+        if (added.nonEmpty) {
+          val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri"
+          logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp")
           postEnvironmentUpdate()
-        } else {
-          logWarning(s"The jar $path has been added already. Overwriting of added jars " +
-            "is not supported in the current version.")
+        }
+        if (existed.nonEmpty) {
+          val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri"
+          logInfo(s"The $jarMessage $path at ${existed.mkString(",")} has been added already." +
+            s" Overwriting of added jar is not supported in the current version.")

Review comment:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750485587


   **[Test build #133323 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133323/testReport)** for PR 29966 at commit [`75ff3ce`](https://github.com/apache/spark/commit/75ff3ce59cd43a9baa6b6c128f7bd4ca8607c277).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750204393


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133291/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r548346741



##########
File path: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
##########
@@ -1348,6 +1348,7 @@ private[spark] object SparkSubmitUtils {
       coordinates: String,
       ivySettings: IvySettings,
       exclusions: Seq[String] = Nil,
+      transitive: Boolean = true,

Review comment:
       > What happens if we remove this default value? If it doesn't make many changes, let's remove this default value.
   
   we can remove this default value and then add this in place where use this method.
   ![image](https://user-images.githubusercontent.com/46485123/103049810-b871a580-45cd-11eb-8750-91eeab214ec4.png)
   




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


[GitHub] [spark] AmplabJenkins commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705447233






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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744397283


   **[Test build #132768 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132768/testReport)** for PR 29966 at commit [`d53f302`](https://github.com/apache/spark/commit/d53f3021de54d4072b450059dde91d6cf79c00f5).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735050130


   **[Test build #131896 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131896/testReport)** for PR 29966 at commit [`7f878c2`](https://github.com/apache/spark/commit/7f878c2f675470fbeae12bdce2bfaf26971a16f4).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732803609






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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750035724


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37874/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532394800



##########
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:
       Short is better

##########
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:
       Done




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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533931068



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,158 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(token =>
+        token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+        throw new URISyntaxException(uri.toString, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive: Boolean = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(token =>
+            token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+            throw new URISyntaxException(uri.toString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val invalidParams = groupedParams
+        .filter(entry => !Seq("transitive", "exclude").contains(entry._1))
+        .keys.toArray.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(
+          s"Invalid parameters `${invalidParams.mkString(",")}` found in URI query `$uriQuery`.")
+      }
+
+      groupedParams.foreach { case (key: String, values: Array[(String, String)]) =>
+        if (key != "transitive" || key != "exclude") {
+          logWarning("Invalid parameter")
+        }
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    try {
+      val ivyProperties = DependencyUtils.getIvyProperties()
+      val authority = uri.getAuthority
+      if (authority == null) {
+        throw new URISyntaxException(

Review comment:
       Why do you use `URISyntaxException` instead of `IllegalArgumentException `?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732844619


   **[Test build #131638 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131638/testReport)** for PR 29966 at commit [`b60ba1e`](https://github.com/apache/spark/commit/b60ba1e3eebd5968fb017d537a98637289ba112b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r534312342



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false

Review comment:
       Maybe:
   ```
   val transitiveParams = groupedParams.get("transitive")
   if (transitiveParams.map(_.size).getOrElse(0) > 1) {
     // log warning
   }
   val transitive = transitiveParams.flatMap(_.takeRight(1).headOption).getOrElse(false)
   ```




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735177484


   **[Test build #131901 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131901/testReport)** for PR 29966 at commit [`2200076`](https://github.com/apache/spark/commit/220007647275c294c9a31c0d0a3f0a52507a585c).


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735786330


   **[Test build #131989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131989/testReport)** for PR 29966 at commit [`8c5cb7c`](https://github.com/apache/spark/commit/8c5cb7ccb21cf3b9fbf4d87da1c5f9991448d035).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-732803609






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r529245969



##########
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)`

Review comment:
       Done




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731976658






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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735219860






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731918610


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131524/
   Test FAILed.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750572936


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133323/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739450063


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36880/
   


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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744498324


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132768/
   


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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705059880


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34124/
   


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533081191



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -366,6 +366,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("add jar local path with comma") {
+    sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
+    sc.addJar("file://Test,UDTF.jar")
+    assert(!sc.listJars().exists(_.contains("UDTF.jar")))

Review comment:
       What does this test mean? Why didn't you do ` assert(sc.listJars().exists(_.contains("Test,UDTF.jar")))`?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705447233






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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533368869



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1919,18 +1919,24 @@ class SparkContext(config: SparkConf) extends Logging {
           // A JAR file which exists only on the driver node
           case "file" => addLocalJarFile(new File(uri.getPath))
           // A JAR file which exists locally on every worker node
-          case "local" => "file:" + uri.getPath
+          case "local" => Array("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)).split(",")
           case _ => checkRemoteJarFile(path)
         }
       }
-      if (key != null) {
+      if (keys.nonEmpty) {
         val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis
-        if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
-          logInfo(s"Added JAR $path at $key with timestamp $timestamp")
-          postEnvironmentUpdate()
-        } else {
-          logWarning(s"The jar $path has been added already. Overwriting of added jars " +
-            "is not supported in the current version.")
+        keys.foreach { key =>
+          if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
+            logInfo(s"Added JAR $path at $key with timestamp $timestamp")
+            postEnvironmentUpdate()
+          } else {
+            logWarning(s"The jar $path has been added already. Overwriting of added jars " +

Review comment:
       Looks fine now, but I think we need to make the error message more general because `dependencies` is only meaningful for the `ivy` schema case.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750204393


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/133291/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739445344


   **[Test build #132279 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132279/testReport)** for PR 29966 at commit [`6034fb2`](https://github.com/apache/spark/commit/6034fb25fbb8cd04b040f0d88ce117b1a54ee0ac).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749988240


   **[Test build #133276 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133276/testReport)** for PR 29966 at commit [`6bd41cd`](https://github.com/apache/spark/commit/6bd41cd85462a6bef0e8be61e9a61c2d628bc8fe).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744196913


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37344/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r532394546



##########
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:
       Yea




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744169973


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37341/
   


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-751134606


   FYI:  @gatorsmile @cloud-fan


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750718735


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37920/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744187053


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37344/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-745016830


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37393/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744474475


   **[Test build #132768 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132768/testReport)** for PR 29966 at commit [`d53f302`](https://github.com/apache/spark/commit/d53f3021de54d4072b450059dde91d6cf79c00f5).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547206149



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
##########
@@ -1219,6 +1219,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
       }
     }
   }
+
+  test("SPARK-33084: Add jar support ivy url in SQL") {

Review comment:
       > Could you add tests for the same purpose in `sql/core`, too?
   
   Add UT `test("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") `
   ```
   class Spark33084 extends UserDefinedAggregateFunction {
     // Data types of input arguments of this aggregate function
     def inputSchema: StructType = StructType(StructField("inputColumn", LongType) :: Nil)
     // Data types of values in the aggregation buffer
     def bufferSchema: StructType = {
       StructType(StructField("sum", LongType) :: StructField("count", LongType) :: Nil)
     }
     // The data type of the returned value
     def dataType: DataType = DoubleType
     // Whether this function always returns the same output on the identical input
     def deterministic: Boolean = true
     // Initializes the given aggregatPARKion buffer. The buffer itself is a `Row` that in addition to
     // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides
     // the opportunity to update its values. Note that arrays and maps inside the buffer are still
     // immutable.
     def initialize(buffer: MutableAggregationBuffer): Unit = {
       buffer(0) = 0L
       buffer(1) = 0L
     }
     // Updates the given aggregation buffer `buffer` with new input data from `input`
     def update(buffer: MutableAggregationBuffer, input: Row): Unit = {
       if (!input.isNullAt(0)) {
         buffer(0) = buffer.getLong(0) + input.getLong(0)
         buffer(1) = buffer.getLong(1) + 1
       }
     }
     // Merges two aggregation buffers and stores the updated buffer values back to `buffer1`
     def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = {
       buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0)
       buffer1(1) = buffer1.getLong(1) + buffer2.getLong(1)
     }
     // Calculates the final result
     def evaluate(buffer: Row): Double = buffer.getLong(0).toDouble / buffer.getLong(1)
   }
   ```




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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739702575


   **[Test build #132337 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132337/testReport)** for PR 29966 at commit [`afea73f`](https://github.com/apache/spark/commit/afea73fcadc1d6641c6feb1b07b7fc874b836b29).


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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731919664


   **[Test build #131531 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131531/testReport)** for PR 29966 at commit [`9161340`](https://github.com/apache/spark/commit/9161340abd46471252e2e933fd019cd72446828b).


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533372737



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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)
+  }
+
+  /**
+   * 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(excludes: Array[String]): String = {
+    excludes.flatMap { excludeString =>
+      val excludes: Array[String] = excludeString.split(",")
+      if (excludes.exists(_.split(":").length != 2)) {
+        throw new URISyntaxException(excludeString,
+          "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(transitives: Array[String]): Boolean = {
+    if (transitives.isEmpty) {
+      false
+    } else {
+      if (transitives.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")
+      }
+      transitives.last.toBoolean

Review comment:
       The current behaviour is the same with hive?




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


[GitHub] [spark] SparkQA removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739089837


   **[Test build #132254 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132254/testReport)** for PR 29966 at commit [`6034fb2`](https://github.com/apache/spark/commit/6034fb25fbb8cd04b040f0d88ce117b1a54ee0ac).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739454958


   **[Test build #132279 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132279/testReport)** for PR 29966 at commit [`6034fb2`](https://github.com/apache/spark/commit/6034fb25fbb8cd04b040f0d88ce117b1a54ee0ac).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705447233






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-731885906


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131522/
   Test FAILed.


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750704929


   retest this please


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547749884



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -445,6 +445,29 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     }
   }
 
+  test("SPARK-33084: add jar when path contains comma") {

Review comment:
       > Do we really need to support this `comma`?
   
   This is a UT added earlier.
   In current code seems we don't need it?
   Just remove?




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


[GitHub] [spark] AmplabJenkins commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-704961919






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-704961928


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129507/
   Test FAILed.


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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705073377


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34124/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735960696


   **[Test build #132004 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132004/testReport)** for PR 29966 at commit [`f460974`](https://github.com/apache/spark/commit/f460974489961bb3657dd66faf7799461488db11).


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


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

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533942826



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,158 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 {
+case class IvyProperties(
+    packagesExclusions: String,
+    packages: String,
+    repositories: String,
+    ivyRepoPath: String,
+    ivySettingsPath: String)
+
+private[spark] object DependencyUtils extends Logging {
+
+  def getIvyProperties(): IvyProperties = {
+    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)
+    IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath)
+  }
+
+  /**
+   * Parse URI query string's parameter value of `transitive` and `exclude`.
+   * Other invalid parameters will be ignored.
+   *
+   * @param uri Ivy uri need to be downloaded.
+   * @return Tuple value of parameter `transitive` and `exclude` value.
+   *
+   *         1. transitive: whether to download dependency jar of ivy URI, default value is false
+   *            and this parameter value is case-sensitive. Invalid value will be treat as false.
+   *            Example: Input:  exclude=org.mortbay.jetty:jetty&transitive=true
+   *            Output:  true
+   *
+   *         2. exclude: comma separated exclusions to apply when resolving transitive dependencies,
+   *            consists of `group:module` pairs separated by commas.
+   *            Example: Input:  excludeorg.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http
+   *            Output:  [org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http]
+   */
+  private def parseQueryParams(uri: URI): (Boolean, String) = {
+    val uriQuery = uri.getQuery
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(token =>
+        token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+        throw new URISyntaxException(uri.toString, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive: Boolean = false
+      groupedParams.get("transitive").foreach { params =>
+        if (params.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")
+        }
+        params.map(_._2).foreach {
+          case "true" => transitive = true
+          case _ => transitive = false
+        }
+      }
+      // Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
+      // in an ivy URL. When download ivy URL jar, Spark won't download transitive jar
+      // in a excluded list.
+      val exclusionList = groupedParams.get("exclude").map { params =>
+        params.map(_._2).flatMap { excludeString =>
+          val excludes = excludeString.split(",")
+          if (excludes.map(_.split(":")).exists(token =>
+            token.length != 2 || StringUtils.isBlank(token(0)) || StringUtils.isBlank(token(1)))) {
+            throw new URISyntaxException(uri.toString, "Invalid exclude string: " +
+              "expected 'org:module,org:module,..', found " + excludeString)
+          }
+          excludes
+        }.mkString(",")
+      }.getOrElse("")
+
+      val invalidParams = groupedParams
+        .filter(entry => !Seq("transitive", "exclude").contains(entry._1))
+        .keys.toArray.sorted
+      if (invalidParams.nonEmpty) {
+        logWarning(
+          s"Invalid parameters `${invalidParams.mkString(",")}` found in URI query `$uriQuery`.")
+      }
+
+      groupedParams.foreach { case (key: String, values: Array[(String, String)]) =>
+        if (key != "transitive" || key != "exclude") {
+          logWarning("Invalid parameter")
+        }
+      }
+
+      (transitive, exclusionList)
+    }
+  }
+
+  /**
+   * Download Ivy URIs dependency jars.
+   *
+   * @param uri Ivy uri need to be downloaded. The URI format should be:
+   *              `ivy://group:module:version[?query]`
+   *            Ivy URI query part format should be:
+   *              `parameter=value&parameter=value...`
+   *            Note that currently ivy URI query part support two parameters:
+   *             1. transitive: whether to download dependent jars related to your ivy URL.
+   *                transitive=false or `transitive=true`, if not set, the default value is false.
+   *             2. exclude: exclusion list when download ivy URL jar and dependency jars.
+   *                The `exclude` parameter content is a ',' separated `group:module` pair string :
+   *                `exclude=group:module,group:module...`
+   * @return Comma separated string list of jars downloaded.
+   */
+  def resolveMavenDependencies(uri: URI): Seq[String] = {
+    try {
+      val ivyProperties = DependencyUtils.getIvyProperties()
+      val authority = uri.getAuthority
+      if (authority == null) {
+        throw new URISyntaxException(
+          uri.toString, "Invalid url: Expected 'org:module:version', found null")
+      }
+      if (authority.split(":").length != 3) {
+        throw new URISyntaxException(
+          uri.toString, "Invalid url: Expected 'org:module:version', found " + authority)
+      }
+
+      val (transitive, exclusionList) = parseQueryParams(uri)
+
+      resolveMavenDependencies(
+        transitive,
+        exclusionList,
+        authority,
+        ivyProperties.repositories,
+        ivyProperties.ivyRepoPath,
+        Option(ivyProperties.ivySettingsPath)
+      ).split(",")
+    } catch {
+      case e: URISyntaxException =>

Review comment:
       Which library API throws this exception? Could you minimize the try-catch block?




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-744169990


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/37341/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-739457605


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/132279/
   


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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r535771777



##########
File path: core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
##########
@@ -15,22 +15,112 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.File
-import java.net.URI
+import java.net.{URI, URISyntaxException}
 
 import org.apache.commons.lang3.StringUtils
 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 parseQueryParams(uriQuery: String): (Boolean, String) = {
+    if (uriQuery == null) {
+      (false, "")
+    } else {
+      val mapTokens = uriQuery.split("&").map(_.split("="))
+      if (mapTokens.exists(_.length != 2)) {
+        throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery")
+      }
+      val groupedParams = mapTokens.map(kv => (kv(0), kv(1))).groupBy(_._1)
+      // Parse transitive parameters (e.g., transitive=true) in an ivy URL, default value is false
+      var transitive = false

Review comment:
       Yea, I have too few ideas on how to make code more concise.
   Simpler is better, Updated!




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-750704177


   **[Test build #133328 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133328/testReport)** for PR 29966 at commit [`4c44dae`](https://github.com/apache/spark/commit/4c44daecc7c77527e150d30051170f7ee8667f70).
    * This patch **fails MiMa tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] SparkQA commented on pull request #29966: [WIP][SPARK-33084][CORE][SQL]Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-705503530


   **[Test build #129551 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129551/testReport)** for PR 29966 at commit [`169e1f8`](https://github.com/apache/spark/commit/169e1f8f07248ed65190f8d98f3f4c805c1177c3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735098071






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-736240432


   > For other reviewers, could you put the screenshot of the updated doc in the PR description?
   
   ![image](https://user-images.githubusercontent.com/46485123/100702546-d12ad900-33dc-11eb-90f3-95ad0200c37c.png)
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29966: [SPARK-33084][CORE][SQL] Add jar support ivy path

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-735354560






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


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

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r533091418



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1869,15 +1869,15 @@ class SparkContext(config: SparkConf) extends Logging {
           throw new IllegalArgumentException(
             s"Directory ${file.getAbsoluteFile} is not allowed for addJar")
         }
-        env.rpcEnv.fileServer.addJar(file)
+        Array(env.rpcEnv.fileServer.addJar(file))
       } catch {
         case NonFatal(e) =>
           logError(s"Failed to add $path to Spark environment", e)
-          null
+          Array.empty

Review comment:
       Yea




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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r547732346



##########
File path: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
##########
@@ -304,8 +304,8 @@ private[spark] class SparkSubmit extends Logging {
       // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files
       // too for packages that include Python code
       val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(
-        args.packagesExclusions, args.packages, args.repositories, args.ivyRepoPath,
-        args.ivySettingsPath)
+        true, args.packagesExclusions, args.packages,

Review comment:
       nit. `true,` -> `transitive = true,`.




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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29966:
URL: https://github.com/apache/spark/pull/29966#discussion_r548127713



##########
File path: core/src/test/scala/org/apache/spark/SparkContextSuite.scala
##########
@@ -1152,7 +1129,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
     assert(sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
   }
 
-  test("SPARK-33084: Add jar support Ivy URI -- test param case sensitive") {
+  test("SPARK-33084: Add jar support Ivy URI -- test transitive value case sensitive") {

Review comment:
       This test name is already used at line 1143.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29966:
URL: https://github.com/apache/spark/pull/29966#issuecomment-749513085


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37811/
   


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