Skip to content

Commit bc69d14

Browse files
author
Mridul Muralidharan
committed
Change from hardcoded path separator
1 parent 10c4d78 commit bc69d14

File tree

6 files changed

+34
-17
lines changed

6 files changed

+34
-17
lines changed

core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -87,10 +87,10 @@ private[spark] class PipedRDD[T: ClassTag](
8787
// When spark.worker.separated.working.directory option is turned on, each
8888
// task will be run in separate directory. This should be resolve file
8989
// access conflict issue
90-
val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString
90+
val taskDirectory = "tasks" + File.separator + java.util.UUID.randomUUID.toString
9191
var workInTaskDirectory = false
9292
logDebug("taskDirectory = " + taskDirectory)
93-
if (separateWorkingDir == true) {
93+
if (separateWorkingDir) {
9494
val currentDir = new File(".")
9595
logDebug("currentDir = " + currentDir.getAbsolutePath())
9696
val taskDirFile = new File(taskDirectory)
@@ -106,13 +106,13 @@ private[spark] class PipedRDD[T: ClassTag](
106106
for (file <- currentDir.list(tasksDirFilter)) {
107107
val fileWithDir = new File(currentDir, file)
108108
Utils.symlink(new File(fileWithDir.getAbsolutePath()),
109-
new File(taskDirectory + "/" + fileWithDir.getName()))
109+
new File(taskDirectory + File.separator + fileWithDir.getName()))
110110
}
111111
pb.directory(taskDirFile)
112112
workInTaskDirectory = true
113113
} catch {
114114
case e: Exception => logError("Unable to setup task working directory: " + e.getMessage +
115-
" (" + taskDirectory + ")")
115+
" (" + taskDirectory + ")", e)
116116
}
117117
}
118118

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 16 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import scala.io.Source
3030
import scala.reflect.ClassTag
3131

3232
import com.google.common.io.Files
33+
import org.apache.commons.lang.SystemUtils
3334
import com.google.common.util.concurrent.ThreadFactoryBuilder
3435
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
3536
import org.json4s._
@@ -45,7 +46,10 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream,
4546
*/
4647
private[spark] object Utils extends Logging {
4748

48-
val osName = System.getProperty("os.name")
49+
def sparkBin(sparkHome: String, which: String): File = {
50+
val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else ""
51+
new File(sparkHome + File.separator + "bin", which + suffix)
52+
}
4953

5054
/** Serialize an object using Java serialization */
5155
def serialize[T](o: T): Array[Byte] = {
@@ -603,7 +607,7 @@ private[spark] object Utils extends Logging {
603607
*/
604608
def isSymlink(file: File): Boolean = {
605609
if (file == null) throw new NullPointerException("File must not be null")
606-
if (osName.startsWith("Windows")) return false
610+
if (SystemUtils.IS_OS_WINDOWS) return false
607611
val fileInCanonicalDir = if (file.getParent() == null) {
608612
file
609613
} else {
@@ -1007,9 +1011,17 @@ private[spark] object Utils extends Logging {
10071011
if (dst.isAbsolute()) {
10081012
throw new IOException("Destination must be relative")
10091013
}
1010-
val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf"
1014+
var cmdSuffix = ""
1015+
val linkCmd = if (SystemUtils.IS_OS_WINDOWS) {
1016+
// refer to http://technet.microsoft.com/en-us/library/cc771254.aspx
1017+
cmdSuffix = " /s /e /k /h /y /i"
1018+
"cmd /c xcopy "
1019+
} else {
1020+
cmdSuffix = ""
1021+
"ln -sf "
1022+
}
10111023
import scala.sys.process._
1012-
(linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line =>
1024+
(linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! ProcessLogger(line =>
10131025
(logInfo(line)))
10141026
}
10151027

repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer
2525
import com.google.common.io.Files
2626
import org.scalatest.FunSuite
2727
import org.apache.spark.SparkContext
28+
import org.apache.commons.lang.StringEscapeUtils
2829

2930

3031
class ReplSuite extends FunSuite {
@@ -185,11 +186,12 @@ class ReplSuite extends FunSuite {
185186
out.close()
186187
val output = runInterpreter("local",
187188
"""
188-
|var file = sc.textFile("%s/input").cache()
189+
|var file = sc.textFile("%s").cache()
189190
|file.count()
190191
|file.count()
191192
|file.count()
192-
""".stripMargin.format(tempDir.getAbsolutePath))
193+
""".stripMargin.format(StringEscapeUtils.escapeJava(
194+
tempDir.getAbsolutePath + File.separator + "input")))
193195
assertDoesNotContain("error:", output)
194196
assertDoesNotContain("Exception", output)
195197
assertContains("res0: Long = 3", output)

sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -100,14 +100,15 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) {
100100
hiveFilesTemp.delete()
101101
hiveFilesTemp.mkdir()
102102

103-
val inRepoTests = if (System.getProperty("user.dir").endsWith("sql/hive")) {
104-
new File("src/test/resources/")
103+
val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) {
104+
new File("src" + File.separator + "test" + File.separator + "resources" + File.separator)
105105
} else {
106-
new File("sql/hive/src/test/resources")
106+
new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" +
107+
File.separator + "resources")
107108
}
108109

109110
def getHiveFile(path: String): File = {
110-
val stripped = path.replaceAll("""\.\.\/""", "")
111+
val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar)
111112
hiveDevHome
112113
.map(new File(_, stripped))
113114
.filter(_.exists)

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ import org.apache.spark.sql.hive.test.TestHive._
2626
* https://amplab.cs.berkeley.edu/benchmark/
2727
*/
2828
class BigDataBenchmarkSuite extends HiveComparisonTest {
29-
val testDataDirectory = new File("target/big-data-benchmark-testdata")
29+
val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata")
3030

3131
val testTables = Seq(
3232
TestTable(

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,14 +19,16 @@ package org.apache.spark.sql.hive.execution
1919

2020
import org.scalatest.BeforeAndAfter
2121

22-
import org.apache.spark.sql.hive.test.TestHive
22+
import org.apache.spark.sql.hive.TestHive
23+
import java.io.File
2324

2425
/**
2526
* Runs the test cases that are included in the hive distribution.
2627
*/
2728
class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
2829
// TODO: bundle in jar files... get from classpath
29-
lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive")
30+
lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" +
31+
File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive")
3032
def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f)
3133

3234
override def beforeAll() {

0 commit comments

Comments
 (0)