diff --git a/README.md b/README.md
index 28ad1e4604d1497d8302b48e31bb57bfca2a9797..456b8060ef3280ab077cbcb3f7dcc17a8f67a6bf 100644
--- a/README.md
+++ b/README.md
@@ -69,7 +69,7 @@ described below.
 
 When developing a Spark application, specify the Hadoop version by adding the
 "hadoop-client" artifact to your project's dependencies. For example, if you're
-using Hadoop 1.0.1 and build your application using SBT, add this entry to
+using Hadoop 1.2.1 and build your application using SBT, add this entry to
 `libraryDependencies`:
 
     "org.apache.hadoop" % "hadoop-client" % "1.2.1"
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index c7819d49324042f9ecd54cf90c5b0cc66984d5be..c16afd6b36a77aa6d4baf920af9001754671be35 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -32,12 +32,26 @@ fi
 
 # Build up classpath
 CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
-if [ -f "$FWDIR/RELEASE" ]; then
-  ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
+
+# First check if we have a dependencies jar. If so, include binary classes with the deps jar
+if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then
+  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes"
+
+  DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar`
+  CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR"
 else
-  ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
+  # Else use spark-assembly jar from either RELEASE or assembly directory
+  if [ -f "$FWDIR/RELEASE" ]; then
+    ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
+  else
+    ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
+  fi
+  CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
 fi
-CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
 
 # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
 if [[ $SPARK_TESTING == 1 ]]; then
diff --git a/bin/slaves.sh b/bin/slaves.sh
index 752565b759f77384b169f917bf212a474abd59ae..c367c2fd8ed9ae152607e54eb51b162788521613 100755
--- a/bin/slaves.sh
+++ b/bin/slaves.sh
@@ -28,7 +28,7 @@
 #   SPARK_SSH_OPTS Options passed to ssh when running remote commands.
 ##
 
-usage="Usage: slaves.sh [--config confdir] command..."
+usage="Usage: slaves.sh [--config <conf-dir>] command..."
 
 # if no args specified, show usage
 if [ $# -le 0 ]; then
@@ -46,6 +46,23 @@ bin=`cd "$bin"; pwd`
 # spark-env.sh. Save it here.
 HOSTLIST=$SPARK_SLAVES
 
+# Check if --config is passed as an argument. It is an optional parameter.
+# Exit if the argument is not a directory.
+if [ "$1" == "--config" ]
+then
+  shift
+  conf_dir=$1
+  if [ ! -d "$conf_dir" ]
+  then
+    echo "ERROR : $conf_dir is not a directory"
+    echo $usage
+    exit 1
+  else
+    export SPARK_CONF_DIR=$conf_dir
+  fi
+  shift
+fi
+
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
 fi
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
index 5bfe967fbfaeb76729ae3a30b81287cfb6021b55..a0c0d44b58d9991f2d94ed16c41269baabe15900 100755
--- a/bin/spark-daemon.sh
+++ b/bin/spark-daemon.sh
@@ -29,7 +29,7 @@
 #   SPARK_NICENESS The scheduling priority for daemons. Defaults to 0.
 ##
 
-usage="Usage: spark-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <spark-command> <spark-instance-number> <args...>"
+usage="Usage: spark-daemon.sh [--config <conf-dir>] (start|stop) <spark-command> <spark-instance-number> <args...>"
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
@@ -43,6 +43,25 @@ bin=`cd "$bin"; pwd`
 . "$bin/spark-config.sh"
 
 # get arguments
+
+# Check if --config is passed as an argument. It is an optional parameter.
+# Exit if the argument is not a directory.
+
+if [ "$1" == "--config" ]
+then
+  shift
+  conf_dir=$1
+  if [ ! -d "$conf_dir" ]
+  then
+    echo "ERROR : $conf_dir is not a directory"
+    echo $usage
+    exit 1
+  else
+    export SPARK_CONF_DIR=$conf_dir
+  fi
+  shift
+fi
+
 startStop=$1
 shift
 command=$1
diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh
index 354eb905a1c069dd428111157098b2ff86aa8e64..64286cb2da4f5acee66db6dd19ed442190d6466c 100755
--- a/bin/spark-daemons.sh
+++ b/bin/spark-daemons.sh
@@ -19,7 +19,7 @@
 
 # Run a Spark command on all slave hosts.
 
-usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
+usage="Usage: spark-daemons.sh [--config <conf-dir>] [start|stop] command instance-number args..."
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
index cfd8132891a11cc49ed46cbe9fbec36d022d254a..172c6e4b1cce198020a314b4f6fb3062b3dd5cff 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
@@ -25,6 +25,7 @@ import io.netty.channel.ChannelInboundMessageHandlerAdapter;
 import io.netty.channel.DefaultFileRegion;
 
 import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.FileSegment;
 
 class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
 
@@ -37,40 +38,34 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
   @Override
   public void messageReceived(ChannelHandlerContext ctx, String blockIdString) {
     BlockId blockId = BlockId.apply(blockIdString);
-    String path = pResolver.getAbsolutePath(blockId.name());
-    // if getFilePath returns null, close the channel
-    if (path == null) {
+    FileSegment fileSegment = pResolver.getBlockLocation(blockId);
+    // if getBlockLocation returns null, close the channel
+    if (fileSegment == null) {
       //ctx.close();
       return;
     }
-    File file = new File(path);
+    File file = fileSegment.file();
     if (file.exists()) {
       if (!file.isFile()) {
-        //logger.info("Not a file : " + file.getAbsolutePath());
         ctx.write(new FileHeader(0, blockId).buffer());
         ctx.flush();
         return;
       }
-      long length = file.length();
+      long length = fileSegment.length();
       if (length > Integer.MAX_VALUE || length <= 0) {
-        //logger.info("too large file : " + file.getAbsolutePath() + " of size "+ length);
         ctx.write(new FileHeader(0, blockId).buffer());
         ctx.flush();
         return;
       }
       int len = new Long(length).intValue();
-      //logger.info("Sending block "+blockId+" filelen = "+len);
-      //logger.info("header = "+ (new FileHeader(len, blockId)).buffer());
       ctx.write((new FileHeader(len, blockId)).buffer());
       try {
         ctx.sendFile(new DefaultFileRegion(new FileInputStream(file)
-          .getChannel(), 0, file.length()));
+          .getChannel(), fileSegment.offset(), fileSegment.length()));
       } catch (Exception e) {
-        //logger.warning("Exception when sending file : " + file.getAbsolutePath());
         e.printStackTrace();
       }
     } else {
-      //logger.warning("File not found: " + file.getAbsolutePath());
       ctx.write(new FileHeader(0, blockId).buffer());
     }
     ctx.flush();
diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
index 94c034cad0119397bc6332dde066fa2be2ab75a8..9f7ced44cf93da741b20d75e1b7f2f454272f8b4 100755
--- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
+++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
@@ -17,13 +17,10 @@
 
 package org.apache.spark.network.netty;
 
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.FileSegment;
 
 public interface PathResolver {
-  /**
-   * Get the absolute path of the file
-   *
-   * @param fileId
-   * @return the absolute path of file
-   */
-  public String getAbsolutePath(String fileId);
+  /** Get the file segment in which the given block resides. */
+  public FileSegment getBlockLocation(BlockId blockId);
 }
diff --git a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala
index f87460039b0217855b0b4e262976078e8cb8d71f..0c47afae54c8b6e81af33a5690f1f8f1a7b3f10b 100644
--- a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala
+++ b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala
@@ -17,20 +17,29 @@
 
 package org.apache.hadoop.mapred
 
+private[apache]
 trait SparkHadoopMapRedUtil {
   def newJobContext(conf: JobConf, jobId: JobID): JobContext = {
-    val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", "org.apache.hadoop.mapred.JobContext");
-    val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[org.apache.hadoop.mapreduce.JobID])
+    val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl",
+      "org.apache.hadoop.mapred.JobContext")
+    val ctor = klass.getDeclaredConstructor(classOf[JobConf],
+      classOf[org.apache.hadoop.mapreduce.JobID])
     ctor.newInstance(conf, jobId).asInstanceOf[JobContext]
   }
 
   def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = {
-    val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl", "org.apache.hadoop.mapred.TaskAttemptContext")
+    val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl",
+      "org.apache.hadoop.mapred.TaskAttemptContext")
     val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[TaskAttemptID])
     ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext]
   }
 
-  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = {
+  def newTaskAttemptID(
+      jtIdentifier: String,
+      jobId: Int,
+      isMap: Boolean,
+      taskId: Int,
+      attemptId: Int) = {
     new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId)
   }
 
diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala
index 93180307fa3a5c4293485cb93e1301fb7e1bdfa1..32429f01acab81e9d4dcdb9fd7289240d9ef44b0 100644
--- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala
+++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala
@@ -17,9 +17,10 @@
 
 package org.apache.hadoop.mapreduce
 
-import org.apache.hadoop.conf.Configuration
 import java.lang.{Integer => JInteger, Boolean => JBoolean}
+import org.apache.hadoop.conf.Configuration
 
+private[apache]
 trait SparkHadoopMapReduceUtil {
   def newJobContext(conf: Configuration, jobId: JobID): JobContext = {
     val klass = firstAvailableClass(
@@ -37,23 +38,31 @@ trait SparkHadoopMapReduceUtil {
     ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext]
   }
 
-  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = {
-    val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID");
+  def newTaskAttemptID(
+      jtIdentifier: String,
+      jobId: Int,
+      isMap: Boolean,
+      taskId: Int,
+      attemptId: Int) = {
+    val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID")
     try {
-      // first, attempt to use the old-style constructor that takes a boolean isMap (not available in YARN)
+      // First, attempt to use the old-style constructor that takes a boolean isMap
+      // (not available in YARN)
       val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], classOf[Boolean],
-          classOf[Int], classOf[Int])
-      ctor.newInstance(jtIdentifier, new JInteger(jobId), new JBoolean(isMap), new JInteger(taskId), new
-        JInteger(attemptId)).asInstanceOf[TaskAttemptID]
+        classOf[Int], classOf[Int])
+      ctor.newInstance(jtIdentifier, new JInteger(jobId), new JBoolean(isMap), new JInteger(taskId),
+        new JInteger(attemptId)).asInstanceOf[TaskAttemptID]
     } catch {
       case exc: NoSuchMethodException => {
-        // failed, look for the new ctor that takes a TaskType (not available in 1.x)
-        val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType").asInstanceOf[Class[Enum[_]]]
-        val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(taskTypeClass, if(isMap) "MAP" else "REDUCE")
+        // If that failed, look for the new constructor that takes a TaskType (not available in 1.x)
+        val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType")
+          .asInstanceOf[Class[Enum[_]]]
+        val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(
+          taskTypeClass, if(isMap) "MAP" else "REDUCE")
         val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], taskTypeClass,
           classOf[Int], classOf[Int])
-        ctor.newInstance(jtIdentifier, new JInteger(jobId), taskType, new JInteger(taskId), new
-          JInteger(attemptId)).asInstanceOf[TaskAttemptID]
+        ctor.newInstance(jtIdentifier, new JInteger(jobId), taskType, new JInteger(taskId),
+          new JInteger(attemptId)).asInstanceOf[TaskAttemptID]
       }
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 1e3f1ebfafaafcc1f35f4179da6aae5b3445590f..5e465fa22c1ac05486c1e7097029307e5fdeac36 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -20,13 +20,11 @@ package org.apache.spark
 import java.io._
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 
-import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
 import akka.actor._
 import akka.dispatch._
 import akka.pattern.ask
-import akka.remote._
 import akka.util.Duration
 
 
@@ -40,11 +38,12 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String
   extends MapOutputTrackerMessage
 private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage
 
-private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Logging {
+private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster)
+  extends Actor with Logging {
   def receive = {
     case GetMapOutputStatuses(shuffleId: Int, requester: String) =>
       logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + requester)
-      sender ! tracker.getSerializedLocations(shuffleId)
+      sender ! tracker.getSerializedMapOutputStatuses(shuffleId)
 
     case StopMapOutputTracker =>
       logInfo("MapOutputTrackerActor stopped!")
@@ -60,22 +59,19 @@ private[spark] class MapOutputTracker extends Logging {
   // Set to the MapOutputTrackerActor living on the driver
   var trackerActor: ActorRef = _
 
-  private var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
+  protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
 
   // Incremented every time a fetch fails so that client nodes know to clear
   // their cache of map output locations if this happens.
-  private var epoch: Long = 0
-  private val epochLock = new java.lang.Object
+  protected var epoch: Long = 0
+  protected val epochLock = new java.lang.Object
 
-  // Cache a serialized version of the output statuses for each shuffle to send them out faster
-  var cacheEpoch = epoch
-  private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
-
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
+  private val metadataCleaner =
+    new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
 
   // Send a message to the trackerActor and get its result within a default timeout, or
   // throw a SparkException if this fails.
-  def askTracker(message: Any): Any = {
+  private def askTracker(message: Any): Any = {
     try {
       val future = trackerActor.ask(message)(timeout)
       return Await.result(future, timeout)
@@ -86,50 +82,12 @@ private[spark] class MapOutputTracker extends Logging {
   }
 
   // Send a one-way message to the trackerActor, to which we expect it to reply with true.
-  def communicate(message: Any) {
+  private def communicate(message: Any) {
     if (askTracker(message) != true) {
       throw new SparkException("Error reply received from MapOutputTracker")
     }
   }
 
-  def registerShuffle(shuffleId: Int, numMaps: Int) {
-    if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) {
-      throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
-    }
-  }
-
-  def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
-    var array = mapStatuses(shuffleId)
-    array.synchronized {
-      array(mapId) = status
-    }
-  }
-
-  def registerMapOutputs(
-      shuffleId: Int,
-      statuses: Array[MapStatus],
-      changeEpoch: Boolean = false) {
-    mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses)
-    if (changeEpoch) {
-      incrementEpoch()
-    }
-  }
-
-  def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
-    var arrayOpt = mapStatuses.get(shuffleId)
-    if (arrayOpt.isDefined && arrayOpt.get != null) {
-      var array = arrayOpt.get
-      array.synchronized {
-        if (array(mapId) != null && array(mapId).location == bmAddress) {
-          array(mapId) = null
-        }
-      }
-      incrementEpoch()
-    } else {
-      throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID")
-    }
-  }
-
   // Remembers which map output locations are currently being fetched on a worker
   private val fetching = new HashSet[Int]
 
@@ -168,7 +126,7 @@ private[spark] class MapOutputTracker extends Logging {
         try {
           val fetchedBytes =
             askTracker(GetMapOutputStatuses(shuffleId, hostPort)).asInstanceOf[Array[Byte]]
-          fetchedStatuses = deserializeStatuses(fetchedBytes)
+          fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes)
           logInfo("Got the output locations")
           mapStatuses.put(shuffleId, fetchedStatuses)
         } finally {
@@ -194,9 +152,8 @@ private[spark] class MapOutputTracker extends Logging {
     }
   }
 
-  private def cleanup(cleanupTime: Long) {
+  protected def cleanup(cleanupTime: Long) {
     mapStatuses.clearOldValues(cleanupTime)
-    cachedSerializedStatuses.clearOldValues(cleanupTime)
   }
 
   def stop() {
@@ -206,15 +163,7 @@ private[spark] class MapOutputTracker extends Logging {
     trackerActor = null
   }
 
-  // Called on master to increment the epoch number
-  def incrementEpoch() {
-    epochLock.synchronized {
-      epoch += 1
-      logDebug("Increasing epoch to " + epoch)
-    }
-  }
-
-  // Called on master or workers to get current epoch number
+  // Called to get current epoch number
   def getEpoch: Long = {
     epochLock.synchronized {
       return epoch
@@ -228,14 +177,62 @@ private[spark] class MapOutputTracker extends Logging {
     epochLock.synchronized {
       if (newEpoch > epoch) {
         logInfo("Updating epoch to " + newEpoch + " and clearing cache")
-        // mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
-        mapStatuses.clear()
         epoch = newEpoch
+        mapStatuses.clear()
+      }
+    }
+  }
+}
+
+private[spark] class MapOutputTrackerMaster extends MapOutputTracker {
+
+  // Cache a serialized version of the output statuses for each shuffle to send them out faster
+  private var cacheEpoch = epoch
+  private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
+
+  def registerShuffle(shuffleId: Int, numMaps: Int) {
+    if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) {
+      throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
+    }
+  }
+
+  def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
+    val array = mapStatuses(shuffleId)
+    array.synchronized {
+      array(mapId) = status
+    }
+  }
+
+  def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) {
+    mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses)
+    if (changeEpoch) {
+      incrementEpoch()
+    }
+  }
+
+  def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
+    val arrayOpt = mapStatuses.get(shuffleId)
+    if (arrayOpt.isDefined && arrayOpt.get != null) {
+      val array = arrayOpt.get
+      array.synchronized {
+        if (array(mapId) != null && array(mapId).location == bmAddress) {
+          array(mapId) = null
+        }
       }
+      incrementEpoch()
+    } else {
+      throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID")
     }
   }
 
-  def getSerializedLocations(shuffleId: Int): Array[Byte] = {
+  def incrementEpoch() {
+    epochLock.synchronized {
+      epoch += 1
+      logDebug("Increasing epoch to " + epoch)
+    }
+  }
+
+  def getSerializedMapOutputStatuses(shuffleId: Int): Array[Byte] = {
     var statuses: Array[MapStatus] = null
     var epochGotten: Long = -1
     epochLock.synchronized {
@@ -253,7 +250,7 @@ private[spark] class MapOutputTracker extends Logging {
     }
     // If we got here, we failed to find the serialized locations in the cache, so we pulled
     // out a snapshot of the locations as "locs"; let's serialize and return that
-    val bytes = serializeStatuses(statuses)
+    val bytes = MapOutputTracker.serializeMapStatuses(statuses)
     logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length))
     // Add them into the table only if the epoch hasn't changed while we were working
     epochLock.synchronized {
@@ -261,13 +258,31 @@ private[spark] class MapOutputTracker extends Logging {
         cachedSerializedStatuses(shuffleId) = bytes
       }
     }
-    return bytes
+    bytes
+  }
+
+  protected override def cleanup(cleanupTime: Long) {
+    super.cleanup(cleanupTime)
+    cachedSerializedStatuses.clearOldValues(cleanupTime)
   }
 
+  override def stop() {
+    super.stop()
+    cachedSerializedStatuses.clear()
+  }
+
+  override def updateEpoch(newEpoch: Long) {
+    // This might be called on the MapOutputTrackerMaster if we're running in local mode.
+  }
+}
+
+private[spark] object MapOutputTracker {
+  private val LOG_BASE = 1.1
+
   // Serialize an array of map output locations into an efficient byte format so that we can send
   // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will
   // generally be pretty compressible because many map outputs will be on the same hostname.
-  private def serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = {
+  def serializeMapStatuses(statuses: Array[MapStatus]): Array[Byte] = {
     val out = new ByteArrayOutputStream
     val objOut = new ObjectOutputStream(new GZIPOutputStream(out))
     // Since statuses can be modified in parallel, sync on it
@@ -278,18 +293,11 @@ private[spark] class MapOutputTracker extends Logging {
     out.toByteArray
   }
 
-  // Opposite of serializeStatuses.
-  def deserializeStatuses(bytes: Array[Byte]): Array[MapStatus] = {
+  // Opposite of serializeMapStatuses.
+  def deserializeMapStatuses(bytes: Array[Byte]): Array[MapStatus] = {
     val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes)))
-    objIn.readObject().
-      // // drop all null's from status - not sure why they are occuring though. Causes NPE downstream in slave if present
-      // comment this out - nulls could be due to missing location ? 
-      asInstanceOf[Array[MapStatus]] // .filter( _ != null )
+    objIn.readObject().asInstanceOf[Array[MapStatus]]
   }
-}
-
-private[spark] object MapOutputTracker {
-  private val LOG_BASE = 1.1
 
   // Convert an array of MapStatuses to locations and sizes for a given reduce ID. If
   // any of the statuses is null (indicating a missing location due to a failed mapper),
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index f674cb397f08b70126078f7fb299e43470c9ffc8..880b49e8efc1754f53216481ae42429b51a3794f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.Map
 import scala.collection.generic.Growable
-import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
@@ -51,25 +51,19 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
 
 import org.apache.mesos.MesosNativeLibrary
 
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.deploy.LocalSparkCluster
+import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
 import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
 import org.apache.spark.rdd._
 import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
-  ClusterScheduler}
-import org.apache.spark.scheduler.local.LocalScheduler
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend,
+  SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend}
 import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
-import org.apache.spark.storage.{StorageUtils, BlockManagerSource}
-import org.apache.spark.ui.SparkUI
-import org.apache.spark.util._
-import org.apache.spark.scheduler.StageInfo
-import org.apache.spark.storage.RDDInfo
-import org.apache.spark.storage.StorageStatus
-import scala.Some
+import org.apache.spark.scheduler.local.LocalScheduler
 import org.apache.spark.scheduler.StageInfo
-import org.apache.spark.storage.RDDInfo
-import org.apache.spark.storage.StorageStatus
+import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType,
+  TimeStampedHashMap, Utils}
 
 /**
  * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@@ -125,7 +119,7 @@ class SparkContext(
   private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
   private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup)
 
-  // Initalize the Spark UI
+  // Initialize the Spark UI
   private[spark] val ui = new SparkUI(this)
   ui.bind()
 
@@ -161,8 +155,10 @@ class SparkContext(
     val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
     // Regular expression for connecting to Spark deploy clusters
     val SPARK_REGEX = """spark://(.*)""".r
-    //Regular expression for connection to Mesos cluster
-    val MESOS_REGEX = """(mesos://.*)""".r
+    // Regular expression for connection to Mesos cluster
+    val MESOS_REGEX = """mesos://(.*)""".r
+    // Regular expression for connection to Simr cluster
+    val SIMR_REGEX = """simr://(.*)""".r
 
     master match {
       case "local" =>
@@ -181,6 +177,12 @@ class SparkContext(
         scheduler.initialize(backend)
         scheduler
 
+      case SIMR_REGEX(simrUrl) =>
+        val scheduler = new ClusterScheduler(this)
+        val backend = new SimrSchedulerBackend(scheduler, this, simrUrl)
+        scheduler.initialize(backend)
+        scheduler
+
       case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
         // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
         val memoryPerSlaveInt = memoryPerSlave.toInt
@@ -213,25 +215,24 @@ class SparkContext(
             throw new SparkException("YARN mode not available ?", th)
           }
         }
-        val backend = new StandaloneSchedulerBackend(scheduler, this.env.actorSystem)
+        val backend = new CoarseGrainedSchedulerBackend(scheduler, this.env.actorSystem)
         scheduler.initialize(backend)
         scheduler
 
-      case _ =>
-        if (MESOS_REGEX.findFirstIn(master).isEmpty) {
-          logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master))
-        }
+      case MESOS_REGEX(mesosUrl) =>
         MesosNativeLibrary.load()
         val scheduler = new ClusterScheduler(this)
         val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
-        val masterWithoutProtocol = master.replaceFirst("^mesos://", "")  // Strip initial mesos://
         val backend = if (coarseGrained) {
-          new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
+          new CoarseMesosSchedulerBackend(scheduler, this, mesosUrl, appName)
         } else {
-          new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
+          new MesosSchedulerBackend(scheduler, this, mesosUrl, appName)
         }
         scheduler.initialize(backend)
         scheduler
+
+      case _ =>
+        throw new SparkException("Could not parse Master URL: '" + master + "'")
     }
   }
   taskScheduler.start()
@@ -244,7 +245,7 @@ class SparkContext(
   /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
   val hadoopConfiguration = {
     val env = SparkEnv.get
-    val conf = env.hadoop.newConfiguration()
+    val conf = SparkHadoopUtil.get.newConfiguration()
     // Explicitly check for S3 environment variables
     if (System.getenv("AWS_ACCESS_KEY_ID") != null &&
         System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
@@ -254,8 +255,10 @@ class SparkContext(
       conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
     }
     // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
-    for (key <- System.getProperties.toMap[String, String].keys if key.startsWith("spark.hadoop.")) {
-      conf.set(key.substring("spark.hadoop.".length), System.getProperty(key))
+    Utils.getSystemProperties.foreach { case (key, value) =>
+      if (key.startsWith("spark.hadoop.")) {
+        conf.set(key.substring("spark.hadoop.".length), value)
+      }
     }
     val bufferSize = System.getProperty("spark.buffer.size", "65536")
     conf.set("io.file.buffer.size", bufferSize)
@@ -288,15 +291,46 @@ class SparkContext(
     Option(localProperties.get).map(_.getProperty(key)).getOrElse(null)
 
   /** Set a human readable description of the current job. */
+  @deprecated("use setJobGroup", "0.8.1")
   def setJobDescription(value: String) {
-    setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value)
+    setJobGroup("", value)
+  }
+
+  /**
+   * Assigns a group id to all the jobs started by this thread until the group id is set to a
+   * different value or cleared.
+   *
+   * Often, a unit of execution in an application consists of multiple Spark actions or jobs.
+   * Application programmers can use this method to group all those jobs together and give a
+   * group description. Once set, the Spark web UI will associate such jobs with this group.
+   *
+   * The application can also use [[org.apache.spark.SparkContext.cancelJobGroup]] to cancel all
+   * running jobs in this group. For example,
+   * {{{
+   * // In the main thread:
+   * sc.setJobGroup("some_job_to_cancel", "some job description")
+   * sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()
+   *
+   * // In a separate thread:
+   * sc.cancelJobGroup("some_job_to_cancel")
+   * }}}
+   */
+  def setJobGroup(groupId: String, description: String) {
+    setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, description)
+    setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId)
+  }
+
+  /** Clear the job group id and its description. */
+  def clearJobGroup() {
+    setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null)
+    setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null)
   }
 
   // Post init
   taskScheduler.postStartHook()
 
-  val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this)
-  val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this)
+  private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this)
+  private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this)
 
   def initDriverMetrics() {
     SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource)
@@ -347,7 +381,7 @@ class SparkContext(
       minSplits: Int = defaultMinSplits
       ): RDD[(K, V)] = {
     // Add necessary security credentials to the JobConf before broadcasting it.
-    SparkEnv.get.hadoop.addCredentials(conf)
+    SparkHadoopUtil.get.addCredentials(conf)
     new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
   }
 
@@ -557,7 +591,8 @@ class SparkContext(
     val uri = new URI(path)
     val key = uri.getScheme match {
       case null | "file" => env.httpFileServer.addFile(new File(uri.getPath))
-      case _ => path
+      case "local"       => "file:" + uri.getPath
+      case _             => path
     }
     addedFiles(key) = System.currentTimeMillis
 
@@ -651,12 +686,11 @@ class SparkContext(
   /**
    * 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.
+   * filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.
    */
   def addJar(path: String) {
     if (path == null) {
-      logWarning("null specified as parameter to addJar",
-        new SparkException("null specified as parameter to addJar"))
+      logWarning("null specified as parameter to addJar")
     } else {
       var key = ""
       if (path.contains("\\")) {
@@ -665,8 +699,9 @@ class SparkContext(
       } else {
         val uri = new URI(path)
         key = uri.getScheme match {
+          // A JAR file which exists only on the driver node
           case null | "file" =>
-            if (env.hadoop.isYarnMode()) {
+            if (SparkHadoopUtil.get.isYarnMode()) {
               // In order for this to work on yarn the user must specify the --addjars option to
               // the client to upload the file into the distributed cache to make it show up in the
               // current working directory.
@@ -682,6 +717,9 @@ class SparkContext(
             } else {
               env.httpFileServer.addJar(new File(uri.getPath))
             }
+          // A JAR file which exists locally on every worker node
+          case "local" =>
+            "file:" + uri.getPath
           case _ =>
             path
         }
@@ -867,13 +905,19 @@ class SparkContext(
       callSite,
       allowLocal = false,
       resultHandler,
-      null)
+      localProperties.get)
     new SimpleFutureAction(waiter, resultFunc)
   }
 
   /**
-   * Cancel all jobs that have been scheduled or are running.
+   * Cancel active jobs for the specified group. See [[org.apache.spark.SparkContext.setJobGroup]]
+   * for more information.
    */
+  def cancelJobGroup(groupId: String) {
+    dagScheduler.cancelJobGroup(groupId)
+  }
+
+  /** Cancel all jobs that have been scheduled or are running.  */
   def cancelAllJobs() {
     dagScheduler.cancelAllJobs()
   }
@@ -895,9 +939,8 @@ class SparkContext(
    * prevent accidental overriding of checkpoint files in the existing directory.
    */
   def setCheckpointDir(dir: String, useExisting: Boolean = false) {
-    val env = SparkEnv.get
     val path = new Path(dir)
-    val fs = path.getFileSystem(env.hadoop.newConfiguration())
+    val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
     if (!useExisting) {
       if (fs.exists(path)) {
         throw new Exception("Checkpoint directory '" + path + "' already exists.")
@@ -934,7 +977,10 @@ class SparkContext(
  * various Spark features.
  */
 object SparkContext {
-  val SPARK_JOB_DESCRIPTION = "spark.job.description"
+
+  private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"
+
+  private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"
 
   implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
     def addInPlace(t1: Double, t2: Double): Double = t1 + t2
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 29968c273c31d59e5b45e1b8237f860c9eb21bdf..ff2df8fb6a2fc7256e40b161f4e3a4bd054a3731 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -25,13 +25,13 @@ import akka.remote.RemoteActorRefProvider
 
 import org.apache.spark.broadcast.BroadcastManager
 import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster}
 import org.apache.spark.network.ConnectionManager
 import org.apache.spark.serializer.{Serializer, SerializerManager}
 import org.apache.spark.util.{Utils, AkkaUtils}
 import org.apache.spark.api.python.PythonWorkerFactory
 
+import com.google.common.collect.MapMaker
 
 /**
  * Holds all the runtime environment objects for a running Spark instance (either master or worker),
@@ -58,18 +58,9 @@ class SparkEnv (
 
   private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
 
-  val hadoop = {
-    val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
-    if(yarnMode) {
-      try {
-        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
-      } catch {
-        case th: Throwable => throw new SparkException("Unable to load YARN support", th)
-      }
-    } else {
-      new SparkHadoopUtil
-    }
-  }
+  // A general, soft-reference map for metadata needed during HadoopRDD split computation
+  // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
+  private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
 
   def stop() {
     pythonWorkers.foreach { case(key, worker) => worker.stop() }
@@ -187,10 +178,14 @@ object SparkEnv extends Logging {
 
     // Have to assign trackerActor after initialization as MapOutputTrackerActor
     // requires the MapOutputTracker itself
-    val mapOutputTracker = new MapOutputTracker()
+    val mapOutputTracker =  if (isDriver) {
+      new MapOutputTrackerMaster()
+    } else {
+      new MapOutputTracker()
+    }
     mapOutputTracker.trackerActor = registerOrLookup(
       "MapOutputTracker",
-      new MapOutputTrackerActor(mapOutputTracker))
+      new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]))
 
     val shuffleFetcher = instantiateClass[ShuffleFetcher](
       "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher")
diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index 2bab9d6e3d9280c2173fa2b6f2498c32d03bb6cc..103a1c205185f1effde97fbac116076221c298b8 100644
--- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -17,14 +17,14 @@
 
 package org.apache.hadoop.mapred
 
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-
+import java.io.IOException
 import java.text.SimpleDateFormat
 import java.text.NumberFormat
-import java.io.IOException
 import java.util.Date
 
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+
 import org.apache.spark.Logging
 import org.apache.spark.SerializableWritable
 
@@ -36,7 +36,11 @@ import org.apache.spark.SerializableWritable
  * Saves the RDD using a JobConf, which should contain an output key class, an output value class,
  * a filename to write to, etc, exactly like in a Hadoop MapReduce job.
  */
-class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable {
+private[apache]
+class SparkHadoopWriter(@transient jobConf: JobConf)
+  extends Logging
+  with SparkHadoopMapRedUtil
+  with Serializable {
 
   private val now = new Date()
   private val conf = new SerializableWritable(jobConf)
@@ -83,13 +87,11 @@ class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkH
     }
 
     getOutputCommitter().setupTask(getTaskContext()) 
-    writer = getOutputFormat().getRecordWriter(
-        fs, conf.value, outputName, Reporter.NULL)
+    writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL)
   }
 
   def write(key: AnyRef, value: AnyRef) {
-    if (writer!=null) {
-      //println (">>> Writing ("+key.toString+": " + key.getClass.toString + ", " + value.toString + ": " + value.getClass.toString + ")")
+    if (writer != null) {
       writer.write(key, value)
     } else {
       throw new IOException("Writer is null, open() has not been called")
@@ -179,6 +181,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkH
   }
 }
 
+private[apache]
 object SparkHadoopWriter {
   def createJobID(time: Date, id: Int): JobID = {
     val formatter = new SimpleDateFormat("yyyyMMddHHmm")
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index 51584d686d1d4d5cb6206465d7345a946c8864c9..cae983ed4c652cc936f6df4f704a04cd9b1cd2b6 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.executor.TaskMetrics
 
 class TaskContext(
-  private[spark] val stageId: Int,
+  val stageId: Int,
   val partitionId: Int,
   val attemptId: Long,
   val runningLocally: Boolean = false,
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
index 5fd1fab5809da1ddc9cf6d2162c73a9532a53cf5..043cb183bad179d864d92f11e68c397fc284bd29 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
@@ -48,6 +48,19 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
    */
   def persist(newLevel: StorageLevel): JavaDoubleRDD = fromRDD(srdd.persist(newLevel))
 
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   * This method blocks until all blocks are deleted.
+   */
+  def unpersist(): JavaDoubleRDD = fromRDD(srdd.unpersist())
+
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   *
+   * @param blocking Whether to block until all blocks are deleted.
+   */
+  def unpersist(blocking: Boolean): JavaDoubleRDD = fromRDD(srdd.unpersist(blocking))
+
   // first() has to be overriden here in order for its return type to be Double instead of Object.
   override def first(): Double = srdd.first()
 
@@ -80,6 +93,17 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
   def coalesce(numPartitions: Int, shuffle: Boolean): JavaDoubleRDD =
     fromRDD(srdd.coalesce(numPartitions, shuffle))
 
+  /**
+   * Return a new RDD that has exactly numPartitions partitions.
+   *
+   * Can increase or decrease the level of parallelism in this RDD. Internally, this uses
+   * a shuffle to redistribute data.
+   *
+   * If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
+   * which can avoid performing a shuffle.
+   */
+  def repartition(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.repartition(numPartitions))
+
   /**
    * Return an RDD with the elements from `this` that are not in `other`.
    * 
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index a6518abf456d3ec9d3da730713c2d71733a6fc14..2142fd73278aca1a2dd3a94d7cfb3508fd8877ca 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -65,6 +65,19 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   def persist(newLevel: StorageLevel): JavaPairRDD[K, V] =
     new JavaPairRDD[K, V](rdd.persist(newLevel))
 
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   * This method blocks until all blocks are deleted.
+   */
+  def unpersist(): JavaPairRDD[K, V] = wrapRDD(rdd.unpersist())
+
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   *
+   * @param blocking Whether to block until all blocks are deleted.
+   */
+  def unpersist(blocking: Boolean): JavaPairRDD[K, V] = wrapRDD(rdd.unpersist(blocking))
+
   // Transformations (return a new RDD)
 
   /**
@@ -94,6 +107,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   def coalesce(numPartitions: Int, shuffle: Boolean): JavaPairRDD[K, V] =
     fromRDD(rdd.coalesce(numPartitions, shuffle))
 
+  /**
+   * Return a new RDD that has exactly numPartitions partitions.
+   *
+   * Can increase or decrease the level of parallelism in this RDD. Internally, this uses
+   * a shuffle to redistribute data.
+   *
+   * If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
+   * which can avoid performing a shuffle.
+   */
+  def repartition(numPartitions: Int): JavaPairRDD[K, V] = fromRDD(rdd.repartition(numPartitions))
+
   /**
    * Return a sampled subset of this RDD.
    */
@@ -598,4 +622,15 @@ object JavaPairRDD {
     new JavaPairRDD[K, V](rdd)
 
   implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd
+
+
+  /** Convert a JavaRDD of key-value pairs to JavaPairRDD. */
+  def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = {
+    implicit val cmk: ClassManifest[K] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+    implicit val cmv: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    new JavaPairRDD[K, V](rdd.rdd)
+  }
+
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index eec58abdd608a462997b6101ac645d8c8be9cefe..3b359a8fd60941b8cbcdcceb843d1d52c8887842 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -41,9 +41,17 @@ JavaRDDLike[T, JavaRDD[T]] {
 
   /**
    * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   * This method blocks until all blocks are deleted.
    */
   def unpersist(): JavaRDD[T] = wrapRDD(rdd.unpersist())
 
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   *
+   * @param blocking Whether to block until all blocks are deleted.
+   */
+  def unpersist(blocking: Boolean): JavaRDD[T] = wrapRDD(rdd.unpersist(blocking))
+
   // Transformations (return a new RDD)
 
   /**
@@ -73,6 +81,17 @@ JavaRDDLike[T, JavaRDD[T]] {
   def coalesce(numPartitions: Int, shuffle: Boolean): JavaRDD[T] =
     rdd.coalesce(numPartitions, shuffle)
 
+  /**
+   * Return a new RDD that has exactly numPartitions partitions.
+   *
+   * Can increase or decrease the level of parallelism in this RDD. Internally, this uses
+   * a shuffle to redistribute data.
+   *
+   * If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
+   * which can avoid performing a shuffle.
+   */
+  def repartition(numPartitions: Int): JavaRDD[T] = rdd.repartition(numPartitions)
+
   /**
    * Return a sampled subset of this RDD.
    */
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
index 4830067f7a0db2a7b483c5c962483a876b0788ff..3e85052cd03e8323f208bed5d37c86979ca75a96 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
@@ -18,8 +18,6 @@
 package org.apache.spark.api.java.function;
 
 
-import scala.runtime.AbstractFunction1;
-
 import java.io.Serializable;
 
 /**
@@ -27,11 +25,7 @@ import java.io.Serializable;
  */
 // DoubleFlatMapFunction does not extend FlatMapFunction because flatMap is
 // overloaded for both FlatMapFunction and DoubleFlatMapFunction.
-public abstract class DoubleFlatMapFunction<T> extends AbstractFunction1<T, Iterable<Double>>
+public abstract class DoubleFlatMapFunction<T> extends WrappedFunction1<T, Iterable<Double>>
   implements Serializable {
-
-  public abstract Iterable<Double> call(T t);
-
-  @Override
-  public final Iterable<Double> apply(T t) { return call(t); }
+    // Intentionally left blank
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
index db34cd190ad3a583c73b0a1825f4118970855d03..5e9b8c48b89aa575d1800159dc5d5208500983cc 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
@@ -18,8 +18,6 @@
 package org.apache.spark.api.java.function;
 
 
-import scala.runtime.AbstractFunction1;
-
 import java.io.Serializable;
 
 /**
@@ -29,6 +27,5 @@ import java.io.Serializable;
 // are overloaded for both Function and DoubleFunction.
 public abstract class DoubleFunction<T> extends WrappedFunction1<T, Double>
   implements Serializable {
-
-  public abstract Double call(T t) throws Exception;
+    // Intentionally left blank
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
index 158539a8461f94aa85ea6440786d50d461436825..2dfda8b09a88068b2af3db5ff62ad5a589f8684d 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
@@ -21,8 +21,5 @@ package org.apache.spark.api.java.function
  * A function that returns zero or more output records from each input record.
  */
 abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] {
-  @throws(classOf[Exception])
-  def call(x: T) : java.lang.Iterable[R]
-
   def elementType() : ClassManifest[R] = ClassManifest.Any.asInstanceOf[ClassManifest[R]]
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
index 5ef6a814f5a4f2aa347dc1f7487b7011308bcfce..528e1c0a7cd648f405d472804a6557fe15030eeb 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
@@ -21,8 +21,5 @@ package org.apache.spark.api.java.function
  * A function that takes two inputs and returns zero or more output records.
  */
 abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] {
-  @throws(classOf[Exception])
-  def call(a: A, b:B) : java.lang.Iterable[C]
-
   def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]]
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
index b9070cfd837eb68743042ae79500ced87815eb4c..ce368ee01b295577d8e288681617215caf4997ef 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
@@ -19,7 +19,6 @@ package org.apache.spark.api.java.function;
 
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
-import scala.runtime.AbstractFunction1;
 
 import java.io.Serializable;
 
@@ -30,8 +29,6 @@ import java.io.Serializable;
  * when mapping RDDs of other types.
  */
 public abstract class Function<T, R> extends WrappedFunction1<T, R> implements Serializable {
-  public abstract R call(T t) throws Exception;
-
   public ClassManifest<R> returnType() {
     return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
   }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
index d4c9154869fe9ac14a4432951ca8cb4a490b5eb2..44ad559d48474c364981212d707fe1a40dae5081 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
@@ -19,7 +19,6 @@ package org.apache.spark.api.java.function;
 
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
-import scala.runtime.AbstractFunction2;
 
 import java.io.Serializable;
 
@@ -29,8 +28,6 @@ import java.io.Serializable;
 public abstract class Function2<T1, T2, R> extends WrappedFunction2<T1, T2, R>
   implements Serializable {
 
-  public abstract R call(T1 t1, T2 t2) throws Exception;
-
   public ClassManifest<R> returnType() {
     return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
   }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
new file mode 100644
index 0000000000000000000000000000000000000000..ac6178924a2bfc7a11c78da08ac4e765dd97b06b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api.java.function;
+
+import scala.reflect.ClassManifest;
+import scala.reflect.ClassManifest$;
+import scala.runtime.AbstractFunction2;
+
+import java.io.Serializable;
+
+/**
+ * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R.
+ */
+public abstract class Function3<T1, T2, T3, R> extends WrappedFunction3<T1, T2, T3, R>
+        implements Serializable {
+
+    public ClassManifest<R> returnType() {
+        return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+    }
+}
+
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
index c0e5544b7dff0d66576243d1e422bf4e57ee15dc..6d76a8f970f7c2e812337f26796df8c0ee5afc68 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
@@ -20,7 +20,6 @@ package org.apache.spark.api.java.function;
 import scala.Tuple2;
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
-import scala.runtime.AbstractFunction1;
 
 import java.io.Serializable;
 
@@ -34,8 +33,6 @@ public abstract class PairFlatMapFunction<T, K, V>
   extends WrappedFunction1<T, Iterable<Tuple2<K, V>>>
   implements Serializable {
 
-  public abstract Iterable<Tuple2<K, V>> call(T t) throws Exception;
-
   public ClassManifest<K> keyType() {
     return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
   }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
index 40480fe8e816055494be1b8138a9bb487b357326..ede7ceefb5d297edfaf63ed365a0a1a3f78a4466 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
@@ -20,7 +20,6 @@ package org.apache.spark.api.java.function;
 import scala.Tuple2;
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
-import scala.runtime.AbstractFunction1;
 
 import java.io.Serializable;
 
@@ -29,12 +28,9 @@ import java.io.Serializable;
  */
 // PairFunction does not extend Function because some UDF functions, like map,
 // are overloaded for both Function and PairFunction.
-public abstract class PairFunction<T, K, V>
-  extends WrappedFunction1<T, Tuple2<K, V>>
+public abstract class PairFunction<T, K, V> extends WrappedFunction1<T, Tuple2<K, V>>
   implements Serializable {
 
-  public abstract Tuple2<K, V> call(T t) throws Exception;
-
   public ClassManifest<K> keyType() {
     return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
   }
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala
new file mode 100644
index 0000000000000000000000000000000000000000..d314dbdf1d980c82f4be6263bcce13746eb4a7aa
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api.java.function
+
+import scala.runtime.AbstractFunction3
+
+/**
+ * Subclass of Function3 for ease of calling from Java. The main thing it does is re-expose the
+ * apply() method as call() and declare that it can throw Exception (since AbstractFunction3.apply
+ * isn't marked to allow that).
+ */
+private[spark] abstract class WrappedFunction3[T1, T2, T3, R]
+  extends AbstractFunction3[T1, T2, T3, R] {
+  @throws(classOf[Exception])
+  def call(t1: T1, t2: T2, t3: T3): R
+
+  final def apply(t1: T1, t2: T2, t3: T3): R = call(t1, t2, t3)
+}
+
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 1f8ad688a649dec0d14228e4db97c600d1877e7f..12b4d94a567ceec4713299fa49a54349b08c0cef 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -308,7 +308,7 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[
  * Internal class that acts as an `AccumulatorParam` for Python accumulators. Inside, it
  * collects a list of pickled strings that we pass to Python through a socket.
  */
-class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int)
+private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int)
   extends AccumulatorParam[JList[Array[Byte]]] {
 
   Utils.checkHost(serverHost, "Expected hostname")
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
deleted file mode 100644
index b6c484bfe1967dbe9d99cacf87a00c796110d402..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
+++ /dev/null
@@ -1,1058 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.broadcast
-
-import java.io._
-import java.net._
-import java.util.{BitSet, Comparator, Timer, TimerTask, UUID}
-import java.util.concurrent.atomic.AtomicInteger
-
-import scala.collection.mutable.{ListBuffer, Map, Set}
-import scala.math
-
-import org.apache.spark._
-import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
-import org.apache.spark.util.Utils
-
-private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
-  extends Broadcast[T](id)
-  with Logging
-  with Serializable {
-
-  def value = value_
-
-  def blockId = BroadcastBlockId(id)
-
-  MultiTracker.synchronized {
-    SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
-  }
-
-  @transient var arrayOfBlocks: Array[BroadcastBlock] = null
-  @transient var hasBlocksBitVector: BitSet = null
-  @transient var numCopiesSent: Array[Int] = null
-  @transient var totalBytes = -1
-  @transient var totalBlocks = -1
-  @transient var hasBlocks = new AtomicInteger(0)
-
-  // Used ONLY by driver to track how many unique blocks have been sent out
-  @transient var sentBlocks = new AtomicInteger(0)
-
-  @transient var listenPortLock = new Object
-  @transient var guidePortLock = new Object
-  @transient var totalBlocksLock = new Object
-
-  @transient var listOfSources = ListBuffer[SourceInfo]()
-
-  @transient var serveMR: ServeMultipleRequests = null
-
-  // Used only in driver
-  @transient var guideMR: GuideMultipleRequests = null
-
-  // Used only in Workers
-  @transient var ttGuide: TalkToGuide = null
-
-  @transient var hostAddress = Utils.localIpAddress
-  @transient var listenPort = -1
-  @transient var guidePort = -1
-
-  @transient var stopBroadcast = false
-
-  // Must call this after all the variables have been created/initialized
-  if (!isLocal) {
-    sendBroadcast()
-  }
-
-  def sendBroadcast() {
-    logInfo("Local host address: " + hostAddress)
-
-    // Create a variableInfo object and store it in valueInfos
-    var variableInfo = MultiTracker.blockifyObject(value_)
-
-    // Prepare the value being broadcasted
-    arrayOfBlocks = variableInfo.arrayOfBlocks
-    totalBytes = variableInfo.totalBytes
-    totalBlocks = variableInfo.totalBlocks
-    hasBlocks.set(variableInfo.totalBlocks)
-
-    // Guide has all the blocks
-    hasBlocksBitVector = new BitSet(totalBlocks)
-    hasBlocksBitVector.set(0, totalBlocks)
-
-    // Guide still hasn't sent any block
-    numCopiesSent = new Array[Int](totalBlocks)
-
-    guideMR = new GuideMultipleRequests
-    guideMR.setDaemon(true)
-    guideMR.start()
-    logInfo("GuideMultipleRequests started...")
-
-    // Must always come AFTER guideMR is created
-    while (guidePort == -1) {
-      guidePortLock.synchronized { guidePortLock.wait() }
-    }
-
-    serveMR = new ServeMultipleRequests
-    serveMR.setDaemon(true)
-    serveMR.start()
-    logInfo("ServeMultipleRequests started...")
-
-    // Must always come AFTER serveMR is created
-    while (listenPort == -1) {
-      listenPortLock.synchronized { listenPortLock.wait() }
-    }
-
-    // Must always come AFTER listenPort is created
-    val driverSource =
-      SourceInfo(hostAddress, listenPort, totalBlocks, totalBytes)
-    hasBlocksBitVector.synchronized {
-      driverSource.hasBlocksBitVector = hasBlocksBitVector
-    }
-
-    // In the beginning, this is the only known source to Guide
-    listOfSources += driverSource
-
-    // Register with the Tracker
-    MultiTracker.registerBroadcast(id,
-      SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes))
-  }
-
-  private def readObject(in: ObjectInputStream) {
-    in.defaultReadObject()
-    MultiTracker.synchronized {
-      SparkEnv.get.blockManager.getSingle(blockId) match {
-        case Some(x) =>
-          value_ = x.asInstanceOf[T]
-
-        case None =>
-          logInfo("Started reading broadcast variable " + id)
-          // Initializing everything because driver will only send null/0 values
-          // Only the 1st worker in a node can be here. Others will get from cache
-          initializeWorkerVariables()
-
-          logInfo("Local host address: " + hostAddress)
-
-          // Start local ServeMultipleRequests thread first
-          serveMR = new ServeMultipleRequests
-          serveMR.setDaemon(true)
-          serveMR.start()
-          logInfo("ServeMultipleRequests started...")
-
-          val start = System.nanoTime
-
-          val receptionSucceeded = receiveBroadcast(id)
-          if (receptionSucceeded) {
-            value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
-            SparkEnv.get.blockManager.putSingle(
-              blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
-          }  else {
-            logError("Reading broadcast variable " + id + " failed")
-          }
-
-          val time = (System.nanoTime - start) / 1e9
-          logInfo("Reading broadcast variable " + id + " took " + time + " s")
-      }
-    }
-  }
-
-  // Initialize variables in the worker node. Driver sends everything as 0/null
-  private def initializeWorkerVariables() {
-    arrayOfBlocks = null
-    hasBlocksBitVector = null
-    numCopiesSent = null
-    totalBytes = -1
-    totalBlocks = -1
-    hasBlocks = new AtomicInteger(0)
-
-    listenPortLock = new Object
-    totalBlocksLock = new Object
-
-    serveMR = null
-    ttGuide = null
-
-    hostAddress = Utils.localIpAddress
-    listenPort = -1
-
-    listOfSources = ListBuffer[SourceInfo]()
-
-    stopBroadcast = false
-  }
-
-  private def getLocalSourceInfo: SourceInfo = {
-    // Wait till hostName and listenPort are OK
-    while (listenPort == -1) {
-      listenPortLock.synchronized { listenPortLock.wait() }
-    }
-
-    // Wait till totalBlocks and totalBytes are OK
-    while (totalBlocks == -1) {
-      totalBlocksLock.synchronized { totalBlocksLock.wait() }
-    }
-
-    var localSourceInfo = SourceInfo(
-      hostAddress, listenPort, totalBlocks, totalBytes)
-
-    localSourceInfo.hasBlocks = hasBlocks.get
-
-    hasBlocksBitVector.synchronized {
-      localSourceInfo.hasBlocksBitVector = hasBlocksBitVector
-    }
-
-    return localSourceInfo
-  }
-
-  // Add new SourceInfo to the listOfSources. Update if it exists already.
-  // Optimizing just by OR-ing the BitVectors was BAD for performance
-  private def addToListOfSources(newSourceInfo: SourceInfo) {
-    listOfSources.synchronized {
-      if (listOfSources.contains(newSourceInfo)) {
-        listOfSources = listOfSources - newSourceInfo
-      }
-      listOfSources += newSourceInfo
-    }
-  }
-
-  private def addToListOfSources(newSourceInfos: ListBuffer[SourceInfo]) {
-    newSourceInfos.foreach { newSourceInfo =>
-      addToListOfSources(newSourceInfo)
-    }
-  }
-
-  class TalkToGuide(gInfo: SourceInfo)
-  extends Thread with Logging {
-    override def run() {
-
-      // Keep exchaning information until all blocks have been received
-      while (hasBlocks.get < totalBlocks) {
-        talkOnce
-        Thread.sleep(MultiTracker.ranGen.nextInt(
-          MultiTracker.MaxKnockInterval - MultiTracker.MinKnockInterval) +
-          MultiTracker.MinKnockInterval)
-      }
-
-      // Talk one more time to let the Guide know of reception completion
-      talkOnce
-    }
-
-    // Connect to Guide and send this worker's information
-    private def talkOnce {
-      var clientSocketToGuide: Socket = null
-      var oosGuide: ObjectOutputStream = null
-      var oisGuide: ObjectInputStream = null
-
-      clientSocketToGuide = new Socket(gInfo.hostAddress, gInfo.listenPort)
-      oosGuide = new ObjectOutputStream(clientSocketToGuide.getOutputStream)
-      oosGuide.flush()
-      oisGuide = new ObjectInputStream(clientSocketToGuide.getInputStream)
-
-      // Send local information
-      oosGuide.writeObject(getLocalSourceInfo)
-      oosGuide.flush()
-
-      // Receive source information from Guide
-      var suitableSources =
-        oisGuide.readObject.asInstanceOf[ListBuffer[SourceInfo]]
-      logDebug("Received suitableSources from Driver " + suitableSources)
-
-      addToListOfSources(suitableSources)
-
-      oisGuide.close()
-      oosGuide.close()
-      clientSocketToGuide.close()
-    }
-  }
-
-  def receiveBroadcast(variableID: Long): Boolean = {
-    val gInfo = MultiTracker.getGuideInfo(variableID)
-
-    if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) {
-      return false
-    }
-
-    // Wait until hostAddress and listenPort are created by the
-    // ServeMultipleRequests thread
-    while (listenPort == -1) {
-      listenPortLock.synchronized { listenPortLock.wait() }
-    }
-
-    // Setup initial states of variables
-    totalBlocks = gInfo.totalBlocks
-    arrayOfBlocks = new Array[BroadcastBlock](totalBlocks)
-    hasBlocksBitVector = new BitSet(totalBlocks)
-    numCopiesSent = new Array[Int](totalBlocks)
-    totalBlocksLock.synchronized { totalBlocksLock.notifyAll() }
-    totalBytes = gInfo.totalBytes
-
-    // Start ttGuide to periodically talk to the Guide
-    var ttGuide = new TalkToGuide(gInfo)
-    ttGuide.setDaemon(true)
-    ttGuide.start()
-    logInfo("TalkToGuide started...")
-
-    // Start pController to run TalkToPeer threads
-    var pcController = new PeerChatterController
-    pcController.setDaemon(true)
-    pcController.start()
-    logInfo("PeerChatterController started...")
-
-    // FIXME: Must fix this. This might never break if broadcast fails.
-    // We should be able to break and send false. Also need to kill threads
-    while (hasBlocks.get < totalBlocks) {
-      Thread.sleep(MultiTracker.MaxKnockInterval)
-    }
-
-    return true
-  }
-
-  class PeerChatterController
-  extends Thread with Logging {
-    private var peersNowTalking = ListBuffer[SourceInfo]()
-    // TODO: There is a possible bug with blocksInRequestBitVector when a
-    // certain bit is NOT unset upon failure resulting in an infinite loop.
-    private var blocksInRequestBitVector = new BitSet(totalBlocks)
-
-    override def run() {
-      var threadPool = Utils.newDaemonFixedThreadPool(MultiTracker.MaxChatSlots)
-
-      while (hasBlocks.get < totalBlocks) {
-        var numThreadsToCreate = 0
-        listOfSources.synchronized {
-          numThreadsToCreate = math.min(listOfSources.size, MultiTracker.MaxChatSlots) -
-          threadPool.getActiveCount
-        }
-
-        while (hasBlocks.get < totalBlocks && numThreadsToCreate > 0) {
-          var peerToTalkTo = pickPeerToTalkToRandom
-
-          if (peerToTalkTo != null)
-            logDebug("Peer chosen: " + peerToTalkTo + " with " + peerToTalkTo.hasBlocksBitVector)
-          else
-            logDebug("No peer chosen...")
-
-          if (peerToTalkTo != null) {
-            threadPool.execute(new TalkToPeer(peerToTalkTo))
-
-            // Add to peersNowTalking. Remove in the thread. We have to do this
-            // ASAP, otherwise pickPeerToTalkTo picks the same peer more than once
-            peersNowTalking.synchronized { peersNowTalking += peerToTalkTo }
-          }
-
-          numThreadsToCreate = numThreadsToCreate - 1
-        }
-
-        // Sleep for a while before starting some more threads
-        Thread.sleep(MultiTracker.MinKnockInterval)
-      }
-      // Shutdown the thread pool
-      threadPool.shutdown()
-    }
-
-    // Right now picking the one that has the most blocks this peer wants
-    // Also picking peer randomly if no one has anything interesting
-    private def pickPeerToTalkToRandom: SourceInfo = {
-      var curPeer: SourceInfo = null
-      var curMax = 0
-
-      logDebug("Picking peers to talk to...")
-
-      // Find peers that are not connected right now
-      var peersNotInUse = ListBuffer[SourceInfo]()
-      listOfSources.synchronized {
-        peersNowTalking.synchronized {
-          peersNotInUse = listOfSources -- peersNowTalking
-        }
-      }
-
-      // Select the peer that has the most blocks that this receiver does not
-      peersNotInUse.foreach { eachSource =>
-        var tempHasBlocksBitVector: BitSet = null
-        hasBlocksBitVector.synchronized {
-          tempHasBlocksBitVector = hasBlocksBitVector.clone.asInstanceOf[BitSet]
-        }
-        tempHasBlocksBitVector.flip(0, tempHasBlocksBitVector.size)
-        tempHasBlocksBitVector.and(eachSource.hasBlocksBitVector)
-
-        if (tempHasBlocksBitVector.cardinality > curMax) {
-          curPeer = eachSource
-          curMax = tempHasBlocksBitVector.cardinality
-        }
-      }
-
-      // Always picking randomly
-      if (curPeer == null && peersNotInUse.size > 0) {
-        // Pick uniformly the i'th required peer
-        var i = MultiTracker.ranGen.nextInt(peersNotInUse.size)
-
-        var peerIter = peersNotInUse.iterator
-        curPeer = peerIter.next
-
-        while (i > 0) {
-          curPeer = peerIter.next
-          i = i - 1
-        }
-      }
-
-      return curPeer
-    }
-
-    // Picking peer with the weight of rare blocks it has
-    private def pickPeerToTalkToRarestFirst: SourceInfo = {
-      // Find peers that are not connected right now
-      var peersNotInUse = ListBuffer[SourceInfo]()
-      listOfSources.synchronized {
-        peersNowTalking.synchronized {
-          peersNotInUse = listOfSources -- peersNowTalking
-        }
-      }
-
-      // Count the number of copies of each block in the neighborhood
-      var numCopiesPerBlock = Array.tabulate [Int](totalBlocks)(_ => 0)
-
-      listOfSources.synchronized {
-        listOfSources.foreach { eachSource =>
-          for (i <- 0 until totalBlocks) {
-            numCopiesPerBlock(i) +=
-              ( if (eachSource.hasBlocksBitVector.get(i)) 1 else 0 )
-          }
-        }
-      }
-
-      // A block is considered rare if there are at most 2 copies of that block
-      // This CONSTANT could be a function of the neighborhood size
-      var rareBlocksIndices = ListBuffer[Int]()
-      for (i <- 0 until totalBlocks) {
-        if (numCopiesPerBlock(i) > 0 && numCopiesPerBlock(i) <= 2) {
-          rareBlocksIndices += i
-        }
-      }
-
-      // Find peers with rare blocks
-      var peersWithRareBlocks = ListBuffer[(SourceInfo, Int)]()
-      var totalRareBlocks = 0
-
-      peersNotInUse.foreach { eachPeer =>
-        var hasRareBlocks = 0
-        rareBlocksIndices.foreach { rareBlock =>
-          if (eachPeer.hasBlocksBitVector.get(rareBlock)) {
-            hasRareBlocks += 1
-          }
-        }
-
-        if (hasRareBlocks > 0) {
-          peersWithRareBlocks += ((eachPeer, hasRareBlocks))
-        }
-        totalRareBlocks += hasRareBlocks
-      }
-
-      // Select a peer from peersWithRareBlocks based on weight calculated from
-      // unique rare blocks
-      var selectedPeerToTalkTo: SourceInfo = null
-
-      if (peersWithRareBlocks.size > 0) {
-        // Sort the peers based on how many rare blocks they have
-        peersWithRareBlocks.sortBy(_._2)
-
-        var randomNumber = MultiTracker.ranGen.nextDouble
-        var tempSum = 0.0
-
-        var i = 0
-        do {
-          tempSum += (1.0 * peersWithRareBlocks(i)._2 / totalRareBlocks)
-          if (tempSum >= randomNumber) {
-            selectedPeerToTalkTo = peersWithRareBlocks(i)._1
-          }
-          i += 1
-        } while (i < peersWithRareBlocks.size && selectedPeerToTalkTo == null)
-      }
-
-      if (selectedPeerToTalkTo == null) {
-        selectedPeerToTalkTo = pickPeerToTalkToRandom
-      }
-
-      return selectedPeerToTalkTo
-    }
-
-    class TalkToPeer(peerToTalkTo: SourceInfo)
-    extends Thread with Logging {
-      private var peerSocketToSource: Socket = null
-      private var oosSource: ObjectOutputStream = null
-      private var oisSource: ObjectInputStream = null
-
-      override def run() {
-        // TODO: There is a possible bug here regarding blocksInRequestBitVector
-        var blockToAskFor = -1
-
-        // Setup the timeout mechanism
-        var timeOutTask = new TimerTask {
-          override def run() {
-            cleanUpConnections()
-          }
-        }
-
-        var timeOutTimer = new Timer
-        timeOutTimer.schedule(timeOutTask, MultiTracker.MaxKnockInterval)
-
-        logInfo("TalkToPeer started... => " + peerToTalkTo)
-
-        try {
-          // Connect to the source
-          peerSocketToSource =
-            new Socket(peerToTalkTo.hostAddress, peerToTalkTo.listenPort)
-          oosSource =
-            new ObjectOutputStream(peerSocketToSource.getOutputStream)
-          oosSource.flush()
-          oisSource =
-            new ObjectInputStream(peerSocketToSource.getInputStream)
-
-          // Receive latest SourceInfo from peerToTalkTo
-          var newPeerToTalkTo = oisSource.readObject.asInstanceOf[SourceInfo]
-          // Update listOfSources
-          addToListOfSources(newPeerToTalkTo)
-
-          // Turn the timer OFF, if the sender responds before timeout
-          timeOutTimer.cancel()
-
-          // Send the latest SourceInfo
-          oosSource.writeObject(getLocalSourceInfo)
-          oosSource.flush()
-
-          var keepReceiving = true
-
-          while (hasBlocks.get < totalBlocks && keepReceiving) {
-            blockToAskFor =
-              pickBlockRandom(newPeerToTalkTo.hasBlocksBitVector)
-
-            // No block to request
-            if (blockToAskFor < 0) {
-              // Nothing to receive from newPeerToTalkTo
-              keepReceiving = false
-            } else {
-              // Let other threads know that blockToAskFor is being requested
-              blocksInRequestBitVector.synchronized {
-                blocksInRequestBitVector.set(blockToAskFor)
-              }
-
-              // Start with sending the blockID
-              oosSource.writeObject(blockToAskFor)
-              oosSource.flush()
-
-              // CHANGED: Driver might send some other block than the one
-              // requested to ensure fast spreading of all blocks.
-              val recvStartTime = System.currentTimeMillis
-              val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock]
-              val receptionTime = (System.currentTimeMillis - recvStartTime)
-
-              logDebug("Received block: " + bcBlock.blockID + " from " + peerToTalkTo + " in " + receptionTime + " millis.")
-
-              if (!hasBlocksBitVector.get(bcBlock.blockID)) {
-                arrayOfBlocks(bcBlock.blockID) = bcBlock
-
-                // Update the hasBlocksBitVector first
-                hasBlocksBitVector.synchronized {
-                  hasBlocksBitVector.set(bcBlock.blockID)
-                  hasBlocks.getAndIncrement
-                }
-
-                // Some block(may NOT be blockToAskFor) has arrived.
-                // In any case, blockToAskFor is not in request any more
-                blocksInRequestBitVector.synchronized {
-                  blocksInRequestBitVector.set(blockToAskFor, false)
-                }
-
-                // Reset blockToAskFor to -1. Else it will be considered missing
-                blockToAskFor = -1
-              }
-
-              // Send the latest SourceInfo
-              oosSource.writeObject(getLocalSourceInfo)
-              oosSource.flush()
-            }
-          }
-        } catch {
-          // EOFException is expected to happen because sender can break
-          // connection due to timeout
-          case eofe: java.io.EOFException => { }
-          case e: Exception => {
-            logError("TalktoPeer had a " + e)
-            // FIXME: Remove 'newPeerToTalkTo' from listOfSources
-            // We probably should have the following in some form, but not
-            // really here. This exception can happen if the sender just breaks connection
-            // listOfSources.synchronized {
-              // logInfo("Exception in TalkToPeer. Removing source: " + peerToTalkTo)
-              // listOfSources = listOfSources - peerToTalkTo
-            // }
-          }
-        } finally {
-          // blockToAskFor != -1 => there was an exception
-          if (blockToAskFor != -1) {
-            blocksInRequestBitVector.synchronized {
-              blocksInRequestBitVector.set(blockToAskFor, false)
-            }
-          }
-
-          cleanUpConnections()
-        }
-      }
-
-      // Right now it picks a block uniformly that this peer does not have
-      private def pickBlockRandom(txHasBlocksBitVector: BitSet): Int = {
-        var needBlocksBitVector: BitSet = null
-
-        // Blocks already present
-        hasBlocksBitVector.synchronized {
-          needBlocksBitVector = hasBlocksBitVector.clone.asInstanceOf[BitSet]
-        }
-
-        // Include blocks already in transmission ONLY IF
-        // MultiTracker.EndGameFraction has NOT been achieved
-        if ((1.0 * hasBlocks.get / totalBlocks) < MultiTracker.EndGameFraction) {
-          blocksInRequestBitVector.synchronized {
-            needBlocksBitVector.or(blocksInRequestBitVector)
-          }
-        }
-
-        // Find blocks that are neither here nor in transit
-        needBlocksBitVector.flip(0, needBlocksBitVector.size)
-
-        // Blocks that should/can be requested
-        needBlocksBitVector.and(txHasBlocksBitVector)
-
-        if (needBlocksBitVector.cardinality == 0) {
-          return -1
-        } else {
-          // Pick uniformly the i'th required block
-          var i = MultiTracker.ranGen.nextInt(needBlocksBitVector.cardinality)
-          var pickedBlockIndex = needBlocksBitVector.nextSetBit(0)
-
-          while (i > 0) {
-            pickedBlockIndex =
-              needBlocksBitVector.nextSetBit(pickedBlockIndex + 1)
-            i -= 1
-          }
-
-          return pickedBlockIndex
-        }
-      }
-
-      // Pick the block that seems to be the rarest across sources
-      private def pickBlockRarestFirst(txHasBlocksBitVector: BitSet): Int = {
-        var needBlocksBitVector: BitSet = null
-
-        // Blocks already present
-        hasBlocksBitVector.synchronized {
-          needBlocksBitVector = hasBlocksBitVector.clone.asInstanceOf[BitSet]
-        }
-
-        // Include blocks already in transmission ONLY IF
-        // MultiTracker.EndGameFraction has NOT been achieved
-        if ((1.0 * hasBlocks.get / totalBlocks) < MultiTracker.EndGameFraction) {
-          blocksInRequestBitVector.synchronized {
-            needBlocksBitVector.or(blocksInRequestBitVector)
-          }
-        }
-
-        // Find blocks that are neither here nor in transit
-        needBlocksBitVector.flip(0, needBlocksBitVector.size)
-
-        // Blocks that should/can be requested
-        needBlocksBitVector.and(txHasBlocksBitVector)
-
-        if (needBlocksBitVector.cardinality == 0) {
-          return -1
-        } else {
-          // Count the number of copies for each block across all sources
-          var numCopiesPerBlock = Array.tabulate [Int](totalBlocks)(_ => 0)
-
-          listOfSources.synchronized {
-            listOfSources.foreach { eachSource =>
-              for (i <- 0 until totalBlocks) {
-                numCopiesPerBlock(i) +=
-                  ( if (eachSource.hasBlocksBitVector.get(i)) 1 else 0 )
-              }
-            }
-          }
-
-          // Find the minimum
-          var minVal = Integer.MAX_VALUE
-          for (i <- 0 until totalBlocks) {
-            if (numCopiesPerBlock(i) > 0 && numCopiesPerBlock(i) < minVal) {
-              minVal = numCopiesPerBlock(i)
-            }
-          }
-
-          // Find the blocks with the least copies that this peer does not have
-          var minBlocksIndices = ListBuffer[Int]()
-          for (i <- 0 until totalBlocks) {
-            if (needBlocksBitVector.get(i) && numCopiesPerBlock(i) == minVal) {
-              minBlocksIndices += i
-            }
-          }
-
-          // Now select a random index from minBlocksIndices
-          if (minBlocksIndices.size == 0) {
-            return -1
-          } else {
-            // Pick uniformly the i'th index
-            var i = MultiTracker.ranGen.nextInt(minBlocksIndices.size)
-            return minBlocksIndices(i)
-          }
-        }
-      }
-
-      private def cleanUpConnections() {
-        if (oisSource != null) {
-          oisSource.close()
-        }
-        if (oosSource != null) {
-          oosSource.close()
-        }
-        if (peerSocketToSource != null) {
-          peerSocketToSource.close()
-        }
-
-        // Delete from peersNowTalking
-        peersNowTalking.synchronized { peersNowTalking -= peerToTalkTo }
-      }
-    }
-  }
-
-  class GuideMultipleRequests
-  extends Thread with Logging {
-    // Keep track of sources that have completed reception
-    private var setOfCompletedSources = Set[SourceInfo]()
-
-    override def run() {
-      var threadPool = Utils.newDaemonCachedThreadPool()
-      var serverSocket: ServerSocket = null
-
-      serverSocket = new ServerSocket(0)
-      guidePort = serverSocket.getLocalPort
-      logInfo("GuideMultipleRequests => " + serverSocket + " " + guidePort)
-
-      guidePortLock.synchronized { guidePortLock.notifyAll() }
-
-      try {
-        while (!stopBroadcast) {
-          var clientSocket: Socket = null
-          try {
-            serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
-            clientSocket = serverSocket.accept()
-          } catch {
-            case e: Exception => {
-              // Stop broadcast if at least one worker has connected and
-              // everyone connected so far are done. Comparing with
-              // listOfSources.size - 1, because it includes the Guide itself
-              listOfSources.synchronized {
-                setOfCompletedSources.synchronized {
-                  if (listOfSources.size > 1 &&
-                    setOfCompletedSources.size == listOfSources.size - 1) {
-                    stopBroadcast = true
-                    logInfo("GuideMultipleRequests Timeout. stopBroadcast == true.")
-                  }
-                }
-              }
-            }
-          }
-          if (clientSocket != null) {
-            logDebug("Guide: Accepted new client connection:" + clientSocket)
-            try {
-              threadPool.execute(new GuideSingleRequest(clientSocket))
-            } catch {
-              // In failure, close the socket here; else, thread will close it
-              case ioe: IOException => {
-                clientSocket.close()
-              }
-            }
-          }
-        }
-
-        // Shutdown the thread pool
-        threadPool.shutdown()
-
-        logInfo("Sending stopBroadcast notifications...")
-        sendStopBroadcastNotifications
-
-        MultiTracker.unregisterBroadcast(id)
-      } finally {
-        if (serverSocket != null) {
-          logInfo("GuideMultipleRequests now stopping...")
-          serverSocket.close()
-        }
-      }
-    }
-
-    private def sendStopBroadcastNotifications() {
-      listOfSources.synchronized {
-        listOfSources.foreach { sourceInfo =>
-
-          var guideSocketToSource: Socket = null
-          var gosSource: ObjectOutputStream = null
-          var gisSource: ObjectInputStream = null
-
-          try {
-            // Connect to the source
-            guideSocketToSource = new Socket(sourceInfo.hostAddress, sourceInfo.listenPort)
-            gosSource = new ObjectOutputStream(guideSocketToSource.getOutputStream)
-            gosSource.flush()
-            gisSource = new ObjectInputStream(guideSocketToSource.getInputStream)
-
-            // Throw away whatever comes in
-            gisSource.readObject.asInstanceOf[SourceInfo]
-
-            // Send stopBroadcast signal. listenPort = SourceInfo.StopBroadcast
-            gosSource.writeObject(SourceInfo("", SourceInfo.StopBroadcast))
-            gosSource.flush()
-          } catch {
-            case e: Exception => {
-              logError("sendStopBroadcastNotifications had a " + e)
-            }
-          } finally {
-            if (gisSource != null) {
-              gisSource.close()
-            }
-            if (gosSource != null) {
-              gosSource.close()
-            }
-            if (guideSocketToSource != null) {
-              guideSocketToSource.close()
-            }
-          }
-        }
-      }
-    }
-
-    class GuideSingleRequest(val clientSocket: Socket)
-    extends Thread with Logging {
-      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
-      oos.flush()
-      private val ois = new ObjectInputStream(clientSocket.getInputStream)
-
-      private var sourceInfo: SourceInfo = null
-      private var selectedSources: ListBuffer[SourceInfo] = null
-
-      override def run() {
-        try {
-          logInfo("new GuideSingleRequest is running")
-          // Connecting worker is sending in its information
-          sourceInfo = ois.readObject.asInstanceOf[SourceInfo]
-
-          // Select a suitable source and send it back to the worker
-          selectedSources = selectSuitableSources(sourceInfo)
-          logDebug("Sending selectedSources:" + selectedSources)
-          oos.writeObject(selectedSources)
-          oos.flush()
-
-          // Add this source to the listOfSources
-          addToListOfSources(sourceInfo)
-        } catch {
-          case e: Exception => {
-            // Assuming exception caused by receiver failure: remove
-            if (listOfSources != null) {
-              listOfSources.synchronized { listOfSources -= sourceInfo }
-            }
-          }
-        } finally {
-          logInfo("GuideSingleRequest is closing streams and sockets")
-          ois.close()
-          oos.close()
-          clientSocket.close()
-        }
-      }
-
-      // Randomly select some sources to send back
-      private def selectSuitableSources(skipSourceInfo: SourceInfo): ListBuffer[SourceInfo] = {
-        var selectedSources = ListBuffer[SourceInfo]()
-
-        // If skipSourceInfo.hasBlocksBitVector has all bits set to 'true'
-        // then add skipSourceInfo to setOfCompletedSources. Return blank.
-        if (skipSourceInfo.hasBlocks == totalBlocks) {
-          setOfCompletedSources.synchronized { setOfCompletedSources += skipSourceInfo }
-          return selectedSources
-        }
-
-        listOfSources.synchronized {
-          if (listOfSources.size <= MultiTracker.MaxPeersInGuideResponse) {
-            selectedSources = listOfSources.clone
-          } else {
-            var picksLeft = MultiTracker.MaxPeersInGuideResponse
-            var alreadyPicked = new BitSet(listOfSources.size)
-
-            while (picksLeft > 0) {
-              var i = -1
-
-              do {
-                i = MultiTracker.ranGen.nextInt(listOfSources.size)
-              } while (alreadyPicked.get(i))
-
-              var peerIter = listOfSources.iterator
-              var curPeer = peerIter.next
-
-              // Set the BitSet before i is decremented
-              alreadyPicked.set(i)
-
-              while (i > 0) {
-                curPeer = peerIter.next
-                i = i - 1
-              }
-
-              selectedSources += curPeer
-
-              picksLeft = picksLeft - 1
-            }
-          }
-        }
-
-        // Remove the receiving source (if present)
-        selectedSources = selectedSources - skipSourceInfo
-
-        return selectedSources
-      }
-    }
-  }
-
-  class ServeMultipleRequests
-  extends Thread with Logging {
-    // Server at most MultiTracker.MaxChatSlots peers
-    var threadPool = Utils.newDaemonFixedThreadPool(MultiTracker.MaxChatSlots)
-
-    override def run() {
-      var serverSocket = new ServerSocket(0)
-      listenPort = serverSocket.getLocalPort
-
-      logInfo("ServeMultipleRequests started with " + serverSocket)
-
-      listenPortLock.synchronized { listenPortLock.notifyAll() }
-
-      try {
-        while (!stopBroadcast) {
-          var clientSocket: Socket = null
-          try {
-            serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
-            clientSocket = serverSocket.accept()
-          } catch {
-            case e: Exception => { }
-          }
-          if (clientSocket != null) {
-            logDebug("Serve: Accepted new client connection:" + clientSocket)
-            try {
-              threadPool.execute(new ServeSingleRequest(clientSocket))
-            } catch {
-              // In failure, close socket here; else, the thread will close it
-              case ioe: IOException => clientSocket.close()
-            }
-          }
-        }
-      } finally {
-        if (serverSocket != null) {
-          logInfo("ServeMultipleRequests now stopping...")
-          serverSocket.close()
-        }
-      }
-      // Shutdown the thread pool
-      threadPool.shutdown()
-    }
-
-    class ServeSingleRequest(val clientSocket: Socket)
-    extends Thread with Logging {
-      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
-      oos.flush()
-      private val ois = new ObjectInputStream(clientSocket.getInputStream)
-
-      logInfo("new ServeSingleRequest is running")
-
-      override def run() {
-        try {
-          // Send latest local SourceInfo to the receiver
-          // In the case of receiver timeout and connection close, this will
-          // throw a java.net.SocketException: Broken pipe
-          oos.writeObject(getLocalSourceInfo)
-          oos.flush()
-
-          // Receive latest SourceInfo from the receiver
-          var rxSourceInfo = ois.readObject.asInstanceOf[SourceInfo]
-
-          if (rxSourceInfo.listenPort == SourceInfo.StopBroadcast) {
-            stopBroadcast = true
-          } else {
-            addToListOfSources(rxSourceInfo)
-          }
-
-          val startTime = System.currentTimeMillis
-          var curTime = startTime
-          var keepSending = true
-          var numBlocksToSend = MultiTracker.MaxChatBlocks
-
-          while (!stopBroadcast && keepSending && numBlocksToSend > 0) {
-            // Receive which block to send
-            var blockToSend = ois.readObject.asInstanceOf[Int]
-
-            // If it is driver AND at least one copy of each block has not been
-            // sent out already, MODIFY blockToSend
-            if (MultiTracker.isDriver && sentBlocks.get < totalBlocks) {
-              blockToSend = sentBlocks.getAndIncrement
-            }
-
-            // Send the block
-            sendBlock(blockToSend)
-            rxSourceInfo.hasBlocksBitVector.set(blockToSend)
-
-            numBlocksToSend -= 1
-
-            // Receive latest SourceInfo from the receiver
-            rxSourceInfo = ois.readObject.asInstanceOf[SourceInfo]
-            logDebug("rxSourceInfo: " + rxSourceInfo + " with " + rxSourceInfo.hasBlocksBitVector)
-            addToListOfSources(rxSourceInfo)
-
-            curTime = System.currentTimeMillis
-            // Revoke sending only if there is anyone waiting in the queue
-            if (curTime - startTime >= MultiTracker.MaxChatTime &&
-                threadPool.getQueue.size > 0) {
-              keepSending = false
-            }
-          }
-        } catch {
-          case e: Exception => logError("ServeSingleRequest had a " + e)
-        } finally {
-          logInfo("ServeSingleRequest is closing streams and sockets")
-          ois.close()
-          oos.close()
-          clientSocket.close()
-        }
-      }
-
-      private def sendBlock(blockToSend: Int) {
-        try {
-          oos.writeObject(arrayOfBlocks(blockToSend))
-          oos.flush()
-        } catch {
-          case e: Exception => logError("sendBlock had a " + e)
-        }
-        logDebug("Sent block: " + blockToSend + " to " + clientSocket)
-      }
-    }
-  }
-}
-
-private[spark] class BitTorrentBroadcastFactory
-extends BroadcastFactory {
-  def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) }
-
-  def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
-    new BitTorrentBroadcast[T](value_, isLocal, id)
-
-  def stop() { MultiTracker.stop() }
-}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala
deleted file mode 100644
index 21ec94659e6011e63b8164b2ec53b91d6effb83c..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala
+++ /dev/null
@@ -1,410 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.broadcast
-
-import java.io._
-import java.net._
-import java.util.Random
-
-import scala.collection.mutable.Map
-
-import org.apache.spark._
-import org.apache.spark.util.Utils
-
-private object MultiTracker
-extends Logging {
-
-  // Tracker Messages
-  val REGISTER_BROADCAST_TRACKER = 0
-  val UNREGISTER_BROADCAST_TRACKER = 1
-  val FIND_BROADCAST_TRACKER = 2
-
-  // Map to keep track of guides of ongoing broadcasts
-  var valueToGuideMap = Map[Long, SourceInfo]()
-
-  // Random number generator
-  var ranGen = new Random
-
-  private var initialized = false
-  private var _isDriver = false
-
-  private var stopBroadcast = false
-
-  private var trackMV: TrackMultipleValues = null
-
-  def initialize(__isDriver: Boolean) {
-    synchronized {
-      if (!initialized) {
-        _isDriver = __isDriver
-
-        if (isDriver) {
-          trackMV = new TrackMultipleValues
-          trackMV.setDaemon(true)
-          trackMV.start()
-        
-          // Set DriverHostAddress to the driver's IP address for the slaves to read
-          System.setProperty("spark.MultiTracker.DriverHostAddress", Utils.localIpAddress)
-        }
-
-        initialized = true
-      }
-    }
-  }
-  
-  def stop() {
-    stopBroadcast = true
-  }
-
-  // Load common parameters
-  private var DriverHostAddress_ = System.getProperty(
-    "spark.MultiTracker.DriverHostAddress", "")
-  private var DriverTrackerPort_ = System.getProperty(
-    "spark.broadcast.driverTrackerPort", "11111").toInt
-  private var BlockSize_ = System.getProperty(
-    "spark.broadcast.blockSize", "4096").toInt * 1024
-  private var MaxRetryCount_ = System.getProperty(
-    "spark.broadcast.maxRetryCount", "2").toInt
-
-  private var TrackerSocketTimeout_ = System.getProperty(
-    "spark.broadcast.trackerSocketTimeout", "50000").toInt
-  private var ServerSocketTimeout_ = System.getProperty(
-    "spark.broadcast.serverSocketTimeout", "10000").toInt
-
-  private var MinKnockInterval_ = System.getProperty(
-    "spark.broadcast.minKnockInterval", "500").toInt
-  private var MaxKnockInterval_ = System.getProperty(
-    "spark.broadcast.maxKnockInterval", "999").toInt
-
-  // Load TreeBroadcast config params
-  private var MaxDegree_ = System.getProperty(
-    "spark.broadcast.maxDegree", "2").toInt
-
-  // Load BitTorrentBroadcast config params
-  private var MaxPeersInGuideResponse_ = System.getProperty(
-    "spark.broadcast.maxPeersInGuideResponse", "4").toInt
-
-  private var MaxChatSlots_ = System.getProperty(
-    "spark.broadcast.maxChatSlots", "4").toInt
-  private var MaxChatTime_ = System.getProperty(
-    "spark.broadcast.maxChatTime", "500").toInt
-  private var MaxChatBlocks_ = System.getProperty(
-    "spark.broadcast.maxChatBlocks", "1024").toInt
-
-  private var EndGameFraction_ = System.getProperty(
-      "spark.broadcast.endGameFraction", "0.95").toDouble
-
-  def isDriver = _isDriver
-
-  // Common config params
-  def DriverHostAddress = DriverHostAddress_
-  def DriverTrackerPort = DriverTrackerPort_
-  def BlockSize = BlockSize_
-  def MaxRetryCount = MaxRetryCount_
-
-  def TrackerSocketTimeout = TrackerSocketTimeout_
-  def ServerSocketTimeout = ServerSocketTimeout_
-
-  def MinKnockInterval = MinKnockInterval_
-  def MaxKnockInterval = MaxKnockInterval_
-
-  // TreeBroadcast configs
-  def MaxDegree = MaxDegree_
-
-  // BitTorrentBroadcast configs
-  def MaxPeersInGuideResponse = MaxPeersInGuideResponse_
-
-  def MaxChatSlots = MaxChatSlots_
-  def MaxChatTime = MaxChatTime_
-  def MaxChatBlocks = MaxChatBlocks_
-
-  def EndGameFraction = EndGameFraction_
-
-  class TrackMultipleValues
-  extends Thread with Logging {
-    override def run() {
-      var threadPool = Utils.newDaemonCachedThreadPool()
-      var serverSocket: ServerSocket = null
-
-      serverSocket = new ServerSocket(DriverTrackerPort)
-      logInfo("TrackMultipleValues started at " + serverSocket)
-
-      try {
-        while (!stopBroadcast) {
-          var clientSocket: Socket = null
-          try {
-            serverSocket.setSoTimeout(TrackerSocketTimeout)
-            clientSocket = serverSocket.accept()
-          } catch {
-            case e: Exception => {
-              if (stopBroadcast) {
-                logInfo("Stopping TrackMultipleValues...")
-              }              
-            }
-          }
-
-          if (clientSocket != null) {
-            try {
-              threadPool.execute(new Thread {
-                override def run() {
-                  val oos = new ObjectOutputStream(clientSocket.getOutputStream)
-                  oos.flush()
-                  val ois = new ObjectInputStream(clientSocket.getInputStream)
-
-                  try {
-                    // First, read message type
-                    val messageType = ois.readObject.asInstanceOf[Int]
-
-                    if (messageType == REGISTER_BROADCAST_TRACKER) {
-                      // Receive Long
-                      val id = ois.readObject.asInstanceOf[Long]
-                      // Receive hostAddress and listenPort
-                      val gInfo = ois.readObject.asInstanceOf[SourceInfo]
-
-                      // Add to the map
-                      valueToGuideMap.synchronized {
-                        valueToGuideMap += (id -> gInfo)
-                      }
-
-                      logInfo ("New broadcast " + id + " registered with TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
-
-                      // Send dummy ACK
-                      oos.writeObject(-1)
-                      oos.flush()
-                    } else if (messageType == UNREGISTER_BROADCAST_TRACKER) {
-                      // Receive Long
-                      val id = ois.readObject.asInstanceOf[Long]
-
-                      // Remove from the map
-                      valueToGuideMap.synchronized {
-                        valueToGuideMap(id) = SourceInfo("", SourceInfo.TxOverGoToDefault)
-                      }
-
-                      logInfo ("Broadcast " + id + " unregistered from TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
-
-                      // Send dummy ACK
-                      oos.writeObject(-1)
-                      oos.flush()
-                    } else if (messageType == FIND_BROADCAST_TRACKER) {
-                      // Receive Long
-                      val id = ois.readObject.asInstanceOf[Long]
-
-                      var gInfo =
-                        if (valueToGuideMap.contains(id)) valueToGuideMap(id)
-                        else SourceInfo("", SourceInfo.TxNotStartedRetry)
-
-                      logDebug("Got new request: " + clientSocket + " for " + id + " : " + gInfo.listenPort)
-
-                      // Send reply back
-                      oos.writeObject(gInfo)
-                      oos.flush()
-                    } else {
-                      throw new SparkException("Undefined messageType at TrackMultipleValues")
-                    }
-                  } catch {
-                    case e: Exception => {
-                      logError("TrackMultipleValues had a " + e)
-                    }
-                  } finally {
-                    ois.close()
-                    oos.close()
-                    clientSocket.close()
-                  }
-                }
-              })
-            } catch {
-              // In failure, close socket here; else, client thread will close
-              case ioe: IOException => clientSocket.close()
-            }
-          }
-        }
-      } finally {
-        serverSocket.close()
-      }
-      // Shutdown the thread pool
-      threadPool.shutdown()
-    }
-  }
-  
-  def getGuideInfo(variableLong: Long): SourceInfo = {
-    var clientSocketToTracker: Socket = null
-    var oosTracker: ObjectOutputStream = null
-    var oisTracker: ObjectInputStream = null
-
-    var gInfo: SourceInfo = SourceInfo("", SourceInfo.TxNotStartedRetry)
-
-    var retriesLeft = MultiTracker.MaxRetryCount
-    do {
-      try {
-        // Connect to the tracker to find out GuideInfo
-        clientSocketToTracker =
-          new Socket(MultiTracker.DriverHostAddress, MultiTracker.DriverTrackerPort)
-        oosTracker =
-          new ObjectOutputStream(clientSocketToTracker.getOutputStream)
-        oosTracker.flush()
-        oisTracker =
-          new ObjectInputStream(clientSocketToTracker.getInputStream)
-
-        // Send messageType/intention
-        oosTracker.writeObject(MultiTracker.FIND_BROADCAST_TRACKER)
-        oosTracker.flush()
-
-        // Send Long and receive GuideInfo
-        oosTracker.writeObject(variableLong)
-        oosTracker.flush()
-        gInfo = oisTracker.readObject.asInstanceOf[SourceInfo]
-      } catch {
-        case e: Exception => logError("getGuideInfo had a " + e)
-      } finally {
-        if (oisTracker != null) {
-          oisTracker.close()
-        }
-        if (oosTracker != null) {
-          oosTracker.close()
-        }
-        if (clientSocketToTracker != null) {
-          clientSocketToTracker.close()
-        }
-      }
-
-      Thread.sleep(MultiTracker.ranGen.nextInt(
-        MultiTracker.MaxKnockInterval - MultiTracker.MinKnockInterval) +
-        MultiTracker.MinKnockInterval)
-
-      retriesLeft -= 1
-    } while (retriesLeft > 0 && gInfo.listenPort == SourceInfo.TxNotStartedRetry)
-
-    logDebug("Got this guidePort from Tracker: " + gInfo.listenPort)
-    return gInfo
-  }
-  
-  def registerBroadcast(id: Long, gInfo: SourceInfo) {
-    val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort)
-    val oosST = new ObjectOutputStream(socket.getOutputStream)
-    oosST.flush()
-    val oisST = new ObjectInputStream(socket.getInputStream)
-
-    // Send messageType/intention
-    oosST.writeObject(REGISTER_BROADCAST_TRACKER)
-    oosST.flush()
-
-    // Send Long of this broadcast
-    oosST.writeObject(id)
-    oosST.flush()
-
-    // Send this tracker's information
-    oosST.writeObject(gInfo)
-    oosST.flush()
-
-    // Receive ACK and throw it away
-    oisST.readObject.asInstanceOf[Int]
-
-    // Shut stuff down
-    oisST.close()
-    oosST.close()
-    socket.close()
-  }
-
-  def unregisterBroadcast(id: Long) {
-    val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort)
-    val oosST = new ObjectOutputStream(socket.getOutputStream)
-    oosST.flush()
-    val oisST = new ObjectInputStream(socket.getInputStream)
-
-    // Send messageType/intention
-    oosST.writeObject(UNREGISTER_BROADCAST_TRACKER)
-    oosST.flush()
-
-    // Send Long of this broadcast
-    oosST.writeObject(id)
-    oosST.flush()
-
-    // Receive ACK and throw it away
-    oisST.readObject.asInstanceOf[Int]
-
-    // Shut stuff down
-    oisST.close()
-    oosST.close()
-    socket.close()
-  }
-
-  // Helper method to convert an object to Array[BroadcastBlock]
-  def blockifyObject[IN](obj: IN): VariableInfo = {
-    val baos = new ByteArrayOutputStream
-    val oos = new ObjectOutputStream(baos)
-    oos.writeObject(obj)
-    oos.close()
-    baos.close()
-    val byteArray = baos.toByteArray
-    val bais = new ByteArrayInputStream(byteArray)
-
-    var blockNum = (byteArray.length / BlockSize)
-    if (byteArray.length % BlockSize != 0)
-      blockNum += 1
-
-    var retVal = new Array[BroadcastBlock](blockNum)
-    var blockID = 0
-
-    for (i <- 0 until (byteArray.length, BlockSize)) {
-      val thisBlockSize = math.min(BlockSize, byteArray.length - i)
-      var tempByteArray = new Array[Byte](thisBlockSize)
-      val hasRead = bais.read(tempByteArray, 0, thisBlockSize)
-
-      retVal(blockID) = new BroadcastBlock(blockID, tempByteArray)
-      blockID += 1
-    }
-    bais.close()
-
-    var variableInfo = VariableInfo(retVal, blockNum, byteArray.length)
-    variableInfo.hasBlocks = blockNum
-
-    return variableInfo
-  }
-
-  // Helper method to convert Array[BroadcastBlock] to object
-  def unBlockifyObject[OUT](arrayOfBlocks: Array[BroadcastBlock],
-                            totalBytes: Int, 
-                            totalBlocks: Int): OUT = {
-
-    var retByteArray = new Array[Byte](totalBytes)
-    for (i <- 0 until totalBlocks) {
-      System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray,
-        i * BlockSize, arrayOfBlocks(i).byteArray.length)
-    }
-    byteArrayToObject(retByteArray)
-  }
-
-  private def byteArrayToObject[OUT](bytes: Array[Byte]): OUT = {
-    val in = new ObjectInputStream (new ByteArrayInputStream (bytes)){
-      override def resolveClass(desc: ObjectStreamClass) =
-        Class.forName(desc.getName, false, Thread.currentThread.getContextClassLoader)
-    }    
-    val retVal = in.readObject.asInstanceOf[OUT]
-    in.close()
-    return retVal
-  }
-}
-
-private[spark] case class BroadcastBlock(blockID: Int, byteArray: Array[Byte]) 
-extends Serializable
-
-private[spark] case class VariableInfo(@transient arrayOfBlocks : Array[BroadcastBlock],
-                        totalBlocks: Int, 
-                        totalBytes: Int) 
-extends Serializable {
- @transient var hasBlocks = 0 
-}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala b/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala
deleted file mode 100644
index baa1fd6da46e820ed1d706198d4ab742bb4c19b6..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.broadcast
-
-import java.util.BitSet
-
-import org.apache.spark._
-
-/**
- * Used to keep and pass around information of peers involved in a broadcast
- */
-private[spark] case class SourceInfo (hostAddress: String,
-                       listenPort: Int,
-                       totalBlocks: Int = SourceInfo.UnusedParam,
-                       totalBytes: Int = SourceInfo.UnusedParam)
-extends Comparable[SourceInfo] with Logging {
-
-  var currentLeechers = 0
-  var receptionFailed = false
-
-  var hasBlocks = 0
-  var hasBlocksBitVector: BitSet = new BitSet (totalBlocks)
-
-  // Ascending sort based on leecher count
-  def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers)
-}
-
-/**
- * Helper Object of SourceInfo for its constants
- */
-private[spark] object SourceInfo {
-  // Broadcast has not started yet! Should never happen.
-  val TxNotStartedRetry = -1
-  // Broadcast has already finished. Try default mechanism.
-  val TxOverGoToDefault = -3
-  // Other constants
-  val StopBroadcast = -2
-  val UnusedParam = 0
-}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
new file mode 100644
index 0000000000000000000000000000000000000000..073a0a50297d068963bc3759a557921ce6ee47a4
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.broadcast
+
+import java.io._
+
+import scala.math
+import scala.util.Random
+
+import org.apache.spark._
+import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel}
+import org.apache.spark.util.Utils
+
+
+private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
+extends Broadcast[T](id) with Logging with Serializable {
+
+  def value = value_
+
+  def broadcastId = BroadcastBlockId(id)
+
+  TorrentBroadcast.synchronized {
+    SparkEnv.get.blockManager.putSingle(broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false)
+  }
+
+  @transient var arrayOfBlocks: Array[TorrentBlock] = null
+  @transient var totalBlocks = -1
+  @transient var totalBytes = -1
+  @transient var hasBlocks = 0
+
+  if (!isLocal) {
+    sendBroadcast()
+  }
+
+  def sendBroadcast() {
+    var tInfo = TorrentBroadcast.blockifyObject(value_)
+
+    totalBlocks = tInfo.totalBlocks
+    totalBytes = tInfo.totalBytes
+    hasBlocks = tInfo.totalBlocks
+
+    // Store meta-info
+    val metaId = BroadcastHelperBlockId(broadcastId, "meta")
+    val metaInfo = TorrentInfo(null, totalBlocks, totalBytes)
+    TorrentBroadcast.synchronized {
+      SparkEnv.get.blockManager.putSingle(
+        metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, true)
+    }
+
+    // Store individual pieces
+    for (i <- 0 until totalBlocks) {
+      val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + i)
+      TorrentBroadcast.synchronized {
+        SparkEnv.get.blockManager.putSingle(
+          pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, true)
+      }
+    }
+  }
+
+  // Called by JVM when deserializing an object
+  private def readObject(in: ObjectInputStream) {
+    in.defaultReadObject()
+    TorrentBroadcast.synchronized {
+      SparkEnv.get.blockManager.getSingle(broadcastId) match {
+        case Some(x) =>
+          value_ = x.asInstanceOf[T]
+
+        case None =>
+          val start = System.nanoTime
+          logInfo("Started reading broadcast variable " + id)
+          
+          // Initialize @transient variables that will receive garbage values from the master.
+          resetWorkerVariables()
+
+          if (receiveBroadcast(id)) {
+            value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
+            
+            // Store the merged copy in cache so that the next worker doesn't need to rebuild it.
+            // This creates a tradeoff between memory usage and latency.
+            // Storing copy doubles the memory footprint; not storing doubles deserialization cost.
+            SparkEnv.get.blockManager.putSingle(
+              broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false)
+
+            // Remove arrayOfBlocks from memory once value_ is on local cache
+            resetWorkerVariables()
+          }  else {
+            logError("Reading broadcast variable " + id + " failed")
+          }
+
+          val time = (System.nanoTime - start) / 1e9
+          logInfo("Reading broadcast variable " + id + " took " + time + " s")
+      }
+    }
+  }
+
+  private def resetWorkerVariables() {
+    arrayOfBlocks = null
+    totalBytes = -1
+    totalBlocks = -1
+    hasBlocks = 0
+  }
+
+  def receiveBroadcast(variableID: Long): Boolean = {
+    // Receive meta-info
+    val metaId = BroadcastHelperBlockId(broadcastId, "meta")
+    var attemptId = 10
+    while (attemptId > 0 && totalBlocks == -1) {
+      TorrentBroadcast.synchronized {
+        SparkEnv.get.blockManager.getSingle(metaId) match {
+          case Some(x) => 
+            val tInfo = x.asInstanceOf[TorrentInfo]
+            totalBlocks = tInfo.totalBlocks
+            totalBytes = tInfo.totalBytes
+            arrayOfBlocks = new Array[TorrentBlock](totalBlocks)
+            hasBlocks = 0
+          
+          case None => 
+            Thread.sleep(500)
+        }
+      }
+      attemptId -= 1
+    }
+    if (totalBlocks == -1) {
+      return false
+    }
+
+    // Receive actual blocks
+    val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList)
+    for (pid <- recvOrder) {
+      val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid)
+      TorrentBroadcast.synchronized {
+        SparkEnv.get.blockManager.getSingle(pieceId) match {
+          case Some(x) => 
+            arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock]
+            hasBlocks += 1
+            SparkEnv.get.blockManager.putSingle(
+              pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true)
+          
+          case None => 
+            throw new SparkException("Failed to get " + pieceId + " of " + broadcastId)
+        }
+      }
+    }
+
+    (hasBlocks == totalBlocks)
+  }
+
+}
+
+private object TorrentBroadcast
+extends Logging {
+
+  private var initialized = false
+
+  def initialize(_isDriver: Boolean) {
+    synchronized {
+      if (!initialized) {
+        initialized = true
+      }
+    }
+  }
+  
+  def stop() {
+    initialized = false
+  }
+
+  val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024
+  
+  def blockifyObject[T](obj: T): TorrentInfo = {
+    val byteArray = Utils.serialize[T](obj)
+    val bais = new ByteArrayInputStream(byteArray)
+
+    var blockNum = (byteArray.length / BLOCK_SIZE)
+    if (byteArray.length % BLOCK_SIZE != 0)
+      blockNum += 1
+
+    var retVal = new Array[TorrentBlock](blockNum)
+    var blockID = 0
+
+    for (i <- 0 until (byteArray.length, BLOCK_SIZE)) {
+      val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i)
+      var tempByteArray = new Array[Byte](thisBlockSize)
+      val hasRead = bais.read(tempByteArray, 0, thisBlockSize)
+
+      retVal(blockID) = new TorrentBlock(blockID, tempByteArray)
+      blockID += 1
+    }
+    bais.close()
+
+    var tInfo = TorrentInfo(retVal, blockNum, byteArray.length)
+    tInfo.hasBlocks = blockNum
+
+    return tInfo
+  }
+
+  def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock],
+                            totalBytes: Int, 
+                            totalBlocks: Int): T = {
+    var retByteArray = new Array[Byte](totalBytes)
+    for (i <- 0 until totalBlocks) {
+      System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray,
+        i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length)
+    }
+    Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader)
+  }
+
+}
+
+private[spark] case class TorrentBlock(
+    blockID: Int, 
+    byteArray: Array[Byte]) 
+  extends Serializable
+
+private[spark] case class TorrentInfo(
+    @transient arrayOfBlocks : Array[TorrentBlock],
+    totalBlocks: Int, 
+    totalBytes: Int) 
+  extends Serializable {
+  
+  @transient var hasBlocks = 0 
+}
+
+private[spark] class TorrentBroadcastFactory
+  extends BroadcastFactory {
+  
+  def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) }
+
+  def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
+    new TorrentBroadcast[T](value_, isLocal, id)
+
+  def stop() { TorrentBroadcast.stop() }
+}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
deleted file mode 100644
index e6674d49a7226bda26c7e1d44f8290a98956a877..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
+++ /dev/null
@@ -1,601 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.broadcast
-
-import java.io._
-import java.net._
-
-import scala.collection.mutable.{ListBuffer, Set}
-
-import org.apache.spark._
-import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
-import org.apache.spark.util.Utils
-
-private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
-extends Broadcast[T](id) with Logging with Serializable {
-
-  def value = value_
-
-  def blockId = BroadcastBlockId(id)
-
-  MultiTracker.synchronized {
-    SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
-  }
-
-  @transient var arrayOfBlocks: Array[BroadcastBlock] = null
-  @transient var totalBytes = -1
-  @transient var totalBlocks = -1
-  @transient var hasBlocks = 0
-
-  @transient var listenPortLock = new Object
-  @transient var guidePortLock = new Object
-  @transient var totalBlocksLock = new Object
-  @transient var hasBlocksLock = new Object
-
-  @transient var listOfSources = ListBuffer[SourceInfo]()
-
-  @transient var serveMR: ServeMultipleRequests = null
-  @transient var guideMR: GuideMultipleRequests = null
-
-  @transient var hostAddress = Utils.localIpAddress
-  @transient var listenPort = -1
-  @transient var guidePort = -1
-
-  @transient var stopBroadcast = false
-
-  // Must call this after all the variables have been created/initialized
-  if (!isLocal) {
-    sendBroadcast()
-  }
-
-  def sendBroadcast() {
-    logInfo("Local host address: " + hostAddress)
-
-    // Create a variableInfo object and store it in valueInfos
-    var variableInfo = MultiTracker.blockifyObject(value_)
-
-    // Prepare the value being broadcasted
-    arrayOfBlocks = variableInfo.arrayOfBlocks
-    totalBytes = variableInfo.totalBytes
-    totalBlocks = variableInfo.totalBlocks
-    hasBlocks = variableInfo.totalBlocks
-
-    guideMR = new GuideMultipleRequests
-    guideMR.setDaemon(true)
-    guideMR.start()
-    logInfo("GuideMultipleRequests started...")
-
-    // Must always come AFTER guideMR is created
-    while (guidePort == -1) {
-      guidePortLock.synchronized { guidePortLock.wait() }
-    }
-
-    serveMR = new ServeMultipleRequests
-    serveMR.setDaemon(true)
-    serveMR.start()
-    logInfo("ServeMultipleRequests started...")
-
-    // Must always come AFTER serveMR is created
-    while (listenPort == -1) {
-      listenPortLock.synchronized { listenPortLock.wait() }
-    }
-
-    // Must always come AFTER listenPort is created
-    val masterSource =
-      SourceInfo(hostAddress, listenPort, totalBlocks, totalBytes)
-    listOfSources += masterSource
-
-    // Register with the Tracker
-    MultiTracker.registerBroadcast(id,
-      SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes))
-  }
-
-  private def readObject(in: ObjectInputStream) {
-    in.defaultReadObject()
-    MultiTracker.synchronized {
-      SparkEnv.get.blockManager.getSingle(blockId) match {
-        case Some(x) =>
-          value_ = x.asInstanceOf[T]
-
-        case None =>
-          logInfo("Started reading broadcast variable " + id)
-          // Initializing everything because Driver will only send null/0 values
-          // Only the 1st worker in a node can be here. Others will get from cache
-          initializeWorkerVariables()
-
-          logInfo("Local host address: " + hostAddress)
-
-          serveMR = new ServeMultipleRequests
-          serveMR.setDaemon(true)
-          serveMR.start()
-          logInfo("ServeMultipleRequests started...")
-
-          val start = System.nanoTime
-
-          val receptionSucceeded = receiveBroadcast(id)
-          if (receptionSucceeded) {
-            value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
-            SparkEnv.get.blockManager.putSingle(
-              blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
-          }  else {
-            logError("Reading broadcast variable " + id + " failed")
-          }
-
-          val time = (System.nanoTime - start) / 1e9
-          logInfo("Reading broadcast variable " + id + " took " + time + " s")
-      }
-    }
-  }
-
-  private def initializeWorkerVariables() {
-    arrayOfBlocks = null
-    totalBytes = -1
-    totalBlocks = -1
-    hasBlocks = 0
-
-    listenPortLock = new Object
-    totalBlocksLock = new Object
-    hasBlocksLock = new Object
-
-    serveMR =  null
-
-    hostAddress = Utils.localIpAddress
-    listenPort = -1
-
-    stopBroadcast = false
-  }
-
-  def receiveBroadcast(variableID: Long): Boolean = {
-    val gInfo = MultiTracker.getGuideInfo(variableID)
-    
-    if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) {
-      return false
-    }
-
-    // Wait until hostAddress and listenPort are created by the
-    // ServeMultipleRequests thread
-    while (listenPort == -1) {
-      listenPortLock.synchronized { listenPortLock.wait() }
-    }
-
-    var clientSocketToDriver: Socket = null
-    var oosDriver: ObjectOutputStream = null
-    var oisDriver: ObjectInputStream = null
-
-    // Connect and receive broadcast from the specified source, retrying the
-    // specified number of times in case of failures
-    var retriesLeft = MultiTracker.MaxRetryCount
-    do {
-      // Connect to Driver and send this worker's Information
-      clientSocketToDriver = new Socket(MultiTracker.DriverHostAddress, gInfo.listenPort)
-      oosDriver = new ObjectOutputStream(clientSocketToDriver.getOutputStream)
-      oosDriver.flush()
-      oisDriver = new ObjectInputStream(clientSocketToDriver.getInputStream)
-
-      logDebug("Connected to Driver's guiding object")
-
-      // Send local source information
-      oosDriver.writeObject(SourceInfo(hostAddress, listenPort))
-      oosDriver.flush()
-
-      // Receive source information from Driver
-      var sourceInfo = oisDriver.readObject.asInstanceOf[SourceInfo]
-      totalBlocks = sourceInfo.totalBlocks
-      arrayOfBlocks = new Array[BroadcastBlock](totalBlocks)
-      totalBlocksLock.synchronized { totalBlocksLock.notifyAll() }
-      totalBytes = sourceInfo.totalBytes
-
-      logDebug("Received SourceInfo from Driver:" + sourceInfo + " My Port: " + listenPort)
-
-      val start = System.nanoTime
-      val receptionSucceeded = receiveSingleTransmission(sourceInfo)
-      val time = (System.nanoTime - start) / 1e9
-
-      // Updating some statistics in sourceInfo. Driver will be using them later
-      if (!receptionSucceeded) {
-        sourceInfo.receptionFailed = true
-      }
-
-      // Send back statistics to the Driver
-      oosDriver.writeObject(sourceInfo)
-
-      if (oisDriver != null) {
-        oisDriver.close()
-      }
-      if (oosDriver != null) {
-        oosDriver.close()
-      }
-      if (clientSocketToDriver != null) {
-        clientSocketToDriver.close()
-      }
-
-      retriesLeft -= 1
-    } while (retriesLeft > 0 && hasBlocks < totalBlocks)
-
-    return (hasBlocks == totalBlocks)
-  }
-
-  /**
-   * Tries to receive broadcast from the source and returns Boolean status.
-   * This might be called multiple times to retry a defined number of times.
-   */
-  private def receiveSingleTransmission(sourceInfo: SourceInfo): Boolean = {
-    var clientSocketToSource: Socket = null
-    var oosSource: ObjectOutputStream = null
-    var oisSource: ObjectInputStream = null
-
-    var receptionSucceeded = false
-    try {
-      // Connect to the source to get the object itself
-      clientSocketToSource = new Socket(sourceInfo.hostAddress, sourceInfo.listenPort)
-      oosSource = new ObjectOutputStream(clientSocketToSource.getOutputStream)
-      oosSource.flush()
-      oisSource = new ObjectInputStream(clientSocketToSource.getInputStream)
-
-      logDebug("Inside receiveSingleTransmission")
-      logDebug("totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks)
-
-      // Send the range
-      oosSource.writeObject((hasBlocks, totalBlocks))
-      oosSource.flush()
-
-      for (i <- hasBlocks until totalBlocks) {
-        val recvStartTime = System.currentTimeMillis
-        val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock]
-        val receptionTime = (System.currentTimeMillis - recvStartTime)
-
-        logDebug("Received block: " + bcBlock.blockID + " from " + sourceInfo + " in " + receptionTime + " millis.")
-
-        arrayOfBlocks(hasBlocks) = bcBlock
-        hasBlocks += 1
-        
-        // Set to true if at least one block is received
-        receptionSucceeded = true
-        hasBlocksLock.synchronized { hasBlocksLock.notifyAll() }
-      }
-    } catch {
-      case e: Exception => logError("receiveSingleTransmission had a " + e)
-    } finally {
-      if (oisSource != null) {
-        oisSource.close()
-      }
-      if (oosSource != null) {
-        oosSource.close()
-      }
-      if (clientSocketToSource != null) {
-        clientSocketToSource.close()
-      }
-    }
-
-    return receptionSucceeded
-  }
-
-  class GuideMultipleRequests
-  extends Thread with Logging {
-    // Keep track of sources that have completed reception
-    private var setOfCompletedSources = Set[SourceInfo]()
-
-    override def run() {
-      var threadPool = Utils.newDaemonCachedThreadPool()
-      var serverSocket: ServerSocket = null
-
-      serverSocket = new ServerSocket(0)
-      guidePort = serverSocket.getLocalPort
-      logInfo("GuideMultipleRequests => " + serverSocket + " " + guidePort)
-
-      guidePortLock.synchronized { guidePortLock.notifyAll() }
-
-      try {
-        while (!stopBroadcast) {
-          var clientSocket: Socket = null
-          try {
-            serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
-            clientSocket = serverSocket.accept
-          } catch {
-            case e: Exception => {
-              // Stop broadcast if at least one worker has connected and
-              // everyone connected so far are done. Comparing with
-              // listOfSources.size - 1, because it includes the Guide itself
-              listOfSources.synchronized {
-                setOfCompletedSources.synchronized {
-                  if (listOfSources.size > 1 &&
-                    setOfCompletedSources.size == listOfSources.size - 1) {
-                    stopBroadcast = true
-                    logInfo("GuideMultipleRequests Timeout. stopBroadcast == true.")
-                  }
-                }
-              }
-            }
-          }
-          if (clientSocket != null) {
-            logDebug("Guide: Accepted new client connection: " + clientSocket)
-            try {
-              threadPool.execute(new GuideSingleRequest(clientSocket))
-            } catch {
-              // In failure, close() the socket here; else, the thread will close() it
-              case ioe: IOException => clientSocket.close()
-            }
-          }
-        }
-
-        logInfo("Sending stopBroadcast notifications...")
-        sendStopBroadcastNotifications
-
-        MultiTracker.unregisterBroadcast(id)
-      } finally {
-        if (serverSocket != null) {
-          logInfo("GuideMultipleRequests now stopping...")
-          serverSocket.close()
-        }
-      }
-      // Shutdown the thread pool
-      threadPool.shutdown()
-    }
-
-    private def sendStopBroadcastNotifications() {
-      listOfSources.synchronized {
-        var listIter = listOfSources.iterator
-        while (listIter.hasNext) {
-          var sourceInfo = listIter.next
-
-          var guideSocketToSource: Socket = null
-          var gosSource: ObjectOutputStream = null
-          var gisSource: ObjectInputStream = null
-
-          try {
-            // Connect to the source
-            guideSocketToSource = new Socket(sourceInfo.hostAddress, sourceInfo.listenPort)
-            gosSource = new ObjectOutputStream(guideSocketToSource.getOutputStream)
-            gosSource.flush()
-            gisSource = new ObjectInputStream(guideSocketToSource.getInputStream)
-
-            // Send stopBroadcast signal
-            gosSource.writeObject((SourceInfo.StopBroadcast, SourceInfo.StopBroadcast))
-            gosSource.flush()
-          } catch {
-            case e: Exception => {
-              logError("sendStopBroadcastNotifications had a " + e)
-            }
-          } finally {
-            if (gisSource != null) {
-              gisSource.close()
-            }
-            if (gosSource != null) {
-              gosSource.close()
-            }
-            if (guideSocketToSource != null) {
-              guideSocketToSource.close()
-            }
-          }
-        }
-      }
-    }
-
-    class GuideSingleRequest(val clientSocket: Socket)
-    extends Thread with Logging {
-      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
-      oos.flush()
-      private val ois = new ObjectInputStream(clientSocket.getInputStream)
-
-      private var selectedSourceInfo: SourceInfo = null
-      private var thisWorkerInfo:SourceInfo = null
-
-      override def run() {
-        try {
-          logInfo("new GuideSingleRequest is running")
-          // Connecting worker is sending in its hostAddress and listenPort it will
-          // be listening to. Other fields are invalid (SourceInfo.UnusedParam)
-          var sourceInfo = ois.readObject.asInstanceOf[SourceInfo]
-
-          listOfSources.synchronized {
-            // Select a suitable source and send it back to the worker
-            selectedSourceInfo = selectSuitableSource(sourceInfo)
-            logDebug("Sending selectedSourceInfo: " + selectedSourceInfo)
-            oos.writeObject(selectedSourceInfo)
-            oos.flush()
-
-            // Add this new (if it can finish) source to the list of sources
-            thisWorkerInfo = SourceInfo(sourceInfo.hostAddress,
-              sourceInfo.listenPort, totalBlocks, totalBytes)
-            logDebug("Adding possible new source to listOfSources: " + thisWorkerInfo)
-            listOfSources += thisWorkerInfo
-          }
-
-          // Wait till the whole transfer is done. Then receive and update source
-          // statistics in listOfSources
-          sourceInfo = ois.readObject.asInstanceOf[SourceInfo]
-
-          listOfSources.synchronized {
-            // This should work since SourceInfo is a case class
-            assert(listOfSources.contains(selectedSourceInfo))
-
-            // Remove first 
-            // (Currently removing a source based on just one failure notification!)
-            listOfSources = listOfSources - selectedSourceInfo
-
-            // Update sourceInfo and put it back in, IF reception succeeded
-            if (!sourceInfo.receptionFailed) {
-              // Add thisWorkerInfo to sources that have completed reception
-              setOfCompletedSources.synchronized {
-                setOfCompletedSources += thisWorkerInfo
-              }
-
-              // Update leecher count and put it back in 
-              selectedSourceInfo.currentLeechers -= 1
-              listOfSources += selectedSourceInfo
-            }
-          }
-        } catch {
-          case e: Exception => {
-            // Remove failed worker from listOfSources and update leecherCount of
-            // corresponding source worker
-            listOfSources.synchronized {
-              if (selectedSourceInfo != null) {
-                // Remove first
-                listOfSources = listOfSources - selectedSourceInfo
-                // Update leecher count and put it back in
-                selectedSourceInfo.currentLeechers -= 1
-                listOfSources += selectedSourceInfo
-              }
-
-              // Remove thisWorkerInfo
-              if (listOfSources != null) {
-                listOfSources = listOfSources - thisWorkerInfo
-              }
-            }
-          }
-        } finally {
-          logInfo("GuideSingleRequest is closing streams and sockets")
-          ois.close()
-          oos.close()
-          clientSocket.close()
-        }
-      }
-
-      // Assuming the caller to have a synchronized block on listOfSources
-      // Select one with the most leechers. This will level-wise fill the tree
-      private def selectSuitableSource(skipSourceInfo: SourceInfo): SourceInfo = {
-        var maxLeechers = -1
-        var selectedSource: SourceInfo = null
-
-        listOfSources.foreach { source =>
-          if ((source.hostAddress != skipSourceInfo.hostAddress || 
-               source.listenPort != skipSourceInfo.listenPort) && 
-            source.currentLeechers < MultiTracker.MaxDegree &&
-            source.currentLeechers > maxLeechers) {
-              selectedSource = source
-              maxLeechers = source.currentLeechers
-            }
-        }
-
-        // Update leecher count
-        selectedSource.currentLeechers += 1
-        return selectedSource
-      }
-    }
-  }
-
-  class ServeMultipleRequests
-  extends Thread with Logging {
-    
-    var threadPool = Utils.newDaemonCachedThreadPool()
-    
-    override def run() {      
-      var serverSocket = new ServerSocket(0)
-      listenPort = serverSocket.getLocalPort
-      
-      logInfo("ServeMultipleRequests started with " + serverSocket)
-
-      listenPortLock.synchronized { listenPortLock.notifyAll() }
-
-      try {
-        while (!stopBroadcast) {
-          var clientSocket: Socket = null
-          try {
-            serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
-            clientSocket = serverSocket.accept
-          } catch {
-            case e: Exception => { }
-          }
-          
-          if (clientSocket != null) {
-            logDebug("Serve: Accepted new client connection: " + clientSocket)
-            try {
-              threadPool.execute(new ServeSingleRequest(clientSocket))
-            } catch {
-              // In failure, close socket here; else, the thread will close it
-              case ioe: IOException => clientSocket.close()
-            }
-          }
-        }
-      } finally {
-        if (serverSocket != null) {
-          logInfo("ServeMultipleRequests now stopping...")
-          serverSocket.close()
-        }
-      }
-      // Shutdown the thread pool
-      threadPool.shutdown()
-    }
-
-    class ServeSingleRequest(val clientSocket: Socket)
-    extends Thread with Logging {
-      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
-      oos.flush()
-      private val ois = new ObjectInputStream(clientSocket.getInputStream)
-
-      private var sendFrom = 0
-      private var sendUntil = totalBlocks
-
-      override def run() {
-        try {
-          logInfo("new ServeSingleRequest is running")
-
-          // Receive range to send
-          var rangeToSend = ois.readObject.asInstanceOf[(Int, Int)]
-          sendFrom = rangeToSend._1
-          sendUntil = rangeToSend._2
-
-          // If not a valid range, stop broadcast
-          if (sendFrom == SourceInfo.StopBroadcast && sendUntil == SourceInfo.StopBroadcast) {
-            stopBroadcast = true
-          } else {
-            sendObject
-          }
-        } catch {
-          case e: Exception => logError("ServeSingleRequest had a " + e)
-        } finally {
-          logInfo("ServeSingleRequest is closing streams and sockets")
-          ois.close()
-          oos.close()
-          clientSocket.close()
-        }
-      }
-
-      private def sendObject() {
-        // Wait till receiving the SourceInfo from Driver
-        while (totalBlocks == -1) {
-          totalBlocksLock.synchronized { totalBlocksLock.wait() }
-        }
-
-        for (i <- sendFrom until sendUntil) {
-          while (i == hasBlocks) {
-            hasBlocksLock.synchronized { hasBlocksLock.wait() }
-          }
-          try {
-            oos.writeObject(arrayOfBlocks(i))
-            oos.flush()
-          } catch {
-            case e: Exception => logError("sendObject had a " + e)
-          }
-          logDebug("Sent block: " + i + " to " + clientSocket)
-        }
-      }
-    }
-  }
-}
-
-private[spark] class TreeBroadcastFactory
-extends BroadcastFactory {
-  def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) }
-
-  def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
-    new TreeBroadcast[T](value_, isLocal, id)
-
-  def stop() { MultiTracker.stop() }
-}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 993ba6bd3dbfad6faeed86d8d4f4ef39535dd00e..6bc846aa92eb4440bd0c61a20d9f7a2aaf2467b7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -17,28 +17,47 @@
 
 package org.apache.spark.deploy
 
-import com.google.common.collect.MapMaker
-
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf
 
+import org.apache.spark.SparkException
 
 /**
- * Contains util methods to interact with Hadoop from spark.
+ * Contains util methods to interact with Hadoop from Spark.
  */
+private[spark]
 class SparkHadoopUtil {
-  // A general, soft-reference map for metadata needed during HadoopRDD split computation
-  // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
-  private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
 
-  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop
-  // subsystems
+  /**
+   * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop
+   * subsystems.
+   */
   def newConfiguration(): Configuration = new Configuration()
 
-  // Add any user credentials to the job conf which are necessary for running on a secure Hadoop
-  // cluster
+  /**
+   * Add any user credentials to the job conf which are necessary for running on a secure Hadoop
+   * cluster.
+   */
   def addCredentials(conf: JobConf) {}
 
   def isYarnMode(): Boolean = { false }
-
+}
+  
+object SparkHadoopUtil {
+  private val hadoop = { 
+    val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
+    if (yarnMode) {
+      try {
+        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
+      } catch {
+       case th: Throwable => throw new SparkException("Unable to load YARN support", th)
+      }
+    } else {
+      new SparkHadoopUtil
+    }
+  }
+  
+  def get: SparkHadoopUtil = {
+    hadoop
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
similarity index 87%
rename from core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
rename to core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index db0bea0472a52eb5a400bb435d0088ecabc0930c..80ff4c59cb484f33e3148d95cc0c9796d71fdaab 100644
--- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -24,11 +24,11 @@ import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClie
 
 import org.apache.spark.{Logging, SparkEnv}
 import org.apache.spark.TaskState.TaskState
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
 import org.apache.spark.util.{Utils, AkkaUtils}
 
 
-private[spark] class StandaloneExecutorBackend(
+private[spark] class CoarseGrainedExecutorBackend(
     driverUrl: String,
     executorId: String,
     hostPort: String,
@@ -80,6 +80,11 @@ private[spark] class StandaloneExecutorBackend(
     case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
       logError("Driver terminated or disconnected! Shutting down.")
       System.exit(1)
+
+    case StopExecutor =>
+      logInfo("Driver commanded a shutdown")
+      context.stop(self)
+      context.system.shutdown()
   }
 
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) {
@@ -87,7 +92,7 @@ private[spark] class StandaloneExecutorBackend(
   }
 }
 
-private[spark] object StandaloneExecutorBackend {
+private[spark] object CoarseGrainedExecutorBackend {
   def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
     // Debug code
     Utils.checkHost(hostname)
@@ -99,7 +104,7 @@ private[spark] object StandaloneExecutorBackend {
     val sparkHostPort = hostname + ":" + boundPort
     System.setProperty("spark.hostPort", sparkHostPort)
     val actor = actorSystem.actorOf(
-      Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
+      Props(new CoarseGrainedExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
       name = "Executor")
     actorSystem.awaitTermination()
   }
@@ -107,7 +112,9 @@ private[spark] object StandaloneExecutorBackend {
   def main(args: Array[String]) {
     if (args.length < 4) {
       //the reason we allow the last frameworkId argument is to make it easy to kill rogue executors
-      System.err.println("Usage: StandaloneExecutorBackend <driverUrl> <executorId> <hostname> <cores> [<appid>]")
+      System.err.println(
+        "Usage: CoarseGrainedExecutorBackend <driverUrl> <executorId> <hostname> <cores> " +
+        "[<appid>]")
       System.exit(1)
     }
     run(args(0), args(1), args(2), args(3).toInt)
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index 20323ea038fc2e6bbf62836eb7052e83c34e5bb1..b773346df305d598a54ebe17be18b7626f4d3025 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -74,30 +74,33 @@ private[spark] class Executor(
   private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader)
   Thread.currentThread.setContextClassLoader(replClassLoader)
 
-  // Make any thread terminations due to uncaught exceptions kill the entire
-  // executor process to avoid surprising stalls.
-  Thread.setDefaultUncaughtExceptionHandler(
-    new Thread.UncaughtExceptionHandler {
-      override def uncaughtException(thread: Thread, exception: Throwable) {
-        try {
-          logError("Uncaught exception in thread " + thread, exception)
-
-          // We may have been called from a shutdown hook. If so, we must not call System.exit().
-          // (If we do, we will deadlock.)
-          if (!Utils.inShutdown()) {
-            if (exception.isInstanceOf[OutOfMemoryError]) {
-              System.exit(ExecutorExitCode.OOM)
-            } else {
-              System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION)
+  if (!isLocal) {
+    // Setup an uncaught exception handler for non-local mode.
+    // Make any thread terminations due to uncaught exceptions kill the entire
+    // executor process to avoid surprising stalls.
+    Thread.setDefaultUncaughtExceptionHandler(
+      new Thread.UncaughtExceptionHandler {
+        override def uncaughtException(thread: Thread, exception: Throwable) {
+          try {
+            logError("Uncaught exception in thread " + thread, exception)
+
+            // We may have been called from a shutdown hook. If so, we must not call System.exit().
+            // (If we do, we will deadlock.)
+            if (!Utils.inShutdown()) {
+              if (exception.isInstanceOf[OutOfMemoryError]) {
+                System.exit(ExecutorExitCode.OOM)
+              } else {
+                System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION)
+              }
             }
+          } catch {
+            case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM)
+            case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE)
           }
-        } catch {
-          case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM)
-          case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE)
         }
       }
-    }
-  )
+    )
+  }
 
   val executorSource = new ExecutorSource(this, executorId)
 
@@ -121,8 +124,7 @@ private[spark] class Executor(
   }
 
   // Start worker thread pool
-  val threadPool = new ThreadPoolExecutor(
-    1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable], Utils.daemonThreadFactory)
+  val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker")
 
   // Maintains the list of running tasks.
   private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index f311141148cb0c03fd6049c5f7df8c56cb566818..0b4892f98f03976e683476d052d1e33b38f15782 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -102,4 +102,9 @@ class ShuffleWriteMetrics extends Serializable {
    * Number of bytes written for a shuffle
    */
   var shuffleBytesWritten: Long = _
+
+  /**
+   * Time spent blocking on writes to disk or buffer cache, in nanoseconds.
+   */
+  var shuffleWriteTime: Long = _
 }
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
index e15a839c4e7a79ca14214fc66ca951726ad6253b..9c2fee4023be613ca8fa04c7cfdfcc5bfa64916e 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -79,7 +79,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
   private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
   private val registerRequests = new SynchronizedQueue[SendingConnection]
 
-  implicit val futureExecContext = ExecutionContext.fromExecutor(Utils.newDaemonCachedThreadPool())
+  implicit val futureExecContext = ExecutionContext.fromExecutor(
+    Utils.newDaemonCachedThreadPool("Connection manager future execution context"))
 
   private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null
 
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
index 1586dff254df2234f799a700b50333df48182f1d..546d921067175a2db075f8896e101782eb6c40c4 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
@@ -21,7 +21,7 @@ import java.io.File
 
 import org.apache.spark.Logging
 import org.apache.spark.util.Utils
-import org.apache.spark.storage.BlockId
+import org.apache.spark.storage.{BlockId, FileSegment}
 
 
 private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging {
@@ -54,8 +54,7 @@ private[spark] object ShuffleSender {
     val localDirs = args.drop(2).map(new File(_))
 
     val pResovler = new PathResolver {
-      override def getAbsolutePath(blockIdString: String): String = {
-        val blockId = BlockId(blockIdString)
+      override def getBlockLocation(blockId: BlockId): FileSegment = {
         if (!blockId.isShuffle) {
           throw new Exception("Block " + blockId + " is not a shuffle block")
         }
@@ -65,7 +64,7 @@ private[spark] object ShuffleSender {
         val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
         val subDir = new File(localDirs(dirId), "%02x".format(subDirId))
         val file = new File(subDir, blockId.name)
-        return file.getAbsolutePath
+        return new FileSegment(file, 0, file.length())
       }
     }
     val sender = new ShuffleSender(port, pResovler)
diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
index f132e2b735e532a5f14cc489a070006cf698b155..70a5a8caff839926704e3a2faf2c2a024963bf23 100644
--- a/core/src/main/scala/org/apache/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -15,6 +15,8 @@
  * limitations under the License.
  */
 
+package org.apache
+
 /**
  * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to
  * Spark, while [[org.apache.spark.rdd.RDD]] is the data type representing a distributed collection,
diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index ccaaecb85b03c6d07bc6a29a0f08868620641898..d3033ea4a627b3bdc3e878488da88fcc9eb43b5f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.rdd
 
 import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.{NullWritable, BytesWritable}
@@ -83,7 +84,7 @@ private[spark] object CheckpointRDD extends Logging {
   def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) {
     val env = SparkEnv.get
     val outputDir = new Path(path)
-    val fs = outputDir.getFileSystem(env.hadoop.newConfiguration())
+    val fs = outputDir.getFileSystem(SparkHadoopUtil.get.newConfiguration())
 
     val finalOutputName = splitIdToFile(ctx.partitionId)
     val finalOutputPath = new Path(outputDir, finalOutputName)
@@ -122,7 +123,7 @@ private[spark] object CheckpointRDD extends Logging {
 
   def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
     val env = SparkEnv.get
-    val fs = path.getFileSystem(env.hadoop.newConfiguration())
+    val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
     val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
     val fileInputStream = fs.open(path, bufferSize)
     val serializer = env.serializer.newInstance()
@@ -145,7 +146,7 @@ private[spark] object CheckpointRDD extends Logging {
     val sc = new SparkContext(cluster, "CheckpointRDD Test")
     val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
     val path = new Path(hdfsPath, "temp")
-    val fs = path.getFileSystem(env.hadoop.newConfiguration())
+    val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
     sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
     val cpRDD = new CheckpointRDD[Int](sc, path.toString)
     assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same")
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index fad042c7ae184daf60d14717bd52a93fed399c26..32901a508f53b34d44c01fc631e9927680bd312a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -29,6 +29,7 @@ import org.apache.hadoop.util.ReflectionUtils
 
 import org.apache.spark._
 import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.util.NextIterator
 import org.apache.hadoop.conf.{Configuration, Configurable}
 
@@ -198,10 +199,10 @@ private[spark] object HadoopRDD {
    * The three methods below are helpers for accessing the local map, a property of the SparkEnv of
    * the local process.
    */
-  def getCachedMetadata(key: String) = SparkEnv.get.hadoop.hadoopJobMetadata.get(key)
+  def getCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.get(key)
 
-  def containsCachedMetadata(key: String) = SparkEnv.get.hadoop.hadoopJobMetadata.containsKey(key)
+  def containsCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.containsKey(key)
 
   def putCachedMetadata(key: String, value: Any) =
-    SparkEnv.get.hadoop.hadoopJobMetadata.put(key, value)
+    SparkEnv.get.hadoopJobMetadata.put(key, value)
 }
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 0355618e435bd3d00357406ca167ab324aae4645..6e88be6f6ac64bb3d99bb56aa276c789dfba7c6a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -265,6 +265,19 @@ abstract class RDD[T: ClassManifest](
 
   def distinct(): RDD[T] = distinct(partitions.size)
 
+  /**
+   * Return a new RDD that has exactly numPartitions partitions.
+   *
+   * Can increase or decrease the level of parallelism in this RDD. Internally, this uses
+   * a shuffle to redistribute data.
+   *
+   * If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
+   * which can avoid performing a shuffle.
+   */
+  def repartition(numPartitions: Int): RDD[T] = {
+    coalesce(numPartitions, true)
+  }
+
   /**
    * Return a new RDD that is reduced into `numPartitions` partitions.
    *
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 7fb614402b7b7fc5f268dc8c1d823833cfc9b075..4cef0825dd6c0aab711df8a58700bd37fb91c0e0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -52,23 +52,29 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH
 private[spark]
 class DAGScheduler(
     taskSched: TaskScheduler,
-    mapOutputTracker: MapOutputTracker,
+    mapOutputTracker: MapOutputTrackerMaster,
     blockManagerMaster: BlockManagerMaster,
     env: SparkEnv)
-  extends TaskSchedulerListener with Logging {
+  extends Logging {
 
   def this(taskSched: TaskScheduler) {
-    this(taskSched, SparkEnv.get.mapOutputTracker, SparkEnv.get.blockManager.master, SparkEnv.get)
+    this(taskSched, SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster],
+      SparkEnv.get.blockManager.master, SparkEnv.get)
   }
-  taskSched.setListener(this)
+  taskSched.setDAGScheduler(this)
 
   // Called by TaskScheduler to report task's starting.
-  override def taskStarted(task: Task[_], taskInfo: TaskInfo) {
+  def taskStarted(task: Task[_], taskInfo: TaskInfo) {
     eventQueue.put(BeginEvent(task, taskInfo))
   }
 
+  // Called to report that a task has completed and results are being fetched remotely.
+  def taskGettingResult(task: Task[_], taskInfo: TaskInfo) {
+    eventQueue.put(GettingResultEvent(task, taskInfo))
+  }
+
   // Called by TaskScheduler to report task completions or failures.
-  override def taskEnded(
+  def taskEnded(
       task: Task[_],
       reason: TaskEndReason,
       result: Any,
@@ -79,18 +85,18 @@ class DAGScheduler(
   }
 
   // Called by TaskScheduler when an executor fails.
-  override def executorLost(execId: String) {
+  def executorLost(execId: String) {
     eventQueue.put(ExecutorLost(execId))
   }
 
   // Called by TaskScheduler when a host is added
-  override def executorGained(execId: String, host: String) {
+  def executorGained(execId: String, host: String) {
     eventQueue.put(ExecutorGained(execId, host))
   }
 
   // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or
   // cancellation of the job itself.
-  override def taskSetFailed(taskSet: TaskSet, reason: String) {
+  def taskSetFailed(taskSet: TaskSet, reason: String) {
     eventQueue.put(TaskSetFailed(taskSet, reason))
   }
 
@@ -182,7 +188,7 @@ class DAGScheduler(
     shuffleToMapStage.get(shuffleDep.shuffleId) match {
       case Some(stage) => stage
       case None =>
-        val stage = newStage(shuffleDep.rdd, Some(shuffleDep), jobId)
+        val stage = newStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, Some(shuffleDep), jobId)
         shuffleToMapStage(shuffleDep.shuffleId) = stage
         stage
     }
@@ -195,6 +201,7 @@ class DAGScheduler(
    */
   private def newStage(
       rdd: RDD[_],
+      numTasks: Int,
       shuffleDep: Option[ShuffleDependency[_,_]],
       jobId: Int,
       callSite: Option[String] = None)
@@ -207,9 +214,10 @@ class DAGScheduler(
       mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
     }
     val id = nextStageId.getAndIncrement()
-    val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
+    val stage =
+      new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
     stageIdToStage(id) = stage
-    stageToInfos(stage) = StageInfo(stage)
+    stageToInfos(stage) = new StageInfo(stage)
     stage
   }
 
@@ -277,11 +285,6 @@ class DAGScheduler(
       resultHandler: (Int, U) => Unit,
       properties: Properties = null): JobWaiter[U] =
   {
-    val jobId = nextJobId.getAndIncrement()
-    if (partitions.size == 0) {
-      return new JobWaiter[U](this, jobId, 0, resultHandler)
-    }
-
     // Check to make sure we are not launching a task on a partition that does not exist.
     val maxPartitions = rdd.partitions.length
     partitions.find(p => p >= maxPartitions).foreach { p =>
@@ -290,6 +293,11 @@ class DAGScheduler(
           "Total number of partitions: " + maxPartitions)
     }
 
+    val jobId = nextJobId.getAndIncrement()
+    if (partitions.size == 0) {
+      return new JobWaiter[U](this, jobId, 0, resultHandler)
+    }
+
     assert(partitions.size > 0)
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
     val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
@@ -342,6 +350,11 @@ class DAGScheduler(
     eventQueue.put(JobCancelled(jobId))
   }
 
+  def cancelJobGroup(groupId: String) {
+    logInfo("Asked to cancel job group " + groupId)
+    eventQueue.put(JobGroupCancelled(groupId))
+  }
+
   /**
    * Cancel all jobs that are running or waiting in the queue.
    */
@@ -356,7 +369,7 @@ class DAGScheduler(
   private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
     event match {
       case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
-        val finalStage = newStage(rdd, None, jobId, Some(callSite))
+        val finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
         val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
         clearCacheLocs()
         logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length +
@@ -381,6 +394,17 @@ class DAGScheduler(
           taskSched.cancelTasks(stage.id)
         }
 
+      case JobGroupCancelled(groupId) =>
+        // Cancel all jobs belonging to this job group.
+        // First finds all active jobs with this group id, and then kill stages for them.
+        val jobIds = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
+          .map(_.jobId)
+        if (!jobIds.isEmpty) {
+          running.filter(stage => jobIds.contains(stage.jobId)).foreach { stage =>
+            taskSched.cancelTasks(stage.id)
+          }
+        }
+
       case AllJobsCancelled =>
         // Cancel all running jobs.
         running.foreach { stage =>
@@ -396,6 +420,9 @@ class DAGScheduler(
       case begin: BeginEvent =>
         listenerBus.post(SparkListenerTaskStart(begin.task, begin.taskInfo))
 
+      case gettingResult: GettingResultEvent =>
+        listenerBus.post(SparkListenerTaskGettingResult(gettingResult.task, gettingResult.taskInfo))
+
       case completion: CompletionEvent =>
         listenerBus.post(SparkListenerTaskEnd(
           completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))
@@ -568,7 +595,7 @@ class DAGScheduler(
 
     // must be run listener before possible NotSerializableException
     // should be "StageSubmitted" first and then "JobEnded"
-    listenerBus.post(SparkListenerStageSubmitted(stage, tasks.size, properties))
+    listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties))
 
     if (tasks.size > 0) {
       // Preemptively serialize a task to make sure it can be serialized. We are catching this
@@ -589,9 +616,7 @@ class DAGScheduler(
       logDebug("New pending tasks: " + myPending)
       taskSched.submitTasks(
         new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties))
-      if (!stage.submissionTime.isDefined) {
-        stage.submissionTime = Some(System.currentTimeMillis())
-      }
+      stageToInfos(stage).submissionTime = Some(System.currentTimeMillis())
     } else {
       logDebug("Stage " + stage + " is actually done; %b %d %d".format(
         stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions))
@@ -613,12 +638,12 @@ class DAGScheduler(
     val stage = stageIdToStage(task.stageId)
 
     def markStageAsFinished(stage: Stage) = {
-      val serviceTime = stage.submissionTime match {
+      val serviceTime = stageToInfos(stage).submissionTime match {
         case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0)
-        case _ => "Unkown"
+        case _ => "Unknown"
       }
       logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime))
-      stage.completionTime = Some(System.currentTimeMillis)
+      stageToInfos(stage).completionTime = Some(System.currentTimeMillis())
       listenerBus.post(StageCompleted(stageToInfos(stage)))
       running -= stage
     }
@@ -788,7 +813,7 @@ class DAGScheduler(
    */
   private def abortStage(failedStage: Stage, reason: String) {
     val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq
-    failedStage.completionTime = Some(System.currentTimeMillis())
+    stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis())
     for (resultStage <- dependentStages) {
       val job = resultStageToJob(resultStage)
       val error = new SparkException("Job aborted: " + reason)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
index ee89bfb38dcd57a5705d25ac8e5cc2a9320101b2..708d221d60caf8cd981780d513a89d372a1baf97 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -46,11 +46,16 @@ private[scheduler] case class JobSubmitted(
 
 private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent
 
+private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent
+
 private[scheduler] case object AllJobsCancelled extends DAGSchedulerEvent
 
 private[scheduler]
 case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent
 
+private[scheduler]
+case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent 
+
 private[scheduler] case class CompletionEvent(
     task: Task[_],
     reason: TaskEndReason,
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 370ccd183c37849ce269798d9aa4a0a891368343..1791ee660db0212ca7afc4dbc99cb18639087009 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.scheduler
 
 import org.apache.spark.{Logging, SparkEnv}
+import org.apache.spark.deploy.SparkHadoopUtil
 import scala.collection.immutable.Set
 import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
 import org.apache.hadoop.security.UserGroupInformation
@@ -87,9 +88,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 
   // This method does not expect failures, since validate has already passed ...
   private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = {
-    val env = SparkEnv.get
     val conf = new JobConf(configuration)
-    env.hadoop.addCredentials(conf)
+    SparkHadoopUtil.get.addCredentials(conf)
     FileInputFormat.setInputPaths(conf, path)
 
     val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] =
@@ -108,9 +108,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 
   // This method does not expect failures, since validate has already passed ...
   private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = {
-    val env = SparkEnv.get
     val jobConf = new JobConf(configuration)
-    env.hadoop.addCredentials(jobConf)
+    SparkHadoopUtil.get.addCredentials(jobConf)
     FileInputFormat.setInputPaths(jobConf, path)
 
     val instance: org.apache.hadoop.mapred.InputFormat[_, _] =
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index 3628b1b078de8b0f21ce95d14f2fe6bade88b0fa..12b0d74fb5346867bdafd27bb3dd877bf277b489 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -24,56 +24,54 @@ import java.text.SimpleDateFormat
 import java.util.{Date, Properties}
 import java.util.concurrent.LinkedBlockingQueue
 
-import scala.collection.mutable.{Map, HashMap, ListBuffer}
-import scala.io.Source
+import scala.collection.mutable.{HashMap, ListBuffer}
 
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.executor.TaskMetrics
 
-// Used to record runtime information for each job, including RDD graph 
-// tasks' start/stop shuffle information and information from outside
-
+/**
+ * A logger class to record runtime information for jobs in Spark. This class outputs one log file
+ * per Spark job with information such as RDD graph, tasks start/stop, shuffle information.
+ *
+ * @param logDirName The base directory for the log files.
+ */
 class JobLogger(val logDirName: String) extends SparkListener with Logging {
-  private val logDir =  
-    if (System.getenv("SPARK_LOG_DIR") != null)  
-      System.getenv("SPARK_LOG_DIR")
-    else 
-      "/tmp/spark"
+
+  private val logDir = Option(System.getenv("SPARK_LOG_DIR")).getOrElse("/tmp/spark")
+
   private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] 
   private val stageIDToJobID = new HashMap[Int, Int]
   private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]]
   private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
   private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents]
-  
+
   createLogDir()
   def this() = this(String.valueOf(System.currentTimeMillis()))
-  
-  def getLogDir = logDir
-  def getJobIDtoPrintWriter = jobIDToPrintWriter
-  def getStageIDToJobID = stageIDToJobID
-  def getJobIDToStages = jobIDToStages
-  def getEventQueue = eventQueue
-  
+
+  // The following 5 functions are used only in testing.
+  private[scheduler] def getLogDir = logDir
+  private[scheduler] def getJobIDtoPrintWriter = jobIDToPrintWriter
+  private[scheduler] def getStageIDToJobID = stageIDToJobID
+  private[scheduler] def getJobIDToStages = jobIDToStages
+  private[scheduler] def getEventQueue = eventQueue
+
   // Create a folder for log files, the folder's name is the creation time of the jobLogger
   protected def createLogDir() {
     val dir = new File(logDir + "/" + logDirName + "/")
-    if (dir.exists()) {
-      return
-    }
-    if (dir.mkdirs() == false) {
-      logError("create log directory error:" + logDir + "/" + logDirName + "/")
+    if (!dir.exists() && !dir.mkdirs()) {
+      logError("Error creating log directory: " + logDir + "/" + logDirName + "/")
     }
   }
   
   // Create a log file for one job, the file name is the jobID
   protected def createLogWriter(jobID: Int) {
-    try{
+    try {
       val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID)
       jobIDToPrintWriter += (jobID -> fileWriter)
-      } catch {
-        case e: FileNotFoundException => e.printStackTrace()
-      }
+    } catch {
+      case e: FileNotFoundException => e.printStackTrace()
+    }
   }
   
   // Close log file, and clean the stage relationship in stageIDToJobID 
@@ -118,10 +116,9 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
     def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = {
       var rddList = new ListBuffer[RDD[_]]
       rddList += rdd
-      rdd.dependencies.foreach{ dep => dep match {
-          case shufDep: ShuffleDependency[_,_] =>
-          case _ => rddList ++= getRddsInStage(dep.rdd)
-        }
+      rdd.dependencies.foreach {
+        case shufDep: ShuffleDependency[_, _] =>
+        case dep: Dependency[_] => rddList ++= getRddsInStage(dep.rdd)
       }
       rddList
     }
@@ -161,29 +158,27 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
   protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) {
     val rddInfo = "RDD_ID=" + rdd.id + "(" + getRddName(rdd) + "," + rdd.generator + ")"
     jobLogInfo(jobID, indentString(indent) + rddInfo, false)
-    rdd.dependencies.foreach{ dep => dep match {
-        case shufDep: ShuffleDependency[_,_] => 
-          val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId
-          jobLogInfo(jobID, indentString(indent + 1) + depInfo, false)
-        case _ => recordRddInStageGraph(jobID, dep.rdd, indent + 1)
-      }
+    rdd.dependencies.foreach {
+      case shufDep: ShuffleDependency[_, _] =>
+        val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId
+        jobLogInfo(jobID, indentString(indent + 1) + depInfo, false)
+      case dep: Dependency[_] => recordRddInStageGraph(jobID, dep.rdd, indent + 1)
     }
   }
   
   protected def recordStageDepGraph(jobID: Int, stage: Stage, indent: Int = 0) {
-    var stageInfo: String = ""
-    if (stage.isShuffleMap) {
-      stageInfo = "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + 
-                  stage.shuffleDep.get.shuffleId
-    }else{
-      stageInfo = "STAGE_ID=" + stage.id + " RESULT_STAGE"
+    val stageInfo = if (stage.isShuffleMap) {
+      "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId
+    } else {
+      "STAGE_ID=" + stage.id + " RESULT_STAGE"
     }
     if (stage.jobId == jobID) {
       jobLogInfo(jobID, indentString(indent) + stageInfo, false)
       recordRddInStageGraph(jobID, stage.rdd, indent)
       stage.parents.foreach(recordStageDepGraph(jobID, _, indent + 2))
-    } else
+    } else {
       jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false)
+    }
   }
   
   // Record task metrics into job log files
@@ -193,39 +188,32 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
                " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + 
                " EXECUTOR_ID=" + taskInfo.executorId +  " HOST=" + taskMetrics.hostname
     val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime
-    val readMetrics = 
-      taskMetrics.shuffleReadMetrics match {
-        case Some(metrics) => 
-          " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + 
-          " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + 
-          " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + 
-          " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + 
-          " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + 
-          " REMOTE_FETCH_TIME=" + metrics.remoteFetchTime + 
-          " REMOTE_BYTES_READ=" + metrics.remoteBytesRead
-        case None => ""
-      }
-    val writeMetrics = 
-      taskMetrics.shuffleWriteMetrics match {
-        case Some(metrics) => 
-          " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten
-        case None => ""
-      }
+    val readMetrics = taskMetrics.shuffleReadMetrics match {
+      case Some(metrics) =>
+        " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime +
+        " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched +
+        " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched +
+        " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched +
+        " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime +
+        " REMOTE_FETCH_TIME=" + metrics.remoteFetchTime +
+        " REMOTE_BYTES_READ=" + metrics.remoteBytesRead
+      case None => ""
+    }
+    val writeMetrics = taskMetrics.shuffleWriteMetrics match {
+      case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten
+      case None => ""
+    }
     stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics)
   }
   
   override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) {
-    stageLogInfo(
-      stageSubmitted.stage.id,
-      "STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format(
-        stageSubmitted.stage.id, stageSubmitted.taskSize))
+    stageLogInfo(stageSubmitted.stage.stageId,"STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format(
+        stageSubmitted.stage.stageId, stageSubmitted.stage.numTasks))
   }
   
   override def onStageCompleted(stageCompleted: StageCompleted) {
-    stageLogInfo(
-      stageCompleted.stageInfo.stage.id,
-      "STAGE_ID=%d STATUS=COMPLETED".format(stageCompleted.stageInfo.stage.id))
-    
+    stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format(
+        stageCompleted.stage.stageId))
   }
 
   override def onTaskStart(taskStart: SparkListenerTaskStart) { }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 802791797a6f3c2e4c29bcc93fbe8cd918c1a473..24d97da6eb6e0fdef25c0b78f5d6c427fc28cb76 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -164,17 +164,19 @@ private[spark] class ShuffleMapTask(
 
       // Commit the writes. Get the size of each bucket block (total block size).
       var totalBytes = 0L
+      var totalTime = 0L
       val compressedSizes: Array[Byte] = buckets.writers.map { writer: BlockObjectWriter =>
         writer.commit()
-        writer.close()
-        val size = writer.size()
+        val size = writer.fileSegment().length
         totalBytes += size
+        totalTime += writer.timeWriting()
         MapOutputTracker.compressSize(size)
       }
 
       // Update shuffle metrics.
       val shuffleMetrics = new ShuffleWriteMetrics
       shuffleMetrics.shuffleBytesWritten = totalBytes
+      shuffleMetrics.shuffleWriteTime = totalTime
       metrics.get.shuffleWriteMetrics = Some(shuffleMetrics)
 
       new MapStatus(blockManager.blockManagerId, compressedSizes)
@@ -188,6 +190,7 @@ private[spark] class ShuffleMapTask(
     } finally {
       // Release the writers back to the shuffle block manager.
       if (shuffle != null && buckets != null) {
+        buckets.writers.foreach(_.close())
         shuffle.releaseWriters(buckets)
       }
       // Execute the callbacks on task completion.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 466baf991314b3b87a8eed078cca0af3ea5127ba..a35081f7b10d7040d8b45302ce50941cef3e7960 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -24,13 +24,16 @@ import org.apache.spark.executor.TaskMetrics
 
 sealed trait SparkListenerEvents
 
-case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties)
+case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties)
      extends SparkListenerEvents
 
-case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents
+case class StageCompleted(val stage: StageInfo) extends SparkListenerEvents
 
 case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
 
+case class SparkListenerTaskGettingResult(
+  task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
+
 case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo,
      taskMetrics: TaskMetrics) extends SparkListenerEvents
 
@@ -56,6 +59,12 @@ trait SparkListener {
    */
   def onTaskStart(taskStart: SparkListenerTaskStart) { }
 
+  /**
+   * Called when a task begins remotely fetching its result (will not be called for tasks that do
+   * not need to fetch the result remotely).
+   */
+ def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { }
+
   /**
    * Called when a task ends
    */
@@ -80,7 +89,7 @@ class StatsReportListener extends SparkListener with Logging {
   override def onStageCompleted(stageCompleted: StageCompleted) {
     import org.apache.spark.scheduler.StatsReportListener._
     implicit val sc = stageCompleted
-    this.logInfo("Finished stage: " + stageCompleted.stageInfo)
+    this.logInfo("Finished stage: " + stageCompleted.stage)
     showMillisDistribution("task runtime:", (info, _) => Some(info.duration))
 
     //shuffle write
@@ -93,7 +102,7 @@ class StatsReportListener extends SparkListener with Logging {
 
     //runtime breakdown
 
-    val runtimePcts = stageCompleted.stageInfo.taskInfos.map{
+    val runtimePcts = stageCompleted.stage.taskInfos.map{
       case (info, metrics) => RuntimePercentage(info.duration, metrics)
     }
     showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%")
@@ -111,7 +120,7 @@ object StatsReportListener extends Logging {
   val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
 
   def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = {
-    Distribution(stage.stageInfo.taskInfos.flatMap{
+    Distribution(stage.stage.taskInfos.flatMap {
       case ((info,metric)) => getMetric(info, metric)})
   }
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index 4d3e4a17ba5620281d0b0e7238fd39907d814e46..d5824e79547974e643b348b12465fa6fe78a2fe0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -49,6 +49,8 @@ private[spark] class SparkListenerBus() extends Logging {
             sparkListeners.foreach(_.onJobEnd(jobEnd))
           case taskStart: SparkListenerTaskStart =>
             sparkListeners.foreach(_.onTaskStart(taskStart))
+          case taskGettingResult: SparkListenerTaskGettingResult =>
+            sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult))
           case taskEnd: SparkListenerTaskEnd =>
             sparkListeners.foreach(_.onTaskEnd(taskEnd))
           case _ =>
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index aa293dc6b31b5c03784cd7745ce5ccd134f1fde4..7cb3fe46e5baf6c9880d804b80cb3015b64a3045 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -39,6 +39,7 @@ import org.apache.spark.storage.BlockManagerId
 private[spark] class Stage(
     val id: Int,
     val rdd: RDD[_],
+    val numTasks: Int,
     val shuffleDep: Option[ShuffleDependency[_,_]],  // Output shuffle if stage is a map stage
     val parents: List[Stage],
     val jobId: Int,
@@ -49,11 +50,6 @@ private[spark] class Stage(
   val numPartitions = rdd.partitions.size
   val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil)
   var numAvailableOutputs = 0
-
-  /** When first task was submitted to scheduler. */
-  var submissionTime: Option[Long] = None
-  var completionTime: Option[Long] = None
-
   private var nextAttemptId = 0
 
   def isAvailable: Boolean = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index b6f11969e575efbe889d13cf8ef2dcc7eaccfe0a..93599dfdc85fff358d497176bdf92519fbea9426 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -21,9 +21,16 @@ import scala.collection._
 
 import org.apache.spark.executor.TaskMetrics
 
-case class StageInfo(
-    val stage: Stage,
+class StageInfo(
+    stage: Stage,
     val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]()
 ) {
-  override def toString = stage.rdd.toString
+  val stageId = stage.id
+  /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */
+  var submissionTime: Option[Long] = None
+  var completionTime: Option[Long] = None
+  val rddName = stage.rdd.name
+  val name = stage.name
+  val numPartitions = stage.numPartitions
+  val numTasks = stage.numTasks
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 1fe0d0e4e20d7affc7b345670fde4dcd361018b4..69b42e86eae3e3dd4c737355d143b10bb18185c3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -45,7 +45,7 @@ import org.apache.spark.util.ByteBufferInputStream
  */
 private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable {
 
-  def run(attemptId: Long): T = {
+  final def run(attemptId: Long): T = {
     context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false)
     if (_killed) {
       kill()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
index 7c2a422affbbfbe4be65817121d22c06f1bb3dfd..4bae26f3a6a885c73bd1639d61d226cbd06a5ea2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
@@ -31,9 +31,25 @@ class TaskInfo(
     val host: String,
     val taskLocality: TaskLocality.TaskLocality) {
 
+  /**
+   * The time when the task started remotely getting the result. Will not be set if the
+   * task result was sent immediately when the task finished (as opposed to sending an
+   * IndirectTaskResult and later fetching the result from the block manager).
+   */
+  var gettingResultTime: Long = 0
+
+  /**
+   * The time when the task has completed successfully (including the time to remotely fetch
+   * results, if necessary).
+   */
   var finishTime: Long = 0
+
   var failed = false
 
+  def markGettingResult(time: Long = System.currentTimeMillis) {
+    gettingResultTime = time
+  }
+
   def markSuccessful(time: Long = System.currentTimeMillis) {
     finishTime = time
   }
@@ -43,6 +59,8 @@ class TaskInfo(
     failed = true
   }
 
+  def gettingResult: Boolean = gettingResultTime != 0
+
   def finished: Boolean = finishTime != 0
 
   def successful: Boolean = finished && !failed
@@ -52,6 +70,8 @@ class TaskInfo(
   def status: String = {
     if (running)
       "RUNNING"
+    else if (gettingResult)
+      "GET RESULT"
     else if (failed)
       "FAILED"
     else if (successful)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 6a51efe8d6c05f9828c62413a0dc6710b35b381e..10e047810827c974e7e9230bca60693c1e67cf40 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -24,8 +24,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
  * Each TaskScheduler schedulers task for a single SparkContext.
  * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage,
  * and are responsible for sending the tasks to the cluster, running them, retrying if there
- * are failures, and mitigating stragglers. They return events to the DAGScheduler through
- * the TaskSchedulerListener interface.
+ * are failures, and mitigating stragglers. They return events to the DAGScheduler.
  */
 private[spark] trait TaskScheduler {
 
@@ -48,8 +47,8 @@ private[spark] trait TaskScheduler {
   // Cancel a stage.
   def cancelTasks(stageId: Int)
 
-  // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called.
-  def setListener(listener: TaskSchedulerListener): Unit
+  // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called.
+  def setDAGScheduler(dagScheduler: DAGScheduler): Unit
 
   // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs.
   def defaultParallelism(): Int
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
deleted file mode 100644
index 593fa9fb93a55624573750e611c265c5de18d8a0..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler
-
-import scala.collection.mutable.Map
-
-import org.apache.spark.TaskEndReason
-import org.apache.spark.executor.TaskMetrics
-
-/**
- * Interface for getting events back from the TaskScheduler.
- */
-private[spark] trait TaskSchedulerListener {
-  // A task has started.
-  def taskStarted(task: Task[_], taskInfo: TaskInfo)
-
-  // A task has finished or failed.
-  def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any],
-                taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit
-
-  // A node was added to the cluster.
-  def executorGained(execId: String, host: String): Unit
-
-  // A node was lost from the cluster.
-  def executorLost(execId: String): Unit
-
-  // The TaskScheduler wants to abort an entire task set.
-  def taskSetFailed(taskSet: TaskSet, reason: String): Unit
-}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
index 7a72ff047442a27e06c249dd3d19ee1214f21160..85033958ef54f4e1568a3023630621e2f9cd7b35 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
@@ -79,7 +79,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   private val executorIdToHost = new HashMap[String, String]
 
   // Listener object to pass upcalls into
-  var listener: TaskSchedulerListener = null
+  var dagScheduler: DAGScheduler = null
 
   var backend: SchedulerBackend = null
 
@@ -94,8 +94,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   // This is a var so that we can reset it for testing purposes.
   private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this)
 
-  override def setListener(listener: TaskSchedulerListener) {
-    this.listener = listener
+  override def setDAGScheduler(dagScheduler: DAGScheduler) {
+    this.dagScheduler = dagScheduler
   }
 
   def initialize(context: SchedulerBackend) {
@@ -297,7 +297,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
     // Update the DAGScheduler without holding a lock on this, since that can deadlock
     if (failedExecutor != None) {
-      listener.executorLost(failedExecutor.get)
+      dagScheduler.executorLost(failedExecutor.get)
       backend.reviveOffers()
     }
     if (taskFailed) {
@@ -306,6 +306,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
   }
 
+  def handleTaskGettingResult(taskSetManager: ClusterTaskSetManager, tid: Long) {
+    taskSetManager.handleTaskGettingResult(tid)
+  }
+
   def handleSuccessfulTask(
     taskSetManager: ClusterTaskSetManager,
     tid: Long,
@@ -397,9 +401,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
          logError("Lost an executor " + executorId + " (already removed): " + reason)
       }
     }
-    // Call listener.executorLost without holding the lock on this to prevent deadlock
+    // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock
     if (failedExecutor != None) {
-      listener.executorLost(failedExecutor.get)
+      dagScheduler.executorLost(failedExecutor.get)
       backend.reviveOffers()
     }
   }
@@ -418,7 +422,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   }
 
   def executorGained(execId: String, host: String) {
-    listener.executorGained(execId, host)
+    dagScheduler.executorGained(execId, host)
   }
 
   def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
index 7bd3499300e98ac5f15791b39496cb7742b36fa3..ee47aaffcae11d1e341626791047c5e1ae2bc9ca 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
@@ -415,11 +415,17 @@ private[spark] class ClusterTaskSetManager(
   }
 
   private def taskStarted(task: Task[_], info: TaskInfo) {
-    sched.listener.taskStarted(task, info)
+    sched.dagScheduler.taskStarted(task, info)
+  }
+
+  def handleTaskGettingResult(tid: Long) = {
+    val info = taskInfos(tid)
+    info.markGettingResult()
+    sched.dagScheduler.taskGettingResult(tasks(info.index), info)
   }
 
   /**
-   * Marks the task as successful and notifies the listener that a task has ended.
+   * Marks the task as successful and notifies the DAGScheduler that a task has ended.
    */
   def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = {
     val info = taskInfos(tid)
@@ -429,7 +435,7 @@ private[spark] class ClusterTaskSetManager(
     if (!successful(index)) {
       logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format(
         tid, info.duration, info.host, tasksSuccessful, numTasks))
-      sched.listener.taskEnded(
+      sched.dagScheduler.taskEnded(
         tasks(index), Success, result.value, result.accumUpdates, info, result.metrics)
 
       // Mark successful and stop if all the tasks have succeeded.
@@ -445,7 +451,8 @@ private[spark] class ClusterTaskSetManager(
   }
 
   /**
-   * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the listener.
+   * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the
+   * DAG Scheduler.
    */
   def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) {
     val info = taskInfos(tid)
@@ -463,7 +470,7 @@ private[spark] class ClusterTaskSetManager(
       reason.foreach {
         case fetchFailed: FetchFailed =>
           logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress)
-          sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null)
+          sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null)
           successful(index) = true
           tasksSuccessful += 1
           sched.taskSetFinished(this)
@@ -472,11 +479,11 @@ private[spark] class ClusterTaskSetManager(
 
         case TaskKilled =>
           logWarning("Task %d was killed.".format(tid))
-          sched.listener.taskEnded(tasks(index), reason.get, null, null, info, null)
+          sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null)
           return
 
         case ef: ExceptionFailure =>
-          sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
+          sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
           val key = ef.description
           val now = clock.getTime()
           val (printFull, dupCount) = {
@@ -504,7 +511,7 @@ private[spark] class ClusterTaskSetManager(
 
         case TaskResultLost =>
           logWarning("Lost result for TID %s on host %s".format(tid, info.host))
-          sched.listener.taskEnded(tasks(index), TaskResultLost, null, null, info, null)
+          sched.dagScheduler.taskEnded(tasks(index), TaskResultLost, null, null, info, null)
 
         case _ => {}
       }
@@ -533,7 +540,7 @@ private[spark] class ClusterTaskSetManager(
     failed = true
     causeOfFailure = message
     // TODO: Kill running tasks if we were not terminated due to a Mesos error
-    sched.listener.taskSetFailed(taskSet, message)
+    sched.dagScheduler.taskSetFailed(taskSet, message)
     removeAllRunningTasks()
     sched.taskSetFinished(this)
   }
@@ -606,7 +613,7 @@ private[spark] class ClusterTaskSetManager(
           addPendingTask(index)
           // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our
           // stage finishes when a total of tasks.size tasks finish.
-          sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null)
+          sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null)
         }
       }
     }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
similarity index 68%
rename from core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
index 12b2fd01c0e02622fb2728e6c1dab1ce01ae50f7..53316dae2a6c81d625184fb991bd51d8967f4889 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
@@ -24,28 +24,28 @@ import org.apache.spark.scheduler.TaskDescription
 import org.apache.spark.util.{Utils, SerializableBuffer}
 
 
-private[spark] sealed trait StandaloneClusterMessage extends Serializable
+private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable
 
-private[spark] object StandaloneClusterMessages {
+private[spark] object CoarseGrainedClusterMessages {
 
   // Driver to executors
-  case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage
+  case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage
 
-  case class KillTask(taskId: Long, executor: String) extends StandaloneClusterMessage
+  case class KillTask(taskId: Long, executor: String) extends CoarseGrainedClusterMessage
 
   case class RegisteredExecutor(sparkProperties: Seq[(String, String)])
-    extends StandaloneClusterMessage
+    extends CoarseGrainedClusterMessage
 
-  case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage
+  case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage
 
   // Executors to driver
   case class RegisterExecutor(executorId: String, hostPort: String, cores: Int)
-    extends StandaloneClusterMessage {
+    extends CoarseGrainedClusterMessage {
     Utils.checkHostPort(hostPort, "Expected host port")
   }
 
   case class StatusUpdate(executorId: String, taskId: Long, state: TaskState,
-    data: SerializableBuffer) extends StandaloneClusterMessage
+    data: SerializableBuffer) extends CoarseGrainedClusterMessage
 
   object StatusUpdate {
     /** Alternate factory method that takes a ByteBuffer directly for the data field */
@@ -56,10 +56,14 @@ private[spark] object StandaloneClusterMessages {
   }
 
   // Internal messages in driver
-  case object ReviveOffers extends StandaloneClusterMessage
+  case object ReviveOffers extends CoarseGrainedClusterMessage
 
-  case object StopDriver extends StandaloneClusterMessage
+  case object StopDriver extends CoarseGrainedClusterMessage
 
-  case class RemoveExecutor(executorId: String, reason: String) extends StandaloneClusterMessage
+  case object StopExecutor extends CoarseGrainedClusterMessage
+
+  case object StopExecutors extends CoarseGrainedClusterMessage
+
+  case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage
 
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
similarity index 83%
rename from core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 08ee2182a2b91c080f0e1065534b0daefea22bc9..70f3f88401ad06663c00cb7fe6220f80fdef8953 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -30,16 +30,19 @@ import akka.util.duration._
 
 import org.apache.spark.{SparkException, Logging, TaskState}
 import org.apache.spark.scheduler.TaskDescription
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
 import org.apache.spark.util.Utils
 
 /**
- * A standalone scheduler backend, which waits for standalone executors to connect to it through
- * Akka. These may be executed in a variety of ways, such as Mesos tasks for the coarse-grained
- * Mesos mode or standalone processes for Spark's standalone deploy mode (spark.deploy.*).
+ * A scheduler backend that waits for coarse grained executors to connect to it through Akka.
+ * This backend holds onto each executor for the duration of the Spark job rather than relinquishing
+ * executors whenever a task is done and asking the scheduler to launch a new executor for
+ * each new task. Executors may be launched in a variety of ways, such as Mesos tasks for the
+ * coarse-grained Mesos mode or standalone processes for Spark's standalone deploy mode
+ * (spark.deploy.*).
  */
 private[spark]
-class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem)
+class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem)
   extends SchedulerBackend with Logging
 {
   // Use an atomic variable to track total number of cores in the cluster for simplicity and speed
@@ -98,6 +101,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
         sender ! true
         context.stop(self)
 
+      case StopExecutors =>
+        logInfo("Asking each executor to shut down")
+        for (executor <- executorActor.values) {
+          executor ! StopExecutor
+        }
+        sender ! true
+
       case RemoveExecutor(executorId, reason) =>
         removeExecutor(executorId, reason)
         sender ! true
@@ -162,16 +172,29 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
       }
     }
     driverActor = actorSystem.actorOf(
-      Props(new DriverActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME)
+      Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME)
   }
 
   private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
 
+  def stopExecutors() {
+    try {
+      if (driverActor != null) {
+        logInfo("Shutting down all executors")
+        val future = driverActor.ask(StopExecutors)(timeout)
+        Await.ready(future, timeout)
+      }
+    } catch {
+      case e: Exception =>
+        throw new SparkException("Error asking standalone scheduler to shut down executors", e)
+    }
+  }
+
   override def stop() {
     try {
       if (driverActor != null) {
         val future = driverActor.ask(StopDriver)(timeout)
-        Await.result(future, timeout)
+        Await.ready(future, timeout)
       }
     } catch {
       case e: Exception =>
@@ -194,7 +217,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
   def removeExecutor(executorId: String, reason: String) {
     try {
       val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout)
-      Await.result(future, timeout)
+      Await.ready(future, timeout)
     } catch {
       case e: Exception =>
         throw new SparkException("Error notifying standalone scheduler's driver actor", e)
@@ -202,6 +225,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
   }
 }
 
-private[spark] object StandaloneSchedulerBackend {
-  val ACTOR_NAME = "StandaloneScheduler"
+private[spark] object CoarseGrainedSchedulerBackend {
+  val ACTOR_NAME = "CoarseGrainedScheduler"
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
new file mode 100644
index 0000000000000000000000000000000000000000..d78bdbaa7a02be024af2dc6cabcb873654694384
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, FileSystem}
+import org.apache.spark.{Logging, SparkContext}
+
+private[spark] class SimrSchedulerBackend(
+    scheduler: ClusterScheduler,
+    sc: SparkContext,
+    driverFilePath: String)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+  with Logging {
+
+  val tmpPath = new Path(driverFilePath + "_tmp")
+  val filePath = new Path(driverFilePath)
+
+  val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt
+
+  override def start() {
+    super.start()
+
+    val driverUrl = "akka://spark@%s:%s/user/%s".format(
+      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+      CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+    val conf = new Configuration()
+    val fs = FileSystem.get(conf)
+
+    logInfo("Writing to HDFS file: "  + driverFilePath)
+    logInfo("Writing Akka address: "  + driverUrl)
+
+    // Create temporary file to prevent race condition where executors get empty driverUrl file
+    val temp = fs.create(tmpPath, true)
+    temp.writeUTF(driverUrl)
+    temp.writeInt(maxCores)
+    temp.close()
+
+    // "Atomic" rename
+    fs.rename(tmpPath, filePath)
+  }
+
+  override def stop() {
+    val conf = new Configuration()
+    val fs = FileSystem.get(conf)
+    fs.delete(new Path(driverFilePath), false)
+    super.stopExecutors()
+    super.stop()
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index cb88159b8d3910bc688af411c8f4acb9b3bf97fa..cefa970bb92f97f60be78dab81ff0ac1ba8276e1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -28,7 +28,7 @@ private[spark] class SparkDeploySchedulerBackend(
     sc: SparkContext,
     masters: Array[String],
     appName: String)
-  extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
   with ClientListener
   with Logging {
 
@@ -44,10 +44,10 @@ private[spark] class SparkDeploySchedulerBackend(
     // The endpoint for executors to talk to us
     val driverUrl = "akka://spark@%s:%s/user/%s".format(
       System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
-      StandaloneSchedulerBackend.ACTOR_NAME)
+      CoarseGrainedSchedulerBackend.ACTOR_NAME)
     val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
     val command = Command(
-      "org.apache.spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
+      "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)
     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
index feec8ecfe499b21808fbd02760cd7f5db675fac0..2064d97b49cc04f35cd638a65c57112abf3d9956 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
@@ -24,33 +24,16 @@ import org.apache.spark._
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult}
 import org.apache.spark.serializer.SerializerInstance
+import org.apache.spark.util.Utils
 
 /**
  * Runs a thread pool that deserializes and remotely fetches (if necessary) task results.
  */
 private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler)
   extends Logging {
-  private val MIN_THREADS = System.getProperty("spark.resultGetter.minThreads", "4").toInt
-  private val MAX_THREADS = System.getProperty("spark.resultGetter.maxThreads", "4").toInt
-  private val getTaskResultExecutor = new ThreadPoolExecutor(
-    MIN_THREADS,
-    MAX_THREADS,
-    0L,
-    TimeUnit.SECONDS,
-    new LinkedBlockingDeque[Runnable],
-    new ResultResolverThreadFactory)
-
-  class ResultResolverThreadFactory extends ThreadFactory {
-    private var counter = 0
-    private var PREFIX = "Result resolver thread"
-
-    override def newThread(r: Runnable): Thread = {
-      val thread = new Thread(r, "%s-%s".format(PREFIX, counter))
-      counter += 1
-      thread.setDaemon(true)
-      return thread
-    }
-  }
+  private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt
+  private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool(
+    THREADS, "Result resolver thread")
 
   protected val serializer = new ThreadLocal[SerializerInstance] {
     override def initialValue(): SerializerInstance = {
@@ -67,6 +50,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
             case directResult: DirectTaskResult[_] => directResult
             case IndirectTaskResult(blockId) =>
               logDebug("Fetching indirect task result for TID %s".format(tid))
+              scheduler.handleTaskGettingResult(taskSetManager, tid)
               val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId)
               if (!serializedTaskResult.isDefined) {
                 /* We won't be able to get the task result if the machine that ran the task failed
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 8f2eef9a535fc062c68ab664721750317dad9ddc..300fe693f1c5556bf5fbb509f34f133d75639bdb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -30,13 +30,14 @@ import org.apache.mesos._
 import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
 
 import org.apache.spark.{SparkException, Logging, SparkContext, TaskState}
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
 
 /**
  * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
  * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever
  * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the
- * StandaloneBackend mechanism. This class is useful for lower and more predictable latency.
+ * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable
+ * latency.
  *
  * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to
  * remove this.
@@ -46,7 +47,7 @@ private[spark] class CoarseMesosSchedulerBackend(
     sc: SparkContext,
     master: String,
     appName: String)
-  extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
   with MScheduler
   with Logging {
 
@@ -122,20 +123,20 @@ private[spark] class CoarseMesosSchedulerBackend(
     val driverUrl = "akka://spark@%s:%s/user/%s".format(
       System.getProperty("spark.driver.host"),
       System.getProperty("spark.driver.port"),
-      StandaloneSchedulerBackend.ACTOR_NAME)
+      CoarseGrainedSchedulerBackend.ACTOR_NAME)
     val uri = System.getProperty("spark.executor.uri")
     if (uri == null) {
       val runScript = new File(sparkHome, "spark-class").getCanonicalPath
       command.setValue(
-        "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
+        "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
     } else {
       // Grab everything to the first '.'. We'll use that and '*' to
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
       command.setValue(
-        "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
-          basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
+        "cd %s*; ./spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
+          .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
     return command.build()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
index b445260d1b3f121ee359cf07e017735ba5b9a874..2699f0b33e8d302fa2acd2237245a6559fcdb3a7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
@@ -81,7 +81,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
 
   val env = SparkEnv.get
   val attemptId = new AtomicInteger
-  var listener: TaskSchedulerListener = null
+  var dagScheduler: DAGScheduler = null
 
   // Application dependencies (added through SparkContext) that we've fetched so far on this node.
   // Each map holds the master's timestamp for the version of that file or JAR we got.
@@ -114,8 +114,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
     localActor = env.actorSystem.actorOf(Props(new LocalActor(this, threads)), "Test")
   }
 
-  override def setListener(listener: TaskSchedulerListener) {
-    this.listener = listener
+  override def setDAGScheduler(dagScheduler: DAGScheduler) {
+    this.dagScheduler = dagScheduler
   }
 
   override def submitTasks(taskSet: TaskSet) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
index f72e77d40fc122890d643eb988fc6ae46d4e38a8..53bf78267e02668f48cf0c2239c8c2313ad6d2ca 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
@@ -133,7 +133,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
   }
 
   def taskStarted(task: Task[_], info: TaskInfo) {
-    sched.listener.taskStarted(task, info)
+    sched.dagScheduler.taskStarted(task, info)
   }
 
   def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) {
@@ -148,7 +148,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
       }
     }
     result.metrics.resultSize = serializedData.limit()
-    sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics)
+    sched.dagScheduler.taskEnded(task, Success, result.value, result.accumUpdates, info,
+      result.metrics)
     numFinished += 1
     decreaseRunningTasks(1)
     finished(index) = true
@@ -165,7 +166,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
     decreaseRunningTasks(1)
     val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](
       serializedData, getClass.getClassLoader)
-    sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null))
+    sched.dagScheduler.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null))
     if (!finished(index)) {
       copiesRunning(index) -= 1
       numFailures(index) += 1
@@ -174,9 +175,9 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
         reason.className, reason.description, locs.mkString("\n")))
       if (numFailures(index) > MAX_TASK_FAILURES) {
         val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(
-          taskSet.id, index, 4, reason.description)
+          taskSet.id, index, MAX_TASK_FAILURES, reason.description)
         decreaseRunningTasks(runningTasks)
-        sched.listener.taskSetFailed(taskSet, errorMessage)
+        sched.dagScheduler.taskSetFailed(taskSet, errorMessage)
         // need to delete failed Taskset from schedule queue
         sched.taskSetFinished(this)
       }
@@ -184,7 +185,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
   }
 
   override def error(message: String) {
-    sched.listener.taskSetFailed(taskSet, message)
+    sched.dagScheduler.taskSetFailed(taskSet, message)
     sched.taskSetFinished(this)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
index c7efc67a4ac24262f67fac80a6f5f95a46f5633d..7156d855d873b3ec63710ca72204922aa564a5d8 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -32,7 +32,7 @@ private[spark] sealed abstract class BlockId {
   def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None
   def isRDD = isInstanceOf[RDDBlockId]
   def isShuffle = isInstanceOf[ShuffleBlockId]
-  def isBroadcast = isInstanceOf[BroadcastBlockId]
+  def isBroadcast = isInstanceOf[BroadcastBlockId] || isInstanceOf[BroadcastHelperBlockId]
 
   override def toString = name
   override def hashCode = name.hashCode
@@ -55,6 +55,10 @@ private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId {
   def name = "broadcast_" + broadcastId
 }
 
+private[spark] case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId {
+  def name = broadcastId.name + "_" + hType
+}
+
 private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId {
   def name = "taskresult_" + taskId
 }
@@ -72,6 +76,7 @@ private[spark] object BlockId {
   val RDD = "rdd_([0-9]+)_([0-9]+)".r
   val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r
   val BROADCAST = "broadcast_([0-9]+)".r
+  val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r
   val TASKRESULT = "taskresult_([0-9]+)".r
   val STREAM = "input-([0-9]+)-([0-9]+)".r
   val TEST = "test_(.*)".r
@@ -84,6 +89,8 @@ private[spark] object BlockId {
       ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt)
     case BROADCAST(broadcastId) =>
       BroadcastBlockId(broadcastId.toLong)
+    case BROADCAST_HELPER(broadcastId, hType) =>
+      BroadcastHelperBlockId(BroadcastBlockId(broadcastId.toLong), hType)
     case TASKRESULT(taskId) =>
       TaskResultBlockId(taskId.toLong)
     case STREAM(streamId, uniqueId) =>
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dbe0bda61589ca6a2d8e138113e48e8acfdaa392
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.storage
+
+import java.util.concurrent.ConcurrentHashMap
+
+private[storage] trait BlockInfo {
+  def level: StorageLevel
+  def tellMaster: Boolean
+  // To save space, 'pending' and 'failed' are encoded as special sizes:
+  @volatile var size: Long = BlockInfo.BLOCK_PENDING
+  private def pending: Boolean = size == BlockInfo.BLOCK_PENDING
+  private def failed: Boolean = size == BlockInfo.BLOCK_FAILED
+  private def initThread: Thread = BlockInfo.blockInfoInitThreads.get(this)
+
+  setInitThread()
+
+  private def setInitThread() {
+    // Set current thread as init thread - waitForReady will not block this thread
+    // (in case there is non trivial initialization which ends up calling waitForReady as part of
+    // initialization itself)
+    BlockInfo.blockInfoInitThreads.put(this, Thread.currentThread())
+  }
+
+  /**
+   * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing).
+   * Return true if the block is available, false otherwise.
+   */
+  def waitForReady(): Boolean = {
+    if (pending && initThread != Thread.currentThread()) {
+      synchronized {
+        while (pending) this.wait()
+      }
+    }
+    !failed
+  }
+
+  /** Mark this BlockInfo as ready (i.e. block is finished writing) */
+  def markReady(sizeInBytes: Long) {
+    require (sizeInBytes >= 0, "sizeInBytes was negative: " + sizeInBytes)
+    assert (pending)
+    size = sizeInBytes
+    BlockInfo.blockInfoInitThreads.remove(this)
+    synchronized {
+      this.notifyAll()
+    }
+  }
+
+  /** Mark this BlockInfo as ready but failed */
+  def markFailure() {
+    assert (pending)
+    size = BlockInfo.BLOCK_FAILED
+    BlockInfo.blockInfoInitThreads.remove(this)
+    synchronized {
+      this.notifyAll()
+    }
+  }
+}
+
+private object BlockInfo {
+  // initThread is logically a BlockInfo field, but we store it here because
+  // it's only needed while this block is in the 'pending' state and we want
+  // to minimize BlockInfo's memory footprint.
+  private val blockInfoInitThreads = new ConcurrentHashMap[BlockInfo, Thread]
+
+  private val BLOCK_PENDING: Long = -1L
+  private val BLOCK_FAILED: Long = -2L
+}
+
+// All shuffle blocks have the same `level` and `tellMaster` properties,
+// so we can save space by not storing them in each instance:
+private[storage] class ShuffleBlockInfo extends BlockInfo {
+  // These need to be defined using 'def' instead of 'val' in order for
+  // the compiler to eliminate the fields:
+  def level: StorageLevel = StorageLevel.DISK_ONLY
+  def tellMaster: Boolean = false
+}
+
+private[storage] class BlockInfoImpl(val level: StorageLevel, val tellMaster: Boolean)
+  extends BlockInfo {
+  // Intentionally left blank
+}
\ No newline at end of file
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 801f88a3dbf341245b42d0ad87c1e2139068645b..76d537f8e838a75ec9f6296e143e13011a46408d 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -20,14 +20,15 @@ package org.apache.spark.storage
 import java.io.{InputStream, OutputStream}
 import java.nio.{ByteBuffer, MappedByteBuffer}
 
-import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet}
+import scala.collection.mutable.{HashMap, ArrayBuffer}
+import scala.util.Random
 
 import akka.actor.{ActorSystem, Cancellable, Props}
 import akka.dispatch.{Await, Future}
 import akka.util.Duration
 import akka.util.duration._
 
-import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream}
 
 import org.apache.spark.{Logging, SparkEnv, SparkException}
 import org.apache.spark.io.CompressionCodec
@@ -45,74 +46,20 @@ private[spark] class BlockManager(
     maxMemory: Long)
   extends Logging {
 
-  private class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
-    @volatile var pending: Boolean = true
-    @volatile var size: Long = -1L
-    @volatile var initThread: Thread = null
-    @volatile var failed = false
-
-    setInitThread()
-
-    private def setInitThread() {
-      // Set current thread as init thread - waitForReady will not block this thread
-      // (in case there is non trivial initialization which ends up calling waitForReady as part of
-      // initialization itself)
-      this.initThread = Thread.currentThread()
-    }
-
-    /**
-     * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing).
-     * Return true if the block is available, false otherwise.
-     */
-    def waitForReady(): Boolean = {
-      if (initThread != Thread.currentThread() && pending) {
-        synchronized {
-          while (pending) this.wait()
-        }
-      }
-      !failed
-    }
-
-    /** Mark this BlockInfo as ready (i.e. block is finished writing) */
-    def markReady(sizeInBytes: Long) {
-      assert (pending)
-      size = sizeInBytes
-      initThread = null
-      failed = false
-      initThread = null
-      pending = false
-      synchronized {
-        this.notifyAll()
-      }
-    }
-
-    /** Mark this BlockInfo as ready but failed */
-    def markFailure() {
-      assert (pending)
-      size = 0
-      initThread = null
-      failed = true
-      initThread = null
-      pending = false
-      synchronized {
-        this.notifyAll()
-      }
-    }
-  }
-
   val shuffleBlockManager = new ShuffleBlockManager(this)
+  val diskBlockManager = new DiskBlockManager(
+    System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
 
   private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
 
   private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
-  private[storage] val diskStore: DiskStore =
-    new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
+  private[storage] val diskStore = new DiskStore(this, diskBlockManager)
 
   // If we use Netty for shuffle, start a new Netty-based shuffle sender service.
   private val nettyPort: Int = {
     val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean
     val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt
-    if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0
+    if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0
   }
 
   val connectionManager = new ConnectionManager(0)
@@ -269,7 +216,7 @@ private[spark] class BlockManager(
   }
 
   /**
-   * Actually send a UpdateBlockInfo message. Returns the mater's response,
+   * Actually send a UpdateBlockInfo message. Returns the master's response,
    * which will be true if the block was successfully recorded and false if
    * the slave needs to re-register.
    */
@@ -320,89 +267,14 @@ private[spark] class BlockManager(
    */
   def getLocal(blockId: BlockId): Option[Iterator[Any]] = {
     logDebug("Getting local block " + blockId)
-    val info = blockInfo.get(blockId).orNull
-    if (info != null) {
-      info.synchronized {
-
-        // In the another thread is writing the block, wait for it to become ready.
-        if (!info.waitForReady()) {
-          // If we get here, the block write failed.
-          logWarning("Block " + blockId + " was marked as failure.")
-          return None
-        }
-
-        val level = info.level
-        logDebug("Level for block " + blockId + " is " + level)
-
-        // Look for the block in memory
-        if (level.useMemory) {
-          logDebug("Getting block " + blockId + " from memory")
-          memoryStore.getValues(blockId) match {
-            case Some(iterator) =>
-              return Some(iterator)
-            case None =>
-              logDebug("Block " + blockId + " not found in memory")
-          }
-        }
-
-        // Look for block on disk, potentially loading it back into memory if required
-        if (level.useDisk) {
-          logDebug("Getting block " + blockId + " from disk")
-          if (level.useMemory && level.deserialized) {
-            diskStore.getValues(blockId) match {
-              case Some(iterator) =>
-                // Put the block back in memory before returning it
-                // TODO: Consider creating a putValues that also takes in a iterator ?
-                val elements = new ArrayBuffer[Any]
-                elements ++= iterator
-                memoryStore.putValues(blockId, elements, level, true).data match {
-                  case Left(iterator2) =>
-                    return Some(iterator2)
-                  case _ =>
-                    throw new Exception("Memory store did not return back an iterator")
-                }
-              case None =>
-                throw new Exception("Block " + blockId + " not found on disk, though it should be")
-            }
-          } else if (level.useMemory && !level.deserialized) {
-            // Read it as a byte buffer into memory first, then return it
-            diskStore.getBytes(blockId) match {
-              case Some(bytes) =>
-                // Put a copy of the block back in memory before returning it. Note that we can't
-                // put the ByteBuffer returned by the disk store as that's a memory-mapped file.
-                // The use of rewind assumes this.
-                assert (0 == bytes.position())
-                val copyForMemory = ByteBuffer.allocate(bytes.limit)
-                copyForMemory.put(bytes)
-                memoryStore.putBytes(blockId, copyForMemory, level)
-                bytes.rewind()
-                return Some(dataDeserialize(blockId, bytes))
-              case None =>
-                throw new Exception("Block " + blockId + " not found on disk, though it should be")
-            }
-          } else {
-            diskStore.getValues(blockId) match {
-              case Some(iterator) =>
-                return Some(iterator)
-              case None =>
-                throw new Exception("Block " + blockId + " not found on disk, though it should be")
-            }
-          }
-        }
-      }
-    } else {
-      logDebug("Block " + blockId + " not registered locally")
-    }
-    return None
+    doGetLocal(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]]
   }
 
   /**
    * Get block from the local block manager as serialized bytes.
    */
   def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = {
-    // TODO: This whole thing is very similar to getLocal; we need to refactor it somehow
     logDebug("Getting local block " + blockId + " as bytes")
-
     // As an optimization for map output fetches, if the block is for a shuffle, return it
     // without acquiring a lock; the disk store never deletes (recent) items so this should work
     if (blockId.isShuffle) {
@@ -413,12 +285,15 @@ private[spark] class BlockManager(
           throw new Exception("Block " + blockId + " not found on disk, though it should be")
       }
     }
+    doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]]
+  }
 
+  private def doGetLocal(blockId: BlockId, asValues: Boolean): Option[Any] = {
     val info = blockInfo.get(blockId).orNull
     if (info != null) {
       info.synchronized {
 
-        // In the another thread is writing the block, wait for it to become ready.
+        // If another thread is writing the block, wait for it to become ready.
         if (!info.waitForReady()) {
           // If we get here, the block write failed.
           logWarning("Block " + blockId + " was marked as failure.")
@@ -431,62 +306,104 @@ private[spark] class BlockManager(
         // Look for the block in memory
         if (level.useMemory) {
           logDebug("Getting block " + blockId + " from memory")
-          memoryStore.getBytes(blockId) match {
-            case Some(bytes) =>
-              return Some(bytes)
+          val result = if (asValues) {
+            memoryStore.getValues(blockId)
+          } else {
+            memoryStore.getBytes(blockId)
+          }
+          result match {
+            case Some(values) =>
+              return Some(values)
             case None =>
               logDebug("Block " + blockId + " not found in memory")
           }
         }
 
-        // Look for block on disk
+        // Look for block on disk, potentially storing it back into memory if required:
         if (level.useDisk) {
-          // Read it as a byte buffer into memory first, then return it
-          diskStore.getBytes(blockId) match {
-            case Some(bytes) =>
-              assert (0 == bytes.position())
-              if (level.useMemory) {
-                if (level.deserialized) {
-                  memoryStore.putBytes(blockId, bytes, level)
-                } else {
-                  // The memory store will hang onto the ByteBuffer, so give it a copy instead of
-                  // the memory-mapped file buffer we got from the disk store
-                  val copyForMemory = ByteBuffer.allocate(bytes.limit)
-                  copyForMemory.put(bytes)
-                  memoryStore.putBytes(blockId, copyForMemory, level)
-                }
-              }
-              bytes.rewind()
-              return Some(bytes)
+          logDebug("Getting block " + blockId + " from disk")
+          val bytes: ByteBuffer = diskStore.getBytes(blockId) match {
+            case Some(bytes) => bytes
             case None =>
               throw new Exception("Block " + blockId + " not found on disk, though it should be")
           }
+          assert (0 == bytes.position())
+
+          if (!level.useMemory) {
+            // If the block shouldn't be stored in memory, we can just return it:
+            if (asValues) {
+              return Some(dataDeserialize(blockId, bytes))
+            } else {
+              return Some(bytes)
+            }
+          } else {
+            // Otherwise, we also have to store something in the memory store:
+            if (!level.deserialized || !asValues) {
+              // We'll store the bytes in memory if the block's storage level includes
+              // "memory serialized", or if it should be cached as objects in memory
+              // but we only requested its serialized bytes:
+              val copyForMemory = ByteBuffer.allocate(bytes.limit)
+              copyForMemory.put(bytes)
+              memoryStore.putBytes(blockId, copyForMemory, level)
+              bytes.rewind()
+            }
+            if (!asValues) {
+              return Some(bytes)
+            } else {
+              val values = dataDeserialize(blockId, bytes)
+              if (level.deserialized) {
+                // Cache the values before returning them:
+                // TODO: Consider creating a putValues that also takes in a iterator?
+                val valuesBuffer = new ArrayBuffer[Any]
+                valuesBuffer ++= values
+                memoryStore.putValues(blockId, valuesBuffer, level, true).data match {
+                  case Left(values2) =>
+                    return Some(values2)
+                  case _ =>
+                    throw new Exception("Memory store did not return back an iterator")
+                }
+              } else {
+                return Some(values)
+              }
+            }
+          }
         }
       }
     } else {
       logDebug("Block " + blockId + " not registered locally")
     }
-    return None
+    None
   }
 
   /**
    * Get block from remote block managers.
    */
   def getRemote(blockId: BlockId): Option[Iterator[Any]] = {
-    if (blockId == null) {
-      throw new IllegalArgumentException("Block Id is null")
-    }
     logDebug("Getting remote block " + blockId)
-    // Get locations of block
-    val locations = master.getLocations(blockId)
+    doGetRemote(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]]
+  }
+
+  /**
+   * Get block from remote block managers as serialized bytes.
+   */
+   def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = {
+    logDebug("Getting remote block " + blockId + " as bytes")
+    doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]]
+   }
 
-    // Get block from remote locations
+  private def doGetRemote(blockId: BlockId, asValues: Boolean): Option[Any] = {
+    require(blockId != null, "BlockId is null")
+    val locations = Random.shuffle(master.getLocations(blockId))
     for (loc <- locations) {
       logDebug("Getting remote block " + blockId + " from " + loc)
       val data = BlockManagerWorker.syncGetBlock(
         GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
       if (data != null) {
-        return Some(dataDeserialize(blockId, data))
+        if (asValues) {
+          return Some(dataDeserialize(blockId, data))
+        } else {
+          return Some(data)
+        }
       }
       logDebug("The value of block " + blockId + " is null")
     }
@@ -494,31 +411,6 @@ private[spark] class BlockManager(
     return None
   }
 
-  /**
-   * Get block from remote block managers as serialized bytes.
-   */
-   def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = {
-     // TODO: As with getLocalBytes, this is very similar to getRemote and perhaps should be
-     // refactored.
-     if (blockId == null) {
-       throw new IllegalArgumentException("Block Id is null")
-     }
-     logDebug("Getting remote block " + blockId + " as bytes")
-     
-     val locations = master.getLocations(blockId)
-     for (loc <- locations) {
-       logDebug("Getting remote block " + blockId + " from " + loc)
-       val data = BlockManagerWorker.syncGetBlock(
-         GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
-       if (data != null) {
-         return Some(data)
-       }
-       logDebug("The value of block " + blockId + " is null")
-     }
-     logDebug("Block " + blockId + " not found")
-     return None
-   }
-
   /**
    * Get a block from the block manager (either local or remote).
    */
@@ -566,16 +458,22 @@ private[spark] class BlockManager(
 
   /**
    * A short circuited method to get a block writer that can write data directly to disk.
+   * The Block will be appended to the File specified by filename.
    * This is currently used for writing shuffle files out. Callers should handle error
    * cases.
    */
-  def getDiskBlockWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
+  def getDiskWriter(blockId: BlockId, filename: String, serializer: Serializer, bufferSize: Int)
     : BlockObjectWriter = {
-    val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize)
+    val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
+    val file = diskBlockManager.createBlockFile(blockId, filename, allowAppending =  true)
+    val writer = new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream)
     writer.registerCloseEventHandler(() => {
-      val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false)
+      if (shuffleBlockManager.consolidateShuffleFiles) {
+        diskBlockManager.mapBlockToFileSegment(blockId, writer.fileSegment())
+      }
+      val myInfo = new ShuffleBlockInfo()
       blockInfo.put(blockId, myInfo)
-      myInfo.markReady(writer.size())
+      myInfo.markReady(writer.fileSegment().length)
     })
     writer
   }
@@ -584,23 +482,30 @@ private[spark] class BlockManager(
    * Put a new block of values to the block manager. Returns its (estimated) size in bytes.
    */
   def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel,
-    tellMaster: Boolean = true) : Long = {
+          tellMaster: Boolean = true) : Long = {
+    require(values != null, "Values is null")
+    doPut(blockId, Left(values), level, tellMaster)
+  }
 
-    if (blockId == null) {
-      throw new IllegalArgumentException("Block Id is null")
-    }
-    if (values == null) {
-      throw new IllegalArgumentException("Values is null")
-    }
-    if (level == null || !level.isValid) {
-      throw new IllegalArgumentException("Storage level is null or invalid")
-    }
+  /**
+   * Put a new block of serialized bytes to the block manager.
+   */
+  def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel,
+               tellMaster: Boolean = true) {
+    require(bytes != null, "Bytes is null")
+    doPut(blockId, Right(bytes), level, tellMaster)
+  }
+
+  private def doPut(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer],
+                    level: StorageLevel, tellMaster: Boolean = true): Long = {
+    require(blockId != null, "BlockId is null")
+    require(level != null && level.isValid, "StorageLevel is null or invalid")
 
     // Remember the block's storage level so that we can correctly drop it to disk if it needs
     // to be dropped right after it got put into memory. Note, however, that other threads will
     // not be able to get() this block until we call markReady on its BlockInfo.
     val myInfo = {
-      val tinfo = new BlockInfo(level, tellMaster)
+      val tinfo = new BlockInfoImpl(level, tellMaster)
       // Do atomically !
       val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo)
 
@@ -610,7 +515,8 @@ private[spark] class BlockManager(
           return oldBlockOpt.get.size
         }
 
-        // TODO: So the block info exists - but previous attempt to load it (?) failed. What do we do now ? Retry on it ?
+        // TODO: So the block info exists - but previous attempt to load it (?) failed.
+        // What do we do now ? Retry on it ?
         oldBlockOpt.get
       } else {
         tinfo
@@ -619,10 +525,10 @@ private[spark] class BlockManager(
 
     val startTimeMs = System.currentTimeMillis
 
-    // If we need to replicate the data, we'll want access to the values, but because our
-    // put will read the whole iterator, there will be no values left. For the case where
-    // the put serializes data, we'll remember the bytes, above; but for the case where it
-    // doesn't, such as deserialized storage, let's rely on the put returning an Iterator.
+    // If we're storing values and we need to replicate the data, we'll want access to the values,
+    // but because our put will read the whole iterator, there will be no values left. For the
+    // case where the put serializes data, we'll remember the bytes, above; but for the case where
+    // it doesn't, such as deserialized storage, let's rely on the put returning an Iterator.
     var valuesAfterPut: Iterator[Any] = null
 
     // Ditto for the bytes after the put
@@ -631,30 +537,51 @@ private[spark] class BlockManager(
     // Size of the block in bytes (to return to caller)
     var size = 0L
 
+    // If we're storing bytes, then initiate the replication before storing them locally.
+    // This is faster as data is already serialized and ready to send.
+    val replicationFuture = if (data.isRight && level.replication > 1) {
+      val bufferView = data.right.get.duplicate() // Doesn't copy the bytes, just creates a wrapper
+      Future {
+        replicate(blockId, bufferView, level)
+      }
+    } else {
+      null
+    }
+
     myInfo.synchronized {
       logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
         + " to get into synchronized block")
 
       var marked = false
       try {
-        if (level.useMemory) {
-          // Save it just to memory first, even if it also has useDisk set to true; we will later
-          // drop it to disk if the memory store can't hold it.
-          val res = memoryStore.putValues(blockId, values, level, true)
-          size = res.size
-          res.data match {
-            case Right(newBytes) => bytesAfterPut = newBytes
-            case Left(newIterator) => valuesAfterPut = newIterator
+        data match {
+          case Left(values) => {
+            if (level.useMemory) {
+              // Save it just to memory first, even if it also has useDisk set to true; we will
+              // drop it to disk later if the memory store can't hold it.
+              val res = memoryStore.putValues(blockId, values, level, true)
+              size = res.size
+              res.data match {
+                case Right(newBytes) => bytesAfterPut = newBytes
+                case Left(newIterator) => valuesAfterPut = newIterator
+              }
+            } else {
+              // Save directly to disk.
+              // Don't get back the bytes unless we replicate them.
+              val askForBytes = level.replication > 1
+              val res = diskStore.putValues(blockId, values, level, askForBytes)
+              size = res.size
+              res.data match {
+                case Right(newBytes) => bytesAfterPut = newBytes
+                case _ =>
+              }
+            }
           }
-        } else {
-          // Save directly to disk.
-          // Don't get back the bytes unless we replicate them.
-          val askForBytes = level.replication > 1
-          val res = diskStore.putValues(blockId, values, level, askForBytes)
-          size = res.size
-          res.data match {
-            case Right(newBytes) => bytesAfterPut = newBytes
-            case _ =>
+          case Right(bytes) => {
+            bytes.rewind()
+            // Store it only in memory at first, even if useDisk is also set to true
+            (if (level.useMemory) memoryStore else diskStore).putBytes(blockId, bytes, level)
+            size = bytes.limit
           }
         }
 
@@ -679,125 +606,39 @@ private[spark] class BlockManager(
     }
     logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs))
 
-    // Replicate block if required
+    // Either we're storing bytes and we asynchronously started replication, or we're storing
+    // values and need to serialize and replicate them now:
     if (level.replication > 1) {
-      val remoteStartTime = System.currentTimeMillis
-      // Serialize the block if not already done
-      if (bytesAfterPut == null) {
-        if (valuesAfterPut == null) {
-          throw new SparkException(
-            "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.")
-        }
-        bytesAfterPut = dataSerialize(blockId, valuesAfterPut)
-      }
-      replicate(blockId, bytesAfterPut, level)
-      logDebug("Put block " + blockId + " remotely took " + Utils.getUsedTimeMs(remoteStartTime))
-    }
-    BlockManager.dispose(bytesAfterPut)
-
-    return size
-  }
-
-
-  /**
-   * Put a new block of serialized bytes to the block manager.
-   */
-  def putBytes(
-    blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {
-
-    if (blockId == null) {
-      throw new IllegalArgumentException("Block Id is null")
-    }
-    if (bytes == null) {
-      throw new IllegalArgumentException("Bytes is null")
-    }
-    if (level == null || !level.isValid) {
-      throw new IllegalArgumentException("Storage level is null or invalid")
-    }
-
-    // Remember the block's storage level so that we can correctly drop it to disk if it needs
-    // to be dropped right after it got put into memory. Note, however, that other threads will
-    // not be able to get() this block until we call markReady on its BlockInfo.
-    val myInfo = {
-      val tinfo = new BlockInfo(level, tellMaster)
-      // Do atomically !
-      val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo)
-
-      if (oldBlockOpt.isDefined) {
-        if (oldBlockOpt.get.waitForReady()) {
-          logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
-          return
-        }
-
-        // TODO: So the block info exists - but previous attempt to load it (?) failed. What do we do now ? Retry on it ?
-        oldBlockOpt.get
-      } else {
-        tinfo
-      }
-    }
-
-    val startTimeMs = System.currentTimeMillis
-
-    // Initiate the replication before storing it locally. This is faster as
-    // data is already serialized and ready for sending
-    val replicationFuture = if (level.replication > 1) {
-      val bufferView = bytes.duplicate() // Doesn't copy the bytes, just creates a wrapper
-      Future {
-        replicate(blockId, bufferView, level)
-      }
-    } else {
-      null
-    }
-
-    myInfo.synchronized {
-      logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
-        + " to get into synchronized block")
-
-      var marked = false
-      try {
-        if (level.useMemory) {
-          // Store it only in memory at first, even if useDisk is also set to true
-          bytes.rewind()
-          memoryStore.putBytes(blockId, bytes, level)
-        } else {
-          bytes.rewind()
-          diskStore.putBytes(blockId, bytes, level)
-        }
-
-        // assert (0 == bytes.position(), "" + bytes)
-
-        // Now that the block is in either the memory or disk store, let other threads read it,
-        // and tell the master about it.
-        marked = true
-        myInfo.markReady(bytes.limit)
-        if (tellMaster) {
-          reportBlockStatus(blockId, myInfo)
-        }
-      } finally {
-        // If we failed at putting the block to memory/disk, notify other possible readers
-        // that it has failed, and then remove it from the block info map.
-        if (! marked) {
-          // Note that the remove must happen before markFailure otherwise another thread
-          // could've inserted a new BlockInfo before we remove it.
-          blockInfo.remove(blockId)
-          myInfo.markFailure()
-          logWarning("Putting block " + blockId + " failed")
+      data match {
+        case Right(bytes) => Await.ready(replicationFuture, Duration.Inf)
+        case Left(values) => {
+          val remoteStartTime = System.currentTimeMillis
+          // Serialize the block if not already done
+          if (bytesAfterPut == null) {
+            if (valuesAfterPut == null) {
+              throw new SparkException(
+                "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.")
+            }
+            bytesAfterPut = dataSerialize(blockId, valuesAfterPut)
+          }
+          replicate(blockId, bytesAfterPut, level)
+          logDebug("Put block " + blockId + " remotely took " +
+            Utils.getUsedTimeMs(remoteStartTime))
         }
       }
     }
 
-    // If replication had started, then wait for it to finish
-    if (level.replication > 1) {
-      Await.ready(replicationFuture, Duration.Inf)
-    }
+    BlockManager.dispose(bytesAfterPut)
 
     if (level.replication > 1) {
-      logDebug("PutBytes for block " + blockId + " with replication took " +
+      logDebug("Put for block " + blockId + " with replication took " +
         Utils.getUsedTimeMs(startTimeMs))
     } else {
-      logDebug("PutBytes for block " + blockId + " without replication took " +
+      logDebug("Put for block " + blockId + " without replication took " +
         Utils.getUsedTimeMs(startTimeMs))
     }
+
+    size
   }
 
   /**
@@ -922,34 +763,20 @@ private[spark] class BlockManager(
 
   private def dropOldNonBroadcastBlocks(cleanupTime: Long) {
     logInfo("Dropping non broadcast blocks older than " + cleanupTime)
-    val iterator = blockInfo.internalMap.entrySet().iterator()
-    while (iterator.hasNext) {
-      val entry = iterator.next()
-      val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2)
-      if (time < cleanupTime && !id.isBroadcast) {
-        info.synchronized {
-          val level = info.level
-          if (level.useMemory) {
-            memoryStore.remove(id)
-          }
-          if (level.useDisk) {
-            diskStore.remove(id)
-          }
-          iterator.remove()
-          logInfo("Dropped block " + id)
-        }
-        reportBlockStatus(id, info)
-      }
-    }
+    dropOldBlocks(cleanupTime, !_.isBroadcast)
   }
 
   private def dropOldBroadcastBlocks(cleanupTime: Long) {
     logInfo("Dropping broadcast blocks older than " + cleanupTime)
+    dropOldBlocks(cleanupTime, _.isBroadcast)
+  }
+
+  private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)) {
     val iterator = blockInfo.internalMap.entrySet().iterator()
     while (iterator.hasNext) {
       val entry = iterator.next()
       val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2)
-      if (time < cleanupTime && id.isBroadcast) {
+      if (time < cleanupTime && shouldDrop(id)) {
         info.synchronized {
           val level = info.level
           if (level.useMemory) {
@@ -987,13 +814,24 @@ private[spark] class BlockManager(
     if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s
   }
 
+  /** Serializes into a stream. */
+  def dataSerializeStream(
+      blockId: BlockId,
+      outputStream: OutputStream,
+      values: Iterator[Any],
+      serializer: Serializer = defaultSerializer) {
+    val byteStream = new FastBufferedOutputStream(outputStream)
+    val ser = serializer.newInstance()
+    ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close()
+  }
+
+  /** Serializes into a byte buffer. */
   def dataSerialize(
       blockId: BlockId,
       values: Iterator[Any],
       serializer: Serializer = defaultSerializer): ByteBuffer = {
     val byteStream = new FastByteArrayOutputStream(4096)
-    val ser = serializer.newInstance()
-    ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close()
+    dataSerializeStream(blockId, byteStream, values, serializer)
     byteStream.trim()
     ByteBuffer.wrap(byteStream.array)
   }
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 633230c0a86f60409e2ad30645ac784a0890a866..f8cf14b503b082490339e8225bd1fc0179cbaad4 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -227,9 +227,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   }
 
   private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
-    if (id.executorId == "<driver>" && !isLocal) {
-      // Got a register message from the master node; don't register it
-    } else if (!blockManagerInfo.contains(id)) {
+    if (!blockManagerInfo.contains(id)) {
       blockManagerIdByExecutor.get(id.executorId) match {
         case Some(manager) =>
           // A block manager of the same executor already exists.
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
index 951503019f765001bd0313d17c663f6b3695afa2..3a65e5573383451bdc3d3a2a1f3e4d94a1c407f9 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
@@ -26,6 +26,7 @@ import org.apache.spark.storage.BlockManagerMessages._
  * An actor to take commands from the master to execute options. For example,
  * this is used to remove blocks from the slave's BlockManager.
  */
+private[storage]
 class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor {
   override def receive = {
 
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index 2a67800c45a4d5d85aaa4436c14dfc2958c0752b..32d2dd06943a0952f7a6763397cbb81000b17933 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -17,6 +17,13 @@
 
 package org.apache.spark.storage
 
+import java.io.{FileOutputStream, File, OutputStream}
+import java.nio.channels.FileChannel
+
+import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
+
+import org.apache.spark.Logging
+import org.apache.spark.serializer.{SerializationStream, Serializer}
 
 /**
  * An interface for writing JVM objects to some underlying storage. This interface allows
@@ -59,7 +66,129 @@ abstract class BlockObjectWriter(val blockId: BlockId) {
   def write(value: Any)
 
   /**
-   * Size of the valid writes, in bytes.
+   * Returns the file segment of committed data that this Writer has written.
+   */
+  def fileSegment(): FileSegment
+
+  /**
+   * Cumulative time spent performing blocking writes, in ns.
    */
-  def size(): Long
+  def timeWriting(): Long
+}
+
+/** BlockObjectWriter which writes directly to a file on disk. Appends to the given file. */
+class DiskBlockObjectWriter(
+                             blockId: BlockId,
+                             file: File,
+                             serializer: Serializer,
+                             bufferSize: Int,
+                             compressStream: OutputStream => OutputStream)
+  extends BlockObjectWriter(blockId)
+  with Logging
+{
+
+  /** Intercepts write calls and tracks total time spent writing. Not thread safe. */
+  private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream {
+    def timeWriting = _timeWriting
+    private var _timeWriting = 0L
+
+    private def callWithTiming(f: => Unit) = {
+      val start = System.nanoTime()
+      f
+      _timeWriting += (System.nanoTime() - start)
+    }
+
+    def write(i: Int): Unit = callWithTiming(out.write(i))
+    override def write(b: Array[Byte]) = callWithTiming(out.write(b))
+    override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len))
+  }
+
+  private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean
+
+  /** The file channel, used for repositioning / truncating the file. */
+  private var channel: FileChannel = null
+  private var bs: OutputStream = null
+  private var fos: FileOutputStream = null
+  private var ts: TimeTrackingOutputStream = null
+  private var objOut: SerializationStream = null
+  private var initialPosition = 0L
+  private var lastValidPosition = 0L
+  private var initialized = false
+  private var _timeWriting = 0L
+
+  override def open(): BlockObjectWriter = {
+    fos = new FileOutputStream(file, true)
+    ts = new TimeTrackingOutputStream(fos)
+    channel = fos.getChannel()
+    initialPosition = channel.position
+    lastValidPosition = initialPosition
+    bs = compressStream(new FastBufferedOutputStream(ts, bufferSize))
+    objOut = serializer.newInstance().serializeStream(bs)
+    initialized = true
+    this
+  }
+
+  override def close() {
+    if (initialized) {
+      if (syncWrites) {
+        // Force outstanding writes to disk and track how long it takes
+        objOut.flush()
+        val start = System.nanoTime()
+        fos.getFD.sync()
+        _timeWriting += System.nanoTime() - start
+      }
+      objOut.close()
+
+      _timeWriting += ts.timeWriting
+
+      channel = null
+      bs = null
+      fos = null
+      ts = null
+      objOut = null
+    }
+    // Invoke the close callback handler.
+    super.close()
+  }
+
+  override def isOpen: Boolean = objOut != null
+
+  override def commit(): Long = {
+    if (initialized) {
+      // NOTE: Flush the serializer first and then the compressed/buffered output stream
+      objOut.flush()
+      bs.flush()
+      val prevPos = lastValidPosition
+      lastValidPosition = channel.position()
+      lastValidPosition - prevPos
+    } else {
+      // lastValidPosition is zero if stream is uninitialized
+      lastValidPosition
+    }
+  }
+
+  override def revertPartialWrites() {
+    if (initialized) {
+      // Discard current writes. We do this by flushing the outstanding writes and
+      // truncate the file to the last valid position.
+      objOut.flush()
+      bs.flush()
+      channel.truncate(lastValidPosition)
+    }
+  }
+
+  override def write(value: Any) {
+    if (!initialized) {
+      open()
+    }
+    objOut.writeObject(value)
+  }
+
+  override def fileSegment(): FileSegment = {
+    val bytesWritten = lastValidPosition - initialPosition
+    new FileSegment(file, initialPosition, bytesWritten)
+  }
+
+  // Only valid if called after close()
+  override def timeWriting() = _timeWriting
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..bcb58ad9467e6c8ff6fcf611ec570edaebb5c735
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.storage
+
+import java.io.File
+import java.text.SimpleDateFormat
+import java.util.{Date, Random}
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.spark.Logging
+import org.apache.spark.executor.ExecutorExitCode
+import org.apache.spark.network.netty.{PathResolver, ShuffleSender}
+import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
+
+/**
+ * Creates and maintains the logical mapping between logical blocks and physical on-disk
+ * locations. By default, one block is mapped to one file with a name given by its BlockId.
+ * However, it is also possible to have a block map to only a segment of a file, by calling
+ * mapBlockToFileSegment().
+ *
+ * @param rootDirs The directories to use for storing block files. Data will be hashed among these.
+ */
+private[spark] class DiskBlockManager(rootDirs: String) extends PathResolver with Logging {
+
+  private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
+  private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt
+
+  // Create one local directory for each path mentioned in spark.local.dir; then, inside this
+  // directory, create multiple subdirectories that we will hash files into, in order to avoid
+  // having really large inodes at the top level.
+  private val localDirs: Array[File] = createLocalDirs()
+  private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
+  private var shuffleSender : ShuffleSender = null
+
+  // Stores only Blocks which have been specifically mapped to segments of files
+  // (rather than the default, which maps a Block to a whole file).
+  // This keeps our bookkeeping down, since the file system itself tracks the standalone Blocks. 
+  private val blockToFileSegmentMap = new TimeStampedHashMap[BlockId, FileSegment]
+
+  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DISK_BLOCK_MANAGER, this.cleanup)
+
+  addShutdownHook()
+
+  /**
+   * Creates a logical mapping from the given BlockId to a segment of a file.
+   * This will cause any accesses of the logical BlockId to be directed to the specified
+   * physical location.
+   */
+  def mapBlockToFileSegment(blockId: BlockId, fileSegment: FileSegment) {
+    blockToFileSegmentMap.put(blockId, fileSegment)
+  }
+
+  /**
+   * Returns the phyiscal file segment in which the given BlockId is located.
+   * If the BlockId has been mapped to a specific FileSegment, that will be returned.
+   * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly.
+   */
+  def getBlockLocation(blockId: BlockId): FileSegment = {
+    if (blockToFileSegmentMap.internalMap.containsKey(blockId)) {
+      blockToFileSegmentMap.get(blockId).get
+    } else {
+      val file = getFile(blockId.name)
+      new FileSegment(file, 0, file.length())
+    }
+  }
+
+  /**
+   * Simply returns a File to place the given Block into. This does not physically create the file.
+   * If filename is given, that file will be used. Otherwise, we will use the BlockId to get
+   * a unique filename.
+   */
+  def createBlockFile(blockId: BlockId, filename: String = "", allowAppending: Boolean): File = {
+    val actualFilename = if (filename == "") blockId.name else filename
+    val file = getFile(actualFilename)
+    if (!allowAppending && file.exists()) {
+      throw new IllegalStateException(
+        "Attempted to create file that already exists: " + actualFilename)
+    }
+    file
+  }
+
+  private def getFile(filename: String): File = {
+    // Figure out which local directory it hashes to, and which subdirectory in that
+    val hash = Utils.nonNegativeHash(filename)
+    val dirId = hash % localDirs.length
+    val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
+
+    // Create the subdirectory if it doesn't already exist
+    var subDir = subDirs(dirId)(subDirId)
+    if (subDir == null) {
+      subDir = subDirs(dirId).synchronized {
+        val old = subDirs(dirId)(subDirId)
+        if (old != null) {
+          old
+        } else {
+          val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
+          newDir.mkdir()
+          subDirs(dirId)(subDirId) = newDir
+          newDir
+        }
+      }
+    }
+
+    new File(subDir, filename)
+  }
+
+  private def createLocalDirs(): Array[File] = {
+    logDebug("Creating local directories at root dirs '" + rootDirs + "'")
+    val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
+    rootDirs.split(",").map { rootDir =>
+      var foundLocalDir = false
+      var localDir: File = null
+      var localDirId: String = null
+      var tries = 0
+      val rand = new Random()
+      while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) {
+        tries += 1
+        try {
+          localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
+          localDir = new File(rootDir, "spark-local-" + localDirId)
+          if (!localDir.exists) {
+            foundLocalDir = localDir.mkdirs()
+          }
+        } catch {
+          case e: Exception =>
+            logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e)
+        }
+      }
+      if (!foundLocalDir) {
+        logError("Failed " + MAX_DIR_CREATION_ATTEMPTS +
+          " attempts to create local dir in " + rootDir)
+        System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR)
+      }
+      logInfo("Created local directory at " + localDir)
+      localDir
+    }
+  }
+
+  private def cleanup(cleanupTime: Long) {
+    blockToFileSegmentMap.clearOldValues(cleanupTime)
+  }
+
+  private def addShutdownHook() {
+    localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir))
+    Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
+      override def run() {
+        logDebug("Shutdown hook called")
+        localDirs.foreach { localDir =>
+          try {
+            if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)
+          } catch {
+            case t: Throwable =>
+              logError("Exception while deleting local spark dir: " + localDir, t)
+          }
+        }
+
+        if (shuffleSender != null) {
+          shuffleSender.stop()
+        }
+      }
+    })
+  }
+
+  private[storage] def startShuffleBlockSender(port: Int): Int = {
+    shuffleSender = new ShuffleSender(port, this)
+    logInfo("Created ShuffleSender binding to port : " + shuffleSender.port)
+    shuffleSender.port
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index b7ca61e9381f22ed6f937b3ff751cfafc254a935..a3c496f9e05c517f198510095471ab6623b40d22 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -17,120 +17,25 @@
 
 package org.apache.spark.storage
 
-import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile}
+import java.io.{FileOutputStream, RandomAccessFile}
 import java.nio.ByteBuffer
-import java.nio.channels.FileChannel
 import java.nio.channels.FileChannel.MapMode
-import java.util.{Random, Date}
-import java.text.SimpleDateFormat
 
 import scala.collection.mutable.ArrayBuffer
 
-import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
-
-import org.apache.spark.executor.ExecutorExitCode
-import org.apache.spark.serializer.{Serializer, SerializationStream}
 import org.apache.spark.Logging
-import org.apache.spark.network.netty.ShuffleSender
-import org.apache.spark.network.netty.PathResolver
+import org.apache.spark.serializer.Serializer
 import org.apache.spark.util.Utils
 
 
 /**
  * Stores BlockManager blocks on disk.
  */
-private class DiskStore(blockManager: BlockManager, rootDirs: String)
+private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager)
   extends BlockStore(blockManager) with Logging {
 
-  class DiskBlockObjectWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
-    extends BlockObjectWriter(blockId) {
-
-    private val f: File = createFile(blockId /*, allowAppendExisting */)
-
-    // The file channel, used for repositioning / truncating the file.
-    private var channel: FileChannel = null
-    private var bs: OutputStream = null
-    private var objOut: SerializationStream = null
-    private var lastValidPosition = 0L
-    private var initialized = false
-
-    override def open(): DiskBlockObjectWriter = {
-      val fos = new FileOutputStream(f, true)
-      channel = fos.getChannel()
-      bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos, bufferSize))
-      objOut = serializer.newInstance().serializeStream(bs)
-      initialized = true
-      this
-    }
-
-    override def close() {
-      if (initialized) {
-        objOut.close()
-        channel = null
-        bs = null
-        objOut = null
-      }
-      // Invoke the close callback handler.
-      super.close()
-    }
-
-    override def isOpen: Boolean = objOut != null
-
-    // Flush the partial writes, and set valid length to be the length of the entire file.
-    // Return the number of bytes written for this commit.
-    override def commit(): Long = {
-      if (initialized) {
-        // NOTE: Flush the serializer first and then the compressed/buffered output stream
-        objOut.flush()
-        bs.flush()
-        val prevPos = lastValidPosition
-        lastValidPosition = channel.position()
-        lastValidPosition - prevPos
-      } else {
-        // lastValidPosition is zero if stream is uninitialized
-        lastValidPosition
-      }
-    }
-
-    override def revertPartialWrites() {
-      if (initialized) { 
-        // Discard current writes. We do this by flushing the outstanding writes and
-        // truncate the file to the last valid position.
-        objOut.flush()
-        bs.flush()
-        channel.truncate(lastValidPosition)
-      }
-    }
-
-    override def write(value: Any) {
-      if (!initialized) {
-        open()
-      }
-      objOut.writeObject(value)
-    }
-
-    override def size(): Long = lastValidPosition
-  }
-
-  private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
-  private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt
-
-  private var shuffleSender : ShuffleSender = null
-  // Create one local directory for each path mentioned in spark.local.dir; then, inside this
-  // directory, create multiple subdirectories that we will hash files into, in order to avoid
-  // having really large inodes at the top level.
-  private val localDirs: Array[File] = createLocalDirs()
-  private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
-
-  addShutdownHook()
-
-  def getBlockWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
-    : BlockObjectWriter = {
-    new DiskBlockObjectWriter(blockId, serializer, bufferSize)
-  }
-
   override def getSize(blockId: BlockId): Long = {
-    getFile(blockId).length()
+    diskManager.getBlockLocation(blockId).length
   }
 
   override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) {
@@ -139,27 +44,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
     val bytes = _bytes.duplicate()
     logDebug("Attempting to put block " + blockId)
     val startTime = System.currentTimeMillis
-    val file = createFile(blockId)
-    val channel = new RandomAccessFile(file, "rw").getChannel()
+    val file = diskManager.createBlockFile(blockId, allowAppending = false)
+    val channel = new FileOutputStream(file).getChannel()
     while (bytes.remaining > 0) {
       channel.write(bytes)
     }
     channel.close()
     val finishTime = System.currentTimeMillis
     logDebug("Block %s stored as %s file on disk in %d ms".format(
-      blockId, Utils.bytesToString(bytes.limit), (finishTime - startTime)))
-  }
-
-  private def getFileBytes(file: File): ByteBuffer = {
-    val length = file.length()
-    val channel = new RandomAccessFile(file, "r").getChannel()
-    val buffer = try {
-      channel.map(MapMode.READ_ONLY, 0, length)
-    } finally {
-      channel.close()
-    }
-
-    buffer
+      file.getName, Utils.bytesToString(bytes.limit), (finishTime - startTime)))
   }
 
   override def putValues(
@@ -171,21 +64,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
 
     logDebug("Attempting to write values for block " + blockId)
     val startTime = System.currentTimeMillis
-    val file = createFile(blockId)
-    val fileOut = blockManager.wrapForCompression(blockId,
-      new FastBufferedOutputStream(new FileOutputStream(file)))
-    val objOut = blockManager.defaultSerializer.newInstance().serializeStream(fileOut)
-    objOut.writeAll(values.iterator)
-    objOut.close()
-    val length = file.length()
+    val file = diskManager.createBlockFile(blockId, allowAppending = false)
+    val outputStream = new FileOutputStream(file)
+    blockManager.dataSerializeStream(blockId, outputStream, values.iterator)
+    val length = file.length
 
     val timeTaken = System.currentTimeMillis - startTime
     logDebug("Block %s stored as %s file on disk in %d ms".format(
-      blockId, Utils.bytesToString(length), timeTaken))
+      file.getName, Utils.bytesToString(length), timeTaken))
 
     if (returnValues) {
       // Return a byte buffer for the contents of the file
-      val buffer = getFileBytes(file)
+      val buffer = getBytes(blockId).get
       PutResult(length, Right(buffer))
     } else {
       PutResult(length, null)
@@ -193,13 +83,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
   }
 
   override def getBytes(blockId: BlockId): Option[ByteBuffer] = {
-    val file = getFile(blockId)
-    val bytes = getFileBytes(file)
-    Some(bytes)
+    val segment = diskManager.getBlockLocation(blockId)
+    val channel = new RandomAccessFile(segment.file, "r").getChannel()
+    val buffer = try {
+      channel.map(MapMode.READ_ONLY, segment.offset, segment.length)
+    } finally {
+      channel.close()
+    }
+    Some(buffer)
   }
 
   override def getValues(blockId: BlockId): Option[Iterator[Any]] = {
-    getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes))
+    getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer))
   }
 
   /**
@@ -211,118 +106,20 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
   }
 
   override def remove(blockId: BlockId): Boolean = {
-    val file = getFile(blockId)
-    if (file.exists()) {
+    val fileSegment = diskManager.getBlockLocation(blockId)
+    val file = fileSegment.file
+    if (file.exists() && file.length() == fileSegment.length) {
       file.delete()
     } else {
+      if (fileSegment.length < file.length()) {
+        logWarning("Could not delete block associated with only a part of a file: " + blockId)
+      }
       false
     }
   }
 
   override def contains(blockId: BlockId): Boolean = {
-    getFile(blockId).exists()
-  }
-
-  private def createFile(blockId: BlockId, allowAppendExisting: Boolean = false): File = {
-    val file = getFile(blockId)
-    if (!allowAppendExisting && file.exists()) {
-      // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task
-      // was rescheduled on the same machine as the old task.
-      logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting")
-      file.delete()
-    }
-    file
-  }
-
-  private def getFile(blockId: BlockId): File = {
-    logDebug("Getting file for block " + blockId)
-
-    // Figure out which local directory it hashes to, and which subdirectory in that
-    val hash = Utils.nonNegativeHash(blockId)
-    val dirId = hash % localDirs.length
-    val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
-
-    // Create the subdirectory if it doesn't already exist
-    var subDir = subDirs(dirId)(subDirId)
-    if (subDir == null) {
-      subDir = subDirs(dirId).synchronized {
-        val old = subDirs(dirId)(subDirId)
-        if (old != null) {
-          old
-        } else {
-          val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
-          newDir.mkdir()
-          subDirs(dirId)(subDirId) = newDir
-          newDir
-        }
-      }
-    }
-
-    new File(subDir, blockId.name)
-  }
-
-  private def createLocalDirs(): Array[File] = {
-    logDebug("Creating local directories at root dirs '" + rootDirs + "'")
-    val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
-    rootDirs.split(",").map { rootDir =>
-      var foundLocalDir = false
-      var localDir: File = null
-      var localDirId: String = null
-      var tries = 0
-      val rand = new Random()
-      while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) {
-        tries += 1
-        try {
-          localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
-          localDir = new File(rootDir, "spark-local-" + localDirId)
-          if (!localDir.exists) {
-            foundLocalDir = localDir.mkdirs()
-          }
-        } catch {
-          case e: Exception =>
-            logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e)
-        }
-      }
-      if (!foundLocalDir) {
-        logError("Failed " + MAX_DIR_CREATION_ATTEMPTS +
-          " attempts to create local dir in " + rootDir)
-        System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR)
-      }
-      logInfo("Created local directory at " + localDir)
-      localDir
-    }
-  }
-
-  private def addShutdownHook() {
-    localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir))
-    Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
-      override def run() {
-        logDebug("Shutdown hook called")
-        localDirs.foreach { localDir =>
-          try {
-            if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)
-          } catch {
-            case t: Throwable =>
-              logError("Exception while deleting local spark dir: " + localDir, t)
-          }
-        }
-        if (shuffleSender != null) {
-          shuffleSender.stop()
-        }
-      }
-    })
-  }
-
-  private[storage] def startShuffleBlockSender(port: Int): Int = {
-    val pResolver = new PathResolver {
-      override def getAbsolutePath(blockIdString: String): String = {
-        val blockId = BlockId(blockIdString)
-        if (!blockId.isShuffle) null
-        else DiskStore.this.getFile(blockId).getAbsolutePath
-      }
-    }
-    shuffleSender = new ShuffleSender(port, pResolver)
-    logInfo("Created ShuffleSender binding to port : "+ shuffleSender.port)
-    shuffleSender.port
+    val file = diskManager.getBlockLocation(blockId).file
+    file.exists()
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
new file mode 100644
index 0000000000000000000000000000000000000000..555486830a7693a42fc1e874bdee647b5b657b58
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.storage
+
+import java.io.File
+
+/**
+ * References a particular segment of a file (potentially the entire file),
+ * based off an offset and a length.
+ */
+private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) {
+  override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length)
+}
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
index f39fcd87fb0c8a7f1c5f8764775b5d144aad6f31..066e45a12b8c7a8e9784a42eba63c373e1b44378 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -17,12 +17,13 @@
 
 package org.apache.spark.storage
 
-import org.apache.spark.serializer.Serializer
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.atomic.AtomicInteger
 
+import org.apache.spark.serializer.Serializer
 
 private[spark]
-class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter])
-
+class ShuffleWriterGroup(val id: Int, val fileId: Int, val writers: Array[BlockObjectWriter])
 
 private[spark]
 trait ShuffleBlocks {
@@ -30,24 +31,66 @@ trait ShuffleBlocks {
   def releaseWriters(group: ShuffleWriterGroup)
 }
 
+/**
+ * Manages assigning disk-based block writers to shuffle tasks. Each shuffle task gets one writer
+ * per reducer.
+ *
+ * As an optimization to reduce the number of physical shuffle files produced, multiple shuffle
+ * blocks are aggregated into the same file. There is one "combined shuffle file" per reducer
+ * per concurrently executing shuffle task. As soon as a task finishes writing to its shuffle files,
+ * it releases them for another task.
+ * Regarding the implementation of this feature, shuffle files are identified by a 3-tuple:
+ *   - shuffleId: The unique id given to the entire shuffle stage.
+ *   - bucketId: The id of the output partition (i.e., reducer id)
+ *   - fileId: The unique id identifying a group of "combined shuffle files." Only one task at a
+ *       time owns a particular fileId, and this id is returned to a pool when the task finishes.
+ */
 private[spark]
 class ShuffleBlockManager(blockManager: BlockManager) {
+  // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file.
+  // TODO: Remove this once the shuffle file consolidation feature is stable.
+  val consolidateShuffleFiles =
+    System.getProperty("spark.shuffle.consolidateFiles", "true").toBoolean
+
+  var nextFileId = new AtomicInteger(0)
+  val unusedFileIds = new ConcurrentLinkedQueue[java.lang.Integer]()
 
-  def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): ShuffleBlocks = {
+  def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) = {
     new ShuffleBlocks {
       // Get a group of writers for a map task.
       override def acquireWriters(mapId: Int): ShuffleWriterGroup = {
         val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
+        val fileId = getUnusedFileId()
         val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
           val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
-          blockManager.getDiskBlockWriter(blockId, serializer, bufferSize)
+          if (consolidateShuffleFiles) {
+            val filename = physicalFileName(shuffleId, bucketId, fileId)
+            blockManager.getDiskWriter(blockId, filename, serializer, bufferSize)
+          } else {
+            blockManager.getDiskWriter(blockId, blockId.name, serializer, bufferSize)
+          }
         }
-        new ShuffleWriterGroup(mapId, writers)
+        new ShuffleWriterGroup(mapId, fileId, writers)
       }
 
-      override def releaseWriters(group: ShuffleWriterGroup) = {
-        // Nothing really to release here.
+      override def releaseWriters(group: ShuffleWriterGroup) {
+        recycleFileId(group.fileId)
       }
     }
   }
+
+  private def getUnusedFileId(): Int = {
+    val fileId = unusedFileIds.poll()
+    if (fileId == null) nextFileId.getAndIncrement() else fileId
+  }
+
+  private def recycleFileId(fileId: Int) {
+    if (consolidateShuffleFiles) {
+      unusedFileIds.add(fileId)
+    }
+  }
+
+  private def physicalFileName(shuffleId: Int, bucketId: Int, fileId: Int) = {
+    "merged_shuffle_%d_%d_%d".format(shuffleId, bucketId, fileId)
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7dcadc380542cbe93413a339d8573f835b633546
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
@@ -0,0 +1,86 @@
+package org.apache.spark.storage
+
+import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.{CountDownLatch, Executors}
+
+import org.apache.spark.serializer.KryoSerializer
+import org.apache.spark.SparkContext
+import org.apache.spark.util.Utils
+
+/**
+ * Utility for micro-benchmarking shuffle write performance.
+ *
+ * Writes simulated shuffle output from several threads and records the observed throughput.
+ */
+object StoragePerfTester {
+  def main(args: Array[String]) = {
+    /** Total amount of data to generate. Distributed evenly amongst maps and reduce splits. */
+    val dataSizeMb = Utils.memoryStringToMb(sys.env.getOrElse("OUTPUT_DATA", "1g"))
+
+    /** Number of map tasks. All tasks execute concurrently. */
+    val numMaps = sys.env.get("NUM_MAPS").map(_.toInt).getOrElse(8)
+
+    /** Number of reduce splits for each map task. */
+    val numOutputSplits = sys.env.get("NUM_REDUCERS").map(_.toInt).getOrElse(500)
+
+    val recordLength = 1000 // ~1KB records
+    val totalRecords = dataSizeMb * 1000
+    val recordsPerMap = totalRecords / numMaps
+
+    val writeData = "1" * recordLength
+    val executor = Executors.newFixedThreadPool(numMaps)
+
+    System.setProperty("spark.shuffle.compress", "false")
+    System.setProperty("spark.shuffle.sync", "true")
+
+    // This is only used to instantiate a BlockManager. All thread scheduling is done manually.
+    val sc = new SparkContext("local[4]", "Write Tester")
+    val blockManager = sc.env.blockManager
+
+    def writeOutputBytes(mapId: Int, total: AtomicLong) = {
+      val shuffle = blockManager.shuffleBlockManager.forShuffle(1, numOutputSplits,
+        new KryoSerializer())
+      val buckets = shuffle.acquireWriters(mapId)
+      for (i <- 1 to recordsPerMap) {
+        buckets.writers(i % numOutputSplits).write(writeData)
+      }
+      buckets.writers.map {w =>
+        w.commit()
+        total.addAndGet(w.fileSegment().length)
+        w.close()
+      }
+
+      shuffle.releaseWriters(buckets)
+    }
+
+    val start = System.currentTimeMillis()
+    val latch = new CountDownLatch(numMaps)
+    val totalBytes = new AtomicLong()
+    for (task <- 1 to numMaps) {
+      executor.submit(new Runnable() {
+        override def run() = {
+          try {
+            writeOutputBytes(task, totalBytes)
+            latch.countDown()
+          } catch {
+            case e: Exception =>
+              println("Exception in child thread: " + e + " " + e.getMessage)
+              System.exit(1)
+          }
+        }
+      })
+    }
+    latch.await()
+    val end = System.currentTimeMillis()
+    val time = (end - start) / 1000.0
+    val bytesPerSecond = totalBytes.get() / time
+    val bytesPerFile = (totalBytes.get() / (numOutputSplits * numMaps.toDouble)).toLong
+
+    System.err.println("files_total\t\t%s".format(numMaps * numOutputSplits))
+    System.err.println("bytes_per_file\t\t%s".format(Utils.bytesToString(bytesPerFile)))
+    System.err.println("agg_throughput\t\t%s/s".format(Utils.bytesToString(bytesPerSecond.toLong)))
+
+    executor.shutdown()
+    sc.stop()
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
index b39c0e9769d48ee27b7d3c96277fad601f1d0c73..ca5a28625b7de522c1975c8381282e8677481ad7 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
@@ -38,7 +38,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {
       val now = System.currentTimeMillis()
 
       var activeTime = 0L
-      for (tasks <- listener.stageToTasksActive.values; t <- tasks) {
+      for (tasks <- listener.stageIdToTasksActive.values; t <- tasks) {
         activeTime += t.timeRunning(now)
       }
 
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index eb3b4e8522804492e6f9ce0412f48d3b3f42c8ab..6b854740d6a2425e51cac0505a2e6ecc58cecf30 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -36,52 +36,52 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
   val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt
   val DEFAULT_POOL_NAME = "default"
 
-  val stageToPool = new HashMap[Stage, String]()
-  val stageToDescription = new HashMap[Stage, String]()
-  val poolToActiveStages = new HashMap[String, HashSet[Stage]]()
+  val stageIdToPool = new HashMap[Int, String]()
+  val stageIdToDescription = new HashMap[Int, String]()
+  val poolToActiveStages = new HashMap[String, HashSet[StageInfo]]()
 
-  val activeStages = HashSet[Stage]()
-  val completedStages = ListBuffer[Stage]()
-  val failedStages = ListBuffer[Stage]()
+  val activeStages = HashSet[StageInfo]()
+  val completedStages = ListBuffer[StageInfo]()
+  val failedStages = ListBuffer[StageInfo]()
 
   // Total metrics reflect metrics only for completed tasks
   var totalTime = 0L
   var totalShuffleRead = 0L
   var totalShuffleWrite = 0L
 
-  val stageToTime = HashMap[Int, Long]()
-  val stageToShuffleRead = HashMap[Int, Long]()
-  val stageToShuffleWrite = HashMap[Int, Long]()
-  val stageToTasksActive = HashMap[Int, HashSet[TaskInfo]]()
-  val stageToTasksComplete = HashMap[Int, Int]()
-  val stageToTasksFailed = HashMap[Int, Int]()
-  val stageToTaskInfos =
+  val stageIdToTime = HashMap[Int, Long]()
+  val stageIdToShuffleRead = HashMap[Int, Long]()
+  val stageIdToShuffleWrite = HashMap[Int, Long]()
+  val stageIdToTasksActive = HashMap[Int, HashSet[TaskInfo]]()
+  val stageIdToTasksComplete = HashMap[Int, Int]()
+  val stageIdToTasksFailed = HashMap[Int, Int]()
+  val stageIdToTaskInfos =
     HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]()
 
   override def onJobStart(jobStart: SparkListenerJobStart) {}
 
   override def onStageCompleted(stageCompleted: StageCompleted) = synchronized {
-    val stage = stageCompleted.stageInfo.stage
-    poolToActiveStages(stageToPool(stage)) -= stage
+    val stage = stageCompleted.stage
+    poolToActiveStages(stageIdToPool(stage.stageId)) -= stage
     activeStages -= stage
     completedStages += stage
     trimIfNecessary(completedStages)
   }
 
   /** If stages is too large, remove and garbage collect old stages */
-  def trimIfNecessary(stages: ListBuffer[Stage]) = synchronized {
+  def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized {
     if (stages.size > RETAINED_STAGES) {
       val toRemove = RETAINED_STAGES / 10
       stages.takeRight(toRemove).foreach( s => {
-        stageToTaskInfos.remove(s.id)
-        stageToTime.remove(s.id)
-        stageToShuffleRead.remove(s.id)
-        stageToShuffleWrite.remove(s.id)
-        stageToTasksActive.remove(s.id)
-        stageToTasksComplete.remove(s.id)
-        stageToTasksFailed.remove(s.id)
-        stageToPool.remove(s)
-        if (stageToDescription.contains(s)) {stageToDescription.remove(s)}
+        stageIdToTaskInfos.remove(s.stageId)
+        stageIdToTime.remove(s.stageId)
+        stageIdToShuffleRead.remove(s.stageId)
+        stageIdToShuffleWrite.remove(s.stageId)
+        stageIdToTasksActive.remove(s.stageId)
+        stageIdToTasksComplete.remove(s.stageId)
+        stageIdToTasksFailed.remove(s.stageId)
+        stageIdToPool.remove(s.stageId)
+        if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)}
       })
       stages.trimEnd(toRemove)
     }
@@ -95,63 +95,69 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
     val poolName = Option(stageSubmitted.properties).map {
       p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME)
     }.getOrElse(DEFAULT_POOL_NAME)
-    stageToPool(stage) = poolName
+    stageIdToPool(stage.stageId) = poolName
 
     val description = Option(stageSubmitted.properties).flatMap {
       p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION))
     }
-    description.map(d => stageToDescription(stage) = d)
+    description.map(d => stageIdToDescription(stage.stageId) = d)
 
-    val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]())
+    val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]())
     stages += stage
   }
   
   override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
     val sid = taskStart.task.stageId
-    val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
+    val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
     tasksActive += taskStart.taskInfo
-    val taskList = stageToTaskInfos.getOrElse(
+    val taskList = stageIdToTaskInfos.getOrElse(
       sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]())
     taskList += ((taskStart.taskInfo, None, None))
-    stageToTaskInfos(sid) = taskList
+    stageIdToTaskInfos(sid) = taskList
   }
- 
+
+  override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult)
+      = synchronized {
+    // Do nothing: because we don't do a deep copy of the TaskInfo, the TaskInfo in
+    // stageToTaskInfos already has the updated status.
+  }
+
   override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
     val sid = taskEnd.task.stageId
-    val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
+    val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
     tasksActive -= taskEnd.taskInfo
     val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) =
       taskEnd.reason match {
         case e: ExceptionFailure =>
-          stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1
+          stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1
           (Some(e), e.metrics)
         case _ =>
-          stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1
+          stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1
           (None, Option(taskEnd.taskMetrics))
       }
 
-    stageToTime.getOrElseUpdate(sid, 0L)
+    stageIdToTime.getOrElseUpdate(sid, 0L)
     val time = metrics.map(m => m.executorRunTime).getOrElse(0)
-    stageToTime(sid) += time
+    stageIdToTime(sid) += time
     totalTime += time
 
-    stageToShuffleRead.getOrElseUpdate(sid, 0L)
+    stageIdToShuffleRead.getOrElseUpdate(sid, 0L)
     val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s =>
       s.remoteBytesRead).getOrElse(0L)
-    stageToShuffleRead(sid) += shuffleRead
+    stageIdToShuffleRead(sid) += shuffleRead
     totalShuffleRead += shuffleRead
 
-    stageToShuffleWrite.getOrElseUpdate(sid, 0L)
+    stageIdToShuffleWrite.getOrElseUpdate(sid, 0L)
     val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s =>
       s.shuffleBytesWritten).getOrElse(0L)
-    stageToShuffleWrite(sid) += shuffleWrite
+    stageIdToShuffleWrite(sid) += shuffleWrite
     totalShuffleWrite += shuffleWrite
 
-    val taskList = stageToTaskInfos.getOrElse(
+    val taskList = stageIdToTaskInfos.getOrElse(
       sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]())
     taskList -= ((taskEnd.taskInfo, None, None))
     taskList += ((taskEnd.taskInfo, metrics, failureInfo))
-    stageToTaskInfos(sid) = taskList
+    stageIdToTaskInfos(sid) = taskList
   }
 
   override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized {
@@ -159,10 +165,15 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
       case end: SparkListenerJobEnd =>
         end.jobResult match {
           case JobFailed(ex, Some(stage)) =>
-            activeStages -= stage
-            poolToActiveStages(stageToPool(stage)) -= stage
-            failedStages += stage
-            trimIfNecessary(failedStages)
+            /* If two jobs share a stage we could get this failure message twice. So we first
+            *  check whether we've already retired this stage. */
+            val stageInfo = activeStages.filter(s => s.stageId == stage.id).headOption
+            stageInfo.foreach {s =>
+              activeStages -= s
+              poolToActiveStages(stageIdToPool(stage.id)) -= s
+              failedStages += s
+              trimIfNecessary(failedStages)
+            }
           case _ =>
         }
       case _ =>
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index 06810d8dbc2926f8ededb16d58ca4767804ed41d..cfeeccda41713c41c8acbf7c7f69dc5291cb38c4 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -21,13 +21,13 @@ import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 import scala.xml.Node
 
-import org.apache.spark.scheduler.{Schedulable, Stage}
+import org.apache.spark.scheduler.{Schedulable, StageInfo}
 import org.apache.spark.ui.UIUtils
 
 /** Table showing list of pools */
 private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) {
 
-  var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages
+  var poolToActiveStages: HashMap[String, HashSet[StageInfo]] = listener.poolToActiveStages
 
   def toNodeSeq(): Seq[Node] = {
     listener.synchronized {
@@ -35,7 +35,7 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
     }
   }
 
-  private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node],
+  private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[StageInfo]]) => Seq[Node],
     rows: Seq[Schedulable]
     ): Seq[Node] = {
     <table class="table table-bordered table-striped table-condensed sortable table-fixed">
@@ -53,7 +53,7 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
     </table>
   }
 
-  private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]])
+  private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[StageInfo]])
     : Seq[Node] = {
     val activeStages = poolToActiveStages.get(p.name) match {
       case Some(stages) => stages.size
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 163a3746ea00ef5d92ce6345f11c66bb1999e456..35b5d5fd59534938d632bd271d883d75fd5fc499 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -40,7 +40,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
       val stageId = request.getParameter("id").toInt
       val now = System.currentTimeMillis()
 
-      if (!listener.stageToTaskInfos.contains(stageId)) {
+      if (!listener.stageIdToTaskInfos.contains(stageId)) {
         val content =
           <div>
             <h4>Summary Metrics</h4> No tasks have started yet
@@ -49,23 +49,23 @@ private[spark] class StagePage(parent: JobProgressUI) {
         return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Stages)
       }
 
-      val tasks = listener.stageToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime)
+      val tasks = listener.stageIdToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime)
 
       val numCompleted = tasks.count(_._1.finished)
-      val shuffleReadBytes = listener.stageToShuffleRead.getOrElse(stageId, 0L)
+      val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L)
       val hasShuffleRead = shuffleReadBytes > 0
-      val shuffleWriteBytes = listener.stageToShuffleWrite.getOrElse(stageId, 0L)
+      val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L)
       val hasShuffleWrite = shuffleWriteBytes > 0
 
       var activeTime = 0L
-      listener.stageToTasksActive(stageId).foreach(activeTime += _.timeRunning(now))
+      listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now))
 
       val summary =
         <div>
           <ul class="unstyled">
             <li>
               <strong>CPU time: </strong>
-              {parent.formatDuration(listener.stageToTime.getOrElse(stageId, 0L) + activeTime)}
+              {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
             </li>
             {if (hasShuffleRead)
               <li>
@@ -83,10 +83,10 @@ private[spark] class StagePage(parent: JobProgressUI) {
         </div>
 
       val taskHeaders: Seq[String] =
-        Seq("Task ID", "Status", "Locality Level", "Executor", "Launch Time", "Duration") ++
-        Seq("GC Time") ++
+        Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++
+        Seq("Duration", "GC Time") ++
         {if (hasShuffleRead) Seq("Shuffle Read")  else Nil} ++
-        {if (hasShuffleWrite) Seq("Shuffle Write") else Nil} ++
+        {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++
         Seq("Errors")
 
       val taskTable = listingTable(taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite), tasks)
@@ -153,6 +153,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
     val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L)
 
     <tr>
+      <td>{info.index}</td>
       <td>{info.taskId}</td>
       <td>{info.status}</td>
       <td>{info.taskLocality}</td>
@@ -169,6 +170,8 @@ private[spark] class StagePage(parent: JobProgressUI) {
           Utils.bytesToString(s.remoteBytesRead)}.getOrElse("")}</td>
       }}
       {if (shuffleWrite) {
+      <td>{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s =>
+        parent.formatDuration(s.shuffleWriteTime / (1000 * 1000))}.getOrElse("")}</td>
         <td>{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s =>
           Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("")}</td>
       }}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index 07db8622da4718be7a73bd56d708eed4d2bd6036..d7d0441c388fab65ee5e6978d481964c6bdf703b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -22,13 +22,13 @@ import java.util.Date
 import scala.xml.Node
 import scala.collection.mutable.HashSet
 
-import org.apache.spark.scheduler.{SchedulingMode, Stage, TaskInfo}
+import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
 
 
 /** Page showing list of all ongoing and recently finished stages */
-private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressUI) {
+private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) {
 
   val listener = parent.listener
   val dateFmt = parent.dateFmt
@@ -73,40 +73,40 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
   }
 
 
-  private def stageRow(s: Stage): Seq[Node] = {
+  private def stageRow(s: StageInfo): Seq[Node] = {
     val submissionTime = s.submissionTime match {
       case Some(t) => dateFmt.format(new Date(t))
       case None => "Unknown"
     }
 
-    val shuffleRead = listener.stageToShuffleRead.getOrElse(s.id, 0L) match {
+    val shuffleRead = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L) match {
       case 0 => ""
       case b => Utils.bytesToString(b)
     }
-    val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match {
+    val shuffleWrite = listener.stageIdToShuffleWrite.getOrElse(s.stageId, 0L) match {
       case 0 => ""
       case b => Utils.bytesToString(b)
     }
 
-    val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size
-    val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0)
-    val failedTasks = listener.stageToTasksFailed.getOrElse(s.id, 0) match {
+    val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashSet[TaskInfo]()).size
+    val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0)
+    val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match {
         case f if f > 0 => "(%s failed)".format(f)
         case _ => ""
     }
-    val totalTasks = s.numPartitions
+    val totalTasks = s.numTasks
 
-    val poolName = listener.stageToPool.get(s)
+    val poolName = listener.stageIdToPool.get(s.stageId)
 
     val nameLink =
-      <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(),s.id)}>{s.name}</a>
-    val description = listener.stageToDescription.get(s)
+      <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(),s.stageId)}>{s.name}</a>
+    val description = listener.stageIdToDescription.get(s.stageId)
       .map(d => <div><em>{d}</em></div><div>{nameLink}</div>).getOrElse(nameLink)
     val finishTime = s.completionTime.getOrElse(System.currentTimeMillis())
     val duration =  s.submissionTime.map(t => finishTime - t)
 
     <tr>
-      <td>{s.id}</td>
+      <td>{s.stageId}</td>
       {if (isFairScheduler) {
         <td><a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),poolName.get)}>
           {poolName.get}</a></td>}
diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index 0ce1394c77f59b2188d4e9f715e7653931b2e651..3f963727d98ddd3aca90c8bb327e410dceb6f546 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -56,9 +56,10 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea
 }
 
 object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask",
-  "ShuffleMapTask", "BlockManager", "BroadcastVars") {
+  "ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") {
 
-  val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, SHUFFLE_MAP_TASK, BLOCK_MANAGER, BROADCAST_VARS = Value
+  val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK,
+    SHUFFLE_MAP_TASK, BLOCK_MANAGER, DISK_BLOCK_MANAGER, BROADCAST_VARS = Value
 
   type MetadataCleanerType = Value
 
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 f384875cc9af80889312ee61be325bc57ab9f7a3..0c5c12b7a83a4f86a8d3118819d2f772aae50981 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -37,6 +37,7 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream,
 import org.apache.spark.deploy.SparkHadoopUtil
 import java.nio.ByteBuffer
 import org.apache.spark.{SparkEnv, SparkException, Logging}
+import java.util.ConcurrentModificationException
 
 
 /**
@@ -280,9 +281,8 @@ private[spark] object Utils extends Logging {
         }
       case _ =>
         // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
-        val env = SparkEnv.get
         val uri = new URI(url)
-        val conf = env.hadoop.newConfiguration()
+        val conf = SparkHadoopUtil.get.newConfiguration()
         val fs = FileSystem.get(uri, conf)
         val in = fs.open(new Path(uri))
         val out = new FileOutputStream(tempFile)
@@ -447,14 +447,17 @@ private[spark] object Utils extends Logging {
     hostPortParseResults.get(hostPort)
   }
 
-  private[spark] val daemonThreadFactory: ThreadFactory =
-    new ThreadFactoryBuilder().setDaemon(true).build()
+  private val daemonThreadFactoryBuilder: ThreadFactoryBuilder =
+    new ThreadFactoryBuilder().setDaemon(true)
 
   /**
-   * Wrapper over newCachedThreadPool.
+   * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a
+   * unique, sequentially assigned integer.
    */
-  def newDaemonCachedThreadPool(): ThreadPoolExecutor =
-    Executors.newCachedThreadPool(daemonThreadFactory).asInstanceOf[ThreadPoolExecutor]
+  def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = {
+    val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build()
+    Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor]
+  }
 
   /**
    * Return the string to tell how long has passed in seconds. The passing parameter should be in
@@ -465,10 +468,13 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Wrapper over newFixedThreadPool.
+   * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a
+   * unique, sequentially assigned integer.
    */
-  def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor =
-    Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor]
+  def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = {
+    val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build()
+    Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor]
+  }
 
   private def listFilesSafely(file: File): Seq[File] = {
     val files = file.listFiles()
@@ -813,4 +819,10 @@ private[spark] object Utils extends Logging {
     // Nothing else to guard against ?
     hashAbs
   }
+
+  /** Returns a copy of the system properties that is thread-safe to iterator over. */
+  def getSystemProperties(): Map[String, String] = {
+    return System.getProperties().clone()
+      .asInstanceOf[java.util.Properties].toMap[String, String]
+  }
 }
diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala
index b3a53d928b46e2b9dc6802c3eb78e70e4f8514c8..e022accee6d082e86a14b2aa4ee8cbd354bd77bf 100644
--- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala
+++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala
@@ -20,8 +20,42 @@ package org.apache.spark
 import org.scalatest.FunSuite
 
 class BroadcastSuite extends FunSuite with LocalSparkContext {
-  
-  test("basic broadcast") {
+
+  override def afterEach() {
+    super.afterEach()
+    System.clearProperty("spark.broadcast.factory")
+  }
+
+  test("Using HttpBroadcast locally") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
+    sc = new SparkContext("local", "test")
+    val list = List(1, 2, 3, 4)
+    val listBroadcast = sc.broadcast(list)
+    val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum))
+    assert(results.collect.toSet === Set((1, 10), (2, 10)))
+  }
+
+  test("Accessing HttpBroadcast variables from multiple threads") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
+    sc = new SparkContext("local[10]", "test")
+    val list = List(1, 2, 3, 4)
+    val listBroadcast = sc.broadcast(list)
+    val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum))
+    assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet)
+  }
+
+  test("Accessing HttpBroadcast variables in a local cluster") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
+    val numSlaves = 4
+    sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test")
+    val list = List(1, 2, 3, 4)
+    val listBroadcast = sc.broadcast(list)
+    val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum))
+    assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet)
+  }
+
+  test("Using TorrentBroadcast locally") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
     sc = new SparkContext("local", "test")
     val list = List(1, 2, 3, 4)
     val listBroadcast = sc.broadcast(list)
@@ -29,11 +63,23 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
     assert(results.collect.toSet === Set((1, 10), (2, 10)))
   }
 
-  test("broadcast variables accessed in multiple threads") {
+  test("Accessing TorrentBroadcast variables from multiple threads") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
     sc = new SparkContext("local[10]", "test")
     val list = List(1, 2, 3, 4)
     val listBroadcast = sc.broadcast(list)
     val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum))
     assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet)
   }
+
+  test("Accessing TorrentBroadcast variables in a local cluster") {
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory")
+    val numSlaves = 4
+    sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test")
+    val list = List(1, 2, 3, 4)
+    val listBroadcast = sc.broadcast(list)
+    val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum))
+    assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet)
+  }
+
 }
diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
index 35d1d41af175b419f1fa18cbff60cdc5bf98eeff..c210dd5c3b4e2093aba8b1fcb522dc173041b750 100644
--- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -120,4 +120,20 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
     }.collect()
     assert(result.toSet === Set((1,2), (2,7), (3,121)))
   }
+
+  test ("Dynamically adding JARS on a standalone cluster using local: URL") {
+    sc = new SparkContext("local-cluster[1,1,512]", "test")
+    val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
+    sc.addJar(sampleJarFile.replace("file", "local"))
+    val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
+    val result = sc.parallelize(testData).reduceByKey { (x,y) =>
+      val fac = Thread.currentThread.getContextClassLoader()
+                                    .loadClass("org.uncommons.maths.Maths")
+                                    .getDeclaredMethod("factorial", classOf[Int])
+      val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
+      val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
+      a + b
+    }.collect()
+    assert(result.toSet === Set((1,2), (2,7), (3,121)))
+  }
 }
diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
index 7b0bb89ab28ce0306e852da05263b4a94b03b9f1..352036f182e24c676c9792f83b369d43f0fdb48b 100644
--- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -472,6 +472,27 @@ public class JavaAPISuite implements Serializable {
     Assert.assertEquals("[3, 7]", partitionSums.collect().toString());
   }
 
+  @Test
+  public void repartition() {
+    // Shrinking number of partitions
+    JavaRDD<Integer> in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2);
+    JavaRDD<Integer> repartitioned1 = in1.repartition(4);
+    List<List<Integer>> result1 = repartitioned1.glom().collect();
+    Assert.assertEquals(4, result1.size());
+    for (List<Integer> l: result1) {
+      Assert.assertTrue(l.size() > 0);
+    }
+
+    // Growing number of partitions
+    JavaRDD<Integer> in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4);
+    JavaRDD<Integer> repartitioned2 = in2.repartition(2);
+    List<List<Integer>> result2 = repartitioned2.glom().collect();
+    Assert.assertEquals(2, result2.size());
+    for (List<Integer> l: result2) {
+      Assert.assertTrue(l.size() > 0);
+    }
+  }
+
   @Test
   public void persist() {
     JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index a1926514913aa77ea1e37c3542b6ce9e3d96e321..d8a0e983b228faf3d6264cf6f0f5cca4a452331b 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -19,6 +19,8 @@ package org.apache.spark
 
 import java.util.concurrent.Semaphore
 
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
 import scala.concurrent.future
 import scala.concurrent.ExecutionContext.Implicits.global
 
@@ -83,6 +85,36 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf
     assert(sc.parallelize(1 to 10, 2).count === 10)
   }
 
+  test("job group") {
+    sc = new SparkContext("local[2]", "test")
+
+    // Add a listener to release the semaphore once any tasks are launched.
+    val sem = new Semaphore(0)
+    sc.dagScheduler.addSparkListener(new SparkListener {
+      override def onTaskStart(taskStart: SparkListenerTaskStart) {
+        sem.release()
+      }
+    })
+
+    // jobA is the one to be cancelled.
+    val jobA = future {
+      sc.setJobGroup("jobA", "this is a job to be cancelled")
+      sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()
+    }
+
+    sc.clearJobGroup()
+    val jobB = sc.parallelize(1 to 100, 2).countAsync()
+
+    // Block until both tasks of job A have started and cancel job A.
+    sem.acquire(2)
+    sc.cancelJobGroup("jobA")
+    val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) }
+    assert(e.getMessage contains "cancel")
+
+    // Once A is cancelled, job B should finish fairly quickly.
+    assert(jobB.get() === 100)
+  }
+
   test("two jobs sharing the same stage") {
     // sem1: make sure cancel is issued after some tasks are launched
     // sem2: make sure the first stage is not finished until cancel is issued
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 6013320eaab730c202acce87de87714c99af65fb..b7eb268bd504c1d8ed9872c60e2394795ea50c56 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -48,15 +48,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
   test("master start and stop") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker()
-    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
+    val tracker = new MapOutputTrackerMaster()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.stop()
   }
 
   test("master register and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker()
-    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
+    val tracker = new MapOutputTrackerMaster()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
@@ -74,19 +74,17 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
   test("master register and unregister and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker()
-    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
+    val tracker = new MapOutputTrackerMaster()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
-    val size1000 = MapOutputTracker.decompressSize(compressedSize1000)
-    val size10000 = MapOutputTracker.decompressSize(compressedSize10000)
     tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0),
         Array(compressedSize1000, compressedSize1000, compressedSize1000)))
     tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0),
         Array(compressedSize10000, compressedSize1000, compressedSize1000)))
 
-    // As if we had two simulatenous fetch failures
+    // As if we had two simultaneous fetch failures
     tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0))
     tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0))
 
@@ -102,9 +100,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
     System.setProperty("spark.driver.port", boundPort.toString)    // Will be cleared by LocalSparkContext
     System.setProperty("spark.hostPort", hostname + ":" + boundPort)
 
-    val masterTracker = new MapOutputTracker()
+    val masterTracker = new MapOutputTrackerMaster()
     masterTracker.trackerActor = actorSystem.actorOf(
-        Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")
+        Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")
 
     val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
     val slaveTracker = new MapOutputTracker()
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 6d1bc5e296e06beb137673088229da3750c0579c..354ab8ae5d7d5c425cd3b62ea689554cc56b3294 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -139,6 +139,26 @@ class RDDSuite extends FunSuite with SharedSparkContext {
     assert(rdd.union(emptyKv).collect().size === 2)
   }
 
+  test("repartitioned RDDs") {
+    val data = sc.parallelize(1 to 1000, 10)
+
+    // Coalesce partitions
+    val repartitioned1 = data.repartition(2)
+    assert(repartitioned1.partitions.size == 2)
+    val partitions1 = repartitioned1.glom().collect()
+    assert(partitions1(0).length > 0)
+    assert(partitions1(1).length > 0)
+    assert(repartitioned1.collect().toSet === (1 to 1000).toSet)
+
+    // Split partitions
+    val repartitioned2 = data.repartition(20)
+    assert(repartitioned2.partitions.size == 20)
+    val partitions2 = repartitioned2.glom().collect()
+    assert(partitions2(0).length > 0)
+    assert(partitions2(19).length > 0)
+    assert(repartitioned2.collect().toSet === (1 to 1000).toSet)
+  }
+
   test("coalesced RDDs") {
     val data = sc.parallelize(1 to 10, 10)
 
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 838179c6b582ef56642e7e59a0eb9ab636a35643..00f2fdd657bfc97561060e0167d8664a33827d93 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -23,7 +23,7 @@ import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.LocalSparkContext
-import org.apache.spark.MapOutputTracker
+import org.apache.spark.MapOutputTrackerMaster
 import org.apache.spark.SparkContext
 import org.apache.spark.Partition
 import org.apache.spark.TaskContext
@@ -60,11 +60,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
       taskSets += taskSet
     }
     override def cancelTasks(stageId: Int) {}
-    override def setListener(listener: TaskSchedulerListener) = {}
+    override def setDAGScheduler(dagScheduler: DAGScheduler) = {}
     override def defaultParallelism() = 2
   }
 
-  var mapOutputTracker: MapOutputTracker = null
+  var mapOutputTracker: MapOutputTrackerMaster = null
   var scheduler: DAGScheduler = null
 
   /**
@@ -99,7 +99,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     taskSets.clear()
     cacheLocations.clear()
     results.clear()
-    mapOutputTracker = new MapOutputTracker()
+    mapOutputTracker = new MapOutputTrackerMaster()
     scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) {
       override def runLocally(job: ActiveJob) {
         // don't bother with the thread while unit testing
diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
index cece60dda726b30d49091cc7950e6d8fd39fa215..8406093246dac26994aecf2ee56f558a8faa2b51 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
@@ -58,10 +58,13 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
     val parentRdd = makeRdd(4, Nil)
     val shuffleDep = new ShuffleDependency(parentRdd, null)
     val rootRdd = makeRdd(4, List(shuffleDep))
-    val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID, None)
-    val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID, None)
-    
-    joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4, null))
+    val shuffleMapStage =
+      new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobID, None)
+    val rootStage =
+      new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None)
+    val rootStageInfo = new StageInfo(rootStage)
+
+    joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null))
     joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName)
     parentRdd.setName("MyRDD")
     joblogger.getRddNameTest(parentRdd) should be ("MyRDD")
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index a549417a47a44472ec3838d3349fb0394150194e..f7f599532a96c3c61ec8a1ed46359dca8a2f5b90 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -17,16 +17,62 @@
 
 package org.apache.spark.scheduler
 
-import org.scalatest.FunSuite
-import org.apache.spark.{SparkContext, LocalSparkContext}
-import scala.collection.mutable
+import scala.collection.mutable.{Buffer, HashSet}
+
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
 import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.{LocalSparkContext, SparkContext}
 import org.apache.spark.SparkContext._
 
-class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
+class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
+    with BeforeAndAfterAll {
+  /** Length of time to wait while draining listener events. */
+  val WAIT_TIMEOUT_MILLIS = 10000
+
+  override def afterAll {
+    System.clearProperty("spark.akka.frameSize")
+  }
+
+  test("basic creation of StageInfo") {
+    sc = new SparkContext("local", "DAGSchedulerSuite")
+    val listener = new SaveStageInfo
+    sc.addSparkListener(listener)
+    val rdd1 = sc.parallelize(1 to 100, 4)
+    val rdd2 = rdd1.map(x => x.toString)
+    rdd2.setName("Target RDD")
+    rdd2.count
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
+    listener.stageInfos.size should be {1}
+    val first = listener.stageInfos.head
+    first.rddName should be {"Target RDD"}
+    first.numTasks should be {4}
+    first.numPartitions should be {4}
+    first.submissionTime should be ('defined)
+    first.completionTime should be ('defined)
+    first.taskInfos.length should be {4}
+  }
+
+  test("StageInfo with fewer tasks than partitions") {
+    sc = new SparkContext("local", "DAGSchedulerSuite")
+    val listener = new SaveStageInfo
+    sc.addSparkListener(listener)
+    val rdd1 = sc.parallelize(1 to 100, 4)
+    val rdd2 = rdd1.map(x => x.toString)
+    sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1), true)
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
+    listener.stageInfos.size should be {1}
+    val first = listener.stageInfos.head
+    first.numTasks should be {2}
+    first.numPartitions should be {4}
+  }
 
   test("local metrics") {
-    sc = new SparkContext("local[4]", "test")
+    sc = new SparkContext("local", "DAGSchedulerSuite")
     val listener = new SaveStageInfo
     sc.addSparkListener(listener)
     sc.addSparkListener(new StatsReportListener)
@@ -39,7 +85,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
 
     val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)}
     d.count()
-    val WAIT_TIMEOUT_MILLIS = 10000
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
     listener.stageInfos.size should be (1)
 
@@ -64,7 +109,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
       checkNonZeroAvg(
         stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong},
         stageInfo + " executorDeserializeTime")
-      if (stageInfo.stage.rdd.name == d4.name) {
+      if (stageInfo.rddName == d4.name) {
         checkNonZeroAvg(
           stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime},
           stageInfo + " fetchWaitTime")
@@ -72,11 +117,11 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
 
       stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) =>
         taskMetrics.resultSize should be > (0l)
-        if (isStage(stageInfo, Set(d2.name, d3.name), Set(d4.name))) {
+        if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) {
           taskMetrics.shuffleWriteMetrics should be ('defined)
           taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l)
         }
-        if (stageInfo.stage.rdd.name == d4.name) {
+        if (stageInfo.rddName == d4.name) {
           taskMetrics.shuffleReadMetrics should be ('defined)
           val sm = taskMetrics.shuffleReadMetrics.get
           sm.totalBlocksFetched should be > (0)
@@ -89,20 +134,73 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
     }
   }
 
-  def checkNonZeroAvg(m: Traversable[Long], msg: String) {
-    assert(m.sum / m.size.toDouble > 0.0, msg)
+  test("onTaskGettingResult() called when result fetched remotely") {
+    // Need to use local cluster mode here, because results are not ever returned through the
+    // block manager when using the LocalScheduler.
+    sc = new SparkContext("local-cluster[1,1,512]", "test")
+
+    val listener = new SaveTaskEvents
+    sc.addSparkListener(listener)
+ 
+    // Make a task whose result is larger than the akka frame size
+    System.setProperty("spark.akka.frameSize", "1")
+    val akkaFrameSize =
+      sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt
+    val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x)
+    assert(result === 1.to(akkaFrameSize).toArray)
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+    val TASK_INDEX = 0
+    assert(listener.startedTasks.contains(TASK_INDEX))
+    assert(listener.startedGettingResultTasks.contains(TASK_INDEX))
+    assert(listener.endedTasks.contains(TASK_INDEX))
   }
 
-  def isStage(stageInfo: StageInfo, rddNames: Set[String], excludedNames: Set[String]) = {
-    val names = Set(stageInfo.stage.rdd.name) ++ stageInfo.stage.rdd.dependencies.map{_.rdd.name}
-    !names.intersect(rddNames).isEmpty && names.intersect(excludedNames).isEmpty
+  test("onTaskGettingResult() not called when result sent directly") {
+    // Need to use local cluster mode here, because results are not ever returned through the
+    // block manager when using the LocalScheduler.
+    sc = new SparkContext("local-cluster[1,1,512]", "test")
+
+    val listener = new SaveTaskEvents
+    sc.addSparkListener(listener)
+ 
+    // Make a task whose result is larger than the akka frame size
+    val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x)
+    assert(result === 2)
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+    val TASK_INDEX = 0
+    assert(listener.startedTasks.contains(TASK_INDEX))
+    assert(listener.startedGettingResultTasks.isEmpty == true)
+    assert(listener.endedTasks.contains(TASK_INDEX))
+  }
+
+  def checkNonZeroAvg(m: Traversable[Long], msg: String) {
+    assert(m.sum / m.size.toDouble > 0.0, msg)
   }
 
   class SaveStageInfo extends SparkListener {
-    val stageInfos = mutable.Buffer[StageInfo]()
+    val stageInfos = Buffer[StageInfo]()
     override def onStageCompleted(stage: StageCompleted) {
-      stageInfos += stage.stageInfo
+      stageInfos += stage.stage
     }
   }
 
+  class SaveTaskEvents extends SparkListener {
+    val startedTasks = new HashSet[Int]()
+    val startedGettingResultTasks = new HashSet[Int]()
+    val endedTasks = new HashSet[Int]()
+
+    override def onTaskStart(taskStart: SparkListenerTaskStart) {
+      startedTasks += taskStart.taskInfo.index
+    }
+
+    override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
+        endedTasks += taskEnd.taskInfo.index
+    }
+
+    override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) {
+      startedGettingResultTasks += taskGettingResult.taskInfo.index
+    }
+  }
 }
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
index 80d0c5a5e929ac2c62c2eef41e70f75b9789d841..b97f2b19b581c3aa1f82899905bf564b5177afad 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
@@ -28,6 +28,30 @@ import org.apache.spark.executor.TaskMetrics
 import java.nio.ByteBuffer
 import org.apache.spark.util.{Utils, FakeClock}
 
+class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) {
+  override def taskStarted(task: Task[_], taskInfo: TaskInfo) {
+    taskScheduler.startedTasks += taskInfo.index
+  }
+
+  override def taskEnded(
+      task: Task[_],
+      reason: TaskEndReason,
+      result: Any,
+      accumUpdates: mutable.Map[Long, Any],
+      taskInfo: TaskInfo,
+      taskMetrics: TaskMetrics) {
+    taskScheduler.endedTasks(taskInfo.index) = reason
+  }
+
+  override def executorGained(execId: String, host: String) {}
+
+  override def executorLost(execId: String) {}
+
+  override def taskSetFailed(taskSet: TaskSet, reason: String) {
+    taskScheduler.taskSetsFailed += taskSet.id
+  }
+}
+
 /**
  * A mock ClusterScheduler implementation that just remembers information about tasks started and
  * feedback received from the TaskSetManagers. Note that it's important to initialize this with
@@ -44,30 +68,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
 
   val executors = new mutable.HashMap[String, String] ++ liveExecutors
 
-  listener = new TaskSchedulerListener {
-    def taskStarted(task: Task[_], taskInfo: TaskInfo) {
-      startedTasks += taskInfo.index
-    }
-
-    def taskEnded(
-        task: Task[_],
-        reason: TaskEndReason,
-        result: Any,
-        accumUpdates: mutable.Map[Long, Any],
-        taskInfo: TaskInfo,
-        taskMetrics: TaskMetrics)
-    {
-      endedTasks(taskInfo.index) = reason
-    }
-
-    def executorGained(execId: String, host: String) {}
-
-    def executorLost(execId: String) {}
-
-    def taskSetFailed(taskSet: TaskSet, reason: String) {
-      taskSetsFailed += taskSet.id
-    }
-  }
+  dagScheduler = new FakeDAGScheduler(this)
 
   def removeExecutor(execId: String): Unit = executors -= execId
 
diff --git a/docker/spark-test/README.md b/docker/spark-test/README.md
index addea277aa6c41015f37e585c89bffc35009e104..ec0baf6e6d4196c511f044f4cbffe478338376e0 100644
--- a/docker/spark-test/README.md
+++ b/docker/spark-test/README.md
@@ -1,10 +1,11 @@
 Spark Docker files usable for testing and development purposes.
 
 These images are intended to be run like so:
-docker run -v $SPARK_HOME:/opt/spark spark-test-master
-docker run -v $SPARK_HOME:/opt/spark spark-test-worker <master_ip>
+
+	docker run -v $SPARK_HOME:/opt/spark spark-test-master
+	docker run -v $SPARK_HOME:/opt/spark spark-test-worker spark://<master_ip>:7077
 
 Using this configuration, the containers will have their Spark directories
-mounted to your actual SPARK_HOME, allowing you to modify and recompile
+mounted to your actual `SPARK_HOME`, allowing you to modify and recompile
 your Spark source and have them immediately usable in the docker images
 (without rebuilding them).
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index f679cad713769d45abfc9f10936fa39c3d724d8b..5927f736f3579bd7a2a505999257d2ea9abbac01 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -13,7 +13,7 @@ object in your main program (called the _driver program_).
 Specifically, to run on a cluster, the SparkContext can connect to several types of _cluster managers_
 (either Spark's own standalone cluster manager or Mesos/YARN), which allocate resources across
 applications. Once connected, Spark acquires *executors* on nodes in the cluster, which are
-worker processes that run computations and store data for your application. 
+worker processes that run computations and store data for your application.
 Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to
 the executors. Finally, SparkContext sends *tasks* for the executors to run.
 
@@ -57,6 +57,18 @@ which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python)
 worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar`
 and `addFile`.
 
+## URIs for addJar / addFile
+
+- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor
+  pulls the file from the driver HTTP server
+- **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected
+- **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node.  This
+  means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker,
+  or shared via NFS, GlusterFS, etc.
+
+Note that JARs and files are copied to the working directory for each SparkContext on the executor nodes.
+Over time this can use up a significant amount of space and will need to be cleaned up.
+
 # Monitoring
 
 Each driver program has a web UI, typically on port 4040, that displays information about running
diff --git a/docs/configuration.md b/docs/configuration.md
index 7940d41a27ab5929c533ee716a1a014a65ecd2d8..97183bafdb3eabee810fe9ec122ffeef5f147611 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -149,7 +149,7 @@ Apart from these, the following properties are also available, and may be useful
   <td>spark.io.compression.codec</td>
   <td>org.apache.spark.io.<br />LZFCompressionCodec</td>
   <td>
-    The compression codec class to use for various compressions. By default, Spark provides two
+    The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, Spark provides two
     codecs: <code>org.apache.spark.io.LZFCompressionCodec</code> and <code>org.apache.spark.io.SnappyCompressionCodec</code>.
   </td>
 </tr>
@@ -319,6 +319,14 @@ Apart from these, the following properties are also available, and may be useful
     Should be greater than or equal to 1. Number of allowed retries = this value - 1.
   </td>
 </tr>
+<tr>
+  <td>spark.broadcast.blockSize</td>
+  <td>4096</td>
+  <td>
+    Size of each piece of a block in kilobytes for <code>TorrentBroadcastFactory</code>. 
+    Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, <code>BlockManager</code> might take a performance hit.
+  </td>
+</tr>
 
 </table>
 
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
index 1e5575d6570eaae02471ae055eb87e4fa57f9365..156a727026790b9e45232fdf1c44c0906921efe3 100644
--- a/docs/ec2-scripts.md
+++ b/docs/ec2-scripts.md
@@ -98,7 +98,7 @@ permissions on your private key file, you can run `launch` with the
     `bin/hadoop` script in that directory. Note that the data in this
     HDFS goes away when you stop and restart a machine.
 -   There is also a *persistent HDFS* instance in
-    `/root/presistent-hdfs` that will keep data across cluster restarts.
+    `/root/persistent-hdfs` that will keep data across cluster restarts.
     Typically each node has relatively little space of persistent data
     (about 3 GB), but you can use the `--ebs-vol-size` option to
     `spark-ec2` to attach a persistent EBS volume to each node for
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 6c2336ad0c233c36cbe1563cca90f2bfbb6dbb39..55e39b1de17a06c4c4a9e025769c66b0977cffb8 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -131,6 +131,17 @@ sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg
 Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.
 Code dependencies can be added to an existing SparkContext using its `addPyFile()` method.
 
+You can set [system properties](configuration.html#system-properties)
+using `SparkContext.setSystemProperty()` class method *before*
+instantiating SparkContext. For example, to set the amount of memory
+per executor process:
+
+{% highlight python %}
+from pyspark import SparkContext
+SparkContext.setSystemProperty('spark.executor.memory', '2g')
+sc = SparkContext("local", "App Name")
+{% endhighlight %}
+
 # API Docs
 
 [API documentation](api/pyspark/index.html) for PySpark is available as Epydoc.
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 03647a2ad26cbfb729d56a082590d388e5e25d0e..94e8563a8b69321bdd8e9922e06580a267229df0 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -142,7 +142,7 @@ All transformations in Spark are <i>lazy</i>, in that they do not compute their
 
 By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options.
 
-The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.RDD) for details):
+The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD) for details):
 
 ### Transformations
 
@@ -211,7 +211,7 @@ The following tables list the transformations and actions currently supported (s
 </tr>
 </table>
 
-A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.RDD).
+A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD).
 
 ### Actions
 
@@ -259,7 +259,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in
 </tr>
 </table>
 
-A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.RDD).
+A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD).
 
 ## RDD Persistence
 
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 835b257238e4bf04d2e0024a37f93ffaec13f0b5..851e30fe761af664cae684acbd86aa56cff88c65 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -72,6 +72,10 @@ DStreams support many of the transformations available on normal Spark RDD's:
   <td> Similar to map, but runs separately on each partition (block) of the DStream, so <i>func</i> must be of type
     Iterator[T] => Iterator[U] when running on an DStream of type T. </td>
 </tr>
+<tr>
+  <td> <b>repartition</b>(<i>numPartitions</i>) </td>
+  <td> Changes the level of parallelism in this DStream by creating more or fewer partitions. </td>
+</tr>
 <tr>
   <td> <b>union</b>(<i>otherStream</i>) </td>
   <td> Return a new DStream that contains the union of the elements in the source DStream and the argument DStream. </td>
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 65868b76b91fcfdd895ba7a14bbf2895407471e4..79848380c04aae6a1a8bc313f2dda0ecb9a74e6a 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -73,7 +73,7 @@ def parse_args():
   parser.add_option("-v", "--spark-version", default="0.8.0",
       help="Version of Spark to use: 'X.Y.Z' or a specific git hash")
   parser.add_option("--spark-git-repo", 
-      default="https://github.com/mesos/spark", 
+      default="https://github.com/apache/incubator-spark",
       help="Github repo from which to checkout supplied commit hash")
   parser.add_option("--hadoop-major-version", default="1",
       help="Major version of Hadoop (default: 1)")
diff --git a/examples/pom.xml b/examples/pom.xml
index b8c020a3211dd6fcb5bcc86b99f6c700301d4798..aee371fbc7d1ae89ab427c55e8be32de1479f7c3 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -32,13 +32,20 @@
   <url>http://spark.incubator.apache.org/</url>
 
   <repositories>
-    <!-- A repository in the local filesystem for the Kafka JAR, which we modified for Scala 2.9 -->
     <repository>
-      <id>lib</id>
-      <url>file://${project.basedir}/lib</url>
+      <id>apache-repo</id>
+      <name>Apache Repository</name>
+      <url>https://repository.apache.org/content/repositories/releases</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
     </repository>
   </repositories>
 
+
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
@@ -81,9 +88,18 @@
     </dependency>
     <dependency>
       <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka</artifactId>
-      <version>0.7.2-spark</version>  <!-- Comes from our in-project repository -->
-      <scope>provided</scope>
+      <artifactId>kafka_2.9.2</artifactId>
+      <version>0.8.0-beta1</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
@@ -137,6 +153,14 @@
           <groupId>org.apache.cassandra.deps</groupId>
           <artifactId>avro</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.sonatype.sisu.inject</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.xerial.snappy</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
   </dependencies>
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
new file mode 100644
index 0000000000000000000000000000000000000000..9a8e4209eddc7da8d87401dafe81b46f3456065a
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.examples;
+
+import java.util.Map;
+import java.util.HashMap;
+
+import com.google.common.collect.Lists;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import scala.Tuple2;
+
+/**
+ * Consumes messages from one or more topics in Kafka and does wordcount.
+ * Usage: JavaKafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>
+ *   <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
+ *   <zkQuorum> is a list of one or more zookeeper servers that make quorum
+ *   <group> is the name of kafka consumer group
+ *   <topics> is a list of one or more kafka topics to consume from
+ *   <numThreads> is the number of threads the kafka consumer should use
+ *
+ * Example:
+ *    `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ *    zoo03 my-consumer-group topic1,topic2 1`
+ */
+
+public class JavaKafkaWordCount {
+  public static void main(String[] args) {
+    if (args.length < 5) {
+      System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>");
+      System.exit(1);
+    }
+
+    // Create the context with a 1 second batch size
+    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
+            new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    int numThreads = Integer.parseInt(args[4]);
+    Map<String, Integer> topicMap = new HashMap<String, Integer>();
+    String[] topics = args[3].split(",");
+    for (String topic: topics) {
+      topicMap.put(topic, numThreads);
+    }
+
+    JavaPairDStream<String, String> messages = ssc.kafkaStream(args[1], args[2], topicMap);
+
+    JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
+      @Override
+      public String call(Tuple2<String, String> tuple2) throws Exception {
+        return tuple2._2();
+      }
+    });
+
+    JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterable<String> call(String x) {
+        return Lists.newArrayList(x.split(" "));
+      }
+    });
+
+    JavaPairDStream<String, Integer> wordCounts = words.map(
+      new PairFunction<String, String, Integer>() {
+        @Override
+        public Tuple2<String, Integer> call(String s) throws Exception {
+          return new Tuple2<String, Integer>(s, 1);
+        }
+      }).reduceByKey(new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer i1, Integer i2) throws Exception {
+          return i1 + i2;
+        }
+      });
+
+    wordCounts.print();
+    ssc.start();
+  }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index 868ff81f67a82837ed0c108374c009d37a9a75bb..529709c2f95383b10bcfded51fc48268a9c9b7e1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -22,12 +22,19 @@ import org.apache.spark.SparkContext
 object BroadcastTest {
   def main(args: Array[String]) {
     if (args.length == 0) {
-      System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
+      System.err.println("Usage: BroadcastTest <master> [slices] [numElem] [broadcastAlgo] [blockSize]")
       System.exit(1)
     }  
     
-    val sc = new SparkContext(args(0), "Broadcast Test",
+    val bcName = if (args.length > 3) args(3) else "Http"
+    val blockSize = if (args.length > 4) args(4) else "4096"
+
+    System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + "BroadcastFactory")
+    System.setProperty("spark.broadcast.blockSize", blockSize)
+
+    val sc = new SparkContext(args(0), "Broadcast Test 2",
       System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    
     val slices = if (args.length > 1) args(1).toInt else 2
     val num = if (args.length > 2) args(2).toInt else 1000000
 
@@ -36,13 +43,15 @@ object BroadcastTest {
       arr1(i) = i
     }
     
-    for (i <- 0 until 2) {
+    for (i <- 0 until 3) {
       println("Iteration " + i)
       println("===========")
+      val startTime = System.nanoTime
       val barr1 = sc.broadcast(arr1)
       sc.parallelize(1 to 10, slices).foreach {
         i => println(barr1.value.size)
       }
+      println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6))
     }
 
     System.exit(0)
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 646682878fda5373027b660c53c17e39176d0416..86dd9ca1b3e58032196de6d669307ab1ce009557 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -21,6 +21,7 @@ import java.util.Random
 import scala.math.exp
 import org.apache.spark.util.Vector
 import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.scheduler.InputFormatInfo
 
 /**
@@ -51,7 +52,7 @@ object SparkHdfsLR {
       System.exit(1)
     }
     val inputPath = args(1)
-    val conf = SparkEnv.get.hadoop.newConfiguration()
+    val conf = SparkHadoopUtil.get.newConfiguration()
     val sc = new SparkContext(args(0), "SparkHdfsLR",
       System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), 
       InputFormatInfo.computePreferredLocations(
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index 5a2bc9b0d08eebc70a35731b593e1e7dd5f7e6b7..a689e5a360b6bf0eae164ebfb63fe37ea83726e2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -38,6 +38,6 @@ object SparkPi {
       if (x*x + y*y < 1) 1 else 0
     }.reduce(_ + _)
     println("Pi is roughly " + 4.0 * count / n)
-    System.exit(0)
+    spark.stop()
   }
 }
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 12f939d5a7e4b529c05c8dfe77d67e8c3f254b40..570ba4c81a1d2c3cc00144727f04bc8a45dc071f 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -18,13 +18,11 @@
 package org.apache.spark.streaming.examples
 
 import java.util.Properties
-import kafka.message.Message
-import kafka.producer.SyncProducerConfig
+
 import kafka.producer._
-import org.apache.spark.SparkContext
+
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.util.RawTextHelper._
 
 /**
@@ -54,9 +52,10 @@ object KafkaWordCount {
     ssc.checkpoint("checkpoint")
 
     val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
-    val lines = ssc.kafkaStream(zkQuorum, group, topicpMap)
+    val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2)
     val words = lines.flatMap(_.split(" "))
-    val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
+    val wordCounts = words.map(x => (x, 1l))
+      .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
     wordCounts.print()
     
     ssc.start()
@@ -68,15 +67,16 @@ object KafkaWordCountProducer {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      System.err.println("Usage: KafkaWordCountProducer <zkQuorum> <topic> <messagesPerSec> <wordsPerMessage>")
+      System.err.println("Usage: KafkaWordCountProducer <metadataBrokerList> <topic> " +
+        "<messagesPerSec> <wordsPerMessage>")
       System.exit(1)
     }
 
-    val Array(zkQuorum, topic, messagesPerSec, wordsPerMessage) = args
+    val Array(brokers, topic, messagesPerSec, wordsPerMessage) = args
 
     // Zookeper connection properties
     val props = new Properties()
-    props.put("zk.connect", zkQuorum)
+    props.put("metadata.broker.list", brokers)
     props.put("serializer.class", "kafka.serializer.StringEncoder")
     
     val config = new ProducerConfig(props)
@@ -85,11 +85,13 @@ object KafkaWordCountProducer {
     // Send some messages
     while(true) {
       val messages = (1 to messagesPerSec.toInt).map { messageNum =>
-        (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString).mkString(" ")
+        val str = (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString)
+          .mkString(" ")
+
+        new KeyedMessage[String, String](topic, str)
       }.toArray
-      println(messages.mkString(","))
-      val data = new ProducerData[String, String](topic, messages)
-      producer.send(data)
+
+      producer.send(messages: _*)
       Thread.sleep(100)
     }
   }
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..af698a01d511871472751f1fdb779e808cb4d6ab
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.examples
+
+import org.apache.spark.streaming.{ Seconds, StreamingContext }
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.dstream.MQTTReceiver
+import org.apache.spark.storage.StorageLevel
+
+import org.eclipse.paho.client.mqttv3.MqttClient
+import org.eclipse.paho.client.mqttv3.MqttClientPersistence
+import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
+import org.eclipse.paho.client.mqttv3.MqttException
+import org.eclipse.paho.client.mqttv3.MqttMessage
+import org.eclipse.paho.client.mqttv3.MqttTopic
+
+/**
+ * A simple Mqtt publisher for demonstration purposes, repeatedly publishes 
+ * Space separated String Message "hello mqtt demo for spark streaming"
+ */
+object MQTTPublisher {
+
+  var client: MqttClient = _
+
+  def main(args: Array[String]) {
+    if (args.length < 2) {
+      System.err.println("Usage: MQTTPublisher <MqttBrokerUrl> <topic>")
+      System.exit(1)
+    }
+
+    val Seq(brokerUrl, topic) = args.toSeq
+
+    try {
+      var peristance:MqttClientPersistence =new MqttDefaultFilePersistence("/tmp")
+      client = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance)
+    } catch {
+      case e: MqttException => println("Exception Caught: " + e)
+    }
+
+    client.connect()
+
+    val msgtopic: MqttTopic = client.getTopic(topic);
+    val msg: String = "hello mqtt demo for spark streaming"
+
+    while (true) {
+      val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes())
+      msgtopic.publish(message);
+      println("Published data. topic: " + msgtopic.getName() + " Message: " + message)
+    }
+   client.disconnect()
+  }
+}
+
+/**
+ * A sample wordcount with MqttStream stream
+ *
+ * To work with Mqtt, Mqtt Message broker/server required.
+ * Mosquitto (http://mosquitto.org/) is an open source Mqtt Broker
+ * In ubuntu mosquitto can be installed using the command  `$ sudo apt-get install mosquitto`
+ * Eclipse paho project provides Java library for Mqtt Client http://www.eclipse.org/paho/
+ * Example Java code for Mqtt Publisher and Subscriber can be found here https://bitbucket.org/mkjinesh/mqttclient
+ * Usage: MQTTWordCount <master> <MqttbrokerUrl> <topic>
+ * In local mode, <master> should be 'local[n]' with n > 1
+ *   <MqttbrokerUrl> and <topic> describe where Mqtt publisher is running.
+ *
+ * To run this example locally, you may run publisher as
+ *    `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ * and run the example as
+ *    `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ */
+object MQTTWordCount {
+
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: MQTTWordCount <master> <MqttbrokerUrl> <topic>" +
+          " In local mode, <master> should be 'local[n]' with n > 1")
+      System.exit(1)
+    }
+
+    val Seq(master, brokerUrl, topic) = args.toSeq
+
+    val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), 
+    Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY)
+
+    val words = lines.flatMap(x => x.toString.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+    wordCounts.print()
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index 884d6d6f3414c7c18d511b45e1c8ea6618fee8be..de70c50473558a5cf07df0eceacbef6f55fb785b 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -17,17 +17,19 @@
 
 package org.apache.spark.streaming.examples.clickstream
 
-import java.net.{InetAddress,ServerSocket,Socket,SocketException}
-import java.io.{InputStreamReader, BufferedReader, PrintWriter}
+import java.net.ServerSocket
+import java.io.PrintWriter
 import util.Random
 
 /** Represents a page view on a website with associated dimension data.*/
-class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) {
+class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int)
+    extends Serializable {
   override def toString() : String = {
     "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID)
   }
 }
-object PageView {
+
+object PageView extends Serializable {
   def fromString(in : String) : PageView = {
     val parts = in.split("\t")
     new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt)
@@ -39,6 +41,9 @@ object PageView {
   * This should be used in tandem with PageViewStream.scala. Example:
   * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
   * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  *
+  * When running this, you may want to set the root logging level to ERROR in
+  * conf/log4j.properties to reduce the verbosity of the output.
   * */
 object PageViewGenerator {
   val pages = Map("http://foo.com/"        -> .7,
diff --git a/pom.xml b/pom.xml
index 5ad7b1befb3c8bdac9e34368e2e5a5855db063ce..53ac82efd0247ebf45accfdc465d3fd0377c9901 100644
--- a/pom.xml
+++ b/pom.xml
@@ -147,6 +147,17 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
+    <repository>
+      <id>mqtt-repo</id>
+      <name>MQTT Repository</name>
+      <url>https://repo.eclipse.org/content/repositories/paho-releases/</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
   </repositories>
   <pluginRepositories>
     <pluginRepository>
@@ -250,16 +261,34 @@
         <groupId>com.typesafe.akka</groupId>
         <artifactId>akka-actor</artifactId>
         <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>com.typesafe.akka</groupId>
         <artifactId>akka-remote</artifactId>
         <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>com.typesafe.akka</groupId>
         <artifactId>akka-slf4j</artifactId>
         <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>it.unimi.dsi</groupId>
@@ -394,19 +423,11 @@
           </exclusion>
           <exclusion>
             <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
+            <artifactId>*</artifactId>
           </exclusion>
           <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>org.sonatype.sisu.inject</groupId>
+            <artifactId>*</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -430,19 +451,11 @@
           </exclusion>
           <exclusion>
             <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
+            <artifactId>*</artifactId>
           </exclusion>
           <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>org.sonatype.sisu.inject</groupId>
+            <artifactId>*</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -461,19 +474,11 @@
           </exclusion>
           <exclusion>
             <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
+            <artifactId>*</artifactId>
           </exclusion>
           <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>org.sonatype.sisu.inject</groupId>
+            <artifactId>*</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -492,19 +497,11 @@
           </exclusion>
           <exclusion>
             <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
+            <artifactId>*</artifactId>
           </exclusion>
           <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>org.sonatype.sisu.inject</groupId>
+            <artifactId>*</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 973f1e2f11d8d40d9e8f3bb8936124d5e5b1639f..45fd30a7c836408813b473df32db3197508c3be0 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -60,6 +60,8 @@ object SparkBuild extends Build {
   lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
     .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
 
+  lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects")
+
   // A configuration to set an alternative publishLocalConfiguration
   lazy val MavenCompile = config("m2r") extend(Compile)
   lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
@@ -74,8 +76,11 @@ object SparkBuild extends Build {
   // Conditionally include the yarn sub-project
   lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
   lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
-  lazy val allProjects = Seq[ProjectReference](
-    core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef
+
+  // Everything except assembly, tools and examples belong to packageProjects
+  lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
+
+  lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj)
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization := "org.apache.spark",
@@ -103,7 +108,10 @@ object SparkBuild extends Build {
     // Shared between both core and streaming.
     resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
 
-    // For Sonatype publishing
+    // Shared between both examples and streaming.
+    resolvers ++= Seq("Mqtt Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
+
+   // For Sonatype publishing
     resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
       "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
 
@@ -273,13 +281,18 @@ object SparkBuild extends Build {
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
     resolvers ++= Seq(
-      "Akka Repository" at "http://repo.akka.io/releases/"
+      "Akka Repository" at "http://repo.akka.io/releases/",
+      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
     ),
     libraryDependencies ++= Seq(
+      "org.eclipse.paho" % "mqtt-client" % "0.4.0",
       "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
-      "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
       "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
-      "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty)
+      "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty),
+      "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"
+        exclude("com.sun.jdmk", "jmxtools")
+        exclude("com.sun.jmx", "jmxri")
+        exclude("net.sf.jopt-simple", "jopt-simple")
     )
   )
 
@@ -303,7 +316,9 @@ object SparkBuild extends Build {
 
   def assemblyProjSettings = sharedSettings ++ Seq(
     name := "spark-assembly",
-    jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }
+    assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn,
+    jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" },
+    jarName in packageDependency <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" }
   ) ++ assemblySettings ++ extraAssemblySettings
 
   def extraAssemblySettings() = Seq(
@@ -311,6 +326,7 @@ object SparkBuild extends Build {
     mergeStrategy in assembly := {
       case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
       case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard
+      case "log4j.properties" => MergeStrategy.discard
       case "META-INF/services/org.apache.hadoop.fs.FileSystem" => MergeStrategy.concat
       case "reference.conf" => MergeStrategy.concat
       case _ => MergeStrategy.first
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index d367f91967ff5379946da1a5226c88b12bce3878..da3d96689aa15dc14707e8f88c9170e51af3cede 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -42,6 +42,13 @@
 >>> a.value
 13
 
+>>> b = sc.accumulator(0)
+>>> def g(x):
+...     b.add(x)
+>>> rdd.foreach(g)
+>>> b.value
+6
+
 >>> from pyspark.accumulators import AccumulatorParam
 >>> class VectorAccumulatorParam(AccumulatorParam):
 ...     def zero(self, value):
@@ -139,9 +146,13 @@ class Accumulator(object):
             raise Exception("Accumulator.value cannot be accessed inside tasks")
         self._value = value
 
+    def add(self, term):
+        """Adds a term to this accumulator's value"""
+        self._value = self.accum_param.addInPlace(self._value, term)
+
     def __iadd__(self, term):
         """The += operator; adds a term to this accumulator's value"""
-        self._value = self.accum_param.addInPlace(self._value, term)
+        self.add(term)
         return self
 
     def __str__(self):
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 597110321a86370c29063052eb892f9213a4bfb3..a7ca8bc888c6759aff5784d26ad7df015d2fe2f4 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -49,6 +49,7 @@ class SparkContext(object):
     _lock = Lock()
     _python_includes = None # zip and egg files that need to be added to PYTHONPATH
 
+
     def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
         environment=None, batchSize=1024):
         """
@@ -66,19 +67,18 @@ class SparkContext(object):
         @param batchSize: The number of Python objects represented as a single
                Java object.  Set 1 to disable batching or -1 to use an
                unlimited batch size.
+
+
+        >>> from pyspark.context import SparkContext
+        >>> sc = SparkContext('local', 'test')
+
+        >>> sc2 = SparkContext('local', 'test2') # doctest: +IGNORE_EXCEPTION_DETAIL
+        Traceback (most recent call last):
+            ...
+        ValueError:...
         """
-        with SparkContext._lock:
-            if SparkContext._active_spark_context:
-                raise ValueError("Cannot run multiple SparkContexts at once")
-            else:
-                SparkContext._active_spark_context = self
-                if not SparkContext._gateway:
-                    SparkContext._gateway = launch_gateway()
-                    SparkContext._jvm = SparkContext._gateway.jvm
-                    SparkContext._writeIteratorToPickleFile = \
-                        SparkContext._jvm.PythonRDD.writeIteratorToPickleFile
-                    SparkContext._takePartition = \
-                        SparkContext._jvm.PythonRDD.takePartition
+        SparkContext._ensure_initialized(self)
+
         self.master = master
         self.jobName = jobName
         self.sparkHome = sparkHome or None # None becomes null in Py4J
@@ -119,6 +119,32 @@ class SparkContext(object):
         self._temp_dir = \
             self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
 
+    @classmethod
+    def _ensure_initialized(cls, instance=None):
+        with SparkContext._lock:
+            if not SparkContext._gateway:
+                SparkContext._gateway = launch_gateway()
+                SparkContext._jvm = SparkContext._gateway.jvm
+                SparkContext._writeIteratorToPickleFile = \
+                    SparkContext._jvm.PythonRDD.writeIteratorToPickleFile
+                SparkContext._takePartition = \
+                    SparkContext._jvm.PythonRDD.takePartition
+
+            if instance:
+                if SparkContext._active_spark_context and SparkContext._active_spark_context != instance:
+                    raise ValueError("Cannot run multiple SparkContexts at once")
+                else:
+                    SparkContext._active_spark_context = instance
+
+    @classmethod
+    def setSystemProperty(cls, key, value):
+        """
+        Set a system property, such as spark.executor.memory. This must be
+        invoked before instantiating SparkContext.
+        """
+        SparkContext._ensure_initialized()
+        SparkContext._jvm.java.lang.System.setProperty(key, value)
+
     @property
     def defaultParallelism(self):
         """
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 36f54a22cf30449d421364ab74d8dcb2db459783..0ced284da68f50bc24a4305dd43668268f7f09a5 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -633,6 +633,20 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
     Result(true, shouldReplay)
   }
 
+  def addAllClasspath(args: Seq[String]): Unit = {
+    var added = false
+    var totalClasspath = ""
+    for (arg <- args) {
+      val f = File(arg).normalize
+      if (f.exists) {
+        added = true
+        addedClasspath = ClassPath.join(addedClasspath, f.path)
+        totalClasspath = ClassPath.join(settings.classpath.value, addedClasspath)
+      }
+    }
+    if (added) replay()
+  }
+
   def addClasspath(arg: String): Unit = {
     val f = File(arg).normalize
     if (f.exists) {
@@ -845,7 +859,14 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
     val jars = Option(System.getenv("ADD_JARS")).map(_.split(','))
                                                 .getOrElse(new Array[String](0))
                                                 .map(new java.io.File(_).getAbsolutePath)
-    sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
+    try {
+      sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()
+        echo("Failed to create SparkContext, exiting...")
+        sys.exit(1)
+    }
     sparkContext
   }
 
diff --git a/spark-class b/spark-class
index e111ef6da7e7c40cbac450d62d4ab3cfcf6b960d..fb9d1a4f8eaaf5e29632969a9682dd162d3f88d2 100755
--- a/spark-class
+++ b/spark-class
@@ -95,10 +95,17 @@ export JAVA_OPTS
 
 if [ ! -f "$FWDIR/RELEASE" ]; then
   # Exit if the user hasn't compiled Spark
-  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
-  if [[ $? != 0 ]]; then
-    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
-    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+  num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l)
+  jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar")
+  if [ "$num_jars" -eq "0" ]; then
+    echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2
+    echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2
+    exit 1
+  fi
+  if [ "$num_jars" -gt "1" ]; then
+    echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2
+    echo "$jars_list"
+    echo "Please remove all but one jar."
     exit 1
   fi
 fi
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar
deleted file mode 100644
index 65f79925a4d06a41b7b98d7e6f92fedb408c9b3a..0000000000000000000000000000000000000000
Binary files a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar and /dev/null differ
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5
deleted file mode 100644
index 29f45f4adb6975e36cb95ee2f50de53ef76b0c5b..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5
+++ /dev/null
@@ -1 +0,0 @@
-18876b8bc2e4cef28b6d191aa49d963f
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1
deleted file mode 100644
index e3bd62bac038f6e56f7bbaf554a544f441dfbbaa..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-06b27270ffa52250a2c08703b397c99127b72060
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom
deleted file mode 100644
index 082d35726a5afe5edb882bc4caac7bc6a7aafdc0..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>org.apache.kafka</groupId>
-  <artifactId>kafka</artifactId>
-  <version>0.7.2-spark</version>
-  <description>POM was created from install:install-file</description>
-</project>
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5
deleted file mode 100644
index 92c4132b5b01c48b0e17458876a7beeeed3e3084..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5
+++ /dev/null
@@ -1 +0,0 @@
-7bc4322266e6032bdf9ef6eebdd8097d
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1
deleted file mode 100644
index 8a1d8a097a113cf674588322a978235f73c0d7f4..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1
+++ /dev/null
@@ -1 +0,0 @@
-d0f79e8eff0db43ca7bcf7dce2c8cd2972685c9d
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml
deleted file mode 100644
index 720cd51c2f5e6408a63eae37c92f52210de58d55..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
-  <groupId>org.apache.kafka</groupId>
-  <artifactId>kafka</artifactId>
-  <versioning>
-    <release>0.7.2-spark</release>
-    <versions>
-      <version>0.7.2-spark</version>
-    </versions>
-    <lastUpdated>20130121015225</lastUpdated>
-  </versioning>
-</metadata>
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5 b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5
deleted file mode 100644
index a4ce5dc9e8d9b21320106941f0f8b36c3e2b6485..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5
+++ /dev/null
@@ -1 +0,0 @@
-e2b9c7c5f6370dd1d21a0aae5e8dcd77
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1 b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1
deleted file mode 100644
index b869eaf2a61de2fbaa3f75c2e6c9df874ac92a5e..0000000000000000000000000000000000000000
--- a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1
+++ /dev/null
@@ -1 +0,0 @@
-2a4341da936b6c07a09383d17ffb185ac558ee91
\ No newline at end of file
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 3b25fb49fbedb694b699ee0195257b7fc51d2d5c..7a9ae6a97ba7ff615e3e24c76923dc941d76b8f3 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -32,10 +32,16 @@
   <url>http://spark.incubator.apache.org/</url>
 
   <repositories>
-    <!-- A repository in the local filesystem for the Kafka JAR, which we modified for Scala 2.9 -->
     <repository>
-      <id>lib</id>
-      <url>file://${project.basedir}/lib</url>
+      <id>apache-repo</id>
+      <name>Apache Repository</name>
+      <url>https://repository.apache.org/content/repositories/releases</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
     </repository>
   </repositories>
 
@@ -56,9 +62,22 @@
     </dependency>
     <dependency>
       <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka</artifactId>
-      <version>0.7.2-spark</version>  <!-- Comes from our in-project repository -->
-      <scope>provided</scope>
+      <artifactId>kafka_2.9.2</artifactId>
+      <version>0.8.0-beta1</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.sf.jopt-simple</groupId>
+          <artifactId>jopt-simple</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.flume</groupId>
@@ -69,17 +88,22 @@
           <groupId>org.jboss.netty</groupId>
           <artifactId>netty</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.xerial.snappy</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>com.github.sgroschupf</groupId>
-      <artifactId>zkclient</artifactId>
-      <version>0.1</version>
-    </dependency>
     <dependency>
       <groupId>org.twitter4j</groupId>
       <artifactId>twitter4j-stream</artifactId>
       <version>3.0.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.scala-lang</groupId>
@@ -89,6 +113,12 @@
       <groupId>com.typesafe.akka</groupId>
       <artifactId>akka-zeromq</artifactId>
       <version>2.0.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
@@ -110,6 +140,11 @@
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.eclipse.paho</groupId>
+      <artifactId>mqtt-client</artifactId>
+       <version>0.4.0</version>
+    </dependency>
   </dependencies>
   <build>
     <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 2d8f07262426f00373a7afc4ea0064887b60ff2f..bb9febad38d03abaea536b413a5f1405c19052af 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration
 
 import org.apache.spark.Logging
 import org.apache.spark.io.CompressionCodec
+import org.apache.spark.util.MetadataCleaner
 
 
 private[streaming]
@@ -40,6 +41,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
   val checkpointDir = ssc.checkpointDir
   val checkpointDuration = ssc.checkpointDuration
   val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
+  val delaySeconds = MetadataCleaner.getDelaySeconds
 
   def validate() {
     assert(master != null, "Checkpoint.master is null")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 80da6bd30b4b99663f7517531e5d005f62ba8c1c..9ceff754c4b7251dfc50e533d75f892505e713e6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -438,6 +438,13 @@ abstract class DStream[T: ClassManifest] (
    */
   def glom(): DStream[Array[T]] = new GlommedDStream(this)
 
+
+  /**
+   * Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
+   * returned DStream has exactly numPartitions partitions.
+   */
+  def repartition(numPartitions: Int): DStream[T] = this.transform(_.repartition(numPartitions))
+
   /**
    * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
    * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
@@ -479,7 +486,7 @@ abstract class DStream[T: ClassManifest] (
 
   /**
    * Apply a function to each RDD in this DStream. This is an output operator, so
-   * this DStream will be registered as an output stream and therefore materialized.
+   * 'this' DStream will be registered as an output stream and therefore materialized.
    */
   def foreach(foreachFunc: RDD[T] => Unit) {
     this.foreach((r: RDD[T], t: Time) => foreachFunc(r))
@@ -487,7 +494,7 @@ abstract class DStream[T: ClassManifest] (
 
   /**
    * Apply a function to each RDD in this DStream. This is an output operator, so
-   * this DStream will be registered as an output stream and therefore materialized.
+   * 'this' DStream will be registered as an output stream and therefore materialized.
    */
   def foreach(foreachFunc: (RDD[T], Time) => Unit) {
     val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
@@ -497,18 +504,52 @@ abstract class DStream[T: ClassManifest] (
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
-    transform((r: RDD[T], t: Time) => transformFunc(r))
+    transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r)))
   }
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
-    new TransformedDStream(this, context.sparkContext.clean(transformFunc))
+    //new TransformedDStream(this, context.sparkContext.clean(transformFunc))
+    val cleanedF = context.sparkContext.clean(transformFunc)
+    val realTransformFunc =  (rdds: Seq[RDD[_]], time: Time) => {
+      assert(rdds.length == 1)
+      cleanedF(rdds.head.asInstanceOf[RDD[T]], time)
+    }
+    new TransformedDStream[U](Seq(this), realTransformFunc)
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[U: ClassManifest, V: ClassManifest](
+      other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V]
+    ): DStream[V] = {
+    val cleanedF = ssc.sparkContext.clean(transformFunc)
+    transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[U: ClassManifest, V: ClassManifest](
+      other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V]
+    ): DStream[V] = {
+    val cleanedF = ssc.sparkContext.clean(transformFunc)
+    val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
+      assert(rdds.length == 2)
+      val rdd1 = rdds(0).asInstanceOf[RDD[T]]
+      val rdd2 = rdds(1).asInstanceOf[RDD[U]]
+      cleanedF(rdd1, rdd2, time)
+    }
+    new TransformedDStream[V](Seq(this, other), realTransformFunc)
   }
 
   /**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 757bc98981ca9dcc1810a34303394afe20e9b04e..8c12fd11efcafc3d0daa87e39f9d75d6d6a1dd90 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -19,7 +19,7 @@ package org.apache.spark.streaming
 
 import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
-import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream}
+import org.apache.spark.streaming.dstream.{ShuffledDStream}
 import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
 
 import org.apache.spark.{Partitioner, HashPartitioner}
@@ -359,7 +359,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of the key.
    * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -398,11 +398,18 @@ extends Serializable {
      new StateDStream(self, ssc.sc.clean(updateFunc), partitioner, rememberPartitioner)
   }
 
-
+  /**
+   * Return a new DStream by applying a map function to the value of each key-value pairs in
+   * 'this' DStream without changing the key.
+   */
   def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = {
     new MapValuedDStream[K, V, U](self, mapValuesFunc)
   }
 
+  /**
+   * Return a new DStream by applying a flatmap function to the value of each key-value pairs in
+   * 'this' DStream without changing the key.
+   */
   def flatMapValues[U: ClassManifest](
       flatMapValuesFunc: V => TraversableOnce[U]
     ): DStream[(K, U)] = {
@@ -410,9 +417,8 @@ extends Serializable {
   }
 
   /**
-   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
-   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
-   * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with Spark's default number
    * of partitions.
    */
   def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
@@ -420,56 +426,132 @@ extends Serializable {
   }
 
   /**
-   * Cogroup `this` DStream with `other` DStream using a partitioner. For each key k in corresponding RDDs of `this`
-   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
-   * key in both RDDs. Partitioner is used to partition each generated RDD.
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   */
+  def cogroup[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (Seq[V], Seq[W]))] = {
+    cogroup(other, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * The supplied [[org.apache.spark.Partitioner]] is used to partition the generated RDDs.
    */
   def cogroup[W: ClassManifest](
       other: DStream[(K, W)],
       partitioner: Partitioner
     ): DStream[(K, (Seq[V], Seq[W]))] = {
-
-    val cgd = new CoGroupedDStream[K](
-      Seq(self.asInstanceOf[DStream[(K, _)]], other.asInstanceOf[DStream[(K, _)]]),
-      partitioner
-    )
-    val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)(
-      classManifest[K],
-      Manifests.seqSeqManifest
+    self.transformWith(
+      other,
+      (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner)
     )
-    pdfs.mapValues {
-      case Seq(vs, ws) =>
-        (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
-    }
   }
 
   /**
-   * Join `this` DStream with `other` DStream. HashPartitioner is used
-   * to partition each generated RDD into default number of partitions.
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
    */
   def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
     join[W](other, defaultPartitioner())
   }
 
   /**
-   * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
-   * be generated by joining RDDs from `this` and other DStream. Uses the given
-   * Partitioner to partition each generated RDD.
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   */
+  def join[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (V, W))] = {
+    join[W](other, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    */
   def join[W: ClassManifest](
       other: DStream[(K, W)],
       partitioner: Partitioner
     ): DStream[(K, (V, W))] = {
-    this.cogroup(other, partitioner)
-        .flatMapValues{
-      case (vs, ws) =>
-        for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
-    }
+    self.transformWith(
+      other,
+      (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.join(rdd2, partitioner)
+    )
+  }
+
+  /**
+   * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
+   * number of partitions.
+   */
+  def leftOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, Option[W]))] = {
+    leftOuterJoin[W](other, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   */
+  def leftOuterJoin[W: ClassManifest](
+      other: DStream[(K, W)],
+      numPartitions: Int
+    ): DStream[(K, (V, Option[W]))] = {
+    leftOuterJoin[W](other, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
+   * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
+   * the partitioning of each RDD.
+   */
+  def leftOuterJoin[W: ClassManifest](
+      other: DStream[(K, W)],
+      partitioner: Partitioner
+    ): DStream[(K, (V, Option[W]))] = {
+    self.transformWith(
+      other,
+      (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.leftOuterJoin(rdd2, partitioner)
+    )
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
+   * number of partitions.
+   */
+  def rightOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Option[V], W))] = {
+    rightOuterJoin[W](other, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   */
+  def rightOuterJoin[W: ClassManifest](
+      other: DStream[(K, W)],
+      numPartitions: Int
+    ): DStream[(K, (Option[V], W))] = {
+    rightOuterJoin[W](other, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
+   * the partitioning of each RDD.
+   */
+  def rightOuterJoin[W: ClassManifest](
+      other: DStream[(K, W)],
+      partitioner: Partitioner
+    ): DStream[(K, (Option[V], W))] = {
+    self.transformWith(
+      other,
+      (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.rightOuterJoin(rdd2, partitioner)
+    )
   }
 
   /**
-   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
-   * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval
+   * is generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
    */
   def saveAsHadoopFiles[F <: OutputFormat[K, V]](
       prefix: String,
@@ -479,8 +561,8 @@ extends Serializable {
   }
 
   /**
-   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
-   * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval
+   * is generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
    */
   def saveAsHadoopFiles(
       prefix: String,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 878725c705db71807c567c909b1b6ab0f48f7f94..70bf902143d8ee8bfd2dd08c700289cc8801c9cb 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -100,6 +100,10 @@ class StreamingContext private (
       "both SparkContext and checkpoint as null")
   }
 
+  if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) {
+    MetadataCleaner.setDelaySeconds(cp_.delaySeconds)
+  }
+
   if (MetadataCleaner.getDelaySeconds < 0) {
     throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
       + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
@@ -252,10 +256,14 @@ class StreamingContext private (
       groupId: String,
       topics: Map[String, Int],
       storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
-    ): DStream[String] = {
+    ): DStream[(String, String)] = {
     val kafkaParams = Map[String, String](
-      "zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000")
-    kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, storageLevel)
+      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
+      "zookeeper.connection.timeout.ms" -> "10000")
+    kafkaStream[String, String, kafka.serializer.StringDecoder, kafka.serializer.StringDecoder](
+      kafkaParams,
+      topics,
+      storageLevel)
   }
 
   /**
@@ -266,12 +274,16 @@ class StreamingContext private (
    *               in its own thread.
    * @param storageLevel  Storage level to use for storing the received objects
    */
-  def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest](
+  def kafkaStream[
+    K: ClassManifest,
+    V: ClassManifest,
+    U <: kafka.serializer.Decoder[_]: Manifest,
+    T <: kafka.serializer.Decoder[_]: Manifest](
       kafkaParams: Map[String, String],
       topics: Map[String, Int],
       storageLevel: StorageLevel
-    ): DStream[T] = {
-    val inputStream = new KafkaInputDStream[T, D](this, kafkaParams, topics, storageLevel)
+    ): DStream[(K, V)] = {
+    val inputStream = new KafkaInputDStream[K, V, U, T](this, kafkaParams, topics, storageLevel)
     registerInputStream(inputStream)
     inputStream
   }
@@ -450,13 +462,39 @@ class StreamingContext private (
     inputStream
   }
 
+/**
+   * Create an input stream that receives messages pushed by a mqtt publisher.
+   * @param brokerUrl Url of remote mqtt publisher
+   * @param topic topic name to subscribe to
+   * @param storageLevel RDD storage level. Defaults to memory-only.
+   */
+
+  def mqttStream(
+    brokerUrl: String,
+    topic: String,
+    storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[String] = {
+    val inputStream = new MQTTInputDStream[String](this, brokerUrl, topic, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
   /**
-   * Create a unified DStream from multiple DStreams of the same type and same interval
+   * Create a unified DStream from multiple DStreams of the same type and same slide duration.
    */
   def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
     new UnionDStream[T](streams.toArray)
   }
 
+  /**
+   * Create a new DStream in which each RDD is generated by applying a function on RDDs of
+   * the DStreams.
+   */
+  def transform[T: ClassManifest](
+      dstreams: Seq[DStream[_]],
+      transformFunc: (Seq[RDD[_]], Time) => RDD[T]
+    ): DStream[T] = {
+    new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc))
+  }
+
   /**
    * Register an input stream that will be started (InputDStream.start() called) to get the
    * input data.
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
index d1932b6b05a093fbf814f274dc6dc488bca73053..1a2aeaa8797e1fcbabe04505fdcd0c78e98612a3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
@@ -94,6 +94,12 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
    */
   def union(that: JavaDStream[T]): JavaDStream[T] =
     dstream.union(that.dstream)
+
+  /**
+   * Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
+   * returned DStream has exactly numPartitions partitions.
+   */
+  def repartition(numPartitions: Int): JavaDStream[T] = dstream.repartition(numPartitions)
 }
 
 object JavaDStream {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index 459695b7cabab6c70da9a646d5697592ae35703b..09189eadd824e5ace83405d450df8ffbd1f694d6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -24,7 +24,8 @@ import scala.collection.JavaConversions._
 
 import org.apache.spark.streaming._
 import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
-import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import org.apache.spark.api.java.function.{Function3 => JFunction3, _}
 import java.util
 import org.apache.spark.rdd.RDD
 import JavaDStream._
@@ -120,10 +121,12 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
    * this DStream. Applying glom() to an RDD coalesces all elements within each partition into
    * an array.
    */
-  def glom(): JavaDStream[JList[T]] =
+  def glom(): JavaDStream[JList[T]] = {
     new JavaDStream(dstream.glom().map(x => new java.util.ArrayList[T](x.toSeq)))
+  }
+
 
-  /** Return the StreamingContext associated with this DStream */
+  /** Return the [[org.apache.spark.streaming.StreamingContext]] associated with this DStream */
   def context(): StreamingContext = dstream.context()
 
   /** Return a new DStream by applying a function to all elements of this DStream. */
@@ -238,7 +241,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Apply a function to each RDD in this DStream. This is an output operator, so
-   * this DStream will be registered as an output stream and therefore materialized.
+   * 'this' DStream will be registered as an output stream and therefore materialized.
    */
   def foreach(foreachFunc: JFunction[R, Void]) {
     dstream.foreach(rdd => foreachFunc.call(wrapRDD(rdd)))
@@ -246,7 +249,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Apply a function to each RDD in this DStream. This is an output operator, so
-   * this DStream will be registered as an output stream and therefore materialized.
+   * 'this' DStream will be registered as an output stream and therefore materialized.
    */
   def foreach(foreachFunc: JFunction2[R, Time, Void]) {
     dstream.foreach((rdd, time) => foreachFunc.call(wrapRDD(rdd), time))
@@ -254,7 +257,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = {
     implicit val cm: ClassManifest[U] =
@@ -266,7 +269,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = {
     implicit val cm: ClassManifest[U] =
@@ -278,7 +281,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]):
   JavaPairDStream[K2, V2] = {
@@ -293,7 +296,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
 
   /**
    * Return a new DStream in which each RDD is generated by applying a function
-   * on each RDD of this DStream.
+   * on each RDD of 'this' DStream.
    */
   def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]):
   JavaPairDStream[K2, V2] = {
@@ -306,6 +309,82 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
     dstream.transform(scalaTransform(_, _))
   }
 
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[U, W](
+      other: JavaDStream[U],
+      transformFunc: JFunction3[R, JavaRDD[U], Time, JavaRDD[W]]
+    ): JavaDStream[W] = {
+    implicit val cmu: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    implicit val cmv: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[W] =
+      transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
+    dstream.transformWith[U, W](other.dstream, scalaTransform(_, _, _))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[U, K2, V2](
+      other: JavaDStream[U],
+      transformFunc: JFunction3[R, JavaRDD[U], Time, JavaPairRDD[K2, V2]]
+    ): JavaPairDStream[K2, V2] = {
+    implicit val cmu: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    implicit val cmk2: ClassManifest[K2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
+    implicit val cmv2: ClassManifest[V2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+    def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[(K2, V2)] =
+      transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
+    dstream.transformWith[U, (K2, V2)](other.dstream, scalaTransform(_, _, _))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[K2, V2, W](
+      other: JavaPairDStream[K2, V2],
+      transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaRDD[W]]
+    ): JavaDStream[W] = {
+    implicit val cmk2: ClassManifest[K2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
+    implicit val cmv2: ClassManifest[V2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+    implicit val cmw: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] =
+      transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
+    dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of 'this' DStream and 'other' DStream.
+   */
+  def transformWith[K2, V2, K3, V3](
+      other: JavaPairDStream[K2, V2],
+      transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaPairRDD[K3, V3]]
+    ): JavaPairDStream[K3, V3] = {
+    implicit val cmk2: ClassManifest[K2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
+    implicit val cmv2: ClassManifest[V2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+    implicit val cmk3: ClassManifest[K3] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K3]]
+    implicit val cmv3: ClassManifest[V3] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V3]]
+    def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[(K3, V3)] =
+      transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
+    dstream.transformWith[(K2, V2), (K3, V3)](other.dstream, scalaTransform(_, _, _))
+  }
+
   /**
    * Enable periodic checkpointing of RDDs of this DStream
    * @param interval Time interval after which generated RDD will be checkpointed
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index 978fca33ad03e344a355c72dd3f3eff0658acc35..c6cd635afa0c87f903f19a07fe18034c668f2292 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConversions._
 
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, Function3 => JFunction3}
 import org.apache.spark.Partitioner
 import org.apache.hadoop.mapred.{JobConf, OutputFormat}
 import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
@@ -36,7 +36,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.PairRDDFunctions
 
 class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
-    implicit val kManifiest: ClassManifest[K],
+    implicit val kManifest: ClassManifest[K],
     implicit val vManifest: ClassManifest[V])
     extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
 
@@ -59,6 +59,12 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   /** Persist the RDDs of this DStream with the given storage level */
   def persist(storageLevel: StorageLevel): JavaPairDStream[K, V] = dstream.persist(storageLevel)
 
+  /**
+   * Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
+   * returned DStream has exactly numPartitions partitions.
+   */
+  def repartition(numPartitions: Int): JavaPairDStream[K, V] = dstream.repartition(numPartitions)
+
   /** Method that generates a RDD for the given Duration */
   def compute(validTime: Time): JavaPairRDD[K, V] = {
     dstream.compute(validTime) match {
@@ -148,7 +154,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
 
   /**
    * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
-   * combineByKey for RDDs. Please refer to combineByKey in [[PairRDDFunctions]] for more
+   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
    * information.
    */
   def combineByKey[C](createCombiner: JFunction[V, C],
@@ -413,7 +419,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
    * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -428,7 +434,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
    * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -436,15 +442,17 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
    * @param numPartitions Number of partitions of each RDD in the new DStream.
    * @tparam S State type
    */
-  def updateStateByKey[S: ClassManifest](
+  def updateStateByKey[S](
       updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
       numPartitions: Int)
   : JavaPairDStream[K, S] = {
+    implicit val cm: ClassManifest[S] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
     dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), numPartitions)
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of the key.
    * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -452,19 +460,30 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
    * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
    * @tparam S State type
    */
-  def updateStateByKey[S: ClassManifest](
+  def updateStateByKey[S](
       updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
       partitioner: Partitioner
   ): JavaPairDStream[K, S] = {
+    implicit val cm: ClassManifest[S] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
     dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner)
   }
 
+
+  /**
+   * Return a new DStream by applying a map function to the value of each key-value pairs in
+   * 'this' DStream without changing the key.
+   */
   def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = {
     implicit val cm: ClassManifest[U] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
     dstream.mapValues(f)
   }
 
+  /**
+   * Return a new DStream by applying a flatmap function to the value of each key-value pairs in
+   * 'this' DStream without changing the key.
+   */
   def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
     import scala.collection.JavaConverters._
     def fn = (x: V) => f.apply(x).asScala
@@ -474,9 +493,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
-   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
-   * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with Spark's default number
    * of partitions.
    */
   def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = {
@@ -486,21 +504,36 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
-   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
-   * key in both RDDs. Partitioner is used to partition each generated RDD.
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    */
-  def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
-  : JavaPairDStream[K, (JList[V], JList[W])] = {
+  def cogroup[W](
+      other: JavaPairDStream[K, W],
+      numPartitions: Int
+    ): JavaPairDStream[K, (JList[V], JList[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.cogroup(other.dstream, numPartitions)
+           .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+  }
+
+  /**
+   * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   */
+  def cogroup[W](
+      other: JavaPairDStream[K, W],
+      partitioner: Partitioner
+    ): JavaPairDStream[K, (JList[V], JList[W])] = {
     implicit val cm: ClassManifest[W] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
     dstream.cogroup(other.dstream, partitioner)
-        .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+           .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
   }
 
   /**
-   * Join `this` DStream with `other` DStream. HashPartitioner is used
-   * to partition each generated RDD into default number of partitions.
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
    */
   def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = {
     implicit val cm: ClassManifest[W] =
@@ -509,17 +542,111 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
-   * be generated by joining RDDs from `this` and other DStream. Uses the given
-   * Partitioner to partition each generated RDD.
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   */
+  def join[W](other: JavaPairDStream[K, W], numPartitions: Int): JavaPairDStream[K, (V, W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.join(other.dstream, numPartitions)
+  }
+
+  /**
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    */
-  def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
-  : JavaPairDStream[K, (V, W)] = {
+  def join[W](
+      other: JavaPairDStream[K, W],
+      partitioner: Partitioner
+    ): JavaPairDStream[K, (V, W)] = {
     implicit val cm: ClassManifest[W] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
     dstream.join(other.dstream, partitioner)
   }
 
+  /**
+   * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
+   * number of partitions.
+   */
+  def leftOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, Optional[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.leftOuterJoin(other.dstream)
+    joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
+  }
+
+  /**
+   * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   */
+  def leftOuterJoin[W](
+      other: JavaPairDStream[K, W],
+      numPartitions: Int
+    ): JavaPairDStream[K, (V, Optional[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.leftOuterJoin(other.dstream, numPartitions)
+    joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
+  }
+
+  /**
+   * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
+   * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
+   */
+  def leftOuterJoin[W](
+      other: JavaPairDStream[K, W],
+      partitioner: Partitioner
+    ): JavaPairDStream[K, (V, Optional[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.leftOuterJoin(other.dstream, partitioner)
+    joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
+   * number of partitions.
+   */
+  def rightOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (Optional[V], W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.rightOuterJoin(other.dstream)
+    joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   */
+  def rightOuterJoin[W](
+      other: JavaPairDStream[K, W],
+      numPartitions: Int
+    ): JavaPairDStream[K, (Optional[V], W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.rightOuterJoin(other.dstream, numPartitions)
+    joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
+  }
+
+  /**
+   * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
+   * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
+   * the partitioning of each RDD.
+   */
+  def rightOuterJoin[W](
+      other: JavaPairDStream[K, W],
+      partitioner: Partitioner
+    ): JavaPairDStream[K, (Optional[V], W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    val joinResult = dstream.rightOuterJoin(other.dstream, partitioner)
+    joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
+  }
+
   /**
    * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
    * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
@@ -590,14 +717,19 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
   }
 
+  /** Convert to a JavaDStream */
+  def toJavaDStream(): JavaDStream[(K, V)] = {
+    new JavaDStream[(K, V)](dstream)
+  }
+
   override val classManifest: ClassManifest[(K, V)] =
     implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
 }
 
 object JavaPairDStream {
-  implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)])
-  :JavaPairDStream[K, V] =
+  implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)]) = {
     new JavaPairDStream[K, V](dstream)
+  }
 
   def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
     implicit val cmk: ClassManifest[K] =
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 54ba3e602590f806906f75f45ec6df22bde66da3..cf30b541e1f92f020952295530aab24b91a2cad5 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -19,7 +19,7 @@ package org.apache.spark.streaming.api.java
 
 import java.lang.{Long => JLong, Integer => JInt}
 import java.io.InputStream
-import java.util.{Map => JMap}
+import java.util.{Map => JMap, List => JList}
 
 import scala.collection.JavaConversions._
 
@@ -33,7 +33,7 @@ import twitter4j.auth.Authorization
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import org.apache.spark.api.java.{JavaSparkContext, JavaRDD}
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.dstream._
 import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy}
@@ -141,7 +141,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     zkQuorum: String,
     groupId: String,
     topics: JMap[String, JInt])
-  : JavaDStream[String] = {
+  : JavaPairDStream[String, String] = {
     implicit val cmt: ClassManifest[String] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
     ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
@@ -162,7 +162,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     groupId: String,
     topics: JMap[String, JInt],
     storageLevel: StorageLevel)
-  : JavaDStream[String] = {
+  : JavaPairDStream[String, String] = {
     implicit val cmt: ClassManifest[String] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
     ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
@@ -171,25 +171,34 @@ class JavaStreamingContext(val ssc: StreamingContext) {
 
   /**
    * Create an input stream that pulls messages form a Kafka Broker.
-   * @param typeClass Type of RDD
-   * @param decoderClass Type of kafka decoder
+   * @param keyTypeClass Key type of RDD
+   * @param valueTypeClass value type of RDD
+   * @param keyDecoderClass Type of kafka key decoder
+   * @param valueDecoderClass Type of kafka value decoder
    * @param kafkaParams Map of kafka configuration paramaters.
    *                    See: http://kafka.apache.org/configuration.html
    * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
    * in its own thread.
    * @param storageLevel RDD storage level. Defaults to memory-only
    */
-  def kafkaStream[T, D <: kafka.serializer.Decoder[_]](
-    typeClass: Class[T],
-    decoderClass: Class[D],
+  def kafkaStream[K, V, U <: kafka.serializer.Decoder[_], T <: kafka.serializer.Decoder[_]](
+    keyTypeClass: Class[K],
+    valueTypeClass: Class[V],
+    keyDecoderClass: Class[U],
+    valueDecoderClass: Class[T],
     kafkaParams: JMap[String, String],
     topics: JMap[String, JInt],
     storageLevel: StorageLevel)
-  : JavaDStream[T] = {
-    implicit val cmt: ClassManifest[T] =
-      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]]
-    ssc.kafkaStream[T, D](
+  : JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassManifest[K] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+    implicit val valueCmt: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+
+    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+    ssc.kafkaStream[K, V, U, T](
       kafkaParams.toMap,
       Map(topics.mapValues(_.intValue()).toSeq: _*),
       storageLevel)
@@ -584,6 +593,77 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
   }
 
+  /**
+   * Create a unified DStream from multiple DStreams of the same type and same slide duration.
+   */
+  def union[T](first: JavaDStream[T], rest: JList[JavaDStream[T]]): JavaDStream[T] = {
+    val dstreams: Seq[DStream[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
+    implicit val cm: ClassManifest[T] = first.classManifest
+    ssc.union(dstreams)(cm)
+  }
+
+  /**
+   * Create a unified DStream from multiple DStreams of the same type and same slide duration.
+   */
+  def union[K, V](
+      first: JavaPairDStream[K, V],
+      rest: JList[JavaPairDStream[K, V]]
+    ): JavaPairDStream[K, V] = {
+    val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
+    implicit val cm: ClassManifest[(K, V)] = first.classManifest
+    implicit val kcm: ClassManifest[K] = first.kManifest
+    implicit val vcm: ClassManifest[V] = first.vManifest
+    new JavaPairDStream[K, V](ssc.union(dstreams)(cm))(kcm, vcm)
+  }
+
+  /**
+   * Create a new DStream in which each RDD is generated by applying a function on RDDs of
+   * the DStreams. The order of the JavaRDDs in the transform function parameter will be the
+   * same as the order of corresponding DStreams in the list. Note that for adding a
+   * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using
+   * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream().
+   * In the transform function, convert the JavaRDD corresponding to that JavaDStream to
+   * a JavaPairRDD using [[org.apache.spark.api.java.JavaPairRDD]].fromJavaRDD().
+   */
+  def transform[T](
+      dstreams: JList[JavaDStream[_]],
+      transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaRDD[T]]
+    ): JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val scalaDStreams = dstreams.map(_.dstream).toSeq
+    val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
+      val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
+      transformFunc.call(jrdds, time).rdd
+    }
+    ssc.transform(scalaDStreams, scalaTransformFunc)
+  }
+
+  /**
+   * Create a new DStream in which each RDD is generated by applying a function on RDDs of
+   * the DStreams. The order of the JavaRDDs in the transform function parameter will be the
+   * same as the order of corresponding DStreams in the list. Note that for adding a
+   * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using
+   * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream().
+   * In the transform function, convert the JavaRDD corresponding to that JavaDStream to
+   * a JavaPairRDD using [[org.apache.spark.api.java.JavaPairRDD]].fromJavaRDD().
+   */
+  def transform[K, V](
+      dstreams: JList[JavaDStream[_]],
+      transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaPairRDD[K, V]]
+    ): JavaPairDStream[K, V] = {
+    implicit val cmk: ClassManifest[K] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+    implicit val cmv: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    val scalaDStreams = dstreams.map(_.dstream).toSeq
+    val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
+      val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
+      transformFunc.call(jrdds, time).rdd
+    }
+    ssc.transform(scalaDStreams, scalaTransformFunc)
+  }
+
   /**
    * Sets the context to periodically checkpoint the DStream operations for master
    * fault-tolerance. The graph will be checkpointed every batch interval.
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
deleted file mode 100644
index 4eddc755b97f46e0e055917c3815363ae8f8b789..0000000000000000000000000000000000000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.dstream
-
-import org.apache.spark.Partitioner
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.CoGroupedRDD
-import org.apache.spark.streaming.{Time, DStream, Duration}
-
-private[streaming]
-class CoGroupedDStream[K : ClassManifest](
-    parents: Seq[DStream[(K, _)]],
-    partitioner: Partitioner
-  ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) {
-
-  if (parents.length == 0) {
-    throw new IllegalArgumentException("Empty array of parents")
-  }
-
-  if (parents.map(_.ssc).distinct.size > 1) {
-    throw new IllegalArgumentException("Array of parents have different StreamingContexts")
-  }
-
-  if (parents.map(_.slideDuration).distinct.size > 1) {
-    throw new IllegalArgumentException("Array of parents have different slide times")
-  }
-
-  override def dependencies = parents.toList
-
-  override def slideDuration: Duration = parents.head.slideDuration
-
-  override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = {
-    val part = partitioner
-    val rdds = parents.flatMap(_.getOrCompute(validTime))
-    if (rdds.size > 0) {
-      val q = new CoGroupedRDD[K](rdds, part)
-      Some(q)
-    } else {
-      None
-    }
-  }
-
-}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
index 51e913675d24acbfd04534e6d94ed2edc2cdc4af..a5de5e1fb549c9e5ee6c75adb203200ffbdea09a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
@@ -19,22 +19,18 @@ package org.apache.spark.streaming.dstream
 
 import org.apache.spark.Logging
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Time, DStreamCheckpointData, StreamingContext}
+import org.apache.spark.streaming.StreamingContext
 
 import java.util.Properties
 import java.util.concurrent.Executors
 
 import kafka.consumer._
-import kafka.message.{Message, MessageSet, MessageAndMetadata}
 import kafka.serializer.Decoder
-import kafka.utils.{Utils, ZKGroupTopicDirs}
-import kafka.utils.ZkUtils._
+import kafka.utils.VerifiableProperties
 import kafka.utils.ZKStringSerializer
 import org.I0Itec.zkclient._
 
 import scala.collection.Map
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
 
 
 /**
@@ -46,25 +42,32 @@ import scala.collection.JavaConversions._
  * @param storageLevel RDD storage level.
  */
 private[streaming]
-class KafkaInputDStream[T: ClassManifest, D <: Decoder[_]: Manifest](
+class KafkaInputDStream[
+  K: ClassManifest,
+  V: ClassManifest,
+  U <: Decoder[_]: Manifest,
+  T <: Decoder[_]: Manifest](
     @transient ssc_ : StreamingContext,
     kafkaParams: Map[String, String],
     topics: Map[String, Int],
     storageLevel: StorageLevel
-  ) extends NetworkInputDStream[T](ssc_ ) with Logging {
+  ) extends NetworkInputDStream[(K, V)](ssc_) with Logging {
 
-
-  def getReceiver(): NetworkReceiver[T] = {
-    new KafkaReceiver[T, D](kafkaParams, topics, storageLevel)
-        .asInstanceOf[NetworkReceiver[T]]
+  def getReceiver(): NetworkReceiver[(K, V)] = {
+    new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
+        .asInstanceOf[NetworkReceiver[(K, V)]]
   }
 }
 
 private[streaming]
-class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
-  kafkaParams: Map[String, String],
-  topics: Map[String, Int],
-  storageLevel: StorageLevel
+class KafkaReceiver[
+  K: ClassManifest,
+  V: ClassManifest,
+  U <: Decoder[_]: Manifest,
+  T <: Decoder[_]: Manifest](
+    kafkaParams: Map[String, String],
+    topics: Map[String, Int],
+    storageLevel: StorageLevel
   ) extends NetworkReceiver[Any] {
 
   // Handles pushing data into the BlockManager
@@ -83,27 +86,34 @@ class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
     // In case we are using multiple Threads to handle Kafka Messages
     val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
 
-    logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("groupid"))
+    logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id"))
 
     // Kafka connection properties
     val props = new Properties()
     kafkaParams.foreach(param => props.put(param._1, param._2))
 
     // Create the connection to the cluster
-    logInfo("Connecting to Zookeper: " + kafkaParams("zk.connect"))
+    logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect"))
     val consumerConfig = new ConsumerConfig(props)
     consumerConnector = Consumer.create(consumerConfig)
-    logInfo("Connected to " + kafkaParams("zk.connect"))
+    logInfo("Connected to " + kafkaParams("zookeeper.connect"))
 
     // When autooffset.reset is defined, it is our responsibility to try and whack the
     // consumer group zk node.
-    if (kafkaParams.contains("autooffset.reset")) {
-      tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid"))
+    if (kafkaParams.contains("auto.offset.reset")) {
+      tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id"))
     }
 
     // Create Threads for each Topic/Message Stream we are listening
-    val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]]
-    val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder)
+    val keyDecoder = manifest[U].erasure.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[K]]
+    val valueDecoder = manifest[T].erasure.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[V]]
+
+    val topicMessageStreams = consumerConnector.createMessageStreams(
+      topics, keyDecoder, valueDecoder)
 
     // Start the messages handler for each partition
     topicMessageStreams.values.foreach { streams =>
@@ -112,11 +122,12 @@ class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
   }
 
   // Handles Kafka Messages
-  private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable {
+  private class MessageHandler[K: ClassManifest, V: ClassManifest](stream: KafkaStream[K, V])
+    extends Runnable {
     def run() {
       logInfo("Starting MessageHandler.")
       for (msgAndMetadata <- stream) {
-        blockGenerator += msgAndMetadata.message
+        blockGenerator += (msgAndMetadata.key, msgAndMetadata.message)
       }
     }
   }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ac0528213d3290832d458d4eea91992d99f3cbe9
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.dstream
+
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{ Time, DStreamCheckpointData, StreamingContext }
+
+import java.util.Properties
+import java.util.concurrent.Executors
+import java.io.IOException
+
+import org.eclipse.paho.client.mqttv3.MqttCallback
+import org.eclipse.paho.client.mqttv3.MqttClient
+import org.eclipse.paho.client.mqttv3.MqttClientPersistence
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
+import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
+import org.eclipse.paho.client.mqttv3.MqttException
+import org.eclipse.paho.client.mqttv3.MqttMessage
+import org.eclipse.paho.client.mqttv3.MqttTopic
+
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+
+/**
+ * Input stream that subscribe messages from a Mqtt Broker.
+ * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
+ * @param brokerUrl Url of remote mqtt publisher
+ * @param topic topic name to subscribe to
+ * @param storageLevel RDD storage level.
+ */
+
+private[streaming] 
+class MQTTInputDStream[T: ClassManifest](
+  @transient ssc_ : StreamingContext,
+  brokerUrl: String,
+  topic: String,
+  storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_) with Logging {
+  
+  def getReceiver(): NetworkReceiver[T] = {
+    new MQTTReceiver(brokerUrl, topic, storageLevel)
+      .asInstanceOf[NetworkReceiver[T]]
+  }
+}
+
+private[streaming] 
+class MQTTReceiver(brokerUrl: String,
+  topic: String,
+  storageLevel: StorageLevel
+  ) extends NetworkReceiver[Any] {
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+  
+  def onStop() {
+    blockGenerator.stop()
+  }
+  
+  def onStart() {
+
+    blockGenerator.start()
+
+    // Set up persistence for messages 
+    var peristance: MqttClientPersistence = new MemoryPersistence()
+
+    // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
+    var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance)
+
+    // Connect to MqttBroker    
+    client.connect()
+
+    // Subscribe to Mqtt topic
+    client.subscribe(topic)
+
+    // Callback automatically triggers as and when new message arrives on specified topic
+    var callback: MqttCallback = new MqttCallback() {
+
+      // Handles Mqtt message 
+      override def messageArrived(arg0: String, arg1: MqttMessage) {
+        blockGenerator += new String(arg1.getPayload())
+      }
+
+      override def deliveryComplete(arg0: IMqttDeliveryToken) {
+      }
+
+      override def connectionLost(arg0: Throwable) {
+        logInfo("Connection lost " + arg0)
+      }
+    }
+
+    // Set up callback for MqttClient
+    client.setCallback(callback)
+  }
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
index 60485adef9594124f4dcfdb4a91115c03dbe5a63..71bcb2b390582beffd41d12ecfeb6c8cde52d6e4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
@@ -21,16 +21,22 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.streaming.{Duration, DStream, Time}
 
 private[streaming]
-class TransformedDStream[T: ClassManifest, U: ClassManifest] (
-    parent: DStream[T],
-    transformFunc: (RDD[T], Time) => RDD[U]
-  ) extends DStream[U](parent.ssc) {
+class TransformedDStream[U: ClassManifest] (
+    parents: Seq[DStream[_]],
+    transformFunc: (Seq[RDD[_]], Time) => RDD[U]
+  ) extends DStream[U](parents.head.ssc) {
 
-  override def dependencies = List(parent)
+  require(parents.length > 0, "List of DStreams to transform is empty")
+  require(parents.map(_.ssc).distinct.size == 1, "Some of the DStreams have different contexts")
+  require(parents.map(_.slideDuration).distinct.size == 1,
+    "Some of the DStreams have different slide durations")
 
-  override def slideDuration: Duration = parent.slideDuration
+  override def dependencies = parents.toList
+
+  override def slideDuration: Duration = parents.head.slideDuration
 
   override def compute(validTime: Time): Option[RDD[U]] = {
-    parent.getOrCompute(validTime).map(transformFunc(_, validTime))
+    val parentRDDs = parents.map(_.getOrCompute(validTime).orNull).toSeq
+    Some(transformFunc(parentRDDs, validTime))
   }
 }
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index c0d729ff8767373549991e872906a2b63d5fd05d..ad4a8b95355b9a185eb76d4d4e68dabb2a34bff5 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Maps;
 import com.google.common.io.Files;
 import kafka.serializer.StringDecoder;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.spark.streaming.api.java.JavaDStreamLike;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -183,6 +184,39 @@ public class JavaAPISuite implements Serializable {
     assertOrderInvariantEquals(expected, result);
   }
 
+  @Test
+  public void testRepartitionMorePartitions() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 2);
+    JavaDStream repartitioned = stream.repartition(4);
+    JavaTestUtils.attachTestOutputStream(repartitioned);
+    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
+    Assert.assertEquals(2, result.size());
+    for (List<List<Integer>> rdd : result) {
+      Assert.assertEquals(4, rdd.size());
+      Assert.assertEquals(
+        10, rdd.get(0).size() + rdd.get(1).size() + rdd.get(2).size() + rdd.get(3).size());
+    }
+  }
+
+  @Test
+  public void testRepartitionFewerPartitions() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 4);
+    JavaDStream repartitioned = stream.repartition(2);
+    JavaTestUtils.attachTestOutputStream(repartitioned);
+    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
+    Assert.assertEquals(2, result.size());
+    for (List<List<Integer>> rdd : result) {
+      Assert.assertEquals(2, rdd.size());
+      Assert.assertEquals(10, rdd.get(0).size() + rdd.get(1).size());
+    }
+  }
+
   @Test
   public void testGlom() {
     List<List<String>> inputData = Arrays.asList(
@@ -223,7 +257,7 @@ public class JavaAPISuite implements Serializable {
       }
     });
     JavaTestUtils.attachTestOutputStream(mapped);
-    List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
 
     Assert.assertEquals(expected, result);
   }
@@ -292,8 +326,8 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList(7,8,9));
 
     JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
-    JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1,2,3));
-    JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4,5,6));
+    JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1, 2, 3));
+    JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4, 5, 6));
     JavaRDD<Integer> rdd3 = ssc.sc().parallelize(Arrays.asList(7,8,9));
 
     LinkedList<JavaRDD<Integer>> rdds = Lists.newLinkedList();
@@ -320,23 +354,335 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList(9,10,11));
 
     JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> transformed =
-        stream.transform(new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
-      @Override
-      public JavaRDD<Integer> call(JavaRDD<Integer> in) throws Exception {
-        return in.map(new Function<Integer, Integer>() {
-          @Override
-          public Integer call(Integer i) throws Exception {
-            return i + 2;
-          }
-        });
-      }});
+    JavaDStream<Integer> transformed = stream.transform(
+      new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+        @Override
+        public JavaRDD<Integer> call(JavaRDD<Integer> in) throws Exception {
+          return in.map(new Function<Integer, Integer>() {
+            @Override
+            public Integer call(Integer i) throws Exception {
+              return i + 2;
+            }
+          });
+        }
+      });
+
     JavaTestUtils.attachTestOutputStream(transformed);
     List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
 
     assertOrderInvariantEquals(expected, result);
   }
 
+  @Test
+  public void testVariousTransform() {
+    // tests whether all variations of transform can be called from Java
+
+    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1));
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData =
+        Arrays.asList(Arrays.asList(new Tuple2<String, Integer>("x", 1)));
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1));
+
+    JavaDStream<Integer> transformed1 = stream.transform(
+        new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+          @Override
+          public JavaRDD<Integer> call(JavaRDD<Integer> in) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Integer> transformed2 = stream.transform(
+      new Function2<JavaRDD<Integer>, Time, JavaRDD<Integer>>() {
+        @Override public JavaRDD<Integer> call(JavaRDD<Integer> in, Time time) throws Exception {
+          return null;
+        }
+      }
+    );
+
+    JavaPairDStream<String, Integer> transformed3 = stream.transform(
+        new Function<JavaRDD<Integer>, JavaPairRDD<String, Integer>>() {
+          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<String, Integer> transformed4 = stream.transform(
+        new Function2<JavaRDD<Integer>, Time, JavaPairRDD<String, Integer>>() {
+          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Integer> pairTransformed1 = pairStream.transform(
+        new Function<JavaPairRDD<String, Integer>, JavaRDD<Integer>>() {
+          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Integer> pairTransformed2 = pairStream.transform(
+        new Function2<JavaPairRDD<String, Integer>, Time, JavaRDD<Integer>>() {
+          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<String, String> pairTransformed3 = pairStream.transform(
+        new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, String>>() {
+          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<String, String> pairTransformed4 = pairStream.transform(
+        new Function2<JavaPairRDD<String, Integer>, Time, JavaPairRDD<String, String>>() {
+          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+  }
+
+  @Test
+  public void testTransformWith() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, String>("california", "dodgers"),
+            new Tuple2<String, String>("new york", "yankees")),
+        Arrays.asList(
+            new Tuple2<String, String>("california", "sharks"),
+            new Tuple2<String, String>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, String>("california", "giants"),
+            new Tuple2<String, String>("new york", "mets")),
+        Arrays.asList(
+            new Tuple2<String, String>("california", "ducks"),
+            new Tuple2<String, String>("new york", "islanders")));
+
+
+    List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Tuple2<String, String>>("california",
+                new Tuple2<String, String>("dodgers", "giants")),
+            new Tuple2<String, Tuple2<String, String>>("new york",
+                    new Tuple2<String, String>("yankees", "mets"))),
+        Arrays.asList(
+            new Tuple2<String, Tuple2<String, String>>("california",
+                new Tuple2<String, String>("sharks", "ducks")),
+            new Tuple2<String, Tuple2<String, String>>("new york",
+                new Tuple2<String, String>("rangers", "islanders"))));
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.transformWith(
+        pairStream2,
+        new Function3<
+            JavaPairRDD<String, String>,
+            JavaPairRDD<String, String>,
+            Time,
+            JavaPairRDD<String, Tuple2<String, String>>
+          >() {
+          @Override
+          public JavaPairRDD<String, Tuple2<String, String>> call(
+              JavaPairRDD<String, String> rdd1,
+              JavaPairRDD<String, String> rdd2,
+              Time time
+          ) throws Exception {
+            return rdd1.join(rdd2);
+          }
+        }
+    );
+
+    JavaTestUtils.attachTestOutputStream(joined);
+    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+
+  @Test
+  public void testVariousTransformWith() {
+    // tests whether all variations of transformWith can be called from Java
+
+    List<List<Integer>> inputData1 = Arrays.asList(Arrays.asList(1));
+    List<List<String>> inputData2 = Arrays.asList(Arrays.asList("x"));
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1);
+    JavaDStream<String> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData1 =
+        Arrays.asList(Arrays.asList(new Tuple2<String, Integer>("x", 1)));
+    List<List<Tuple2<Double, Character>>> pairInputData2 =
+        Arrays.asList(Arrays.asList(new Tuple2<Double, Character>(1.0, 'x')));
+    JavaPairDStream<String, Integer> pairStream1 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1));
+    JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1));
+
+    JavaDStream<Double> transformed1 = stream1.transformWith(
+        stream2,
+        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Double> transformed2 = stream1.transformWith(
+        pairStream1,
+        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<Double, Double> transformed3 = stream1.transformWith(
+        stream2,
+        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<Double, Double> transformed4 = stream1.transformWith(
+        pairStream1,
+        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Double> pairTransformed1 = pairStream1.transformWith(
+        stream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaDStream<Double> pairTransformed2_ = pairStream1.transformWith(
+        pairStream1,
+        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<Double, Double> pairTransformed3 = pairStream1.transformWith(
+        stream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+
+    JavaPairDStream<Double, Double> pairTransformed4 = pairStream1.transformWith(
+        pairStream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<Double, Character>, Time, JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, JavaPairRDD<Double, Character> rdd2, Time time) throws Exception {
+            return null;
+          }
+        }
+    );
+  }
+
+  @Test
+  public void testStreamingContextTransform(){
+    List<List<Integer>> stream1input = Arrays.asList(
+        Arrays.asList(1),
+        Arrays.asList(2)
+    );
+
+    List<List<Integer>> stream2input = Arrays.asList(
+        Arrays.asList(3),
+        Arrays.asList(4)
+    );
+
+    List<List<Tuple2<Integer, String>>> pairStream1input = Arrays.asList(
+        Arrays.asList(new Tuple2<Integer, String>(1, "x")),
+        Arrays.asList(new Tuple2<Integer, String>(2, "y"))
+    );
+
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<Integer, Tuple2<Integer, String>>(1, new Tuple2<Integer, String>(1, "x"))),
+        Arrays.asList(new Tuple2<Integer, Tuple2<Integer, String>>(2, new Tuple2<Integer, String>(2, "y")))
+    );
+
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1);
+    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1);
+    JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1));
+
+    List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2);
+
+    // This is just to test whether this transform to JavaStream compiles
+    JavaDStream<Long> transformed1 = ssc.transform(
+      listOfDStreams1,
+      new Function2<List<JavaRDD<?>>, Time, JavaRDD<Long>>() {
+        public JavaRDD<Long> call(List<JavaRDD<?>> listOfRDDs, Time time) {
+          assert(listOfRDDs.size() == 2);
+          return null;
+        }
+      }
+    );
+
+    List<JavaDStream<?>> listOfDStreams2 =
+        Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream());
+
+    JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transform(
+      listOfDStreams2,
+      new Function2<List<JavaRDD<?>>, Time, JavaPairRDD<Integer, Tuple2<Integer, String>>>() {
+        public JavaPairRDD<Integer, Tuple2<Integer, String>> call(List<JavaRDD<?>> listOfRDDs, Time time) {
+          assert(listOfRDDs.size() == 3);
+          JavaRDD<Integer> rdd1 = (JavaRDD<Integer>)listOfRDDs.get(0);
+          JavaRDD<Integer> rdd2 = (JavaRDD<Integer>)listOfRDDs.get(1);
+          JavaRDD<Tuple2<Integer, String>> rdd3 = (JavaRDD<Tuple2<Integer, String>>)listOfRDDs.get(2);
+          JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3);
+          PairFunction<Integer, Integer, Integer> mapToTuple = new PairFunction<Integer, Integer, Integer>() {
+            @Override
+            public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+              return new Tuple2<Integer, Integer>(i, i);
+            }
+          };
+          return rdd1.union(rdd2).map(mapToTuple).join(prdd3);
+        }
+      }
+    );
+    JavaTestUtils.attachTestOutputStream(transformed2);
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    Assert.assertEquals(expected, result);
+  }
+
   @Test
   public void testFlatMap() {
     List<List<String>> inputData = Arrays.asList(
@@ -1099,7 +1445,7 @@ public class JavaAPISuite implements Serializable {
 
     JavaPairDStream<String, Tuple2<List<String>, List<String>>> grouped = pairStream1.cogroup(pairStream2);
     JavaTestUtils.attachTestOutputStream(grouped);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    List<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
 
     Assert.assertEquals(expected, result);
   }
@@ -1142,7 +1488,38 @@ public class JavaAPISuite implements Serializable {
 
     JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.join(pairStream2);
     JavaTestUtils.attachTestOutputStream(joined);
-    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testLeftOuterJoin() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+            new Tuple2<String, String>("new york", "yankees")),
+        Arrays.asList(new Tuple2<String, String>("california", "sharks") ));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "giants") ),
+        Arrays.asList(new Tuple2<String, String>("new york", "islanders") )
+
+    );
+
+    List<List<Long>> expected = Arrays.asList(Arrays.asList(2L), Arrays.asList(1L));
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, Optional<String>>> joined = pairStream1.leftOuterJoin(pairStream2);
+    JavaDStream<Long> counted = joined.count();
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 2, 2);
 
     Assert.assertEquals(expected, result);
   }
@@ -1220,14 +1597,20 @@ public class JavaAPISuite implements Serializable {
   @Test
   public void testKafkaStream() {
     HashMap<String, Integer> topics = Maps.newHashMap();
-    JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics);
-    JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics,
+    JavaPairDStream<String, String> test1 = ssc.kafkaStream("localhost:12345", "group", topics);
+    JavaPairDStream<String, String> test2 = ssc.kafkaStream("localhost:12345", "group", topics,
       StorageLevel.MEMORY_AND_DISK());
 
     HashMap<String, String> kafkaParams = Maps.newHashMap();
-    kafkaParams.put("zk.connect","localhost:12345");
-    kafkaParams.put("groupid","consumer-group");
-    JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics,
+    kafkaParams.put("zookeeper.connect","localhost:12345");
+    kafkaParams.put("group.id","consumer-group");
+    JavaPairDStream<String, String> test3 = ssc.kafkaStream(
+      String.class,
+      String.class,
+      StringDecoder.class,
+      StringDecoder.class,
+      kafkaParams,
+      topics,
       StorageLevel.MEMORY_AND_DISK());
   }
 
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
index 8a6604904de57d39626eac6ec5f2b19405dad833..5e384eeee45f385ff83bba34f93c4b2228f5faaa 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -33,9 +33,9 @@ trait JavaTestBase extends TestSuiteBase {
    * The stream will be derived from the supplied lists of Java objects.
    **/
   def attachTestInputStream[T](
-    ssc: JavaStreamingContext,
-    data: JList[JList[T]],
-    numPartitions: Int) = {
+      ssc: JavaStreamingContext,
+      data: JList[JList[T]],
+      numPartitions: Int) = {
     val seqData = data.map(Seq(_:_*))
 
     implicit val cm: ClassManifest[T] =
@@ -50,12 +50,11 @@ trait JavaTestBase extends TestSuiteBase {
    * [[org.apache.spark.streaming.TestOutputStream]].
    **/
   def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]](
-    dstream: JavaDStreamLike[T, This, R]) =
+      dstream: JavaDStreamLike[T, This, R]) =
   {
     implicit val cm: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    val ostream = new TestOutputStream(dstream.dstream,
-      new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]])
+    val ostream = new TestOutputStreamWithPartitions(dstream.dstream)
     dstream.dstream.ssc.registerOutputStream(ostream)
   }
 
@@ -63,9 +62,11 @@ trait JavaTestBase extends TestSuiteBase {
    * Process all registered streams for a numBatches batches, failing if
    * numExpectedOutput RDD's are not generated. Generated RDD's are collected
    * and returned, represented as a list for each batch interval.
+   *
+   * Returns a list of items for each RDD.
    */
   def runStreams[V](
-    ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
+      ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
     implicit val cm: ClassManifest[V] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
     val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
@@ -73,6 +74,27 @@ trait JavaTestBase extends TestSuiteBase {
     res.map(entry => out.append(new ArrayList[V](entry)))
     out
   }
+
+  /**
+   * Process all registered streams for a numBatches batches, failing if
+   * numExpectedOutput RDD's are not generated. Generated RDD's are collected
+   * and returned, represented as a list for each batch interval.
+   *
+   * Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each
+   * representing one partition.
+   */
+  def runStreamsWithPartitions[V](ssc: JavaStreamingContext, numBatches: Int,
+      numExpectedOutput: Int): JList[JList[JList[V]]] = {
+    implicit val cm: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    val res = runStreamsWithPartitions[V](ssc.ssc, numBatches, numExpectedOutput)
+    val out = new ArrayList[JList[JList[V]]]()
+    res.map{entry =>
+      val lists = entry.map(new ArrayList[V](_))
+      out.append(new ArrayList[JList[V]](lists))
+    }
+    out
+  }
 }
 
 object JavaTestUtils extends JavaTestBase {
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 11586f72b6ddb081be0e4bc9e29f989ada9e7876..259ef1608cbc5b179372995ef4a426df0101247d 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -18,7 +18,10 @@
 package org.apache.spark.streaming
 
 import org.apache.spark.streaming.StreamingContext._
-import scala.runtime.RichInt
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.SparkContext._
+
 import util.ManualClock
 
 class BasicOperationsSuite extends TestSuiteBase {
@@ -82,6 +85,44 @@ class BasicOperationsSuite extends TestSuiteBase {
     testOperation(input, operation, output, true)
   }
 
+  test("repartition (more partitions)") {
+    val input = Seq(1 to 100, 101 to 200, 201 to 300)
+    val operation = (r: DStream[Int]) => r.repartition(5)
+    val ssc = setupStreams(input, operation, 2)
+    val output = runStreamsWithPartitions(ssc, 3, 3)
+    assert(output.size === 3)
+    val first = output(0)
+    val second = output(1)
+    val third = output(2)
+
+    assert(first.size === 5)
+    assert(second.size === 5)
+    assert(third.size === 5)
+
+    assert(first.flatten.toSet === (1 to 100).toSet)
+    assert(second.flatten.toSet === (101 to 200).toSet)
+    assert(third.flatten.toSet === (201 to 300).toSet)
+  }
+
+  test("repartition (fewer partitions)") {
+    val input = Seq(1 to 100, 101 to 200, 201 to 300)
+    val operation = (r: DStream[Int]) => r.repartition(2)
+    val ssc = setupStreams(input, operation, 5)
+    val output = runStreamsWithPartitions(ssc, 3, 3)
+    assert(output.size === 3)
+    val first = output(0)
+    val second = output(1)
+    val third = output(2)
+
+    assert(first.size === 2)
+    assert(second.size === 2)
+    assert(third.size === 2)
+
+    assert(first.flatten.toSet === (1 to 100).toSet)
+    assert(second.flatten.toSet === (101 to 200).toSet)
+    assert(third.flatten.toSet === (201 to 300).toSet)
+  }
+
   test("groupByKey") {
     testOperation(
       Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
@@ -143,6 +184,72 @@ class BasicOperationsSuite extends TestSuiteBase {
     )
   }
 
+  test("union") {
+    val input = Seq(1 to 4, 101 to 104, 201 to 204)
+    val output = Seq(1 to 8, 101 to 108, 201 to 208)
+    testOperation(
+      input,
+      (s: DStream[Int]) => s.union(s.map(_ + 4)) ,
+      output
+    )
+  }
+
+  test("StreamingContext.union") {
+    val input = Seq(1 to 4, 101 to 104, 201 to 204)
+    val output = Seq(1 to 12, 101 to 112, 201 to 212)
+    // union over 3 DStreams
+    testOperation(
+      input,
+      (s: DStream[Int]) => s.context.union(Seq(s, s.map(_ + 4), s.map(_ + 8))),
+      output
+    )
+  }
+
+  test("transform") {
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    testOperation(
+      input,
+      (r: DStream[Int]) => r.transform(rdd => rdd.map(_.toString)),   // RDD.map in transform
+      input.map(_.map(_.toString))
+    )
+  }
+
+  test("transformWith") {
+    val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() )
+    val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("")   )
+    val outputData = Seq(
+      Seq( ("a", (1, "x")), ("b", (1, "x")) ),
+      Seq( ("", (1, "x")) ),
+      Seq(  ),
+      Seq(  )
+    )
+    val operation = (s1: DStream[String], s2: DStream[String]) => {
+      val t1 = s1.map(x => (x, 1))
+      val t2 = s2.map(x => (x, "x"))
+      t1.transformWith(           // RDD.join in transform
+        t2,
+        (rdd1: RDD[(String, Int)], rdd2: RDD[(String, String)]) => rdd1.join(rdd2)
+      )
+    }
+    testOperation(inputData1, inputData2, operation, outputData, true)
+  }
+
+  test("StreamingContext.transform") {
+    val input = Seq(1 to 4, 101 to 104, 201 to 204)
+    val output = Seq(1 to 12, 101 to 112, 201 to 212)
+
+    // transform over 3 DStreams by doing union of the 3 RDDs
+    val operation = (s: DStream[Int]) => {
+      s.context.transform(
+        Seq(s, s.map(_ + 4), s.map(_ + 8)),   // 3 DStreams
+        (rdds: Seq[RDD[_]], time: Time) =>
+          rdds.head.context.union(rdds.map(_.asInstanceOf[RDD[Int]]))  // union of RDDs
+      )
+    }
+
+    testOperation(input, operation, output)
+  }
+
   test("cogroup") {
     val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() )
     val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq()   )
@@ -168,7 +275,37 @@ class BasicOperationsSuite extends TestSuiteBase {
       Seq(  )
     )
     val operation = (s1: DStream[String], s2: DStream[String]) => {
-      s1.map(x => (x,1)).join(s2.map(x => (x,"x")))
+      s1.map(x => (x, 1)).join(s2.map(x => (x, "x")))
+    }
+    testOperation(inputData1, inputData2, operation, outputData, true)
+  }
+
+  test("leftOuterJoin") {
+    val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() )
+    val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("")   )
+    val outputData = Seq(
+      Seq( ("a", (1, Some("x"))), ("b", (1, Some("x"))) ),
+      Seq( ("", (1, Some("x"))), ("a", (1, None)) ),
+      Seq( ("", (1, None)) ),
+      Seq(  )
+    )
+    val operation = (s1: DStream[String], s2: DStream[String]) => {
+      s1.map(x => (x, 1)).leftOuterJoin(s2.map(x => (x, "x")))
+    }
+    testOperation(inputData1, inputData2, operation, outputData, true)
+  }
+
+  test("rightOuterJoin") {
+    val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() )
+    val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("")   )
+    val outputData = Seq(
+      Seq( ("a", (Some(1), "x")), ("b", (Some(1), "x")) ),
+      Seq( ("", (Some(1), "x")), ("b", (None, "x")) ),
+      Seq(  ),
+      Seq( ("", (None, "x")) )
+    )
+    val operation = (s1: DStream[String], s2: DStream[String]) => {
+      s1.map(x => (x, 1)).rightOuterJoin(s2.map(x => (x, "x")))
     }
     testOperation(inputData1, inputData2, operation, outputData, true)
   }
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index a327de80b3eb3a0f98bcbd2486e0580225b93a8a..beb20831bd7b49d5a9c9443149a4c1eb9171384a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -366,7 +366,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     logInfo("Manual clock after advancing = " + clock.time)
     Thread.sleep(batchDuration.milliseconds)
 
-    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
-    outputStream.output
+    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]]
+    outputStream.output.map(_.flatten)
   }
 }
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 42e3e51e3fa15b18de981fe46f40fe999786a276..c29b75ece69f05211be5dc977b0f2e860282cca0 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -268,8 +268,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK)
 
     // Test specifying decoder
-    val kafkaParams = Map("zk.connect"->"localhost:12345","groupid"->"consumer-group")
-    val test3 = ssc.kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
+    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
+    val test3 = ssc.kafkaStream[
+      String,
+      String,
+      kafka.serializer.StringDecoder,
+      kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
   }
 }
 
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 37dd9c4cc61d29d28f247bbdf9044b80e8fce76b..be140699c2964456a747a076120987c9abc26f39 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -60,8 +60,11 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
 /**
  * This is a output stream just for the testsuites. All the output is collected into a
  * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ *
+ * The buffer contains a sequence of RDD's, each containing a sequence of items
  */
-class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
+class TestOutputStream[T: ClassManifest](parent: DStream[T],
+    val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]())
   extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
     val collected = rdd.collect()
     output += collected
@@ -75,6 +78,30 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBu
   }
 }
 
+/**
+ * This is a output stream just for the testsuites. All the output is collected into a
+ * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ *
+ * The buffer contains a sequence of RDD's, each containing a sequence of partitions, each
+ * containing a sequence of items.
+ */
+class TestOutputStreamWithPartitions[T: ClassManifest](parent: DStream[T],
+    val output: ArrayBuffer[Seq[Seq[T]]] = ArrayBuffer[Seq[Seq[T]]]())
+  extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
+    val collected = rdd.glom().collect().map(_.toSeq)
+    output += collected
+  }) {
+
+  // This is to clear the output buffer every it is read from a checkpoint
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    ois.defaultReadObject()
+    output.clear()
+  }
+
+  def toTestOutputStream = new TestOutputStream[T](this.parent, this.output.map(_.flatten))
+}
+
 /**
  * This is the base trait for Spark Streaming testsuites. This provides basic functionality
  * to run user-defined set of input on user-defined stream operations, and verify the output.
@@ -108,7 +135,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
    */
   def setupStreams[U: ClassManifest, V: ClassManifest](
       input: Seq[Seq[U]],
-      operation: DStream[U] => DStream[V]
+      operation: DStream[U] => DStream[V],
+      numPartitions: Int = numInputPartitions
     ): StreamingContext = {
 
     // Create StreamingContext
@@ -118,9 +146,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
     }
 
     // Setup the stream computation
-    val inputStream = new TestInputStream(ssc, input, numInputPartitions)
+    val inputStream = new TestInputStream(ssc, input, numPartitions)
     val operatedStream = operation(inputStream)
-    val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[V]] with SynchronizedBuffer[Seq[V]])
+    val outputStream = new TestOutputStreamWithPartitions(operatedStream,
+      new ArrayBuffer[Seq[Seq[V]]] with SynchronizedBuffer[Seq[Seq[V]]])
     ssc.registerInputStream(inputStream)
     ssc.registerOutputStream(outputStream)
     ssc
@@ -146,7 +175,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
     val inputStream1 = new TestInputStream(ssc, input1, numInputPartitions)
     val inputStream2 = new TestInputStream(ssc, input2, numInputPartitions)
     val operatedStream = operation(inputStream1, inputStream2)
-    val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[W]] with SynchronizedBuffer[Seq[W]])
+    val outputStream = new TestOutputStreamWithPartitions(operatedStream,
+      new ArrayBuffer[Seq[Seq[W]]] with SynchronizedBuffer[Seq[Seq[W]]])
     ssc.registerInputStream(inputStream1)
     ssc.registerInputStream(inputStream2)
     ssc.registerOutputStream(outputStream)
@@ -157,18 +187,37 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
    * Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
    * returns the collected output. It will wait until `numExpectedOutput` number of
    * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
+   *
+   * Returns a sequence of items for each RDD.
    */
   def runStreams[V: ClassManifest](
       ssc: StreamingContext,
       numBatches: Int,
       numExpectedOutput: Int
     ): Seq[Seq[V]] = {
+    // Flatten each RDD into a single Seq
+    runStreamsWithPartitions(ssc, numBatches, numExpectedOutput).map(_.flatten.toSeq)
+  }
+
+  /**
+   * Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
+   * returns the collected output. It will wait until `numExpectedOutput` number of
+   * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
+   *
+   * Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each
+   * representing one partition.
+   */
+  def runStreamsWithPartitions[V: ClassManifest](
+      ssc: StreamingContext,
+      numBatches: Int,
+      numExpectedOutput: Int
+    ): Seq[Seq[Seq[V]]] = {
     assert(numBatches > 0, "Number of batches to run stream computation is zero")
     assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero")
     logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput)
 
     // Get the output buffer
-    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
+    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]]
     val output = outputStream.output
 
     try {
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 8afb3e39cb8a6a0e7df4c4e41e03083867601b71..1a380ae714534bc06146d62c0921a404bc64ce5e 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -265,11 +265,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val env = new HashMap[String, String]()
 
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name,
-      Environment.PWD.$() + Path.SEPARATOR + "*")
-
-    Client.populateHadoopClasspath(yarnConf, env)
+    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_JAR_PATH") = 
       localResources("spark.jar").getResource().getScheme.toString() + "://" +
@@ -451,4 +447,30 @@ object Client {
       Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
     }
   }
+
+  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+    // If log4j present, ensure ours overrides all others
+    if (addLog4j) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + "log4j.properties")
+    }
+    // normally the users app.jar is last in case conflicts with spark jars
+    val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
+      .toBoolean
+    if (userClasspathFirst) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + "app.jar")
+    }
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Path.SEPARATOR + "spark.jar")
+    Client.populateHadoopClasspath(conf, env)
+
+    if (!userClasspathFirst) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + "app.jar")
+    }
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Path.SEPARATOR + "*")
+  }
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 8dac9e02ac0b712fcf712c60575b43b3d2729f9c..ba352daac485d0381a2df87f391b8435a315bc61 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -121,7 +121,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ?
       " -XX:OnOutOfMemoryError='kill %p' " +
       JAVA_OPTS +
-      " org.apache.spark.executor.StandaloneExecutorBackend " +
+      " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
       masterAddress + " " +
       slaveId + " " +
       hostname + " " +
@@ -216,10 +216,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, 
-      Environment.PWD.$() + Path.SEPARATOR + "*")
-    Client.populateHadoopClasspath(yarnConf, env)
+    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
     // allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 6d6ef149cc2d8b6351013bce769c0bf421cd9eab..25da9aa917d9510fbead2d7f34a42989144a741e 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -22,7 +22,7 @@ import org.apache.spark.util.Utils
 import org.apache.spark.scheduler.SplitInfo
 import scala.collection
 import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container}
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
 import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
 import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
@@ -211,7 +211,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           val workerId = workerIdCounter.incrementAndGet().toString
           val driverUrl = "akka://spark@%s:%s/user/%s".format(
             System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
-            StandaloneSchedulerBackend.ACTOR_NAME)
+            CoarseGrainedSchedulerBackend.ACTOR_NAME)
 
           logInfo("launching container on " + containerId + " host " + workerHostname)
           // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but ..