You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2021/04/07 18:43:49 UTC
[spark] branch master updated: [SPARK-34955][SQL] ADD JAR command
cannot add jar files which contains whitespaces in the path
This is an automated email from the ASF dual-hosted git repository.
dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new e5d972e [SPARK-34955][SQL] ADD JAR command cannot add jar files which contains whitespaces in the path
e5d972e is described below
commit e5d972e84e973d9a2e62312dc471df30c35269bc
Author: Kousuke Saruta <sa...@oss.nttdata.com>
AuthorDate: Wed Apr 7 11:43:03 2021 -0700
[SPARK-34955][SQL] ADD JAR command cannot add jar files which contains whitespaces in the path
### What changes were proposed in this pull request?
This PR fixes an issue that `ADD JAR` command can't add jar files which contain whitespaces in the path though `ADD FILE` and `ADD ARCHIVE` work with such files.
If we have `/some/path/test file.jar` and execute the following command:
```
ADD JAR "/some/path/test file.jar";
```
The following exception is thrown.
```
21/04/05 10:40:38 ERROR SparkSQLDriver: Failed in [add jar "/some/path/test file.jar"]
java.lang.IllegalArgumentException: Illegal character in path at index 9: /some/path/test file.jar
at java.net.URI.create(URI.java:852)
at org.apache.spark.sql.hive.HiveSessionResourceLoader.addJar(HiveSessionStateBuilder.scala:129)
at org.apache.spark.sql.execution.command.AddJarCommand.run(resources.scala:34)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
```
This is because `HiveSessionStateBuilder` and `SessionStateBuilder` don't check whether the form of the path is URI or plain path and it always regards the path as URI form.
Whitespces should be encoded to `%20` so `/some/path/test file.jar` is rejected.
We can resolve this part by checking whether the given path is URI form or not.
Unfortunatelly, if we fix this part, another problem occurs.
When we execute `ADD JAR` command, Hive's `ADD JAR` command is executed in `HiveClientImpl.addJar` and `AddResourceProcessor.run` is transitively invoked.
In `AddResourceProcessor.run`, the command line is just split by `
s+` and the path is also split into `/some/path/test` and `file.jar` and passed to `ss.add_resources`.
https://github.com/apache/hive/blob/f1e87137034e4ecbe39a859d4ef44319800016d7/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java#L56-L75
So, the command still fails.
Even if we convert the form of the path to URI like `file:/some/path/test%20file.jar` and execute the following command:
```
ADD JAR "file:/some/path/test%20file";
```
The following exception is thrown.
```
21/04/05 10:40:53 ERROR SessionState: file:/some/path/test%20file.jar does not exist
java.lang.IllegalArgumentException: file:/some/path/test%20file.jar does not exist
at org.apache.hadoop.hive.ql.session.SessionState.validateFiles(SessionState.java:1168)
at org.apache.hadoop.hive.ql.session.SessionState$ResourceType.preHook(SessionState.java:1289)
at org.apache.hadoop.hive.ql.session.SessionState$ResourceType$1.preHook(SessionState.java:1278)
at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1378)
at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1336)
at org.apache.hadoop.hive.ql.processors.AddResourceProcessor.run(AddResourceProcessor.java:74)
```
The reason is `Utilities.realFile` invoked in `SessionState.validateFiles` returns `null` as the result of `fs.exists(path)` is `false`.
https://github.com/apache/hive/blob/f1e87137034e4ecbe39a859d4ef44319800016d7/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java#L1052-L1064
`fs.exists` checks the existence of the given path by comparing the string representation of Hadoop's `Path`.
The string representation of `Path` is similar to URI but it's actually different.
`Path` doesn't encode the given path.
For example, the URI form of `/some/path/jar file.jar` is `file:/some/path/jar%20file.jar` but the `Path` form of it is `file:/some/path/jar file.jar`. So `fs.exists` returns false.
So the solution I come up with is removing Hive's `ADD JAR` from `HiveClientimpl.addJar`.
I think Hive's `ADD JAR` was used to add jar files to the class loader for metadata and isolate the class loader from the one for execution.
https://github.com/apache/spark/pull/6758/files#diff-cdb07de713c84779a5308f65be47964af865e15f00eb9897ccf8a74908d581bbR94-R103
But, as of SPARK-10810 and SPARK-10902 (#8909) are resolved, the class loaders for metadata and execution seem to be isolated with different way.
https://github.com/apache/spark/pull/8909/files#diff-8ef7cabf145d3fe7081da799fa415189d9708892ed76d4d13dd20fa27021d149R635-R641
In the current implementation, such class loaders seem to be isolated by `SharedState.jarClassLoader` and `IsolatedClientLoader.classLoader`.
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala#L173-L188
https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L956-L967
So I wonder we can remove Hive's `ADD JAR` from `HiveClientImpl.addJar`.
### Why are the changes needed?
This is a bug.
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes #32052 from sarutak/add-jar-whitespace.
Authored-by: Kousuke Saruta <sa...@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
.../scala/org/apache/spark/sql/internal/SessionState.scala | 4 ++--
.../apache/spark/sql/hive/HiveSessionStateBuilder.scala | 5 ++---
.../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 14 +++-----------
.../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 12 ++++++++++++
4 files changed, 19 insertions(+), 16 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 f4741fc..319b226 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
+import org.apache.spark.util.{DependencyUtils, Utils}
/**
* A class that holds all session-specific state in a given [[SparkSession]].
@@ -172,7 +172,7 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade
* [[SessionState]].
*/
def addJar(path: String): Unit = {
- val uri = URI.create(path)
+ val uri = Utils.resolveURI(path)
resolveJars(uri).foreach { p =>
session.sparkContext.addJar(p)
val uri = new Path(p).toUri
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 7b721e0..7bf9b28 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,8 +17,6 @@
package org.apache.spark.sql.hive
-import java.net.URI
-
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog}
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener
@@ -34,6 +32,7 @@ import org.apache.spark.sql.execution.streaming.ResolveWriteToStream
import org.apache.spark.sql.hive.client.HiveClient
import org.apache.spark.sql.hive.execution.PruneHiveTablePartitions
import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState}
+import org.apache.spark.util.Utils
/**
* Builder that produces a Hive-aware `SessionState`.
@@ -127,7 +126,7 @@ class HiveSessionResourceLoader(
extends SessionResourceLoader(session) {
private lazy val client = clientBuilder()
override def addJar(path: String): Unit = {
- val uri = URI.create(path)
+ val uri = Utils.resolveURI(path)
resolveJars(uri).foreach { p =>
client.addJar(p)
super.addJar(p)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 3658e38..11ee0ff 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.hive.client
-import java.io.{File, PrintStream}
+import java.io.PrintStream
import java.lang.{Iterable => JIterable}
import java.lang.reflect.InvocationTargetException
import java.nio.charset.StandardCharsets.UTF_8
@@ -954,16 +954,8 @@ private[hive] class HiveClientImpl(
}
def addJar(path: String): Unit = {
- 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
- }
- clientLoader.addJar(jarURL)
- runSqlHive(s"ADD JAR $path")
+ val jarURI = Utils.resolveURI(path)
+ clientLoader.addJar(jarURI.toURL)
}
def newSession(): HiveClientImpl = {
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 9968e8d..511992d 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
@@ -830,6 +830,18 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
assert(sql(s"list jar $testJar").count() == 1)
}
+ test("SPARK-34955: ADD JAR should treat paths which contains white spaces") {
+ withTempDir { dir =>
+ val file = File.createTempFile("someprefix1", "somesuffix1", dir)
+ Files.write(file.toPath, "test_file1".getBytes)
+ val jarFile = new File(dir, "test file.jar")
+ TestUtils.createJar(Seq(file), jarFile)
+ sql(s"ADD JAR ${jarFile.getAbsolutePath}")
+ assert(sql("LIST JARS").
+ filter(_.getString(0).contains(s"${jarFile.getName}".replace(" ", "%20"))).count() > 0)
+ }
+ }
+
test("CREATE TEMPORARY FUNCTION") {
val funcJar = TestHive.getHiveFile("TestUDTF.jar")
val jarURL = funcJar.toURI.toURL
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org