aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/util/Utils.scala
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache/spark/util/Utils.scala')
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala25
1 files changed, 13 insertions, 12 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 3f7858d2de..5f1253100b 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -36,14 +36,13 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
import org.apache.spark.deploy.SparkHadoopUtil
import java.nio.ByteBuffer
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging}
/**
* Various utility methods used by Spark.
*/
private[spark] object Utils extends Logging {
-
/** Serialize an object using Java serialization */
def serialize[T](o: T): Array[Byte] = {
val bos = new ByteArrayOutputStream()
@@ -239,9 +238,9 @@ private[spark] object Utils extends Logging {
* Throws SparkException if the target file already exists and has different contents than
* the requested file.
*/
- def fetchFile(url: String, targetDir: File) {
+ def fetchFile(url: String, targetDir: File, conf: SparkConf) {
val filename = url.split("/").last
- val tempDir = getLocalDir
+ val tempDir = getLocalDir(conf)
val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir))
val targetFile = new File(targetDir, filename)
val uri = new URI(url)
@@ -311,8 +310,8 @@ private[spark] object Utils extends Logging {
* return a single directory, even though the spark.local.dir property might be a list of
* multiple paths.
*/
- def getLocalDir: String = {
- System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0)
+ def getLocalDir(conf: SparkConf): String = {
+ conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0)
}
/**
@@ -397,13 +396,12 @@ private[spark] object Utils extends Logging {
InetAddress.getByName(address).getHostName
}
- def localHostPort(): String = {
- val retval = System.getProperty("spark.hostPort", null)
+ def localHostPort(conf: SparkConf): String = {
+ val retval = conf.get("spark.hostPort", null)
if (retval == null) {
logErrorWithStack("spark.hostPort not set but invoking localHostPort")
return localHostName()
}
-
retval
}
@@ -415,9 +413,12 @@ private[spark] object Utils extends Logging {
assert(hostPort.indexOf(':') != -1, message)
}
- // Used by DEBUG code : remove when all testing done
def logErrorWithStack(msg: String) {
- try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+ try {
+ throw new Exception
+ } catch {
+ case ex: Exception => logError(msg, ex)
+ }
}
// Typically, this will be of order of number of nodes in cluster
@@ -837,7 +838,7 @@ private[spark] object Utils extends Logging {
}
}
- /**
+ /**
* Timing method based on iterations that permit JVM JIT optimization.
* @param numIters number of iterations
* @param f function to be executed