2010-03-29 19:17:55 -04:00
|
|
|
package spark
|
|
|
|
|
|
|
|
import java.io._
|
2012-12-08 03:33:11 -05:00
|
|
|
import java.net.{NetworkInterface, InetAddress, URL, URI}
|
2012-09-26 00:46:58 -04:00
|
|
|
import java.util.{Locale, Random, UUID}
|
2011-04-28 01:13:01 -04:00
|
|
|
import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
|
2012-08-30 14:01:43 -04:00
|
|
|
import org.apache.hadoop.conf.Configuration
|
2012-09-04 21:52:07 -04:00
|
|
|
import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
|
2010-10-04 15:01:05 -04:00
|
|
|
import scala.collection.mutable.ArrayBuffer
|
2012-12-08 03:33:11 -05:00
|
|
|
import scala.collection.JavaConversions._
|
2012-07-01 20:13:31 -04:00
|
|
|
import scala.io.Source
|
2012-12-28 19:13:23 -05:00
|
|
|
import com.google.common.io.Files
|
2010-10-04 15:01:05 -04:00
|
|
|
|
2010-10-16 19:14:13 -04:00
|
|
|
/**
|
|
|
|
* Various utility methods used by Spark.
|
|
|
|
*/
|
2012-09-28 20:42:00 -04:00
|
|
|
private object Utils extends Logging {
|
2012-07-06 18:23:26 -04:00
|
|
|
/** Serialize an object using Java serialization */
|
2012-04-10 16:29:46 -04:00
|
|
|
def serialize[T](o: T): Array[Byte] = {
|
|
|
|
val bos = new ByteArrayOutputStream()
|
|
|
|
val oos = new ObjectOutputStream(bos)
|
|
|
|
oos.writeObject(o)
|
2012-06-29 21:47:12 -04:00
|
|
|
oos.close()
|
2012-04-10 16:29:46 -04:00
|
|
|
return bos.toByteArray
|
|
|
|
}
|
2012-04-10 00:59:56 -04:00
|
|
|
|
2012-07-06 18:23:26 -04:00
|
|
|
/** Deserialize an object using Java serialization */
|
2012-04-10 16:29:46 -04:00
|
|
|
def deserialize[T](bytes: Array[Byte]): T = {
|
|
|
|
val bis = new ByteArrayInputStream(bytes)
|
|
|
|
val ois = new ObjectInputStream(bis)
|
|
|
|
return ois.readObject.asInstanceOf[T]
|
|
|
|
}
|
2010-03-29 19:17:55 -04:00
|
|
|
|
2012-07-06 18:23:26 -04:00
|
|
|
/** Deserialize an object using Java serialization and the given ClassLoader */
|
2010-03-29 19:17:55 -04:00
|
|
|
def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = {
|
2012-04-10 16:29:46 -04:00
|
|
|
val bis = new ByteArrayInputStream(bytes)
|
|
|
|
val ois = new ObjectInputStream(bis) {
|
|
|
|
override def resolveClass(desc: ObjectStreamClass) =
|
|
|
|
Class.forName(desc.getName, false, loader)
|
|
|
|
}
|
|
|
|
return ois.readObject.asInstanceOf[T]
|
2010-03-29 19:17:55 -04:00
|
|
|
}
|
2010-10-04 15:01:05 -04:00
|
|
|
|
2012-02-09 18:50:26 -05:00
|
|
|
def isAlpha(c: Char): Boolean = {
|
2010-10-04 15:01:05 -04:00
|
|
|
(c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z')
|
|
|
|
}
|
|
|
|
|
2012-07-01 20:13:31 -04:00
|
|
|
/** Split a string into words at non-alphabetic characters */
|
2010-10-04 15:01:05 -04:00
|
|
|
def splitWords(s: String): Seq[String] = {
|
|
|
|
val buf = new ArrayBuffer[String]
|
|
|
|
var i = 0
|
|
|
|
while (i < s.length) {
|
|
|
|
var j = i
|
|
|
|
while (j < s.length && isAlpha(s.charAt(j))) {
|
|
|
|
j += 1
|
|
|
|
}
|
|
|
|
if (j > i) {
|
2012-06-29 21:47:12 -04:00
|
|
|
buf += s.substring(i, j)
|
2010-10-04 15:01:05 -04:00
|
|
|
}
|
|
|
|
i = j
|
|
|
|
while (i < s.length && !isAlpha(s.charAt(i))) {
|
|
|
|
i += 1
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return buf
|
|
|
|
}
|
2010-10-16 19:14:13 -04:00
|
|
|
|
2012-07-01 20:13:31 -04:00
|
|
|
/** Create a temporary directory inside the given parent directory */
|
2012-02-10 11:19:53 -05:00
|
|
|
def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = {
|
2010-10-16 19:14:13 -04:00
|
|
|
var attempts = 0
|
|
|
|
val maxAttempts = 10
|
|
|
|
var dir: File = null
|
|
|
|
while (dir == null) {
|
|
|
|
attempts += 1
|
|
|
|
if (attempts > maxAttempts) {
|
2012-10-07 03:54:38 -04:00
|
|
|
throw new IOException("Failed to create a temp directory after " + maxAttempts +
|
2012-02-10 11:19:53 -05:00
|
|
|
" attempts!")
|
2010-10-16 19:14:13 -04:00
|
|
|
}
|
|
|
|
try {
|
|
|
|
dir = new File(root, "spark-" + UUID.randomUUID.toString)
|
|
|
|
if (dir.exists() || !dir.mkdirs()) {
|
|
|
|
dir = null
|
|
|
|
}
|
|
|
|
} catch { case e: IOException => ; }
|
|
|
|
}
|
2012-06-09 18:58:07 -04:00
|
|
|
// Add a shutdown hook to delete the temp dir when the JVM exits
|
|
|
|
Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) {
|
|
|
|
override def run() {
|
|
|
|
Utils.deleteRecursively(dir)
|
|
|
|
}
|
|
|
|
})
|
2010-10-16 19:14:13 -04:00
|
|
|
return dir
|
|
|
|
}
|
|
|
|
|
2012-07-01 20:13:31 -04:00
|
|
|
/** Copy all data from an InputStream to an OutputStream */
|
2010-10-16 19:14:13 -04:00
|
|
|
def copyStream(in: InputStream,
|
|
|
|
out: OutputStream,
|
|
|
|
closeStreams: Boolean = false)
|
|
|
|
{
|
|
|
|
val buf = new Array[Byte](8192)
|
|
|
|
var n = 0
|
|
|
|
while (n != -1) {
|
|
|
|
n = in.read(buf)
|
|
|
|
if (n != -1) {
|
|
|
|
out.write(buf, 0, n)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (closeStreams) {
|
|
|
|
in.close()
|
|
|
|
out.close()
|
|
|
|
}
|
|
|
|
}
|
2010-11-04 01:45:44 -04:00
|
|
|
|
2012-07-06 18:23:26 -04:00
|
|
|
/** Copy a file on the local file system */
|
|
|
|
def copyFile(source: File, dest: File) {
|
|
|
|
val in = new FileInputStream(source)
|
|
|
|
val out = new FileOutputStream(dest)
|
|
|
|
copyStream(in, out, true)
|
|
|
|
}
|
2012-09-28 20:42:00 -04:00
|
|
|
|
|
|
|
/** Download a file from a given URL to the local filesystem */
|
2012-08-30 14:01:43 -04:00
|
|
|
def downloadFile(url: URL, localPath: String) {
|
|
|
|
val in = url.openStream()
|
|
|
|
val out = new FileOutputStream(localPath)
|
|
|
|
Utils.copyStream(in, out, true)
|
|
|
|
}
|
2012-10-07 03:54:38 -04:00
|
|
|
|
2012-08-30 14:01:43 -04:00
|
|
|
/**
|
|
|
|
* Download a file requested by the executor. Supports fetching the file in a variety of ways,
|
|
|
|
* including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
|
2012-12-28 20:37:13 -05:00
|
|
|
*
|
|
|
|
* Throws SparkException if the target file already exists and has different contents than
|
|
|
|
* the requested file.
|
2012-08-30 14:01:43 -04:00
|
|
|
*/
|
|
|
|
def fetchFile(url: String, targetDir: File) {
|
|
|
|
val filename = url.split("/").last
|
2012-12-28 19:13:23 -05:00
|
|
|
val tempDir = System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))
|
|
|
|
val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir))
|
2012-08-30 14:01:43 -04:00
|
|
|
val targetFile = new File(targetDir, filename)
|
2012-09-04 21:52:07 -04:00
|
|
|
val uri = new URI(url)
|
|
|
|
uri.getScheme match {
|
|
|
|
case "http" | "https" | "ftp" =>
|
2012-12-28 19:13:23 -05:00
|
|
|
logInfo("Fetching " + url + " to " + tempFile)
|
2012-09-04 21:52:07 -04:00
|
|
|
val in = new URL(url).openStream()
|
2012-12-28 19:13:23 -05:00
|
|
|
val out = new FileOutputStream(tempFile)
|
2012-09-04 21:52:07 -04:00
|
|
|
Utils.copyStream(in, out, true)
|
2012-12-28 19:13:23 -05:00
|
|
|
if (targetFile.exists && !Files.equal(tempFile, targetFile)) {
|
|
|
|
tempFile.delete()
|
2012-12-28 20:37:13 -05:00
|
|
|
throw new SparkException("File " + targetFile + " exists and does not match contents of" +
|
|
|
|
" " + url)
|
2012-12-28 19:13:23 -05:00
|
|
|
} else {
|
|
|
|
Files.move(tempFile, targetFile)
|
|
|
|
}
|
2012-09-04 21:52:07 -04:00
|
|
|
case "file" | null =>
|
2012-12-28 19:13:23 -05:00
|
|
|
val sourceFile = if (uri.isAbsolute) {
|
|
|
|
new File(uri)
|
|
|
|
} else {
|
|
|
|
new File(url)
|
|
|
|
}
|
|
|
|
if (targetFile.exists && !Files.equal(sourceFile, targetFile)) {
|
2012-12-28 20:37:13 -05:00
|
|
|
throw new SparkException("File " + targetFile + " exists and does not match contents of" +
|
|
|
|
" " + url)
|
2012-10-07 03:54:38 -04:00
|
|
|
} else {
|
2012-12-28 19:13:23 -05:00
|
|
|
// Remove the file if it already exists
|
|
|
|
targetFile.delete()
|
|
|
|
// Symlink the file locally.
|
|
|
|
if (uri.isAbsolute) {
|
|
|
|
// url is absolute, i.e. it starts with "file:///". Extract the source
|
|
|
|
// file's absolute path from the url.
|
|
|
|
val sourceFile = new File(uri)
|
|
|
|
logInfo("Symlinking " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath)
|
|
|
|
FileUtil.symLink(sourceFile.getAbsolutePath, targetFile.getAbsolutePath)
|
|
|
|
} else {
|
|
|
|
// url is not absolute, i.e. itself is the path to the source file.
|
|
|
|
logInfo("Symlinking " + url + " to " + targetFile.getAbsolutePath)
|
|
|
|
FileUtil.symLink(url, targetFile.getAbsolutePath)
|
|
|
|
}
|
2012-10-07 03:54:38 -04:00
|
|
|
}
|
2012-09-04 21:52:07 -04:00
|
|
|
case _ =>
|
|
|
|
// Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
|
|
|
|
val uri = new URI(url)
|
|
|
|
val conf = new Configuration()
|
|
|
|
val fs = FileSystem.get(uri, conf)
|
|
|
|
val in = fs.open(new Path(uri))
|
2012-12-28 19:13:23 -05:00
|
|
|
val out = new FileOutputStream(tempFile)
|
2012-09-04 21:52:07 -04:00
|
|
|
Utils.copyStream(in, out, true)
|
2012-12-28 19:13:23 -05:00
|
|
|
if (targetFile.exists && !Files.equal(tempFile, targetFile)) {
|
|
|
|
tempFile.delete()
|
2012-12-28 20:37:13 -05:00
|
|
|
throw new SparkException("File " + targetFile + " exists and does not match contents of" +
|
|
|
|
" " + url)
|
2012-12-28 19:13:23 -05:00
|
|
|
} else {
|
|
|
|
Files.move(tempFile, targetFile)
|
|
|
|
}
|
2012-08-30 14:01:43 -04:00
|
|
|
}
|
|
|
|
// Decompress the file if it's a .tar or .tar.gz
|
|
|
|
if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) {
|
|
|
|
logInfo("Untarring " + filename)
|
|
|
|
Utils.execute(Seq("tar", "-xzf", filename), targetDir)
|
|
|
|
} else if (filename.endsWith(".tar")) {
|
|
|
|
logInfo("Untarring " + filename)
|
|
|
|
Utils.execute(Seq("tar", "-xf", filename), targetDir)
|
|
|
|
}
|
2012-09-17 13:08:37 -04:00
|
|
|
// Make the file executable - That's necessary for scripts
|
|
|
|
FileUtil.chmod(filename, "a+x")
|
2012-08-30 14:01:43 -04:00
|
|
|
}
|
2012-07-06 18:23:26 -04:00
|
|
|
|
2012-07-01 20:13:31 -04:00
|
|
|
/**
|
|
|
|
* Shuffle the elements of a collection into a random order, returning the
|
|
|
|
* result in a new collection. Unlike scala.util.Random.shuffle, this method
|
|
|
|
* uses a local random number generator, avoiding inter-thread contention.
|
|
|
|
*/
|
2012-09-26 00:46:58 -04:00
|
|
|
def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = {
|
|
|
|
randomizeInPlace(seq.toArray)
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* Shuffle the elements of an array into a random order, modifying the
|
|
|
|
* original array. Returns the original array.
|
|
|
|
*/
|
|
|
|
def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = {
|
|
|
|
for (i <- (arr.length - 1) to 1 by -1) {
|
2010-11-04 01:45:44 -04:00
|
|
|
val j = rand.nextInt(i)
|
2012-09-26 00:46:58 -04:00
|
|
|
val tmp = arr(j)
|
|
|
|
arr(j) = arr(i)
|
|
|
|
arr(i) = tmp
|
2010-11-04 01:45:44 -04:00
|
|
|
}
|
2012-09-26 00:46:58 -04:00
|
|
|
arr
|
2010-11-04 01:45:44 -04:00
|
|
|
}
|
2010-11-09 16:46:30 -05:00
|
|
|
|
|
|
|
/**
|
2012-02-10 01:58:24 -05:00
|
|
|
* Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4).
|
2010-11-09 16:46:30 -05:00
|
|
|
*/
|
2012-12-08 03:33:11 -05:00
|
|
|
lazy val localIpAddress: String = findLocalIpAddress()
|
|
|
|
|
|
|
|
private def findLocalIpAddress(): String = {
|
2012-11-19 14:52:10 -05:00
|
|
|
val defaultIpOverride = System.getenv("SPARK_LOCAL_IP")
|
2012-12-08 03:33:11 -05:00
|
|
|
if (defaultIpOverride != null) {
|
2012-11-15 16:44:13 -05:00
|
|
|
defaultIpOverride
|
2012-12-08 03:33:11 -05:00
|
|
|
} else {
|
|
|
|
val address = InetAddress.getLocalHost
|
|
|
|
if (address.isLoopbackAddress) {
|
|
|
|
// Address resolves to something like 127.0.1.1, which happens on Debian; try to find
|
|
|
|
// a better address using the local network interfaces
|
|
|
|
for (ni <- NetworkInterface.getNetworkInterfaces) {
|
|
|
|
for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && !addr.isLoopbackAddress) {
|
|
|
|
// We've found an address that looks reasonable!
|
|
|
|
logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
|
|
|
|
" a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress +
|
|
|
|
" instead (on interface " + ni.getName + ")")
|
|
|
|
logWarning("Set SPARK_LOCAL_IP if you need to bind to another address")
|
|
|
|
return addr.getHostAddress
|
|
|
|
}
|
|
|
|
}
|
|
|
|
logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
|
|
|
|
" a loopback address: " + address.getHostAddress + ", but we couldn't find any" +
|
|
|
|
" external IP address!")
|
|
|
|
logWarning("Set SPARK_LOCAL_IP if you need to bind to another address")
|
|
|
|
}
|
|
|
|
address.getHostAddress
|
|
|
|
}
|
2012-11-15 16:44:13 -05:00
|
|
|
}
|
2012-06-07 03:25:47 -04:00
|
|
|
|
|
|
|
private var customHostname: Option[String] = None
|
|
|
|
|
|
|
|
/**
|
|
|
|
* Allow setting a custom host name because when we run on Mesos we need to use the same
|
|
|
|
* hostname it reports to the master.
|
|
|
|
*/
|
|
|
|
def setCustomHostname(hostname: String) {
|
|
|
|
customHostname = Some(hostname)
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
2012-07-01 20:13:31 -04:00
|
|
|
* Get the local machine's hostname.
|
2012-06-07 03:25:47 -04:00
|
|
|
*/
|
|
|
|
def localHostName(): String = {
|
|
|
|
customHostname.getOrElse(InetAddress.getLocalHost.getHostName)
|
|
|
|
}
|
2012-10-07 03:54:38 -04:00
|
|
|
|
2011-04-28 01:13:01 -04:00
|
|
|
/**
|
2012-02-10 01:58:24 -05:00
|
|
|
* Returns a standard ThreadFactory except all threads are daemons.
|
2011-04-28 01:13:01 -04:00
|
|
|
*/
|
|
|
|
private def newDaemonThreadFactory: ThreadFactory = {
|
|
|
|
new ThreadFactory {
|
|
|
|
def newThread(r: Runnable): Thread = {
|
|
|
|
var t = Executors.defaultThreadFactory.newThread (r)
|
|
|
|
t.setDaemon (true)
|
|
|
|
return t
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
2012-02-10 01:58:24 -05:00
|
|
|
* Wrapper over newCachedThreadPool.
|
2011-04-28 01:13:01 -04:00
|
|
|
*/
|
|
|
|
def newDaemonCachedThreadPool(): ThreadPoolExecutor = {
|
2012-02-10 11:19:53 -05:00
|
|
|
var threadPool = Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor]
|
2011-04-28 01:13:01 -04:00
|
|
|
|
|
|
|
threadPool.setThreadFactory (newDaemonThreadFactory)
|
|
|
|
|
|
|
|
return threadPool
|
|
|
|
}
|
2012-10-07 03:54:38 -04:00
|
|
|
|
2012-06-07 03:25:47 -04:00
|
|
|
/**
|
2012-10-07 03:54:38 -04:00
|
|
|
* Return the string to tell how long has passed in seconds. The passing parameter should be in
|
|
|
|
* millisecond.
|
2012-06-07 03:25:47 -04:00
|
|
|
*/
|
|
|
|
def getUsedTimeMs(startTimeMs: Long): String = {
|
|
|
|
return " " + (System.currentTimeMillis - startTimeMs) + " ms "
|
|
|
|
}
|
2011-04-28 01:13:01 -04:00
|
|
|
|
|
|
|
/**
|
2012-02-10 01:58:24 -05:00
|
|
|
* Wrapper over newFixedThreadPool.
|
2011-04-28 01:13:01 -04:00
|
|
|
*/
|
|
|
|
def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = {
|
2012-02-10 11:19:53 -05:00
|
|
|
var threadPool = Executors.newFixedThreadPool(nThreads).asInstanceOf[ThreadPoolExecutor]
|
2011-04-28 01:13:01 -04:00
|
|
|
|
|
|
|
threadPool.setThreadFactory(newDaemonThreadFactory)
|
|
|
|
|
|
|
|
return threadPool
|
2010-11-09 16:46:30 -05:00
|
|
|
}
|
2011-02-27 22:15:52 -05:00
|
|
|
|
|
|
|
/**
|
2012-02-10 01:58:24 -05:00
|
|
|
* Delete a file or directory and its contents recursively.
|
|
|
|
*/
|
|
|
|
def deleteRecursively(file: File) {
|
|
|
|
if (file.isDirectory) {
|
|
|
|
for (child <- file.listFiles()) {
|
|
|
|
deleteRecursively(child)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (!file.delete()) {
|
|
|
|
throw new IOException("Failed to delete: " + file)
|
|
|
|
}
|
|
|
|
}
|
2012-05-14 21:39:04 -04:00
|
|
|
|
|
|
|
/**
|
2012-06-30 19:43:27 -04:00
|
|
|
* Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes.
|
|
|
|
* This is used to figure out how much memory to claim from Mesos based on the SPARK_MEM
|
|
|
|
* environment variable.
|
|
|
|
*/
|
|
|
|
def memoryStringToMb(str: String): Int = {
|
|
|
|
val lower = str.toLowerCase
|
|
|
|
if (lower.endsWith("k")) {
|
|
|
|
(lower.substring(0, lower.length-1).toLong / 1024).toInt
|
|
|
|
} else if (lower.endsWith("m")) {
|
|
|
|
lower.substring(0, lower.length-1).toInt
|
|
|
|
} else if (lower.endsWith("g")) {
|
|
|
|
lower.substring(0, lower.length-1).toInt * 1024
|
|
|
|
} else if (lower.endsWith("t")) {
|
|
|
|
lower.substring(0, lower.length-1).toInt * 1024 * 1024
|
|
|
|
} else {// no suffix, so it's just a number in bytes
|
|
|
|
(lower.toLong / 1024 / 1024).toInt
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* Convert a memory quantity in bytes to a human-readable string such as "4.0 MB".
|
2012-05-14 21:39:04 -04:00
|
|
|
*/
|
2012-05-15 03:31:52 -04:00
|
|
|
def memoryBytesToString(size: Long): String = {
|
2012-06-30 17:45:55 -04:00
|
|
|
val TB = 1L << 40
|
2012-05-14 21:39:04 -04:00
|
|
|
val GB = 1L << 30
|
|
|
|
val MB = 1L << 20
|
|
|
|
val KB = 1L << 10
|
2012-05-19 09:13:20 -04:00
|
|
|
|
2012-05-14 21:39:04 -04:00
|
|
|
val (value, unit) = {
|
2012-06-30 17:45:55 -04:00
|
|
|
if (size >= 2*TB) {
|
|
|
|
(size.asInstanceOf[Double] / TB, "TB")
|
|
|
|
} else if (size >= 2*GB) {
|
2012-05-14 21:39:04 -04:00
|
|
|
(size.asInstanceOf[Double] / GB, "GB")
|
|
|
|
} else if (size >= 2*MB) {
|
|
|
|
(size.asInstanceOf[Double] / MB, "MB")
|
|
|
|
} else if (size >= 2*KB) {
|
|
|
|
(size.asInstanceOf[Double] / KB, "KB")
|
|
|
|
} else {
|
|
|
|
(size.asInstanceOf[Double], "B")
|
|
|
|
}
|
|
|
|
}
|
2012-06-30 17:45:55 -04:00
|
|
|
"%.1f %s".formatLocal(Locale.US, value, unit)
|
2012-05-14 21:39:04 -04:00
|
|
|
}
|
2012-06-30 19:43:27 -04:00
|
|
|
|
|
|
|
/**
|
|
|
|
* Convert a memory quantity in megabytes to a human-readable string such as "4.0 MB".
|
|
|
|
*/
|
|
|
|
def memoryMegabytesToString(megabytes: Long): String = {
|
|
|
|
memoryBytesToString(megabytes * 1024L * 1024L)
|
|
|
|
}
|
2012-07-01 20:13:31 -04:00
|
|
|
|
|
|
|
/**
|
|
|
|
* Execute a command in the given working directory, throwing an exception if it completes
|
|
|
|
* with an exit code other than 0.
|
|
|
|
*/
|
|
|
|
def execute(command: Seq[String], workingDir: File) {
|
|
|
|
val process = new ProcessBuilder(command: _*)
|
|
|
|
.directory(workingDir)
|
|
|
|
.redirectErrorStream(true)
|
|
|
|
.start()
|
|
|
|
new Thread("read stdout for " + command(0)) {
|
|
|
|
override def run() {
|
|
|
|
for (line <- Source.fromInputStream(process.getInputStream).getLines) {
|
|
|
|
System.err.println(line)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}.start()
|
|
|
|
val exitCode = process.waitFor()
|
|
|
|
if (exitCode != 0) {
|
|
|
|
throw new SparkException("Process " + command + " exited with code " + exitCode)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* Execute a command in the current working directory, throwing an exception if it completes
|
|
|
|
* with an exit code other than 0.
|
|
|
|
*/
|
|
|
|
def execute(command: Seq[String]) {
|
|
|
|
execute(command, new File("."))
|
|
|
|
}
|
2012-09-28 20:42:00 -04:00
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
* When called inside a class in the spark package, returns the name of the user code class
|
|
|
|
* (outside the spark package) that called into Spark, as well as which Spark method they called.
|
|
|
|
* This is used, for example, to tell users where in their code each RDD got created.
|
|
|
|
*/
|
|
|
|
def getSparkCallSite: String = {
|
2012-09-28 21:28:13 -04:00
|
|
|
val trace = Thread.currentThread.getStackTrace().filter( el =>
|
|
|
|
(!el.getMethodName.contains("getStackTrace")))
|
2012-09-28 20:42:00 -04:00
|
|
|
|
|
|
|
// Keep crawling up the stack trace until we find the first function not inside of the spark
|
|
|
|
// package. We track the last (shallowest) contiguous Spark method. This might be an RDD
|
|
|
|
// transformation, a SparkContext function (such as parallelize), or anything else that leads
|
|
|
|
// to instantiation of an RDD. We also track the first (deepest) user method, file, and line.
|
|
|
|
var lastSparkMethod = "<unknown>"
|
|
|
|
var firstUserFile = "<unknown>"
|
|
|
|
var firstUserLine = 0
|
|
|
|
var finished = false
|
|
|
|
|
|
|
|
for (el <- trace) {
|
|
|
|
if (!finished) {
|
2012-09-29 02:28:16 -04:00
|
|
|
if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) {
|
|
|
|
lastSparkMethod = if (el.getMethodName == "<init>") {
|
|
|
|
// Spark method is a constructor; get its class name
|
|
|
|
el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1)
|
|
|
|
} else {
|
|
|
|
el.getMethodName
|
|
|
|
}
|
2012-09-28 20:42:00 -04:00
|
|
|
}
|
|
|
|
else {
|
2012-09-28 21:28:13 -04:00
|
|
|
firstUserLine = el.getLineNumber
|
|
|
|
firstUserFile = el.getFileName
|
2012-09-28 20:42:00 -04:00
|
|
|
finished = true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
"%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine)
|
|
|
|
}
|
2010-03-29 19:17:55 -04:00
|
|
|
}
|