Repository: spark
Updated Branches:
  refs/heads/master 45f4c6612 -> 00e730b94


[SPARK-6300][Spark Core] sc.addFile(path) does not support the relative path.

when i run cmd like that sc.addFile("../test.txt"), it did not work and throwed 
an exception:
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path 
in absolute URI: file:../test.txt
at org.apache.hadoop.fs.Path.initialize(Path.java:206)
at org.apache.hadoop.fs.Path.<init>(Path.java:172)
........
.......
Caused by: java.net.URISyntaxException: Relative path in absolute URI: 
file:../test.txt
at java.net.URI.checkPath(URI.java:1804)
at java.net.URI.<init>(URI.java:752)
at org.apache.hadoop.fs.Path.initialize(Path.java:203)

Author: DoingDone9 <799203...@qq.com>

Closes #4993 from DoingDone9/relativePath and squashes the following commits:

ee375cd [DoingDone9] Update SparkContextSuite.scala
d594e16 [DoingDone9] Update SparkContext.scala
0ff3fa8 [DoingDone9] test for add file
dced8eb [DoingDone9] Update SparkContext.scala
e4a13fe [DoingDone9] getCanonicalPath
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00e730b9
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00e730b9
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00e730b9

Branch: refs/heads/master
Commit: 00e730b94cba1202a73af1e2476ff5a44af4b6b2
Parents: 45f4c66
Author: DoingDone9 <799203...@qq.com>
Authored: Mon Mar 16 12:27:15 2015 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Mon Mar 16 12:27:15 2015 +0000

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala   |  2 +-
 .../org/apache/spark/SparkContextSuite.scala    | 51 ++++++++++++++------
 2 files changed, 38 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/00e730b9/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 8121aab..4457f40 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1093,7 +1093,7 @@ class SparkContext(config: SparkConf) extends Logging 
with ExecutorAllocationCli
   def addFile(path: String, recursive: Boolean): Unit = {
     val uri = new URI(path)
     val schemeCorrectedPath = uri.getScheme match {
-      case null | "local" => "file:" + uri.getPath
+      case null | "local" => new File(path).getCanonicalFile.toURI.toString
       case _              => path
     }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/00e730b9/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala 
b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 50f347f..b8e3e83 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -79,26 +79,49 @@ class SparkContextSuite extends FunSuite with 
LocalSparkContext {
     val byteArray2 = converter.convert(bytesWritable)
     assert(byteArray2.length === 0)
   }
-
+  
   test("addFile works") {
-    val file = File.createTempFile("someprefix", "somesuffix")
-    val absolutePath = file.getAbsolutePath
+    val file1 = File.createTempFile("someprefix1", "somesuffix1")
+    val absolutePath1 = file1.getAbsolutePath
+
+    val pluto = Utils.createTempDir()
+    val file2 = File.createTempFile("someprefix2", "somesuffix2", pluto)
+    val relativePath = file2.getParent + "/../" + file2.getParentFile.getName 
+ "/" + file2.getName
+    val absolutePath2 = file2.getAbsolutePath
+
     try {
-      Files.write("somewords", file, UTF_8)
-      val length = file.length()
+      Files.write("somewords1", file1, UTF_8)
+      Files.write("somewords2", file2, UTF_8)
+      val length1 = file1.length()
+      val length2 = file2.length()
+
       sc = new SparkContext(new 
SparkConf().setAppName("test").setMaster("local"))
-      sc.addFile(file.getAbsolutePath)
+      sc.addFile(file1.getAbsolutePath)
+      sc.addFile(relativePath)
       sc.parallelize(Array(1), 1).map(x => {
-        val gotten = new File(SparkFiles.get(file.getName))
-        if (!gotten.exists()) {
-          throw new SparkException("file doesn't exist")
+        val gotten1 = new File(SparkFiles.get(file1.getName))
+        val gotten2 = new File(SparkFiles.get(file2.getName))
+        if (!gotten1.exists()) {
+          throw new SparkException("file doesn't exist : " + absolutePath1)
+        }
+        if (!gotten2.exists()) {
+          throw new SparkException("file doesn't exist : " + absolutePath2)
         }
-        if (length != gotten.length()) {
+
+        if (length1 != gotten1.length()) {
+          throw new SparkException(
+            s"file has different length $length1 than added file 
${gotten1.length()} : " + absolutePath1)
+        }
+        if (length2 != gotten2.length()) {
           throw new SparkException(
-            s"file has different length $length than added file 
${gotten.length()}")
+            s"file has different length $length2 than added file 
${gotten2.length()} : " + absolutePath2)
         }
-        if (absolutePath == gotten.getAbsolutePath) {
-          throw new SparkException("file should have been copied")
+
+        if (absolutePath1 == gotten1.getAbsolutePath) {
+          throw new SparkException("file should have been copied :" + 
absolutePath1)
+        }
+        if (absolutePath2 == gotten2.getAbsolutePath) {
+          throw new SparkException("file should have been copied : " + 
absolutePath2)
         }
         x
       }).count()
@@ -106,7 +129,7 @@ class SparkContextSuite extends FunSuite with 
LocalSparkContext {
       sc.stop()
     }
   }
-
+  
   test("addFile recursive works") {
     val pluto = Utils.createTempDir()
     val neptune = Utils.createTempDir(pluto.getAbsolutePath)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to