From afaf7bd11c0cb6a20542eb5e7c7fbcfca49319e9 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 7 Oct 2020 19:31:19 +0800 Subject: [PATCH 01/50] [SPARK-33084][CORE][SQL]Add jar support ivy path --- .../scala/org/apache/spark/SparkContext.scala | 24 ++++--- .../org/apache/spark/deploy/SparkSubmit.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 65 ++++++++++++++++++- .../org/apache/spark/SparkContextSuite.scala | 14 ++++ .../spark/sql/internal/SessionState.scala | 33 +++++++--- .../sql/hive/HiveSessionStateBuilder.scala | 6 +- 6 files changed, 123 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 409e3065492b0..37779d8f07209 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast -import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil, SparkSubmitUtils} import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging @@ -1899,7 +1899,7 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { - val key = if (path.contains("\\")) { + val keys = if (path.contains("\\")) { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { @@ -1915,17 +1915,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)` + Utils.resolveMavenDependencies(URI.create(path)) case _ => checkRemoteJarFile(path) } } - if (key != null) { + if (keys != null) { 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.split(",").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 " + + "is not supported in the current version.") + } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 8363d570d7320..158a0a13ce1f8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1342,6 +1342,7 @@ private[spark] object SparkSubmitUtils { coordinates: String, ivySettings: IvySettings, exclusions: Seq[String] = Nil, + transitive: Boolean = true, isTest: Boolean = false): String = { if (coordinates == null || coordinates.trim.isEmpty) { "" @@ -1370,7 +1371,7 @@ private[spark] object SparkSubmitUtils { val ivy = Ivy.newInstance(ivySettings) // Set resolve options to download transitive dependencies as well val resolveOptions = new ResolveOptions - resolveOptions.setTransitive(true) + resolveOptions.setTransitive(transitive) val retrieveOptions = new RetrieveOptions // Turn downloading and logging off for testing if (isTest) { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index b8b044bbad30e..c5a304a33bcf6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -58,7 +58,7 @@ import org.eclipse.jetty.util.MultiException import org.slf4j.Logger import org.apache.spark._ -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.{SparkHadoopUtil, SparkSubmitUtils} import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Streaming._ @@ -2971,6 +2971,69 @@ private[spark] object Utils extends Logging { metadata.append("]") metadata.toString } + + 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)) + } + + /** + * @param queryString + * @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] = { + if (queryString == null || queryString.isEmpty) { + Array.empty[String] + } else { + val mapTokens: Array[String] = queryString.split("&") + assert(mapTokens.forall(_.split("=").length == 2), "Invalid query string: " + queryString) + mapTokens.map(_.split("=")).map(kv => (kv(0), kv(1))).filter(_._1 == "exclude") + .flatMap { case (_, excludeString) => + val excludes: Array[String] = excludeString.split(",") + assert(excludes.forall(_.split(":").length == 2), + "Invalid exclude string: expected 'org:module,org:module,..', found " + excludeString) + excludes + } + } + } + + /** + * @param queryString + * @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 parseTransitive(queryString: String): Boolean = { + if (queryString == null || queryString.isEmpty) { + false + } else { + val mapTokens: Array[String] = queryString.split("&") + assert(mapTokens.forall(_.split("=").length == 2), "Invalid query string: " + queryString) + val transitive = mapTokens.map(_.split("=")).map(kv => (kv(0), kv(1))) + .filter(_._1 == "transitive") + if (transitive.isEmpty) { + false + } else { + transitive.last._2.toBoolean + } + } + } } private[util] object CallerContext extends Logging { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ebdf2f59a2770..2b0d64cfe91c8 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -955,6 +955,20 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop() } } + + test("ivy path") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) + sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0") + assert(sc.listJars().find(_.contains("scalajs-test-interface_2.12")).nonEmpty) + + sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=true") + assert(sc.listJars().find(_.contains("scalajs-library_2.12")).nonEmpty) + + sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7" + + "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=false") + assert(sc.listJars().find(_.contains("org.apache.hive_hive-contrib-2.3.7.jar")).nonEmpty) + assert(sc.listJars().find(_.contains("org.apache.hive_hive-exec-2.3.7.jar")).isEmpty) + } } object SparkContextSuite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 0f9a89741c192..d92e7c028a07d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import java.io.File +import java.net.URI import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -34,6 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListener} +import org.apache.spark.util.Utils /** * A class that holds all session-specific state in a given [[SparkSession]]. @@ -158,6 +160,17 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade } } + def resolveJars(path: String): List[String] = { + val uri = new Path(path).toUri + uri.getScheme match { + case "ivy" => + Utils.resolveMavenDependencies(URI.create(path)) + .split(",").toList + case _ => + path :: Nil + } + } + /** * Add a jar path to [[SparkContext]] and the classloader. * @@ -166,16 +179,18 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade * [[SessionState]]. */ def addJar(path: String): Unit = { - session.sparkContext.addJar(path) - val uri = new Path(path).toUri - val jarURL = if (uri.getScheme == null) { - // `path` is a local file path without a URL scheme - new File(path).toURI.toURL - } else { - // `path` is a URL with a scheme - uri.toURL + resolveJars(path).foreach { p => + session.sparkContext.addJar(p) + val uri = new Path(p).toUri + val jarURL = if (uri.getScheme == null) { + // `path` is a local file path without a URL scheme + new File(p).toURI.toURL + } else { + // `path` is a URL with a scheme + uri.toURL + } + session.sharedState.jarClassLoader.addURL(jarURL) } - session.sharedState.jarClassLoader.addURL(jarURL) Thread.currentThread().setContextClassLoader(session.sharedState.jarClassLoader) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index b9135733856a5..fe26e7da3516a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -127,7 +127,9 @@ class HiveSessionResourceLoader( extends SessionResourceLoader(session) { private lazy val client = clientBuilder() override def addJar(path: String): Unit = { - client.addJar(path) - super.addJar(path) + resolveJars(path).foreach { p => + client.addJar(p) + super.addJar(p) + } } } From 3579de0444d8ea8e320aa7bade1ef15cc8ae9d5f Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 8 Oct 2020 00:00:14 +0800 Subject: [PATCH 02/50] Update SparkContextSuite.scala --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 2b0d64cfe91c8..ed62e31d0d10e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -956,7 +956,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } - test("ivy path") { + test("SPARK-33084: Add jar support ivy url") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0") assert(sc.listJars().find(_.contains("scalajs-test-interface_2.12")).nonEmpty) From 169e1f8f07248ed65190f8d98f3f4c805c1177c3 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 8 Oct 2020 16:05:02 +0800 Subject: [PATCH 03/50] Update Utils.scala --- .../main/scala/org/apache/spark/util/Utils.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c5a304a33bcf6..9dba175ad614e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2972,6 +2972,12 @@ 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( @@ -2992,7 +2998,7 @@ private[spark] object Utils extends Logging { } /** - * @param queryString + * @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] @@ -3014,10 +3020,10 @@ private[spark] object Utils extends Logging { } /** - * @param queryString - * @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] + * @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 = { if (queryString == null || queryString.isEmpty) { From b3e3211cd0d3d96bcf7a021d41553619dc894125 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 23 Nov 2020 09:21:59 +0800 Subject: [PATCH 04/50] resolve conflict --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dc8f26a302a08..166db99f41b91 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast -import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil, SparkSubmitUtils} +import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.executor.{Executor, ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index f8e69ba88e0f4..d95b9110e9918 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -3046,6 +3046,10 @@ private[spark] object Utils extends Logging { } else { transitive.last._2.toBoolean } + } + } + + /** * Convert MEMORY_OFFHEAP_SIZE to MB Unit, return 0 if MEMORY_OFFHEAP_ENABLED is false. */ def executorOffHeapMemorySizeAsMb(sparkConf: SparkConf): Int = { From 9161340abd46471252e2e933fd019cd72446828b Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 23 Nov 2020 09:32:13 +0800 Subject: [PATCH 05/50] Update SessionState.scala --- .../main/scala/org/apache/spark/sql/internal/SessionState.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index a8775da5a6156..f6a694ca8efce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager -import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListener} +import org.apache.spark.sql.util.ExecutionListenerManager import org.apache.spark.util.Utils /** From 0e3c1ec7476804beec25eb47e674d6ed4d8dcb7a Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 24 Nov 2020 10:01:19 +0800 Subject: [PATCH 06/50] Update sql-ref-syntax-aux-resource-mgmt-add-jar.md --- docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 4694bff99daf5..130edc8124496 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/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. ### Examples @@ -42,6 +42,10 @@ ADD JAR /tmp/test.jar; ADD JAR "/path/to/some.jar"; ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; +ADD JAR "ivy://group:module:version"; +ADD JAR "ivy://group:module:version?transitive=true"; +ADD JAR "ivy://group:module:version?exclusin=group:module,group:module"; +ADD JAR "ivy://group:module:version?exclusin=group:module,group:module&transitive=false"; ``` ### Related Statements From 300ca5633ea403028cb0ce51b164708e2d50b350 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 24 Nov 2020 15:35:58 +0800 Subject: [PATCH 07/50] follow comment --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 17 ++++++++++++----- .../sql-ref-syntax-aux-resource-mgmt-add-jar.md | 6 +++--- .../spark/sql/internal/SessionState.scala | 12 ++++-------- 5 files changed, 21 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 166db99f41b91..a954dc4b3894d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1922,7 +1922,7 @@ class SparkContext(config: SparkConf) extends Logging { case "local" => "file:" + uri.getPath case "ivy" => // Since `new Path(path).toUri` will lose query information, - // so here we use `URI>create(path)` + // so here we use `URI.create(path)` Utils.resolveMavenDependencies(URI.create(path)) case _ => checkRemoteJarFile(path) } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d95b9110e9918..4d087b5acf919 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -3015,7 +3015,7 @@ private[spark] object Utils extends Logging { if (queryString == null || queryString.isEmpty) { Array.empty[String] } else { - val mapTokens: Array[String] = queryString.split("&") + 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 == "exclude") .flatMap { case (_, excludeString) => diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ed62e31d0d10e..6ab03361fa4bb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -958,16 +958,23 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("SPARK-33084: Add jar support ivy url") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) - sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0") - assert(sc.listJars().find(_.contains("scalajs-test-interface_2.12")).nonEmpty) + // default transitive=false, only download specified jar + sc.addJar("ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") + assert( + sc.listJars().exists(_.contains("org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar"))) + + // test download ivy URL jar return multiple jars sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=true") - assert(sc.listJars().find(_.contains("scalajs-library_2.12")).nonEmpty) + assert(sc.listJars().exists(_.contains("scalajs-library_2.12"))) + assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7" + "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=false") - assert(sc.listJars().find(_.contains("org.apache.hive_hive-contrib-2.3.7.jar")).nonEmpty) - assert(sc.listJars().find(_.contains("org.apache.hive_hive-exec-2.3.7.jar")).isEmpty) + assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) + // when transitive=true, `org.apache.hive_hive-exec-2.3.7.jar` will be downloaded too + assert(!sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) + assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) } } diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 130edc8124496..6c5fbfc85f88e 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/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 or a ivy URL. + 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. ### Examples @@ -44,8 +44,8 @@ ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; ADD JAR "ivy://group:module:version"; ADD JAR "ivy://group:module:version?transitive=true"; -ADD JAR "ivy://group:module:version?exclusin=group:module,group:module"; -ADD JAR "ivy://group:module:version?exclusin=group:module,group:module&transitive=false"; +ADD JAR "ivy://group:module:version?exclusion=group:module,group:module"; +ADD JAR "ivy://group:module:version?exclusion=group:module,group:module&transitive=false"; ``` ### Related Statements diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index f6a694ca8efce..fe91629660d7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -161,14 +161,10 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade } } - def resolveJars(path: String): List[String] = { - val uri = new Path(path).toUri - uri.getScheme match { - case "ivy" => - Utils.resolveMavenDependencies(URI.create(path)) - .split(",").toList - case _ => - path :: Nil + protected def resolveJars(path: String): List[String] = { + new Path(path).toUri.getScheme match { + case "ivy" => Utils.resolveMavenDependencies(URI.create(path)).split(",").toList + case _ => path :: Nil } } From 63e877b64f7a61888ea59a2c632ef5e9373e087e Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 24 Nov 2020 15:55:05 +0800 Subject: [PATCH 08/50] https://github.com/apache/spark/pull/29966#discussion_r529242208 --- .../scala/org/apache/spark/util/Utils.scala | 48 ++++++++++--------- 1 file changed, 25 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4d087b5acf919..20303c19cc117 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -3005,47 +3005,49 @@ private[spark] object Utils extends Logging { 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] = { - 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 == "exclude") - .flatMap { case (_, excludeString) => - val excludes: Array[String] = excludeString.split(",") - assert(excludes.forall(_.split(":").length == 2), - "Invalid exclude string: expected 'org:module,org:module,..', found " + excludeString) - excludes - } - } + 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 = { - if (queryString == null || queryString.isEmpty) { + val transitive = parseURLQueryParameter(queryString, "transitive") + if (transitive.isEmpty) { false } else { - val mapTokens: Array[String] = queryString.split("&") - assert(mapTokens.forall(_.split("=").length == 2), "Invalid query string: " + queryString) - val transitive = mapTokens.map(_.split("=")).map(kv => (kv(0), kv(1))) - .filter(_._1 == "transitive") - if (transitive.isEmpty) { - false - } else { - transitive.last._2.toBoolean - } + transitive.last.toBoolean } } From 733e62c9d02b0301663ba02423db22242e12f8a9 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 24 Nov 2020 16:37:32 +0800 Subject: [PATCH 09/50] follow comment --- .../org/apache/spark/SparkContextSuite.scala | 5 ++--- ...ql-ref-syntax-aux-resource-mgmt-add-jar.md | 20 +++++++++++++++---- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 6ab03361fa4bb..2f1d4f13bb2ca 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -970,10 +970,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7" + - "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=false") + "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=true") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) - // when transitive=true, `org.apache.hive_hive-exec-2.3.7.jar` will be downloaded too - assert(!sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) + assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) } } diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 6c5fbfc85f88e..1981c16eae7cd 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -34,7 +34,19 @@ 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 or an ivy URL. + Apache Ivy is a popular dependency manager focusing on flexibility and simplicity. Now we support two parameter in URL query string: + 1. transitive: whether to download dependent jars related to your ivy URL. + 2. exclude: exclusion list when download ivy URL jar and dependent jars. + + User can write ivy URL such as: + + 1. ivy://group:module:version + 2. ivy://group:module:version?transitive=true + 3. ivy://group:module:version?exclude=group:module,group:module + 4. ivy://group:module:version?exclude=group:module,group:module&transitive=false + + ### Examples ```sql @@ -42,10 +54,10 @@ ADD JAR /tmp/test.jar; ADD JAR "/path/to/some.jar"; ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; -ADD JAR "ivy://group:module:version"; -ADD JAR "ivy://group:module:version?transitive=true"; -ADD JAR "ivy://group:module:version?exclusion=group:module,group:module"; -ADD JAR "ivy://group:module:version?exclusion=group:module,group:module&transitive=false"; +ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7"; +ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?transitive=false" +ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?transitive=true" +ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=true" ``` ### Related Statements From b60ba1e3eebd5968fb017d537a98637289ba112b Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 24 Nov 2020 16:45:40 +0800 Subject: [PATCH 10/50] Update sql-ref-syntax-aux-resource-mgmt-add-jar.md --- docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 1981c16eae7cd..b1f299ac6a473 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -45,7 +45,6 @@ ADD JAR file_name 2. ivy://group:module:version?transitive=true 3. ivy://group:module:version?exclude=group:module,group:module 4. ivy://group:module:version?exclude=group:module,group:module&transitive=false - ### Examples From 883b9d39b39920079a6a166ffd1608f070782b5a Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 14:26:59 +0800 Subject: [PATCH 11/50] fix uri re-use --- .../org/apache/spark/sql/internal/SessionState.scala | 11 ++++++----- .../spark/sql/hive/HiveSessionStateBuilder.scala | 5 ++++- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index fe91629660d7a..ee81fcfaab7d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -161,10 +161,10 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade } } - protected def resolveJars(path: String): List[String] = { - new Path(path).toUri.getScheme match { - case "ivy" => Utils.resolveMavenDependencies(URI.create(path)).split(",").toList - case _ => path :: Nil + protected def resolveJars(path: URI): List[String] = { + path.getScheme match { + case "ivy" => Utils.resolveMavenDependencies(path).split(",").toList + case _ => path.toString :: Nil } } @@ -176,7 +176,8 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade * [[SessionState]]. */ def addJar(path: String): Unit = { - resolveJars(path).foreach { p => + val uri = URI.create(path) + resolveJars(uri).foreach { p => session.sparkContext.addJar(p) val uri = new Path(p).toUri val jarURL = if (uri.getScheme == null) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 345f9c0452621..454eef3dcaf8c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.net.URI + import org.apache.spark.annotation.Unstable import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} @@ -126,7 +128,8 @@ class HiveSessionResourceLoader( extends SessionResourceLoader(session) { private lazy val client = clientBuilder() override def addJar(path: String): Unit = { - resolveJars(path).foreach { p => + val uri = URI.create(path) + resolveJars(uri).foreach { p => client.addJar(p) super.addJar(p) } From 208afc2789a1efd4179a6b2161fd081bdf31d266 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 14:28:20 +0800 Subject: [PATCH 12/50] follow comment --- .../apache/spark/deploy/DependencyUtils.scala | 80 ++++++++++++++++++- .../org/apache/spark/deploy/SparkSubmit.scala | 7 +- .../spark/deploy/worker/DriverWrapper.scala | 11 +-- .../scala/org/apache/spark/util/Utils.scala | 73 +---------------- .../spark/deploy/SparkSubmitUtilsSuite.scala | 2 +- 5 files changed, 88 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index 5a17a6b6e169c..1a9bfe5ebd65b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -30,7 +30,84 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils} private[deploy] 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 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 { + 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), + uri.getAuthority, + repositories, + ivyRepoPath, + Some(ivySettingsPath) + ) + } + def resolveMavenDependencies( + packagesTransitive: Boolean, packagesExclusions: String, packages: String, repositories: String, @@ -51,7 +128,8 @@ private[deploy] object DependencyUtils extends Logging { SparkSubmitUtils.buildIvySettings(Option(repositories), Option(ivyRepoPath)) } - SparkSubmitUtils.resolveMavenCoordinates(packages, ivySettings, exclusions = exclusions) + SparkSubmitUtils.resolveMavenCoordinates(packages, ivySettings, + transitive = packagesTransitive, exclusions = exclusions) } def resolveAndDownloadJars( diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d4d1f61098107..1263927fcb134 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/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, + args.repositories, args.ivyRepoPath, args.ivySettingsPath) if (!StringUtils.isBlank(resolvedMavenCoordinates)) { // In K8s client mode, when in the driver, add resolved jars early as we might need @@ -1340,6 +1340,7 @@ private[spark] object SparkSubmitUtils { * Resolves any dependencies that were supplied through maven coordinates * @param coordinates Comma-delimited string of maven coordinates * @param ivySettings An IvySettings containing resolvers to use + * @param transitive Whether resolving transitive dependencies, default is true * @param exclusions Exclusions to apply when resolving transitive dependencies * @return The comma-delimited path to the jars of the given maven artifacts including their * transitive dependencies @@ -1347,8 +1348,8 @@ private[spark] object SparkSubmitUtils { def resolveMavenCoordinates( coordinates: String, ivySettings: IvySettings, - exclusions: Seq[String] = Nil, transitive: Boolean = true, + exclusions: Seq[String] = Nil, isTest: Boolean = false): String = { if (coordinates == null || coordinates.trim.isEmpty) { "" diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 45ffdde58d6c3..568faa9e306cd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/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)) val jars = { val jarsProp = sys.props.get(config.JARS.key).orNull diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 20303c19cc117..6ccf65b737c1a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -58,7 +58,7 @@ import org.eclipse.jetty.util.MultiException import org.slf4j.Logger import org.apache.spark._ -import org.apache.spark.deploy.{SparkHadoopUtil, SparkSubmitUtils} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Streaming._ @@ -2980,77 +2980,6 @@ 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 - } - } - /** * Convert MEMORY_OFFHEAP_SIZE to MB Unit, return 0 if MEMORY_OFFHEAP_ENABLED is false. */ diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 2a37f75d86a41..d8b963c3b8072 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -214,7 +214,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val files = SparkSubmitUtils.resolveMavenCoordinates( main.toString, SparkSubmitUtils.buildIvySettings(Some(repo), Some(tempIvyPath)), - Seq("my.great.dep:mydep"), + exclusions = Seq("my.great.dep:mydep"), isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") assert(files.indexOf("my.great.dep") < 0, "Returned excluded artifact") From ba9ea296e78c852fb959b7979157b59d22afeacb Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 14:35:01 +0800 Subject: [PATCH 13/50] add warn message whe multiple trasitive --- .../main/scala/org/apache/spark/deploy/DependencyUtils.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index 1a9bfe5ebd65b..5f90af7e80fb9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -83,6 +83,10 @@ private[deploy] object DependencyUtils extends Logging { 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 } } From 10b373783bc5015ff9e5fc01b53ad7d0bde02c01 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 15:11:23 +0800 Subject: [PATCH 14/50] move DependencyUtils --- .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/deploy/worker/DriverWrapper.scala | 2 +- .../spark/{deploy => util}/DependencyUtils.scala | 14 ++++++++------ .../org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- 4 files changed, 11 insertions(+), 9 deletions(-) rename core/src/main/scala/org/apache/spark/{deploy => util}/DependencyUtils.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a954dc4b3894d..16d347ae24560 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1923,7 +1923,7 @@ class SparkContext(config: SparkConf) extends Logging { case "ivy" => // Since `new Path(path).toUri` will lose query information, // so here we use `URI.create(path)` - Utils.resolveMavenDependencies(URI.create(path)) + DependencyUtils.resolveMavenDependencies(URI.create(path)) case _ => checkRemoteJarFile(path) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 568faa9e306cd..a7814d1b0237b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -22,7 +22,7 @@ import java.io.File import org.apache.commons.lang3.StringUtils import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util._ diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala rename to core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 5f90af7e80fb9..44c6cbca24921 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy +package org.apache.spark.util import java.io.File import java.net.URI @@ -25,10 +25,10 @@ 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( @@ -46,7 +46,8 @@ private[deploy] object DependencyUtils extends Logging { Array.empty[String] } else { val mapTokens = queryString.split("&") - assert(mapTokens.forall(_.split("=").length == 2), "Invalid query string: " + queryString) + 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) } } @@ -65,7 +66,8 @@ private[deploy] object DependencyUtils extends Logging { .flatMap { excludeString => val excludes: Array[String] = excludeString.split(",") assert(excludes.forall(_.split(":").length == 2), - "Invalid exclude string: expected 'org:module,org:module,..', found " + excludeString) + "Invalid exclude string: expected 'org:module,org:module,..'," + + " found [ " + excludeString + " ]") excludes }.mkString(":") } @@ -106,7 +108,7 @@ private[deploy] object DependencyUtils extends Logging { uri.getAuthority, repositories, ivyRepoPath, - Some(ivySettingsPath) + Option(ivySettingsPath) ) } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index b5b3751439750..e2d5a0cd46307 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -47,7 +47,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} +import org.apache.spark.util.{CommandLineUtils, DependencyUtils, ResetSystemProperties, Utils} trait TestPrematureExit { suite: SparkFunSuite => From 7f878c2f675470fbeae12bdce2bfaf26971a16f4 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 15:11:32 +0800 Subject: [PATCH 15/50] add ut --- .../org/apache/spark/SparkContextSuite.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 2f1d4f13bb2ca..cc1615465a3ce 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -974,6 +974,22 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) + + val e1 = intercept[AssertionError] { + sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") + }.getMessage + assert(e1.contains("Invalid URI query string: [ foo= ]")) + + val e2 = intercept[AssertionError] { + sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") + }.getMessage + assert(e2.contains("Invalid URI query string: [ bar=&baz=foo ]")) + + val e3 = intercept[AssertionError] { + sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") + }.getMessage + assert(e3.contains("Invalid exclude string: expected 'org:module,org:module,..'," + + " found [ org.pentaho ]")) } } From d2c1950451e5d9b1a27e9b6d8a6d3540b9a60b4d Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 16:07:08 +0800 Subject: [PATCH 16/50] Update SessionState.scala --- .../scala/org/apache/spark/sql/internal/SessionState.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index ee81fcfaab7d8..d38f9d7bd80fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.ExecutionListenerManager -import org.apache.spark.util.Utils +import org.apache.spark.util.{DependencyUtils, Utils} /** * A class that holds all session-specific state in a given [[SparkSession]]. @@ -163,7 +163,7 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade protected def resolveJars(path: URI): List[String] = { path.getScheme match { - case "ivy" => Utils.resolveMavenDependencies(path).split(",").toList + case "ivy" => DependencyUtils.resolveMavenDependencies(path).split(",").toList case _ => path.toString :: Nil } } From 220007647275c294c9a31c0d0a3f0a52507a585c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 28 Nov 2020 16:37:30 +0800 Subject: [PATCH 17/50] Update SessionState.scala --- .../main/scala/org/apache/spark/sql/internal/SessionState.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index d38f9d7bd80fd..ff5b9fb15f217 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.ExecutionListenerManager -import org.apache.spark.util.{DependencyUtils, Utils} +import org.apache.spark.util.DependencyUtils /** * A class that holds all session-specific state in a given [[SparkSession]]. From 5a9cc30d225f213c6c5bb114a9484d1eacba73e8 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 29 Nov 2020 10:55:24 +0800 Subject: [PATCH 18/50] Update DependencyUtils.scala --- .../org/apache/spark/util/DependencyUtils.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 44c6cbca24921..23518ccef7c72 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -94,9 +94,18 @@ private[spark] object DependencyUtils extends Logging { } /** - * Download Ivy URIs dependent jars. + * Download Ivy URIs dependency jars. * - * @param uri Ivy uri need to be downloaded. + * @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¶meter=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 = { From 875d8a7b640b8112d280e49d827677215e3fb1b8 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 29 Nov 2020 11:07:01 +0800 Subject: [PATCH 19/50] Add end to end test --- .../org/apache/spark/sql/SQLQuerySuite.scala | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 727482e551a8b..542db619fe332 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -22,7 +22,8 @@ import java.net.{MalformedURLException, URL} import java.sql.{Date, Timestamp} import java.util.concurrent.atomic.AtomicBoolean -import org.apache.spark.{AccumulatorSuite, SparkException} +import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext, SparkException} + import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} @@ -3718,6 +3719,41 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-33084: Add jar support ivy url in SQL") { + val sc = spark.sparkContext + // default transitive=false, only download specified jar + sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") + assert(sc.listJars() + .exists(_.contains("org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.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") + assert(sc.listJars().exists(_.contains("scalajs-library_2.12"))) + assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) + + sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7" + + "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=true") + assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) + assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) + assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) + + val e1 = intercept[AssertionError] { + sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?foo=") + }.getMessage + assert(e1.contains("Invalid URI query string: [ foo= ]")) + + val e2 = intercept[AssertionError] { + sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") + }.getMessage + assert(e2.contains("Invalid URI query string: [ bar=&baz=foo ]")) + + val e3 = intercept[AssertionError] { + sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") + }.getMessage + assert(e3.contains("Invalid exclude string: expected 'org:module,org:module,..'," + + " found [ org.pentaho ]")) + } } case class Foo(bar: Option[String]) From e92124581e76767929883d54e2eb6d97e0502aac Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 29 Nov 2020 11:49:49 +0800 Subject: [PATCH 20/50] Update SQLQuerySuite.scala --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 542db619fe332..b678b4ac36953 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -22,8 +22,7 @@ import java.net.{MalformedURLException, URL} import java.sql.{Date, Timestamp} import java.util.concurrent.atomic.AtomicBoolean -import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext, SparkException} - +import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} From 614a86517475e6a9531e56c55bc003f5711031a4 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 30 Nov 2020 16:40:43 +0800 Subject: [PATCH 21/50] follow comment --- .../spark/deploy/worker/DriverWrapper.scala | 3 +- .../apache/spark/util/DependencyUtils.scala | 68 +++++++++++-------- .../org/apache/spark/SparkContextSuite.scala | 27 +++++--- ...ql-ref-syntax-aux-resource-mgmt-add-jar.md | 14 ++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 16 ----- 5 files changed, 65 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index a7814d1b0237b..fe655903dfa3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -83,8 +83,7 @@ object DriverWrapper extends Logging { DependencyUtils.getIvyProperties() val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies( - true, packagesExclusions, - packages, repositories, ivyRepoPath, Option(ivySettingsPath)) + true, packagesExclusions, packages, repositories, ivyRepoPath, Option(ivySettingsPath)) val jars = { val jarsProp = sys.props.get(config.JARS.key).orNull if (!StringUtils.isBlank(resolvedMavenCoordinates)) { diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 23518ccef7c72..180435722fef2 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -18,7 +18,7 @@ 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 @@ -40,18 +40,6 @@ private[spark] object DependencyUtils extends Logging { ).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. @@ -61,15 +49,15 @@ private[spark] object DependencyUtils extends Logging { * 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(":") + 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(":") } /** @@ -80,16 +68,15 @@ private[spark] object DependencyUtils extends Logging { * 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) { + private def parseTransitive(transitives: Array[String]): Boolean = { + if (transitives.isEmpty) { false } else { - if (transitive.length > 1) { + 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") } - transitive.last.toBoolean + transitives.last.toBoolean } } @@ -111,10 +98,31 @@ private[spark] object DependencyUtils extends Logging { 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(uri.getQuery), - parseExcludeList(uri.getQuery), - uri.getAuthority, + parseTransitive(queryParams.filter(_._1.equals("transitive")).map(_._2)), + parseExcludeList(queryParams.filter(_._1.equals("exclude")).map(_._2)), + authority, repositories, ivyRepoPath, Option(ivySettingsPath) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index cc1615465a3ce..ccca77384e925 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark import java.io.File -import java.net.{MalformedURLException, URI} +import java.net.{MalformedURLException, URI, URISyntaxException} import java.nio.charset.StandardCharsets import java.util.concurrent.{CountDownLatch, Semaphore, TimeUnit} @@ -975,21 +975,32 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) - val e1 = intercept[AssertionError] { + val e1 = intercept[IllegalArgumentException] { + sc.addJar("ivy://") + }.getMessage + assert(e1.contains("Expected authority at index 6: ivy://")) + + val e2 = intercept[URISyntaxException] { + sc.addJar("ivy://org.apache.hive:hive-contrib") + }.getMessage + assert(e2.contains("Invalid url: Expected 'org:module:version'," + + " found org.apache.hive:hive-contrib")) + + val e3 = intercept[URISyntaxException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") }.getMessage - assert(e1.contains("Invalid URI query string: [ foo= ]")) + assert(e3.contains("Invalid query string: foo=")) - val e2 = intercept[AssertionError] { + val e4 = intercept[URISyntaxException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") }.getMessage - assert(e2.contains("Invalid URI query string: [ bar=&baz=foo ]")) + assert(e4.contains("Invalid query string: bar=&baz=foo")) - val e3 = intercept[AssertionError] { + val e5 = intercept[URISyntaxException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") }.getMessage - assert(e3.contains("Invalid exclude string: expected 'org:module,org:module,..'," + - " found [ org.pentaho ]")) + assert(e5.contains( + "Invalid exclude string: expected 'org:module,org:module,..', found org.pentaho")) } } diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index b1f299ac6a473..2ff6d24f73f01 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -34,17 +34,17 @@ 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 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: + Apache Ivy is a popular dependency manager focusing on flexibility and simplicity. Now we support two parameter in URL query string: - 1. transitive: whether to download dependent jars related to your ivy URL. - 2. exclude: exclusion list when download ivy URL jar and dependent jars. + * transitive: whether to download dependent jars related to your ivy URL. + * exclude: exclusion list when download ivy URL jar and dependent jars. User can write ivy URL such as: - 1. ivy://group:module:version - 2. ivy://group:module:version?transitive=true - 3. ivy://group:module:version?exclude=group:module,group:module - 4. ivy://group:module:version?exclude=group:module,group:module&transitive=false + ivy://group:module:version + ivy://group:module:version?transitive=true + ivy://group:module:version?exclude=group:module,group:module + ivy://group:module:version?exclude=group:module,group:module&transitive=false ### Examples diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index b678b4ac36953..a9272be1ff864 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3736,22 +3736,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) - - val e1 = intercept[AssertionError] { - sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?foo=") - }.getMessage - assert(e1.contains("Invalid URI query string: [ foo= ]")) - - val e2 = intercept[AssertionError] { - sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") - }.getMessage - assert(e2.contains("Invalid URI query string: [ bar=&baz=foo ]")) - - val e3 = intercept[AssertionError] { - sql("ADD JAR ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") - }.getMessage - assert(e3.contains("Invalid exclude string: expected 'org:module,org:module,..'," + - " found [ org.pentaho ]")) } } From 8c5cb7ccb21cf3b9fbf4d87da1c5f9991448d035 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 30 Nov 2020 20:59:50 +0800 Subject: [PATCH 22/50] Update SparkContext.scala --- .../main/scala/org/apache/spark/SparkContext.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 16d347ae24560..5ba2ae8904571 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1906,7 +1906,7 @@ class SparkContext(config: SparkConf) extends Logging { } else { val keys = if (path.contains("\\") && Utils.isWindows) { // For local paths with backslashes on Windows, URI throws an exception - addLocalJarFile(new File(path)) + Array(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 @@ -1915,21 +1915,22 @@ class SparkContext(config: SparkConf) extends Logging { // 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)) + Array(addLocalJarFile(new File(uri.getPath))) // A JAR file which exists only on the driver node - case "file" => addLocalJarFile(new File(uri.getPath)) + case "file" => Array(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)) - case _ => checkRemoteJarFile(path) + .split(",") + case _ => Array(checkRemoteJarFile(path)) } } if (keys != null) { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis - keys.split(",").foreach { key => + keys.foreach { key => if (addedJars.putIfAbsent(key, timestamp).isEmpty) { logInfo(s"Added JAR $path at $key with timestamp $timestamp") postEnvironmentUpdate() From f460974489961bb3657dd66faf7799461488db11 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 00:46:00 +0800 Subject: [PATCH 23/50] fix local path with comma --- .../scala/org/apache/spark/SparkContext.scala | 22 +++++++++---------- .../org/apache/spark/SparkContextSuite.scala | 6 +++++ 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 16d347ae24560..538ab70d5ac6b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/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] = { try { if (!file.exists()) { throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not found") @@ -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] = { val hadoopPath = new Path(path) val scheme = hadoopPath.toUri.getScheme if (!Array("http", "https", "ftp").contains(scheme)) { @@ -1890,14 +1890,14 @@ class SparkContext(config: SparkConf) extends Logging { throw new IllegalArgumentException( s"Directory ${path} is not allowed for addJar") } - path + Array(path) } catch { case NonFatal(e) => logError(s"Failed to add $path to Spark environment", e) - null + Array.empty } } else { - path + Array(path) } } @@ -1919,17 +1919,17 @@ 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)) + DependencyUtils.resolveMavenDependencies(URI.create(path)).split(",") case _ => checkRemoteJarFile(path) } } - if (keys != null) { + if (keys.nonEmpty) { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis - keys.split(",").foreach { key => + keys.foreach { key => if (addedJars.putIfAbsent(key, timestamp).isEmpty) { logInfo(s"Added JAR $path at $key with timestamp $timestamp") postEnvironmentUpdate() diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ccca77384e925..9c96f71345064 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/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"))) + } + test("SPARK-22585 addJar argument without scheme is interpreted literally without url decoding") { withTempDir { dir => val tmpDir = new File(dir, "host%3A443") From 050c410d8670e4ec5091b38fad4cd1b575533c10 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 15:25:30 +0800 Subject: [PATCH 24/50] follow comment --- .../scala/org/apache/spark/SparkContext.scala | 18 ++-- .../apache/spark/util/DependencyUtils.scala | 91 +++++++++---------- .../org/apache/spark/SparkContextSuite.scala | 36 ++++++-- ...ql-ref-syntax-aux-resource-mgmt-add-jar.md | 4 +- 4 files changed, 83 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 538ab70d5ac6b..35ce40c28dad6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/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): Array[String] = { + def addLocalJarFile(file: File): Seq[String] = { try { if (!file.exists()) { throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not found") @@ -1869,15 +1869,15 @@ class SparkContext(config: SparkConf) extends Logging { throw new IllegalArgumentException( s"Directory ${file.getAbsoluteFile} is not allowed for addJar") } - Array(env.rpcEnv.fileServer.addJar(file)) + Seq(env.rpcEnv.fileServer.addJar(file)) } catch { case NonFatal(e) => logError(s"Failed to add $path to Spark environment", e) - Array.empty + Nil } } - def checkRemoteJarFile(path: String): Array[String] = { + def checkRemoteJarFile(path: String): Seq[String] = { val hadoopPath = new Path(path) val scheme = hadoopPath.toUri.getScheme if (!Array("http", "https", "ftp").contains(scheme)) { @@ -1890,14 +1890,14 @@ class SparkContext(config: SparkConf) extends Logging { throw new IllegalArgumentException( s"Directory ${path} is not allowed for addJar") } - Array(path) + Seq(path) } catch { case NonFatal(e) => logError(s"Failed to add $path to Spark environment", e) - Array.empty + Nil } } else { - Array(path) + Seq(path) } } @@ -1919,11 +1919,11 @@ 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" => Array("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)).split(",") + DependencyUtils.resolveMavenDependencies(URI.create(path)) case _ => checkRemoteJarFile(path) } } diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 180435722fef2..80e4e27f8efb0 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -40,43 +40,45 @@ private[spark] object DependencyUtils extends Logging { ).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 + def parseQueryParams(uriQuery: String): (Boolean, String) = { + if (uriQuery == null) { + (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") + 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 + } + }) } - transitives.last.toBoolean + // 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) } } @@ -95,7 +97,7 @@ private[spark] object DependencyUtils extends Logging { * `exclude=group:module,group:module...` * @return Comma separated string list of URIs of downloaded jars */ - def resolveMavenDependencies(uri: URI): String = { + def resolveMavenDependencies(uri: URI): Seq[String] = { val Seq(_, _, repositories, ivyRepoPath, ivySettingsPath) = DependencyUtils.getIvyProperties() val authority = uri.getAuthority @@ -108,25 +110,16 @@ private[spark] object DependencyUtils extends Logging { 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))) - } + val (transitive, exclusionList) = parseQueryParams(uri.getQuery) resolveMavenDependencies( - parseTransitive(queryParams.filter(_._1.equals("transitive")).map(_._2)), - parseExcludeList(queryParams.filter(_._1.equals("exclude")).map(_._2)), + transitive, + exclusionList, authority, repositories, ivyRepoPath, Option(ivySettingsPath) - ) + ).split(",") } def resolveMavenDependencies( diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 9c96f71345064..ff66cf737390d 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -366,10 +366,27 @@ 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"))) + test("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] { + sc.addJar(badURL) + } + assert(e1.getMessage.contains(badURL)) + assert(!sc.listJars().exists(_.contains(s"Test,UDTF_${scheme}.jar"))) + } + } } test("SPARK-22585 addJar argument without scheme is interpreted literally without url decoding") { @@ -965,18 +982,25 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("SPARK-33084: Add jar support ivy url") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) - // default transitive=false, only download specified jar + // Default transitive=false, only download specified jar sc.addJar("ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") assert( sc.listJars().exists(_.contains("org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar"))) + // Invalid transitive value, will use default value `false` + sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=foo") + assert(!sc.listJars().exists(_.contains("scalajs-library_2.12"))) + assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) + // test download ivy URL jar return multiple jars sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=true") assert(sc.listJars().exists(_.contains("scalajs-library_2.12"))) assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) + // test multiple transitive params of false, invalid value and finally true sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7" + - "?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=true") + "?exclude=org.pentaho:pentaho-aggdesigner-algorithm" + + "&transitive=false&transitive=foo&transitive=true") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 2ff6d24f73f01..68c8c421e1b64 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -42,9 +42,9 @@ ADD JAR file_name User can write ivy URL such as: ivy://group:module:version - ivy://group:module:version?transitive=true + ivy://group:module:version?transitive=[true|false] ivy://group:module:version?exclude=group:module,group:module - ivy://group:module:version?exclude=group:module,group:module&transitive=false + ivy://group:module:version?exclude=group:module,group:module&transitive=[true|false] ### Examples From ff611a673051a903ab6fc897eff517aa184099fb Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 15:33:55 +0800 Subject: [PATCH 25/50] Update SessionState.scala --- .../scala/org/apache/spark/sql/internal/SessionState.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index ff5b9fb15f217..3cb542b3397e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -161,9 +161,9 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade } } - protected def resolveJars(path: URI): List[String] = { + protected def resolveJars(path: URI): Seq[String] = { path.getScheme match { - case "ivy" => DependencyUtils.resolveMavenDependencies(path).split(",").toList + case "ivy" => DependencyUtils.resolveMavenDependencies(path) case _ => path.toString :: Nil } } From 03aca3b71f0fb2af9766490c36bca9fbf951afa5 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 23:17:31 +0800 Subject: [PATCH 26/50] split UT and use simply dependency ivy path --- .../org/apache/spark/SparkContextSuite.scala | 109 ++++++++++++++---- 1 file changed, 85 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ff66cf737390d..e1112f1940bbe 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -366,7 +366,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } - test("add jar when path contains comma") { + 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")) @@ -979,32 +979,93 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } - test("SPARK-33084: Add jar support ivy url") { + 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"))) - // Default transitive=false, only download specified jar - sc.addJar("ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") - assert( - sc.listJars().exists(_.contains("org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar"))) - - // Invalid transitive value, will use default value `false` - sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=foo") - assert(!sc.listJars().exists(_.contains("scalajs-library_2.12"))) - assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) - - // test download ivy URL jar return multiple jars - sc.addJar("ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0?transitive=true") - assert(sc.listJars().exists(_.contains("scalajs-library_2.12"))) - assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) - - // test multiple transitive params of false, invalid value and finally true - sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7" + - "?exclude=org.pentaho:pentaho-aggdesigner-algorithm" + - "&transitive=false&transitive=foo&transitive=true") - assert(sc.listJars().exists(_.contains("org.apache.hive_hive-contrib-2.3.7.jar"))) - assert(sc.listJars().exists(_.contains("org.apache.hive_hive-exec-2.3.7.jar"))) - assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) + 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"))) + } + + + test("SPARK-33084: Add jar support ivy url -- test invalid ivy URI") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) val e1 = intercept[IllegalArgumentException] { sc.addJar("ivy://") }.getMessage From 653b919e7d6dd1975873d3da232ca56abd1edd45 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 23:18:34 +0800 Subject: [PATCH 27/50] Update DependencyUtils.scala --- .../org/apache/spark/util/DependencyUtils.scala | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 80e4e27f8efb0..a186ebd65c149 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -40,7 +40,7 @@ private[spark] object DependencyUtils extends Logging { ).map(sys.props.get(_).orNull) } - def parseQueryParams(uriQuery: String): (Boolean, String) = { + private def parseQueryParams(uriQuery: String): (Boolean, String) = { if (uriQuery == null) { (false, "") } else { @@ -56,13 +56,10 @@ private[spark] object DependencyUtils extends Logging { 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 - } - }) + 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 From 6e4827502b80d7635b6e93bf74b0bed2eb29f0c6 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 1 Dec 2020 23:33:03 +0800 Subject: [PATCH 28/50] Update SparkContext.scala --- .../scala/org/apache/spark/SparkContext.scala | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 35ce40c28dad6..f56b99663c0a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1929,14 +1929,16 @@ class SparkContext(config: SparkConf) extends Logging { } if (keys.nonEmpty) { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis - 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 " + - "is not supported in the current version.") - } + val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty) + if (added.nonEmpty) { + logInfo(s"Added jar or dependency jars of ivy URI with $path" + + s" at ${added.mkString(",")} with timestamp $timestamp") + postEnvironmentUpdate() + } + if (existed.nonEmpty) { + logWarning(s"The jar or dependency jars of ivy URI with $path at" + + s" ${existed.mkString(",")} has been added already." + + s" Overwriting of added jars is not supported in the current version.") } } } From bdc50356077c9c9db1cbfab249083053dbe1a7dd Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 11:26:09 +0800 Subject: [PATCH 29/50] follow comment --- .../spark/deploy/worker/DriverWrapper.scala | 8 +- .../apache/spark/util/DependencyUtils.scala | 105 +++++++++++++----- .../org/apache/spark/SparkContextSuite.scala | 19 ++-- 3 files changed, 89 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index fe655903dfa3d..c1288d64c53f7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -79,11 +79,11 @@ object DriverWrapper extends Logging { val secMgr = new SecurityManager(sparkConf) val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) - val Seq(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath) = - DependencyUtils.getIvyProperties() + val ivyProperties = DependencyUtils.getIvyProperties() - val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies( - true, packagesExclusions, packages, repositories, ivyRepoPath, Option(ivySettingsPath)) + val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(true, + ivyProperties.packagesExclusions, ivyProperties.packages, ivyProperties.repositories, + ivyProperties.ivyRepoPath, Option(ivyProperties.ivySettingsPath)) val jars = { val jarsProp = sys.props.get(config.JARS.key).orNull if (!StringUtils.isBlank(resolvedMavenCoordinates)) { diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index a186ebd65c149..10c17ec5a5dd4 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -28,29 +28,56 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.internal.Logging +case class IvyProperties( + packagesExclusions: String, + packages: String, + repositories: String, + ivyRepoPath: String, + ivySettingsPath: String) + private[spark] object DependencyUtils extends Logging { - def getIvyProperties(): Seq[String] = { - Seq( + 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 parseQueryParams(uriQuery: String): (Boolean, String) = { + /** + * 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(_.length != 2)) { - throw new URISyntaxException(uriQuery, s"Invalid query string: $uriQuery") + 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 = 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." + @@ -65,16 +92,30 @@ private[spark] object DependencyUtils extends Logging { // 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) => + params.map(_._2).flatMap { excludeString => val excludes = excludeString.split(",") - if (excludes.exists(_.split(":").length != 2)) { - throw new URISyntaxException(excludeString, "Invalid exclude string: " + + 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 + 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) } } @@ -92,31 +133,35 @@ private[spark] object DependencyUtils extends Logging { * 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 + * @return Comma separated string list of jars downloaded. */ 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") - } - if (authority.split(":").length != 3) { - throw new URISyntaxException( - authority, "Invalid url: Expected 'org:module:version', found " + authority) - } + 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.getQuery) + val (transitive, exclusionList) = parseQueryParams(uri) - resolveMavenDependencies( - transitive, - exclusionList, - authority, - repositories, - ivyRepoPath, - Option(ivySettingsPath) - ).split(",") + resolveMavenDependencies( + transitive, + exclusionList, + authority, + ivyProperties.repositories, + ivyProperties.ivyRepoPath, + Option(ivyProperties.ivySettingsPath) + ).split(",") + } catch { + case e: URISyntaxException => + throw new IllegalArgumentException(e.getMessage) + } } def resolveMavenDependencies( diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index e1112f1940bbe..eed8c9f97c052 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark import java.io.File -import java.net.{MalformedURLException, URI, URISyntaxException} +import java.net.{MalformedURLException, URI} import java.nio.charset.StandardCharsets import java.util.concurrent.{CountDownLatch, Semaphore, TimeUnit} @@ -380,10 +380,10 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu Seq("http", "https", "ftp").foreach { scheme => val badURL = s"$scheme://user:pwd/path/Test,UDTF_${scheme}.jar" - val e1 = intercept[MalformedURLException] { + val e = intercept[MalformedURLException] { sc.addJar(badURL) } - assert(e1.getMessage.contains(badURL)) + assert(e.getMessage.contains(badURL)) assert(!sc.listJars().exists(_.contains(s"Test,UDTF_${scheme}.jar"))) } } @@ -1017,8 +1017,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu 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.7.0.jar"))) assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.6.0.jar"))) } @@ -1030,11 +1030,13 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu 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]")) + // transitive=invalidValue will win and treated as false 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"))) + // transitive=true will win 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"))) @@ -1063,7 +1065,6 @@ 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 url -- test invalid ivy URI") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) val e1 = intercept[IllegalArgumentException] { @@ -1071,23 +1072,23 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu }.getMessage assert(e1.contains("Expected authority at index 6: ivy://")) - val e2 = intercept[URISyntaxException] { + val e2 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib") }.getMessage assert(e2.contains("Invalid url: Expected 'org:module:version'," + " found org.apache.hive:hive-contrib")) - val e3 = intercept[URISyntaxException] { + val e3 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") }.getMessage assert(e3.contains("Invalid query string: foo=")) - val e4 = intercept[URISyntaxException] { + val e4 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") }.getMessage assert(e4.contains("Invalid query string: bar=&baz=foo")) - val e5 = intercept[URISyntaxException] { + val e5 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") }.getMessage assert(e5.contains( From 9c228823e0be56a87ebc498c254c627babc9db45 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 13:14:22 +0800 Subject: [PATCH 30/50] follow comment --- .../scala/org/apache/spark/SparkContext.scala | 23 +++++++--- .../apache/spark/util/DependencyUtils.scala | 7 +-- .../org/apache/spark/SparkContextSuite.scala | 46 +++++++++++++++---- 3 files changed, 59 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f56b99663c0a6..10530245c8158 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1904,6 +1904,7 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { + var schema = "" val keys = if (path.contains("\\") && Utils.isWindows) { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) @@ -1911,7 +1912,8 @@ class SparkContext(config: SparkConf) extends Logging { 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 @@ -1931,14 +1933,23 @@ class SparkContext(config: SparkConf) extends Logging { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty) if (added.nonEmpty) { - logInfo(s"Added jar or dependency jars of ivy URI with $path" + - s" at ${added.mkString(",")} with timestamp $timestamp") + 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") + } postEnvironmentUpdate() } if (existed.nonEmpty) { - logWarning(s"The jar or dependency jars of ivy URI with $path at" + - s" ${existed.mkString(",")} has been added already." + - s" Overwriting of added jars is not supported in the current version.") + if (schema != "ivy") { + logWarning(s"The jar $path has been added already. Overwriting of added jars " + + "is not supported in the current version.") + } else { + logWarning(s"The dependency jars of ivy URI with $path at" + + s" ${existed.mkString(",")} has been added already." + + s" Overwriting of added jars is not supported in the current version.") + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 10c17ec5a5dd4..02e685d788b4d 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -103,11 +103,12 @@ private[spark] object DependencyUtils extends Logging { }.mkString(",") }.getOrElse("") - val invalidParams = - groupedParams.filter(entry => Seq("transitive", "exclude").contains(entry._1)).keys + 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.") + s"Invalid parameters `${invalidParams.mkString(",")}` found in URI query `$uriQuery`.") } groupedParams.foreach { case (key: String, values: Array[(String, String)]) => diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index eed8c9f97c052..dadfe35e963c7 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -998,11 +998,33 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } 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"))) + val logAppender = new LogAppender("transitive=true will download dependency jars") + withLogAppender(logAppender) { + 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") + val dependencyJars = Array( + "org.apache.hive_hive-storage-api-2.7.0.jar", + "org.slf4j_slf4j-api-1.7.10.jar", + "commons-lang_commons-lang-2.6.jar") + + dependencyJars.foreach(jar => assert(sc.listJars().exists(_.contains(jar)))) + + assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( + "Added dependency jars of ivy uri" + + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) + + // test dependency jars exist + sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true") + assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( + "The dependency jars of ivy URI with" + + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) + val existMsg = logAppender.loggingEvents.filter(_.getRenderedMessage.contains( + "The dependency jars of ivy URI with" + + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) + .head.getRenderedMessage + dependencyJars.foreach(jar => assert(existMsg.contains(jar))) + } } test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") { @@ -1023,9 +1045,17 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } 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"))) + val logAppender = new LogAppender("test log when have invalid parameter") + withLogAppender(logAppender) { + 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?" + + "invalidParam1=foo&invalidParam2=boo") + assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) + assert(logAppender.loggingEvents.exists(_.getRenderedMessage.contains( + "Invalid parameters `invalidParam1,invalidParam2` found in URI query" + + " `invalidParam1=foo&invalidParam2=boo`"))) + } } test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") { From 9c88f8d90b0121d158f8b9ff1597f620ff571dba Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 23:09:59 +0800 Subject: [PATCH 31/50] follow comment --- .../scala/org/apache/spark/SparkContext.scala | 29 +++---- .../apache/spark/util/DependencyUtils.scala | 82 +++++++++---------- 2 files changed, 48 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 10530245c8158..fa54e3101d588 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1904,16 +1904,15 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { - var schema = "" - val keys = 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) - schema = uri.getScheme - schema 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 @@ -1928,28 +1927,20 @@ class SparkContext(config: SparkConf) extends Logging { DependencyUtils.resolveMavenDependencies(URI.create(path)) case _ => checkRemoteJarFile(path) } + (jarPaths, uriSchema) } if (keys.nonEmpty) { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis 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") - } + val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri" + logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp") postEnvironmentUpdate() } if (existed.nonEmpty) { - if (schema != "ivy") { - logWarning(s"The jar $path has been added already. Overwriting of added jars " + - "is not supported in the current version.") - } else { - logWarning(s"The dependency jars of ivy URI with $path at" + - s" ${existed.mkString(",")} has been added already." + - s" Overwriting of added jars is not supported in the current version.") - } + 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.") } } } diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 02e685d788b4d..5378673f9030a 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.io.File -import java.net.{URI, URISyntaxException} +import java.net.URI import org.apache.commons.lang3.StringUtils import org.apache.hadoop.conf.Configuration @@ -48,6 +48,10 @@ private[spark] object DependencyUtils extends Logging { IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath) } + private def checkInvalidQueryString(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. @@ -71,13 +75,13 @@ private[spark] object DependencyUtils extends Logging { (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") + if (mapTokens.exists(checkInvalidQueryString)) { + 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 - var transitive: Boolean = 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." + @@ -94,27 +98,20 @@ private[spark] object DependencyUtils extends Logging { 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) + if (excludes.map(_.split(":")).exists(checkInvalidQueryString)) { + throw new IllegalArgumentException( + s"Invalid exclude string in ivy uri ${uri.toString}:" + + s" expected 'org:module,org:module,..', found " + excludeString) } excludes }.mkString(",") }.getOrElse("") - val invalidParams = groupedParams - .filter(entry => !Seq("transitive", "exclude").contains(entry._1)) - .keys.toArray.sorted + val validParams = Set("transitive", "exclude") + val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.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") - } + logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " + + s"in ivy URI query `$uriQuery`.") } (transitive, exclusionList) @@ -137,32 +134,29 @@ private[spark] object DependencyUtils extends Logging { * @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 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}:" + + s" Expected 'org:module:version', found $authority.") + } - val (transitive, exclusionList) = parseQueryParams(uri) + val (transitive, exclusionList) = parseQueryParams(uri) - resolveMavenDependencies( - transitive, - exclusionList, - authority, - ivyProperties.repositories, - ivyProperties.ivyRepoPath, - Option(ivyProperties.ivySettingsPath) - ).split(",") - } catch { - case e: URISyntaxException => - throw new IllegalArgumentException(e.getMessage) - } + resolveMavenDependencies( + transitive, + exclusionList, + authority, + ivyProperties.repositories, + ivyProperties.ivyRepoPath, + Option(ivyProperties.ivySettingsPath) + ).split(",") } def resolveMavenDependencies( From 8220e5ae03aaf6d06edcb83a9e583379f30ec90f Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 23:21:06 +0800 Subject: [PATCH 32/50] Update SparkContextSuite.scala --- .../org/apache/spark/SparkContextSuite.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index dadfe35e963c7..19df591ad420b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1017,10 +1017,10 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu // test dependency jars exist sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true") assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( - "The dependency jars of ivy URI with" + + "The dependency jars of ivy uri" + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) val existMsg = logAppender.loggingEvents.filter(_.getRenderedMessage.contains( - "The dependency jars of ivy URI with" + + "The dependency jars of ivy uri" + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) .head.getRenderedMessage dependencyJars.foreach(jar => assert(existMsg.contains(jar))) @@ -1053,8 +1053,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu "invalidParam1=foo&invalidParam2=boo") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) assert(logAppender.loggingEvents.exists(_.getRenderedMessage.contains( - "Invalid parameters `invalidParam1,invalidParam2` found in URI query" + - " `invalidParam1=foo&invalidParam2=boo`"))) + "Invalid parameters `invalidParam1,invalidParam2` found in ivy URI query" + + " `invalidParam1=foo&invalidParam2=boo`."))) } } @@ -1105,24 +1105,27 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val e2 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib") }.getMessage - assert(e2.contains("Invalid url: Expected 'org:module:version'," + - " found org.apache.hive:hive-contrib")) + 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] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") }.getMessage - assert(e3.contains("Invalid query string: foo=")) + assert(e3.contains("Invalid query string in ivy uri" + + " ivy://org.apache.hive:hive-contrib:2.3.7?foo=:")) val e4 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") }.getMessage - assert(e4.contains("Invalid query string: bar=&baz=foo")) + assert(e4.contains("Invalid query string in ivy uri" + + " ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo: bar=&baz=foo")) val e5 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") }.getMessage - assert(e5.contains( - "Invalid exclude string: expected 'org:module,org:module,..', found org.pentaho")) + assert(e5.contains("Invalid exclude string in ivy uri" + + " ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:" + + " expected 'org:module,org:module,..', found org.pentaho")) } } From 49ac62cdd7cfe97bef8ce1ac8bb2ecf80d446362 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 23:27:19 +0800 Subject: [PATCH 33/50] follow comment --- .../main/scala/org/apache/spark/util/DependencyUtils.scala | 4 ++-- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 5378673f9030a..d702b64265344 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -77,7 +77,7 @@ private[spark] object DependencyUtils extends Logging { val mapTokens = uriQuery.split("&").map(_.split("=")) if (mapTokens.exists(checkInvalidQueryString)) { throw new IllegalArgumentException( - s"Invalid query string in ivy uri ${uri.toString}: $uriQuery") + 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 @@ -111,7 +111,7 @@ private[spark] object DependencyUtils extends Logging { 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`.") + s"in ivy uri query `$uriQuery`.") } (transitive, exclusionList) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 19df591ad420b..4562476fdafec 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1053,7 +1053,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu "invalidParam1=foo&invalidParam2=boo") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) assert(logAppender.loggingEvents.exists(_.getRenderedMessage.contains( - "Invalid parameters `invalidParam1,invalidParam2` found in ivy URI query" + + "Invalid parameters `invalidParam1,invalidParam2` found in ivy uri query" + " `invalidParam1=foo&invalidParam2=boo`."))) } } @@ -1095,7 +1095,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 url -- test invalid ivy URI") { + test("SPARK-33084: Add jar support ivy url -- test invalid ivy uri") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) val e1 = intercept[IllegalArgumentException] { sc.addJar("ivy://") From b69a62ea633653e71269f3875c1a4a8af8f1665b Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 2 Dec 2020 23:28:06 +0800 Subject: [PATCH 34/50] Update DependencyUtils.scala --- .../main/scala/org/apache/spark/util/DependencyUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index d702b64265344..64b7cdd68db1a 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -48,7 +48,7 @@ private[spark] object DependencyUtils extends Logging { IvyProperties(packagesExclusions, packages, repositories, ivyRepoPath, ivySettingsPath) } - private def checkInvalidQueryString(tokens: Array[String]): Boolean = { + private def isInvalidQueryString(tokens: Array[String]): Boolean = { tokens.length != 2 || StringUtils.isBlank(tokens(0)) || StringUtils.isBlank(tokens(1)) } @@ -75,7 +75,7 @@ private[spark] object DependencyUtils extends Logging { (false, "") } else { val mapTokens = uriQuery.split("&").map(_.split("=")) - if (mapTokens.exists(checkInvalidQueryString)) { + if (mapTokens.exists(isInvalidQueryString)) { throw new IllegalArgumentException( s"Invalid query string in ivy uri ${uri.toString}: $uriQuery") } @@ -98,7 +98,7 @@ private[spark] object DependencyUtils extends Logging { val exclusionList = groupedParams.get("exclude").map { params => params.map(_._2).flatMap { excludeString => val excludes = excludeString.split(",") - if (excludes.map(_.split(":")).exists(checkInvalidQueryString)) { + 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) From 273a5ac15c04c8826babd151caf25ccc39b286e2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 3 Dec 2020 09:38:06 +0800 Subject: [PATCH 35/50] Follow comment --- .../apache/spark/util/DependencyUtils.scala | 22 ++++++++++--------- .../org/apache/spark/SparkContextSuite.scala | 2 +- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 64b7cdd68db1a..1c18f86e7064c 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -80,18 +80,20 @@ private[spark] object DependencyUtils extends Logging { 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 - 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 - } + 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) + // 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. diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 4562476fdafec..f7e6bf0465779 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1062,7 +1062,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) // transitive=invalidValue will win and treated as false sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" + - "transitive=true&transitive=false&transitive=invalidValue") + "transitive=true&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"))) From ebe1c9ccb75898b5c68a570d43d89444d4c92970 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 4 Dec 2020 09:37:25 +0800 Subject: [PATCH 36/50] Update DependencyUtils.scala --- .../scala/org/apache/spark/util/DependencyUtils.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 1c18f86e7064c..789e5bf43c551 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -87,12 +87,8 @@ private[spark] object DependencyUtils extends Logging { 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) + 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 From 6034fb25fbb8cd04b040f0d88ce117b1a54ee0ac Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 5 Dec 2020 08:25:45 +0800 Subject: [PATCH 37/50] Update sql-ref-syntax-aux-resource-mgmt-add-jar.md --- docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 68c8c421e1b64..8e7f7afef50a4 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -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 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. + * 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: From e22e3986dc62ccc6ac71b5be7144703d4f414241 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 7 Dec 2020 11:20:22 +0800 Subject: [PATCH 38/50] Update SparkContext.scala --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fa54e3101d588..decd019cb40f3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1925,6 +1925,7 @@ class SparkContext(config: SparkConf) extends Logging { // Since `new Path(path).toUri` will lose query information, // so here we use `URI.create(path)` DependencyUtils.resolveMavenDependencies(URI.create(path)) + .map(e => env.rpcEnv.fileServer.addJar(new File(e))) case _ => checkRemoteJarFile(path) } (jarPaths, uriSchema) From afea73fcadc1d6641c6feb1b07b7fc874b836b29 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 7 Dec 2020 11:24:04 +0800 Subject: [PATCH 39/50] Update SparkContext.scala --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index decd019cb40f3..afc48d859cd14 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1925,7 +1925,7 @@ class SparkContext(config: SparkConf) extends Logging { // Since `new Path(path).toUri` will lose query information, // so here we use `URI.create(path)` DependencyUtils.resolveMavenDependencies(URI.create(path)) - .map(e => env.rpcEnv.fileServer.addJar(new File(e))) + .map(jar => env.rpcEnv.fileServer.addJar(new File(jar))) case _ => checkRemoteJarFile(path) } (jarPaths, uriSchema) From bce3d4098fbcd36faeb376b6ad1ba565389a2c2e Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 14 Dec 2020 11:45:37 +0800 Subject: [PATCH 40/50] Update SparkContext.scala --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index afc48d859cd14..46bbbe55a93e1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1925,7 +1925,7 @@ class SparkContext(config: SparkConf) extends Logging { // 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))) + .flatMap(jar => addLocalJarFile(new File(jar))) case _ => checkRemoteJarFile(path) } (jarPaths, uriSchema) From 57c351d076da51fd8c41323f16ff8d8fbc4b964b Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 15 Dec 2020 17:02:13 +0800 Subject: [PATCH 41/50] Update HiveQuerySuite.scala --- .../sql/hive/execution/HiveQuerySuite.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 1cabf6033e8d8..2dca29bb67711 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/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") + assert(sql("list jars").filter(_.getString(0).contains( + "org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar")).count() > 0) + assert(sql("list jar"). + filter(_.getString(0).contains( + "org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar")).count() > 0) + } } // for SPARK-2180 test From 8c53b83d1650a69b4225cdbca4fd26d1d5537d94 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 22 Dec 2020 15:21:28 +0800 Subject: [PATCH 42/50] follow comment --- .../scala/org/apache/spark/SparkContext.scala | 6 +-- .../apache/spark/util/DependencyUtils.scala | 38 +++++++++---------- .../org/apache/spark/SparkContextSuite.scala | 36 +++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++------ 5 files changed, 54 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 473a4a9ca80c4..6b75d5960ed66 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2003,14 +2003,14 @@ class SparkContext(config: SparkConf) extends Logging { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty) if (added.nonEmpty) { - val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri" + val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of Ivy URI" logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp") postEnvironmentUpdate() } if (existed.nonEmpty) { - val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of ivy uri" + 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.") + " Overwriting of added jar is not supported in the current version.") } } } diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index 789e5bf43c551..9956ccedf5842 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -56,10 +56,10 @@ private[spark] object DependencyUtils extends Logging { * 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. + * @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 + * 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 @@ -77,39 +77,39 @@ private[spark] object DependencyUtils extends Logging { val mapTokens = uriQuery.split("&").map(_.split("=")) if (mapTokens.exists(isInvalidQueryString)) { throw new IllegalArgumentException( - s"Invalid query string in ivy uri ${uri.toString}: $uriQuery") + 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 + // 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 URL query only once." + + 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) // 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 an Ivy URI. When download Ivy URI 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) + s"Invalid exclude string in Ivy URI ${uri.toString}:" + + " expected 'org:module,org:module,..', found " + excludeString) } excludes }.mkString(",") }.getOrElse("") val validParams = Set("transitive", "exclude") - val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq.sorted + val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq if (invalidParams.nonEmpty) { - logWarning(s"Invalid parameters `${invalidParams.mkString(",")}` found " + - s"in ivy uri query `$uriQuery`.") + logWarning(s"Invalid parameters `${invalidParams.sorted.mkString(",")}` found " + + s"in Ivy URI query `$uriQuery`.") } (transitive, exclusionList) @@ -117,16 +117,16 @@ private[spark] object DependencyUtils extends Logging { } /** - * Download Ivy URIs dependency jars. + * Download Ivy URI's dependency jars. * - * @param uri Ivy uri need to be downloaded. The URI format should be: + * @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¶meter=value...` - * Note that currently ivy URI query part support two parameters: - * 1. transitive: whether to download dependent jars related to your ivy URL. + * 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. - * 2. exclude: exclusion list when download ivy URL jar and dependency jars. + * 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...` * @return Comma separated string list of jars downloaded. @@ -136,12 +136,12 @@ private[spark] object DependencyUtils extends Logging { 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.") + s"Invalid Ivy URI authority in uri ${uri.toString}:" + + " Expected 'org:module:version', found null.") } if (authority.split(":").length != 3) { throw new IllegalArgumentException( - s"Invalid ivy uri authority in uri ${uri.toString}:" + + s"Invalid Ivy URI authority in uri ${uri.toString}:" + s" Expected 'org:module:version', found $authority.") } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 956efc717325e..c8a5a16c343ee 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1058,7 +1058,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } - test("SPARK-33084: Add jar support ivy url -- default transitive = false") { + test("SPARK-33084: Add jar support Ivy URI -- 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"))) @@ -1068,7 +1068,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 url -- invalid transitive use default false") { + test("SPARK-33084: Add jar support Ivy URI -- 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"))) @@ -1076,7 +1076,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 url -- transitive=true will download dependency jars") { + test("SPARK-33084: Add jar support Ivy URI -- transitive=true will download dependency jars") { val logAppender = new LogAppender("transitive=true will download dependency jars") withLogAppender(logAppender) { sc = new SparkContext( @@ -1090,23 +1090,23 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu dependencyJars.foreach(jar => assert(sc.listJars().exists(_.contains(jar)))) assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( - "Added dependency jars of ivy uri" + + "Added dependency jars of Ivy URI" + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) // test dependency jars exist sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true") assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( - "The dependency jars of ivy uri" + + "The dependency jars of Ivy URI" + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) val existMsg = logAppender.loggingEvents.filter(_.getRenderedMessage.contains( - "The dependency jars of ivy uri" + + "The dependency jars of Ivy URI" + " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) .head.getRenderedMessage dependencyJars.foreach(jar => assert(existMsg.contains(jar))) } } - test("SPARK-33084: Add jar support ivy url -- test exclude param when transitive=true") { + 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" + "?exclude=commons-lang:commons-lang&transitive=true") @@ -1115,7 +1115,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 url -- test different version") { + test("SPARK-33084: Add jar support Ivy URI -- 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") sc.addJar("ivy://org.apache.hive:hive-storage-api:2.6.0") @@ -1123,7 +1123,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu 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") { + test("SPARK-33084: Add jar support Ivy URI -- test invalid param") { val logAppender = new LogAppender("test log when have invalid parameter") withLogAppender(logAppender) { sc = new SparkContext( @@ -1132,12 +1132,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu "invalidParam1=foo&invalidParam2=boo") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) assert(logAppender.loggingEvents.exists(_.getRenderedMessage.contains( - "Invalid parameters `invalidParam1,invalidParam2` found in ivy uri query" + + "Invalid parameters `invalidParam1,invalidParam2` found in Ivy URI query" + " `invalidParam1=foo&invalidParam2=boo`."))) } } - test("SPARK-33084: Add jar support ivy url -- test multiple transitive params") { + test("SPARK-33084: Add jar support Ivy URI -- test multiple transitive params") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) // transitive=invalidValue will win and treated as false sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?" + @@ -1152,7 +1152,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 url -- test param case sensitive") { + test("SPARK-33084: Add jar support Ivy URI -- 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"))) @@ -1163,7 +1163,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 url -- test transitive value case sensitive") { + test("SPARK-33084: Add jar support Ivy URI -- 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"))) @@ -1174,7 +1174,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 url -- test invalid ivy uri") { + test("SPARK-33084: Add jar support Ivy URI -- test invalid ivy uri") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) val e1 = intercept[IllegalArgumentException] { sc.addJar("ivy://") @@ -1184,25 +1184,25 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val e2 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib") }.getMessage - assert(e2.contains("Invalid ivy uri authority in uri ivy://org.apache.hive:hive-contrib:" + + 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] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") }.getMessage - assert(e3.contains("Invalid query string in ivy uri" + + assert(e3.contains("Invalid query string in Ivy URI" + " ivy://org.apache.hive:hive-contrib:2.3.7?foo=:")) val e4 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") }.getMessage - assert(e4.contains("Invalid query string in ivy uri" + + assert(e4.contains("Invalid query string in Ivy URI" + " ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo: bar=&baz=foo")) val e5 = intercept[IllegalArgumentException] { sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") }.getMessage - assert(e5.contains("Invalid exclude string in ivy uri" + + assert(e5.contains("Invalid exclude string in Ivy URI" + " ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:" + " expected 'org:module,org:module,..', found org.pentaho")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a6a27c3dfbb8c..4b5967729dcd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3719,7 +3719,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } - test("SPARK-33084: Add jar support ivy url in SQL") { + test("SPARK-33084: Add jar support Ivy URI in SQL") { val sc = spark.sparkContext // default transitive=false, only download specified jar sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2dca29bb67711..21cc6af398eec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1220,20 +1220,21 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } } - test("SPARK-33084: Add jar support ivy url in SQL") { + test("SPARK-33084: Add jar support Ivy URI 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") - assert(sql("list jars").filter(_.getString(0).contains( - "org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar")).count() > 0) - assert(sql("list jar"). - filter(_.getString(0).contains( + withTable("t") { + sql("ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:2.3.7") + sql( + """CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert(sql("LIST JARS").filter(_.getString(0).contains( "org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar")).count() > 0) + assert(sql("LIST JAR"). + filter(_.getString(0).contains( + "org.apache.hive.hcatalog_hive-hcatalog-core-2.3.7.jar")).count() > 0) + } } } From 4048c5bb166f82efc39c0d8823e8857e3d0ddff2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 22 Dec 2020 18:46:42 +0800 Subject: [PATCH 43/50] https://github.com/apache/spark/pull/29966#discussion_r547040115 --- sql/core/src/test/resources/SPARK-33084.jar | Bin 0 -> 6322 bytes .../org/apache/spark/sql/SQLQuerySuite.scala | 35 ++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 sql/core/src/test/resources/SPARK-33084.jar diff --git a/sql/core/src/test/resources/SPARK-33084.jar b/sql/core/src/test/resources/SPARK-33084.jar new file mode 100644 index 0000000000000000000000000000000000000000..1dc5e9303b707f9b788c28fc23a0f0aa5b2c323b GIT binary patch literal 6322 zcmbVR2|Sc*`yN~NeQB)8R>m4*36(u&7z~Y_WM?ppbwUZ#A~{(@)+GBf_AJ?Dm!*=j zi^@T^VURzk)H$cl`M&Re&+lF4H}`ct_x(Qa``qt+jXsowlnQWY&Qb9Of1G^VQ63z1 zHH=imv|*YO`rq6r08R&P5w-k4&x4p;cCU(iCu zKz%*^WpK%1>B%`SQApo&QK*}(X zM!=b&aKpytdt|pe*>kBHZrkSuP_<*SdC@9KaPN9JPG!?%fxVT}jAPa7vlkKIC`15& z;`U!NAw3Au!^`n|51S_(elGka|t{Vtysh`*z=N@0Xu)k4t4bTYQO;t8=H3+3923Ocu-&_+Lz5yDcWdw-yeWtrBE)d}tidKZpVjd$gLzDk4}+mE zVh@pxZq(>dmq^nAZ?@M3o>b!!A;ZHu29PyU?*;j(TM^ndYOB|KrIOGq$bAbNVM!h; zhU0B@AI}#v4(i671?eiOk*s_1YA75NV`j{DJ#A1_;Lwk_AjGiWoDA_Le296T@bE%u z$#g$LXz%mz8x-E%?Lv0p)}x#7{oNO@DCP9V`=4*q%1Eo$StVt^B+=#^k;{{MyRs`p zc$fGVZP2Z~tWe*qUvmquZo(&9ihNhdJrE!GEJ|#>N2G(hsp^Fz+8)|dTlPLpoI0PR zXZ8!SsTmiPs#hu|DAOPvy|^}bF7j0F><~jQ?%aFK3D?Mkw(+HRogoPq<02OqTlsF6 z-TMMRt>%DP(2`4r#10xRH5hrJuso|SgOkX!DX21{;p5iT2}0yn3HX3E`kA&fT2!Hn8SaGo?+v?UaYCu^8xUYm0aX?>bfdQTcB@fj2qtjU}6=q^)9kRT+JkklwQwa9p_ z3w_BdZ=qa0S8AWZ?AZOgNw2TP_CK%WRAunJLC;5;DpTB_X)#i)SLbxL6ftDfpwC); zsq{`r1k>fRv=>suvQyVR9|%m+Wu}VCJR@~@6IF(gSG~UdTj|U^&n3*jQVsB(THb|e1OiomY;AVP(}2e%5v!%V_`h|𝔄e>Y*Uq;UTJf%* znp31=O}3}5OevvMBwccDxtNmHxU|pQP^&99E6rb0?)ibqACtyLa-wur-TqQOA|}U) zR^pT2>r-2A$jDb{8yHws+}h12F!NYqmJP#%^*tts#bDxv1|cvAsG6NPVd9$gss`0% zEW_Wdh9_;5CqpdGs_j&vXRx+83L(UGb{3jV;QSNzYn)y?@7h|%=Pd-7g1UjFpY}@{r>6?i z_a9!@wF&G;RnpNc0m8&HJ}pmTAv)1S)#u&K+B!r+5ODF_CFV3P z;jBmsO*7MnsXeA|!4F^m;hnE3*u(f2@}NSW63AOH!vKfuY#bb!WPF;jS`0>DU8xbDd1yB%MsuaG$z5 z+r?wdNXMAY%tU(Ya_y-dr=d^y)ie(~+2MF9>+VGQe&g+8BX2>*28ZTns=1u|9qq-eqRglJv?4?7G*FyPwck?3(VaygV?zYh9m0J8^1KatouJ+3N?kmp7#h&f5L;+ySV>Yx++ba$_ zKg*WNx4V`JiW&`~Ukz-o0`Uat^7Rei>ED-_ztBxNdDbW2zsmbuOM0Bb zOT)_x4JK!3yH~`eTgz(BV$U6)1Cz1dxdX%iF*Ny8SM8hgnGMC0UF{hf$77Be#GdOY z_Adhqtq*at!!0oExReYBD&fVZ^JGC_MHt@RfEApmq+92)1#g~DOKsYZg{u?U;^T4vdx>rIu>i_P1DixHn{ z*O4)$IvqPyLMucq;~;#s;S<~Tjk{E#AUZBJxyXL4`8jUhh_YYw|gM1{=O@Ss#Gi5W9Hr+^9MH7 zD`_%Bf7*FUAk>gO3{bF=TG(>@LUi{4hHUrOnT?nd9vnV6{+ z7(zQxN6@i)sKNPOj18ZnFzk`u=N$4h<+lbhOJtwf(%iH3RW2|$wF6`3M6Z^ssY%$DE}MBW5Ew`BRigsQemnJRKCir)BwERs2qiHM=H45GzO`J%>Z zl5!%nr6I?)IqY0cTns9>!*R8M?_|-NR+(DJ%X-15yHEBAN`4MLAJW7} zjr!0}ZEposzrN_WkZD<8sEDy#XDMA0yNa<7fOv!nxr}b6P^C$>W>txrT^Y>eOw%)? zmesgabU94m)B>KyC;YWiBpm!Pd!HulMM2v5wDqdsmHzt}p;PKuW7SZ_S(ZomMg@ZP zbmQK7h7XrzM<;Eaa9Yhg5L9TZKGs-6Yq1rsW4rY*^{N2whJ?TqApN9e?sc=1H>#Wj zVawA(yN}%_=WdD=p2)uCcgz3w&PUm|xk5`(Ob)`gR7+IRGyv}QfM3C~~@F1Sfm z;rkTJ)Ei;`2z@6QVV4x^bf1Y9n!!Zd-3y7=YQq_+zT*NVQmoy7Nld9%aeX6qZ!Nxu z4*Zyp`~G9+YZ3b}jk{fM+{JKlg&OflFrmC`nnWwH@3M;Yzj^*fJ#xm?c_iFp!7}}sFNQP=X+LU+J8Ngr)^eX!qMtF* zM~IJ4BzDR?E;2{xaotF~QdW z4VIRD-j|8^o3n1v*l-JWYMHG>^XE`Fl5@Qr>nKq!S4zRp@A-?B&8+9Vf>Nd0V4}i@ zM-7vCiHIDuY9=oQ+<&Z~n_>{JzpJHOZ)>7|dX?h-VE5?n{sb8gx*f+Slq)DTb9W~C z9s}qyOJ;<)ZrsEfmfWQ-Q)1Pg67Mx=f%FyUc#hvF;J{b6VbdXBh7#3Ys}o-s1`sD$ zFtxSGnv$GTY<%7YPZohcG;}u0wYTq;4JCs)#-ki5>P3d;W5&xnV*%Z{Bp{WcUhaxj zVl+lTvQ7U=BA~?^r<~JmF4q=Y-bI@O0 zN4Me1Q$9P2&$L#_n`n->t*UY_Skq@r8sQH$CZ6h7Ul4XzIz=`( zb6;}aJmqP{awbc>%S%S*6$M!PTx!{E7pn2S)Z#gxX^rF_;)NZuHfDIYZ4j}z=SApcFAem#9W|fNJ^&| zeFP~fkxCSvC#F%9Kbua=R!3W?q8{LJbZ!z6lQLJx({ z8JEwPMfzhd)$4_dK}5B64Ycx%R&+(7y)$q+Vckn8ZC!+<6kHNqB~?)_ z1vW&{(HTfeNlHzEzR7zs2kIU~qDw+-MW?h^0`dwfs-@GXH6kLeCEE2KA`0r%f$E48 zb&-+F9}D=z!O3az0ClL@5fQVH{ImXekjsHF0Df0FJpM_IJc9jNk34ifw5*U*2m9}^ zKdF;{9{5*s;UDln2JSjI{VEtOV&spR`E_&igS+xT zp+20Mzg({G`5wmkgZo>n^Mm~7QhHc-zZwHzf8YP$>-QVys0trt{WF;N2zzkP?C(GR z8T^Z?{LgHMb@{6uvd#P#Y=5oNN6f#j(?{G{Az}2G c-v{`y80kaFjvaE*9ej8WMEu~x{~7@Q2O^~=-v9sr literal 0 HcmV?d00001 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4b5967729dcd6..d692acf5979bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -22,6 +22,8 @@ import java.net.{MalformedURLException, URL} import java.sql.{Date, Timestamp} import java.util.concurrent.atomic.AtomicBoolean +import org.apache.commons.io.FileUtils + import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.expressions.GenericRow @@ -3751,6 +3753,39 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(sql("SELECT s LIKE 'm@@ca' ESCAPE '@' FROM df"), Row(true)) } } + + 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 => { + System.setProperty("ivy.home", dir.getAbsolutePath) + val sourceJar = new File(Thread.currentThread().getContextClassLoader + .getResource("SPARK-33084.jar").getFile) + val targetCacheJarDir = new File(dir.getAbsolutePath + + "/local/org.apache.spark/SPARK-33084/1.0/jars/") + targetCacheJarDir.mkdir() + // copy jar to local cache + FileUtils.copyFileToDirectory(sourceJar, targetCacheJarDir) + withTempView("v1") { + withUserDefinedFunction( + s"default.$functionName" -> false, + functionName -> true) { + // create temporary function without class + val e = intercept[AnalysisException] { + sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$sumFuncClass'") + }.getMessage + assert(e.contains("Can not load class 'org.apache.spark.examples.sql.Spark33084")) + sql("ADD JAR ivy://org.apache.spark:SPARK-33084:1.0") + sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$sumFuncClass'") + // create a view using a function in 'default' database + sql(s"CREATE TEMPORARY VIEW v1 AS SELECT $functionName(col1) FROM VALUES (1), (2), (3)") + // view v1 should still using function defined in `default` database + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(2.0))) + } + } + } + } + } } case class Foo(bar: Option[String]) From 2ffb4319bcd651f0d63d513f4f9c72e116b37318 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 22 Dec 2020 18:51:31 +0800 Subject: [PATCH 44/50] Update SQLQuerySuite.scala --- sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 2da64e2d06388..4eae3b901e46c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3822,6 +3822,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(sql("SELECT * FROM v1"), Seq(Row(2.0))) } } + System.clearProperty("ivy.home") } } } From 8c18cdf5ed942741909c414013a4774e2953cefe Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 23 Dec 2020 15:02:57 +0800 Subject: [PATCH 45/50] Update SparkContext.scala --- .../main/scala/org/apache/spark/SparkContext.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6b75d5960ed66..aae340953c5b2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1973,15 +1973,15 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { - val (keys, schema) = if (path.contains("\\") && Utils.isWindows) { + val (keys, scheme) = if (path.contains("\\") && Utils.isWindows) { // For local paths with backslashes on Windows, URI throws an exception (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) - val uriSchema = uri.getScheme - val jarPaths = uriSchema match { + val uriScheme = uri.getScheme + val jarPaths = uriScheme match { // A JAR file which exists only on the driver node case null => // SPARK-22585 path without schema is not url encoded @@ -1997,18 +1997,18 @@ class SparkContext(config: SparkConf) extends Logging { .flatMap(jar => addLocalJarFile(new File(jar))) case _ => checkRemoteJarFile(path) } - (jarPaths, uriSchema) + (jarPaths, uriScheme) } if (keys.nonEmpty) { val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis val (added, existed) = keys.partition(addedJars.putIfAbsent(_, timestamp).isEmpty) if (added.nonEmpty) { - val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of Ivy URI" + val jarMessage = if (scheme != "ivy") "JAR" else "dependency jars of Ivy URI" logInfo(s"Added $jarMessage $path at ${added.mkString(",")} with timestamp $timestamp") postEnvironmentUpdate() } if (existed.nonEmpty) { - val jarMessage = if (schema != "ivy") "JAR" else "dependency jars of Ivy URI" + val jarMessage = if (scheme != "ivy") "JAR" else "dependency jars of Ivy URI" logInfo(s"The $jarMessage $path at ${existed.mkString(",")} has been added already." + " Overwriting of added jar is not supported in the current version.") } From 6bd41cd85462a6bef0e8be61e9a61c2d628bc8fe Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 23 Dec 2020 15:05:15 +0800 Subject: [PATCH 46/50] Update SparkSubmit.scala --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e22a15ae7d787..178a25c2c794e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -304,7 +304,7 @@ 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( - true, args.packagesExclusions, args.packages, + packagesTransitive = true, args.packagesExclusions, args.packages, args.repositories, args.ivyRepoPath, args.ivySettingsPath) if (!StringUtils.isBlank(resolvedMavenCoordinates)) { From fbc236cc2b33593d3cbd61e57e354486f55e870a Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 23 Dec 2020 17:19:25 +0800 Subject: [PATCH 47/50] follwo comment --- .../org/apache/spark/SparkContextSuite.scala | 58 +----------------- .../apache/spark/util/DependencyUtils.scala | 60 +++++++++++++++++++ ...ql-ref-syntax-aux-resource-mgmt-add-jar.md | 19 +++--- .../spark/sql/internal/SessionState.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 3 +- 5 files changed, 72 insertions(+), 70 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/DependencyUtils.scala diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index c8a5a16c343ee..bfd95b0fedd5e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -445,29 +445,6 @@ 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 e = intercept[MalformedURLException] { - sc.addJar(badURL) - } - assert(e.getMessage.contains(badURL)) - assert(!sc.listJars().exists(_.contains(s"Test,UDTF_${scheme}.jar"))) - } - } - } - test("SPARK-22585 addJar argument without scheme is interpreted literally without url decoding") { withTempDir { dir => val tmpDir = new File(dir, "host%3A443") @@ -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") { 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"))) @@ -1173,39 +1150,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu 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 invalid ivy uri") { - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) - val e1 = intercept[IllegalArgumentException] { - sc.addJar("ivy://") - }.getMessage - assert(e1.contains("Expected authority at index 6: ivy://")) - - val e2 = intercept[IllegalArgumentException] { - sc.addJar("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] { - sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?foo=") - }.getMessage - assert(e3.contains("Invalid query string in Ivy URI" + - " ivy://org.apache.hive:hive-contrib:2.3.7?foo=:")) - - val e4 = intercept[IllegalArgumentException] { - sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo") - }.getMessage - assert(e4.contains("Invalid query string in Ivy URI" + - " ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo: bar=&baz=foo")) - - val e5 = intercept[IllegalArgumentException] { - sc.addJar("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho") - }.getMessage - assert(e5.contains("Invalid exclude string in Ivy URI" + - " ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:" + - " expected 'org:module,org:module,..', found org.pentaho")) - } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala new file mode 100644 index 0000000000000..9ff6b64038fcf --- /dev/null +++ b/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 + +object DependencyUtils 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=")) + }.getMessage + assert(e3.contains("Invalid query string in Ivy URI" + + " ivy://org.apache.hive:hive-contrib:2.3.7?foo=:")) + + val e4 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo")) + }.getMessage + assert(e4.contains("Invalid query string in Ivy URI" + + " ivy://org.apache.hive:hive-contrib:2.3.7?bar=&baz=foo: bar=&baz=foo")) + + val e5 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho")) + }.getMessage + assert(e5.contains("Invalid exclude string in Ivy URI" + + " ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:" + + " expected 'org:module,org:module,..', found org.pentaho")) + } +} diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 8e7f7afef50a4..6d31125fd612d 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -33,18 +33,17 @@ 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 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: + 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. - * exclude: exclusion list when download ivy URL jar and dependent jars. + * exclude: exclusion list during downloading Ivy URI jar and dependent jars. - User can write ivy URL such as: + User can write Ivy URI such as: ivy://group:module:version ivy://group:module:version?transitive=[true|false] - ivy://group:module:version?exclude=group:module,group:module - ivy://group:module:version?exclude=group:module,group:module&transitive=[true|false] + ivy://group:module:version?transitive=[true|false]&exclude=group:module,group:module ### Examples @@ -53,10 +52,10 @@ ADD JAR /tmp/test.jar; ADD JAR "/path/to/some.jar"; ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; -ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7"; -ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?transitive=false" -ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?transitive=true" -ADD JAR "ivy://org.apache.hive:hive-contrib:2.3.7?exclude=org.pentaho:pentaho-aggdesigner-algorithm&transitive=true" +ADD JAR "ivy://group:module:version"; +ADD JAR "ivy://group:module:version?transitive=false" +ADD JAR "ivy://group:module:version?transitive=true" +ADD JAR "ivy://group:module:version?exclude=group:module&transitive=true" ``` ### Related Statements diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 3cb542b3397e8..60ca06dbe0d52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -161,7 +161,7 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade } } - protected def resolveJars(path: URI): Seq[String] = { + def resolveJars(path: URI): Seq[String] = { path.getScheme match { case "ivy" => DependencyUtils.resolveMavenDependencies(path) case _ => path.toString :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4eae3b901e46c..0ba58e1634f06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3796,7 +3796,7 @@ 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 => { + withTempDir { dir => System.setProperty("ivy.home", dir.getAbsolutePath) val sourceJar = new File(Thread.currentThread().getContextClassLoader .getResource("SPARK-33084.jar").getFile) @@ -3824,7 +3824,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } System.clearProperty("ivy.home") } - } } } From 90491d5fd822658222d9d8156a10c6554c6e1427 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 23 Dec 2020 17:21:44 +0800 Subject: [PATCH 48/50] Update DependencyUtils.scala --- core/src/test/scala/org/apache/spark/util/DependencyUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala index 9ff6b64038fcf..d181d4d8ce669 100644 --- a/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/test/scala/org/apache/spark/util/DependencyUtils.scala @@ -21,7 +21,7 @@ import java.net.URI import org.apache.spark.SparkFunSuite -object DependencyUtils extends SparkFunSuite { +class DependencyUtilsSuite extends SparkFunSuite { test("SPARK-33084: Add jar support Ivy URI -- test invalid ivy uri") { val e1 = intercept[IllegalArgumentException] { From 75ff3ce59cd43a9baa6b6c128f7bd4ca8607c277 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 24 Dec 2020 04:57:59 +0800 Subject: [PATCH 49/50] Update SparkContextSuite.scala --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index bfd95b0fedd5e..770ffeef4106f 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1129,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 transitive value case sensitive") { + 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") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) From 4c44daecc7c77527e150d30051170f7ee8667f70 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 24 Dec 2020 09:52:55 +0800 Subject: [PATCH 50/50] follow comment remove default value --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../apache/spark/deploy/SparkSubmitUtilsSuite.scala | 12 +++++++++++- .../spark/sql/hive/client/IsolatedClientLoader.scala | 1 + 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 178a25c2c794e..ad95b18ecaeb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1368,7 +1368,7 @@ private[spark] object SparkSubmitUtils { def resolveMavenCoordinates( coordinates: String, ivySettings: IvySettings, - transitive: Boolean = true, + transitive: Boolean, exclusions: Seq[String] = Nil, isTest: Boolean = false): String = { if (coordinates == null || coordinates.trim.isEmpty) { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index d8b963c3b8072..eaa06ce2aa057 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -135,6 +135,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val jarPath = SparkSubmitUtils.resolveMavenCoordinates( main.toString, SparkSubmitUtils.buildIvySettings(Option(repo), Some(tempIvyPath)), + transitive = true, isTest = true) assert(jarPath.indexOf(tempIvyPath) >= 0, "should use non-default ivy path") } @@ -148,6 +149,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val jarPath = SparkSubmitUtils.resolveMavenCoordinates( main.toString, SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + transitive = true, isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -159,6 +161,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val jarPath = SparkSubmitUtils.resolveMavenCoordinates( main.toString, SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + transitive = true, isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -171,6 +174,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val jarPath = SparkSubmitUtils.resolveMavenCoordinates( main.toString, SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + transitive = true, isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path") @@ -183,6 +187,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { SparkSubmitUtils.resolveMavenCoordinates( "a:b:c", SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + transitive = true, isTest = true) } } @@ -195,6 +200,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val path = SparkSubmitUtils.resolveMavenCoordinates( coordinates, SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + transitive = true, isTest = true) assert(path === "", "should return empty path") val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.12", "1.2.0") @@ -202,6 +208,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val files = SparkSubmitUtils.resolveMavenCoordinates( coordinates + "," + main.toString, SparkSubmitUtils.buildIvySettings(Some(repo), Some(tempIvyPath)), + transitive = true, isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") } @@ -215,6 +222,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { main.toString, SparkSubmitUtils.buildIvySettings(Some(repo), Some(tempIvyPath)), exclusions = Seq("my.great.dep:mydep"), + transitive = true, isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") assert(files.indexOf("my.great.dep") < 0, "Returned excluded artifact") @@ -250,7 +258,8 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { testUtilSettings.setDefaultIvyUserDir(new File(tempIvyPath)) IvyTestUtils.withRepository(main, Some(dep), Some(dummyIvyLocal), useIvyLayout = true, ivySettings = testUtilSettings) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, settings, isTest = true) + val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, settings, + transitive = true, isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -265,6 +274,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val jarPath = SparkSubmitUtils.resolveMavenCoordinates( main.toString, ivySettings, + transitive = true, isTest = true) val r = """.*org.apache.spark-spark-submit-parent-.*""".r assert(!ivySettings.getDefaultCache.listFiles.map(_.getName) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index c0758dcdfc879..97e685efd27de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -124,6 +124,7 @@ private[hive] object IsolatedClientLoader extends Logging { SparkSubmitUtils.buildIvySettings( Some(remoteRepos), ivyPath), + transitive = true, exclusions = version.exclusions) } val allFiles = classpath.split(",").map(new File(_)).toSet