diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 0afab522af82912407231c481e5367cf5b194c02..4fd81bc63bc1d46a9454551f3868d80dcea1f1f8 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -419,8 +419,9 @@ class SparkContext(
     }
     addedFiles(key) = System.currentTimeMillis
 
-    // Fetch the file locally in case the task is executed locally
-    val filename = new File(path.split("/").last)
+    // Fetch the file locally in case a job is executed locally.
+    // Jobs that run through LocalScheduler will already fetch the required dependencies,
+    // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here.
     Utils.fetchFile(path, new File("."))
 
     logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
@@ -437,11 +438,10 @@ class SparkContext(
   }
 
   /**
-   * Clear the job's list of files added by `addFile` so that they do not get donwloaded to
+   * Clear the job's list of files added by `addFile` so that they do not get downloaded to
    * any new nodes.
    */
   def clearFiles() {
-    addedFiles.keySet.map(_.split("/").last).foreach { k => new File(k).delete() }
     addedFiles.clear()
   }
 
@@ -465,7 +465,6 @@ class SparkContext(
    * any new nodes.
    */
   def clearJars() {
-    addedJars.keySet.map(_.split("/").last).foreach { k => new File(k).delete() }
     addedJars.clear()
   }
 
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index 6d64b32174f05f68ea769c39ab691fd4c156a59a..0e7007459d58dcc2f2f8bf45443cd90c828e8dc6 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -9,6 +9,7 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.JavaConversions._
 import scala.io.Source
+import com.google.common.io.Files
 
 /**
  * Various utility methods used by Spark.
@@ -127,31 +128,53 @@ private object Utils extends Logging {
   /**
    * 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.
+   *
+   * Throws SparkException if the target file already exists and has different contents than
+   * the requested file.
    */
   def fetchFile(url: String, targetDir: File) {
     val filename = url.split("/").last
+    val tempDir = System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))
+    val tempFile =  File.createTempFile("fetchFileTemp", null, new File(tempDir))
     val targetFile = new File(targetDir, filename)
     val uri = new URI(url)
     uri.getScheme match {
       case "http" | "https" | "ftp" =>
-        logInfo("Fetching " + url + " to " + targetFile)
+        logInfo("Fetching " + url + " to " + tempFile)
         val in = new URL(url).openStream()
-        val out = new FileOutputStream(targetFile)
+        val out = new FileOutputStream(tempFile)
         Utils.copyStream(in, out, true)
+        if (targetFile.exists && !Files.equal(tempFile, targetFile)) {
+          tempFile.delete()
+          throw new SparkException("File " + targetFile + " exists and does not match contents of" +
+            " " + url)
+        } else {
+          Files.move(tempFile, targetFile)
+        }
       case "file" | null =>
-        // 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)
+        val sourceFile = if (uri.isAbsolute) {
+          new File(uri)
+        } else {
+          new File(url)
+        }
+        if (targetFile.exists && !Files.equal(sourceFile, targetFile)) {
+          throw new SparkException("File " + targetFile + " exists and does not match contents of" +
+            " " + url)
         } 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)
+          // 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)
+          }
         }
       case _ =>
         // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
@@ -159,8 +182,15 @@ private object Utils extends Logging {
         val conf = new Configuration()
         val fs = FileSystem.get(uri, conf)
         val in = fs.open(new Path(uri))
-        val out = new FileOutputStream(targetFile)
+        val out = new FileOutputStream(tempFile)
         Utils.copyStream(in, out, true)
+        if (targetFile.exists && !Files.equal(tempFile, targetFile)) {
+          tempFile.delete()
+          throw new SparkException("File " + targetFile + " exists and does not match contents of" +
+            " " + url)
+        } else {
+          Files.move(tempFile, targetFile)
+        }
     }
     // Decompress the file if it's a .tar or .tar.gz
     if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) {
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index edbb187b1ba6e59d512966d7303dc510589b54de..b7725313c423a5ef28be7e2227f36677f6510794 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -301,6 +301,40 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
    * (in that order of preference). If neither of these is set, return None.
    */
   def getSparkHome(): Option[String] = sc.getSparkHome()
+
+  /**
+   * Add a file to be downloaded into the working directory of this Spark job on every node.
+   * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
+   * filesystems), or an HTTP, HTTPS or FTP URI.
+   */
+  def addFile(path: String) {
+    sc.addFile(path)
+  }
+
+  /**
+   * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
+   * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
+   * filesystems), or an HTTP, HTTPS or FTP URI.
+   */
+  def addJar(path: String) {
+    sc.addJar(path)
+  }
+
+  /**
+   * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to
+   * any new nodes.
+   */
+  def clearJars() {
+    sc.clearJars()
+  }
+
+  /**
+   * Clear the job's list of files added by `addFile` so that they do not get downloaded to
+   * any new nodes.
+   */
+  def clearFiles() {
+    sc.clearFiles()
+  }
 }
 
 object JavaSparkContext {
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
index eb20fe41b2a3007b0d9f33fe35ed99da02521fe9..2593c0e3a0d239e843b295f74761f23e18c39af9 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -108,22 +108,24 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
    * SparkContext. Also adds any new JARs we fetched to the class loader.
    */
   private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
-    // Fetch missing dependencies
-    for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
-      logInfo("Fetching " + name + " with timestamp " + timestamp)
-      Utils.fetchFile(name, new File("."))
-      currentFiles(name) = timestamp
-    }
-    for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
-      logInfo("Fetching " + name + " with timestamp " + timestamp)
-      Utils.fetchFile(name, new File("."))
-      currentJars(name) = timestamp
-      // Add it to our class loader
-      val localName = name.split("/").last
-      val url = new File(".", localName).toURI.toURL
-      if (!classLoader.getURLs.contains(url)) {
-        logInfo("Adding " + url + " to class loader")
-        classLoader.addURL(url)
+    synchronized {
+      // Fetch missing dependencies
+      for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
+        logInfo("Fetching " + name + " with timestamp " + timestamp)
+        Utils.fetchFile(name, new File("."))
+        currentFiles(name) = timestamp
+      }
+      for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
+        logInfo("Fetching " + name + " with timestamp " + timestamp)
+        Utils.fetchFile(name, new File("."))
+        currentJars(name) = timestamp
+        // Add it to our class loader
+        val localName = name.split("/").last
+        val url = new File(".", localName).toURI.toURL
+        if (!classLoader.getURLs.contains(url)) {
+          logInfo("Adding " + url + " to class loader")
+          classLoader.addURL(url)
+        }
       }
     }
   }