Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[SPARK-6568] spark-shell.cmd --jars option does not accept the jar that has space in its path #5447

Closed
wants to merge 13 commits into from
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ object PythonRunner {
s"spark-submit is currently only supported for local files: $path")
}
val windows = Utils.isWindows || testWindows
var formattedPath = if (windows) Utils.formatWindowsPath(path) else path
var formattedPath = Utils.formatPath(path, windows)
Copy link
Contributor

Choose a reason for hiding this comment

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

This is the only use of formatPath leftover and it doesn't feel necessary. Instead, how about:

val formattedPath = Try(new URI(formattedPath).getPath()).getOrElse(formattedPath)

Then you can remove Utils.formatPath. Maybe even Utils.windowsDrive.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's right. I'll try to remove them.


// Strip the URI scheme from the path
formattedPath =
Expand Down
14 changes: 9 additions & 5 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1659,9 +1659,14 @@ private[spark] object Utils extends Logging {
val windowsDrive = "([a-zA-Z])".r

/**
* Format a Windows path such that it can be safely passed to a URI.
* Format a path such that it can be safely passed to a URI.
*/
def formatWindowsPath(path: String): String = path.replace("\\", "/")
def formatPath(path: String, windows: Boolean): String = {
val formatted = path.replace(" ", "%20")
Copy link
Member

Choose a reason for hiding this comment

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

I think this is OK for now and solves the immediate problem. I wonder if we will need to do more complete URI escaping later -- although, it's not clear what URI scheme we assume anyway. LGTM.

Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of this, how about:

scala> new URI(null, "/a b c", null)
res6: java.net.URI = /a%20b%20c

Copy link
Contributor Author

Choose a reason for hiding this comment

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

When path contains #, it doesn't work.

Copy link
Member

Choose a reason for hiding this comment

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

new java.net.URI(null, "/a#b", null).toString
res1: String = /a%23b

... is technically correct, if this is what you mean. # denotes the start of the fragment portion of a URI, so when it occurs in the path, it must be escaped. The URIs we create here will not have a fragment. So I think @vanzin's change sounds more robust to similar cases like this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Do you mean we should assume path never contains fragment?
If we execute bin/spark-shell --jars hdfs:/path/to/jar1.jar, URI-style string is passed to path.
So I thought path may have fragment. (though I wonder if there are any good working example of the path with fragment...)

Copy link
Member

Choose a reason for hiding this comment

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

@steveloughran I think the issue is that we don't start with a File but with a String that may not be a file: URI. @tsudukim fragment is the bit after # in a URI, such as in file:/foo/bar#baz There's no fragment in your example and I would never expect to support such a thing. In fact I was a little surprised that this constructor handled the scheme in a scheme-specific part, but it seems to:

scala> val uri = new java.net.URI(null, "hdfs:/foo/b ar#baz", null)
uri: java.net.URI = hdfs:/foo/b%20ar%23baz

scala> uri.toString
res12: String = hdfs:/foo/b%20ar%23baz

scala> uri.getScheme
res13: String = hdfs

scala> uri.getPath
res14: String = /foo/b ar#baz

scala> uri.getFragment
res15: String = null

This still seems like the way to go as far as I can tell.


// In Windows, the file separator is a backslash, but this is inconsistent with the URI format
if (windows) formatted.replace("\\", "/") else formatted
}

/**
* Indicates whether Spark is currently running unit tests.
Expand Down Expand Up @@ -1762,9 +1767,8 @@ private[spark] object Utils extends Logging {
*/
def resolveURI(path: String, testWindows: Boolean = false): URI = {
Copy link
Member

Choose a reason for hiding this comment

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

testWindows is now unused?


// In Windows, the file separator is a backslash, but this is inconsistent with the URI format
val windows = isWindows || testWindows
val formattedPath = if (windows) formatWindowsPath(path) else path
val formattedPath = formatPath(path, windows)

val uri = new URI(formattedPath)
if (uri.getPath == null) {
Expand Down Expand Up @@ -1801,7 +1805,7 @@ private[spark] object Utils extends Logging {
Array.empty
} else {
paths.split(",").filter { p =>
val formattedPath = if (windows) formatWindowsPath(p) else p
val formattedPath = formatPath(p, windows)
val uri = new URI(formattedPath)
Copy link
Contributor

Choose a reason for hiding this comment

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

You should use resolveURI here; then you shouldn't need formatPath at all.

Option(uri.getScheme).getOrElse("file") match {
case windowsDrive(d) if windows => false
Copy link
Contributor

Choose a reason for hiding this comment

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

Not your fault, but I'm not sure this will ever match anything, since the match is on the URI's scheme. But probably doesn't hurt to leave here if you don't want to double-check that.

Expand Down
39 changes: 31 additions & 8 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import java.util.Locale
import com.google.common.base.Charsets.UTF_8
import com.google.common.io.Files
import org.scalatest.FunSuite
import org.apache.commons.lang3.SystemUtils

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -233,13 +234,16 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(new URI(Utils.resolveURIs(before, testWindows)) === new URI(after))
assert(new URI(Utils.resolveURIs(after, testWindows)) === new URI(after))
}
val cwd = System.getProperty("user.dir")
val rawCwd = System.getProperty("user.dir")
val cwd = if (SystemUtils.IS_OS_WINDOWS) s"/$rawCwd".replace("\\", "/") else rawCwd
assertResolves("hdfs:/root/spark.jar", "hdfs:/root/spark.jar")
assertResolves("hdfs:///root/spark.jar#app.jar", "hdfs:/root/spark.jar#app.jar")
assertResolves("spark.jar", s"file:$cwd/spark.jar")
assertResolves("spark.jar#app.jar", s"file:$cwd/spark.jar#app.jar")
assertResolves("path to/file.txt", s"file:$cwd/path%20to/file.txt")
assertResolves("C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true)
assertResolves("C:\\path\\to\\file.txt", "file:/C:/path/to/file.txt", testWindows = true)
assertResolves("C:/path to/file.txt", "file:/C:/path%20to/file.txt", testWindows = true)
assertResolves("file:/C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true)
assertResolves("file:///C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true)
assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true)
Expand All @@ -258,14 +262,16 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(resolve(resolve(after)) === after)
assert(resolve(resolve(resolve(after))) === after)
}
val cwd = System.getProperty("user.dir")
val rawCwd = System.getProperty("user.dir")
val cwd = if (SystemUtils.IS_OS_WINDOWS) s"/$rawCwd".replace("\\", "/") else rawCwd
assertResolves("jar1,jar2", s"file:$cwd/jar1,file:$cwd/jar2")
assertResolves("file:/jar1,file:/jar2", "file:/jar1,file:/jar2")
assertResolves("hdfs:/jar1,file:/jar2,jar3", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3")
assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5",
s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5")
assertResolves("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi",
s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi", testWindows = true)
assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5,path to/jar6",
s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5,file:$cwd/path%20to/jar6")
assertResolves("""hdfs:/jar1,file:/jar2,jar3,C:\pi.py#py.pi,C:\path to\jar4""",
s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi,file:/C:/path%20to/jar4",
testWindows = true)
}

test("nonLocalPaths") {
Expand All @@ -280,6 +286,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(Utils.nonLocalPaths("local:/spark.jar,file:/smart.jar,family.py") === Array.empty)
assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar") ===
Array("hdfs:/spark.jar", "s3:/smart.jar"))
assert(Utils.nonLocalPaths("hdfs:/path to/spark.jar,path to/a.jar,s3:/path to/smart.jar") ===
Array("hdfs:/path to/spark.jar", "s3:/path to/smart.jar"))
assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar,local.py,file:/hello/pi.py") ===
Array("hdfs:/spark.jar", "s3:/smart.jar"))
assert(Utils.nonLocalPaths("local.py,hdfs:/spark.jar,file:/hello/pi.py,s3:/smart.jar") ===
Expand All @@ -293,6 +301,11 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(Utils.nonLocalPaths("local:///C:/some/path.jar", testWindows = true) === Array.empty)
assert(Utils.nonLocalPaths("hdfs:/a.jar,C:/my.jar,s3:/another.jar", testWindows = true) ===
Array("hdfs:/a.jar", "s3:/another.jar"))
assert(Utils.nonLocalPaths(
"hdfs:/path to/spark.jar,C:\\path to\\a.jar,s3:/path to/smart.jar"
, testWindows = true
) ===
Array("hdfs:/path to/spark.jar", "s3:/path to/smart.jar"))
assert(Utils.nonLocalPaths("D:/your.jar,hdfs:/a.jar,s3:/another.jar", testWindows = true) ===
Array("hdfs:/a.jar", "s3:/another.jar"))
assert(Utils.nonLocalPaths("hdfs:/a.jar,s3:/another.jar,e:/our.jar", testWindows = true) ===
Expand Down Expand Up @@ -392,7 +405,12 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
val targetDir = new File(tempDir, "target-dir")
Files.write("some text", sourceFile, UTF_8)

val path = new Path("file://" + sourceDir.getAbsolutePath)
val path =
if (SystemUtils.IS_OS_WINDOWS) {
new Path("file:/" + sourceDir.getAbsolutePath.replace("\\", "/"))
} else {
new Path("file://" + sourceDir.getAbsolutePath)
}
val conf = new Configuration()
val fs = Utils.getHadoopFileSystem(path.toString, conf)

Expand All @@ -412,7 +430,12 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
val destInnerFile = new File(destInnerDir, sourceFile.getName)
assert(destInnerFile.isFile())

val filePath = new Path("file://" + sourceFile.getAbsolutePath)
val filePath =
if (SystemUtils.IS_OS_WINDOWS) {
new Path("file:/" + sourceFile.getAbsolutePath.replace("\\", "/"))
} else {
new Path("file://" + sourceFile.getAbsolutePath)
}
val testFileDir = new File(tempDir, "test-filename")
val testFileName = "testFName"
val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,8 @@ class SparkILoop(
// e.g. file:/C:/my/path.jar -> C:/my/path.jar
SparkILoop.getAddedJars.map { jar => new URI(jar).getPath.stripPrefix("/") }
} else {
SparkILoop.getAddedJars
// We need new URI(jar).getPath here for the case that `jar` includes encoded white space (%20).
SparkILoop.getAddedJars.map { jar => new URI(jar).getPath}
Copy link
Member

Choose a reason for hiding this comment

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

Does the scala-2.11 REPL need the same treatment?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

scala-2.11 REPL seems not have the equivelent code.

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: space before }

}
// work around for Scala bug
val totalClassPath = addedJars.foldLeft(
Expand Down Expand Up @@ -1109,7 +1110,7 @@ object SparkILoop extends Logging {
if (settings.classpath.isDefault)
settings.classpath.value = sys.props("java.class.path")

getAddedJars.foreach(settings.classpath.append(_))
getAddedJars.map(jar => new URI(jar).getPath).foreach(settings.classpath.append(_))

repl process settings
}
Expand Down