diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala
index 25db395c22128013d259aae8722d567e3c0ff76f..a09c97806869ef707a6b374a0c253d9f81410519 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/bagel/BagelSuite.scala
@@ -23,6 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
     }
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   test("halting by voting") {
diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala
index ff00dd05dd7875e78f426b4e4f9b6bec20cfd079..76d5258b02b9712fae6d5a5935f6036a1a23fb44 100644
--- a/core/src/test/scala/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/spark/LocalSparkContext.scala
@@ -27,6 +27,7 @@ object LocalSparkContext {
     sc.stop()
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */
@@ -38,4 +39,4 @@ object LocalSparkContext {
     }
   }
 
-}
\ No newline at end of file
+}
diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala
index 43559b96d3a807a220053264cc3f34fb68af0c07..1c64f9b98d099d618eedffc559f4cfc7114bfa1f 100644
--- a/repl/src/test/scala/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/spark/repl/ReplSuite.scala
@@ -32,6 +32,7 @@ class ReplSuite extends FunSuite {
       interp.sparkContext.stop()
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     return out.toString
   }
   
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index e303e33e5e4014e7b252b491edc9d5090dc5e88f..66e67cbfa1d786931580a7379e6aa0b10cb80d5b 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -38,11 +38,20 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
 private[streaming]
 class CheckpointWriter(checkpointDir: String) extends Logging {
   val file = new Path(checkpointDir, "graph")
+  // The file to which we actually write - and then "move" to file.
+  private val writeFile = new Path(file.getParent, file.getName + ".next")
+  private val bakFile = new Path(file.getParent, file.getName + ".bk")
+
+  private var stopped = false
+
   val conf = new Configuration()
   var fs = file.getFileSystem(conf)
   val maxAttempts = 3
   val executor = Executors.newFixedThreadPool(1)
 
+  // Removed code which validates whether there is only one CheckpointWriter per path 'file' since 
+  // I did not notice any errors - reintroduce it ?
+
   class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
     def run() {
       var attempts = 0
@@ -51,15 +60,17 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
         attempts += 1
         try {
           logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
-          if (fs.exists(file)) {
-            val bkFile = new Path(file.getParent, file.getName + ".bk")
-            FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
-            logDebug("Moved existing checkpoint file to " + bkFile)
-          }
-          val fos = fs.create(file)
+          // This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast.
+          val fos = fs.create(writeFile)
           fos.write(bytes)
           fos.close()
-          fos.close()
+          if (fs.exists(file) && fs.rename(file, bakFile)) {
+            logDebug("Moved existing checkpoint file to " + bakFile)
+          }
+          // paranoia
+          fs.delete(file, false)
+          fs.rename(writeFile, file)
+
           val finishTime = System.currentTimeMillis();
           logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
             "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
@@ -84,7 +95,15 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
   }
 
   def stop() {
+    synchronized {
+      if (stopped) return ;
+      stopped = true
+    }
     executor.shutdown()
+    val startTime = System.currentTimeMillis()
+    val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS)
+    val endTime = System.currentTimeMillis()
+    logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.")
   }
 }
 
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
index adb7f3a24d25f6fcbd1453c2e75b56b9a22d10b4..3b331956f5973c07af145939d556baa7875a72a0 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -54,8 +54,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
         throw new Exception("Batch duration already set as " + batchDuration +
           ". cannot set it again.")
       }
+      batchDuration = duration
     }
-    batchDuration = duration
   }
 
   def remember(duration: Duration) {
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
index f673e5be15485b839d5f894912d3290eecc21637..426a9b6f71d88852aa62d4fa920018a4a344a667 100644
--- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
@@ -159,6 +159,7 @@ object MasterFailureTest extends Logging {
 
     // Setup the streaming computation with the given operation
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map())
     ssc.checkpoint(checkpointDir.toString)
     val inputStream = ssc.textFileStream(testDir.toString)
@@ -205,6 +206,7 @@ object MasterFailureTest extends Logging {
         // (iii) Its not timed out yet
         System.clearProperty("spark.streaming.clock")
         System.clearProperty("spark.driver.port")
+        System.clearProperty("spark.hostPort")
         ssc.start()
         val startTime = System.currentTimeMillis()
         while (!killed && !isLastOutputGenerated && !isTimedOut) {
@@ -357,13 +359,16 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
         // Write the data to a local file and then move it to the target test directory
         val localFile = new File(localTestDir, (i+1).toString)
         val hadoopFile = new Path(testDir, (i+1).toString)
+        val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString)
         FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
         var tries = 0
 	var done = false
         while (!done && tries < maxTries) {
           tries += 1
           try {
-            fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+            // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+            fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
+            fs.rename(tempHadoopFile, hadoopFile)
 	    done = true
 	  } catch {
 	    case ioe: IOException => { 
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
index cf2ed8b1d4bef11018e27785702cb44254527188..e7352deb81da3a140710d8fb0c223e50121ed7b1 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -15,6 +15,7 @@ class BasicOperationsSuite extends TestSuiteBase {
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   test("map") {
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
index cac86deeaf3492cb298c26d9812cc82e71fd83cf..607dea77ec24b6bfd90d59571b702b4933239f91 100644
--- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -31,6 +31,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   var ssc: StreamingContext = null
@@ -325,6 +326,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     )
     ssc = new StreamingContext(checkpointDir)
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     ssc.start()
     val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches)
     // the first element will be re-processed data of the last batch before restart
@@ -350,4 +352,4 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
     outputStream.output
   }
-}
\ No newline at end of file
+}
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
index 67dca2ac3121596d0b012ec0e749880a6db94e90..0acb6db6f2e484c213ea0bc15d291fb6f8a2dab6 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -41,6 +41,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
 
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
index 1b66f3bda20ad0f112295e529f4a3f1419c167ed..80d827706f63c2cfdee90db2fe6f1bb5d114dd41 100644
--- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -16,6 +16,7 @@ class WindowOperationsSuite extends TestSuiteBase {
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   val largerSlideInput = Seq(