Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-34506][CORE] ADD JAR with ivy coordinates should be compatible with Hive transitive behavior #31623

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 7 additions & 6 deletions core/src/main/scala/org/apache/spark/util/DependencyUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -59,8 +59,8 @@ private[spark] object DependencyUtils extends Logging {
* @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.
* 1. transitive: whether to download dependency jar of Ivy URI, default value is true
* and this parameter value is case-insensitive. Invalid value will be treat as false.
HyukjinKwon marked this conversation as resolved.
Show resolved Hide resolved
* Example: Input: exclude=org.mortbay.jetty:jetty&transitive=true
* Output: true
*
Expand All @@ -72,7 +72,7 @@ private[spark] object DependencyUtils extends Logging {
private def parseQueryParams(uri: URI): (Boolean, String) = {
val uriQuery = uri.getQuery
if (uriQuery == null) {
(false, "")
(true, "")
} else {
val mapTokens = uriQuery.split("&").map(_.split("="))
if (mapTokens.exists(isInvalidQueryString)) {
Expand All @@ -81,14 +81,15 @@ private[spark] object DependencyUtils extends Logging {
}
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
// Parse transitive parameters (e.g., transitive=true) in an Ivy URI, default value is true
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)
transitiveParams.flatMap(_.takeRight(1).map(_._2.equalsIgnoreCase("true")).headOption)
.getOrElse(true)

// Parse an excluded list (e.g., exclude=org.mortbay.jetty:jetty,org.eclipse.jetty:jetty-http)
// in an Ivy URI. When download Ivy URI jar, Spark won't download transitive jar
Expand Down Expand Up @@ -125,7 +126,7 @@ private[spark] object DependencyUtils extends Logging {
* `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 URI.
* transitive=false or `transitive=true`, if not set, the default value is false.
* transitive=false or `transitive=true`, if not set, the default value is true.
* 2. exclude: exclusion list when download Ivy URI jar and dependency jars.
* The `exclude` parameter content is a ',' separated `group:module` pair string :
* `exclude=group:module,group:module...`
Expand Down
33 changes: 23 additions & 10 deletions core/src/test/scala/org/apache/spark/SparkContextSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1035,13 +1035,10 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
}
}

test("SPARK-33084: Add jar support Ivy URI -- default transitive = false") {
test("SPARK-33084: Add jar support Ivy URI -- default 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")
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")))
}

Expand Down Expand Up @@ -1083,6 +1080,22 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu
}
}

test("SPARK-34506: Add jar support Ivy URI -- transitive=false will not 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=false")
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-34506: Add jar support Ivy URI -- test exclude param when transitive unspecified") {
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")
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 URI -- 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" +
Expand Down Expand Up @@ -1131,24 +1144,24 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu

test("SPARK-33084: Add jar support Ivy URI -- test param key 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")
sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=false")
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")
sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?TRANSITIVE=false")
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 URI -- test transitive value case sensitive") {
test("SPARK-33084: Add jar support Ivy URI -- test transitive value case insensitive") {
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")
sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=FALSE")
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")
sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=false")
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")))
assert(!sc.listJars().exists(_.contains("commons-lang_commons-lang-2.6.jar")))
}

test("SPARK-34346: hadoop configuration priority for spark/hive/hadoop configs") {
Expand Down
2 changes: 1 addition & 1 deletion docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ ADD JAR 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 or an Ivy URI.
Apache Ivy is a popular dependency manager focusing on flexibility and simplicity. Now we support two parameter in URI 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.
* transitive: whether to download dependent jars related to your ivy URL. The parameter name is case-sensitive, parameter value is case-insensitive and only take last one if multiple transitive parameters are specified.
HyukjinKwon marked this conversation as resolved.
Show resolved Hide resolved
* exclude: exclusion list during downloading Ivy URI jar and dependent jars.

User can write Ivy URI such as:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3726,13 +3726,13 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
test("SPARK-33084: Add jar support Ivy URI in SQL") {
val sc = spark.sparkContext
val hiveVersion = "2.3.8"
// default transitive=false, only download specified jar
sql(s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion")
// transitive=false, only download specified jar
sql(s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion?transitive=false")
assert(sc.listJars()
.exists(_.contains(s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")))

// test download ivy URL jar return multiple jars
sql("ADD JAR ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=true")
// default transitive=true, test download ivy URL jar return multiple jars
sql("ADD JAR ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0")
assert(sc.listJars().exists(_.contains("scalajs-library_2.12")))
assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12")))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1224,7 +1224,10 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
test("SPARK-33084: Add jar support Ivy URI in SQL") {
val testData = TestHive.getHiveFile("data/files/sample.json").toURI
withTable("t") {
sql(s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion")
// exclude org.pentaho:pentaho-aggdesigner-algorithm as this transitive dependency does
// not exist on mavencentral and hence cannot be found in the test environment
sql(s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion" +
"?exclude=org.pentaho:pentaho-aggdesigner-algorithm")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is odd. Any idea why hive-hcatalog-core has a dependency on a package that doesn't exist?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a very infamous dependency brought in by calcite. It does not exist on mavencentral but it does exist on conjars.org. Here is a Stack overflow asking about the same issue. https://stackoverflow.com/questions/32587769/dependency-resolution-issue-in-gradle

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yikes.

sql(
"""CREATE TABLE t(a string, b string)
|ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
Expand Down