diff --git a/README.md b/README.md
index 7d3f9d4845d247299b9c16ac41fbaa2b9c78e453..80bbe311a94a1cdb05ce7198132f1d280e1c018c 100644
--- a/README.md
+++ b/README.md
@@ -54,7 +54,7 @@ versions without YARN, use:
     # Cloudera CDH 4.2.0 with MapReduce v1
     $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly
 
-For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
+For Apache Hadoop 2.0.X, 2.1.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
 with YARN, also set `SPARK_YARN=true`:
 
     # Apache Hadoop 2.0.5-alpha
@@ -63,12 +63,14 @@ with YARN, also set `SPARK_YARN=true`:
     # Cloudera CDH 4.2.0 with MapReduce v2
     $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly
 
-For convenience, these variables may also be set through the `conf/spark-env.sh` file
-described below.
+When building for Hadoop 2.2.X and newer, you'll need to include the additional `new-yarn` profile:
+
+    # Apache Hadoop 2.2.X and newer
+    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
 
 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/assembly/pom.xml b/assembly/pom.xml
index 28b0692dff10350dab30a6863dee475e9fb5234f..c2cda41c6db776aa50b2c38eff4f5f18305af103 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-assembly_${scala-short.version}</artifactId>
+  <artifactId>spark-assembly_2.10</artifactId>
   <name>Spark Project Assembly</name>
   <url>http://spark.incubator.apache.org/</url>
 
@@ -41,27 +41,27 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-bagel_${scala-short.version}</artifactId>
+      <artifactId>spark-bagel_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-mllib_${scala-short.version}</artifactId>
+      <artifactId>spark-mllib_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-repl_${scala-short.version}</artifactId>
+      <artifactId>spark-repl_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala-short.version}</artifactId>
+      <artifactId>spark-streaming_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -79,7 +79,7 @@
         <artifactId>maven-shade-plugin</artifactId>
         <configuration>
           <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
+          <outputFile>${project.build.directory}/scala-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
           <artifactSet>
             <includes>
               <include>*:*</include>
@@ -128,7 +128,7 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.spark</groupId>
-          <artifactId>spark-yarn_${scala-short.version}</artifactId>
+          <artifactId>spark-yarn_2.10</artifactId>
           <version>${project.version}</version>
         </dependency>
       </dependencies>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index c8b9c4f4cdd979c32724e642e75ad49515fe0927..0f550d70d65c7a91d8e1a912f0934766defc4b70 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-bagel_${scala-short.version}</artifactId>
+  <artifactId>spark-bagel_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Bagel</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -43,18 +43,18 @@
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.scalatest</groupId>
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index 4fe3d0ef3aa3c7ba4a165279a0c424e50302f3ba..40555089fcfe4608f53130c0744094a06090c2c4 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/bin/stop-slaves.sh b/bin/stop-slaves.sh
index 03e416a13274d5c94caa381257332ec42bb31fd1..fcb8555d4e5706a6f4f2ec9670525437a3a2acae 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -17,8 +17,6 @@
 # limitations under the License.
 #
 
-# Starts the master on the machine this script is executed on.
-
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 
diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template
index ae10f615d1312be7f8dea3cca7862fcb98cc297d..1c3d94e1b0831d69e4635e328e69a0511a94e3f4 100644
--- a/conf/metrics.properties.template
+++ b/conf/metrics.properties.template
@@ -80,6 +80,14 @@
 #     /metrics/aplications/json # App information
 #     /metrics/master/json      # Master information
 
+# org.apache.spark.metrics.sink.GraphiteSink
+#   Name:     Default:      Description:
+#   host      NONE          Hostname of Graphite server
+#   port      NONE          Port of Graphite server
+#   period    10            Poll period
+#   unit      seconds       Units of poll period
+#   prefix    EMPTY STRING  Prefix to prepend to metric name
+
 ## Examples
 # Enable JmxSink for all instances by class name
 #*.sink.jmx.class=org.apache.spark.metrics.sink.JmxSink
diff --git a/core/pom.xml b/core/pom.xml
index 595240b5e5fd143160a2ea81ac3a7c94719be252..3fe48fd2afed52637aeacd900bd5a9fc5da0b3ae 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-core_${scala-short.version}</artifactId>
+  <artifactId>spark-core_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Core</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -48,6 +48,10 @@
       <groupId>org.apache.avro</groupId>
       <artifactId>avro-ipc</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
@@ -82,7 +86,7 @@
     </dependency>
     <dependency>
       <groupId>com.twitter</groupId>
-      <artifactId>chill_${scala-short.version}</artifactId>
+      <artifactId>chill_2.10</artifactId>
       <version>0.3.1</version>
     </dependency>
     <dependency>
@@ -91,12 +95,16 @@
       <version>0.3.1</version>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
-      <artifactId>akka-remote_${scala-short.version}</artifactId>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-actor_2.10</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
-      <artifactId>akka-slf4j_${scala-short.version}</artifactId>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-remote_2.10</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-slf4j_2.10</artifactId>
     </dependency>
     <dependency>
       <groupId>org.scala-lang</groupId>
@@ -104,7 +112,7 @@
     </dependency>
     <dependency>
       <groupId>net.liftweb</groupId>
-      <artifactId>lift-json_${scala-short.version}</artifactId>
+      <artifactId>lift-json_2.10</artifactId>
     </dependency>
     <dependency>
       <groupId>it.unimi.dsi</groupId>
@@ -114,10 +122,6 @@
       <groupId>colt</groupId>
       <artifactId>colt</artifactId>
     </dependency>
-    <dependency>
-      <groupId>com.github.scala-incubator.io</groupId>
-      <artifactId>scala-io-file_${scala-short.version}</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.mesos</groupId>
       <artifactId>mesos</artifactId>
@@ -146,6 +150,10 @@
       <groupId>com.codahale.metrics</groupId>
       <artifactId>metrics-ganglia</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-graphite</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.derby</groupId>
       <artifactId>derby</artifactId>
@@ -158,12 +166,12 @@
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -183,8 +191,8 @@
     </dependency>
   </dependencies>
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
index c4aa2669e05d36bea4bd6631ef5c96bd63634219..8a09210245fff7eccfdd50fa551be2a9d53e9954 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundByteHandlerAdapter;
 
+import org.apache.spark.storage.BlockId;
 
 abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
 
@@ -33,7 +34,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
   }
 
   public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header);
-  public abstract void handleError(String blockId);
+  public abstract void handleError(BlockId blockId);
 
   @Override
   public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) {
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 d3d57a02555720000bb90583bfde2ab4f32d07d1..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
@@ -24,6 +24,8 @@ import io.netty.channel.ChannelHandlerContext;
 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> {
 
@@ -34,41 +36,36 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
   }
 
   @Override
-  public void messageReceived(ChannelHandlerContext ctx, String blockId) {
-    String path = pResolver.getAbsolutePath(blockId);
-    // if getFilePath returns null, close the channel
-    if (path == null) {
+  public void messageReceived(ChannelHandlerContext ctx, String blockIdString) {
+    BlockId blockId = BlockId.apply(blockIdString);
+    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/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 3ef402926e8985e30744da8f0e67a30715b21c4e..1a2ec55876c35089d3b448a078d2212cafda08f4 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,43 +17,42 @@
 
 package org.apache.spark
 
-import java.util.{HashMap => JHashMap}
+import org.apache.spark.util.AppendOnlyMap
 
-import scala.collection.JavaConversions._
-
-/** A set of functions used to aggregate data.
-  * 
-  * @param createCombiner function to create the initial value of the aggregation.
-  * @param mergeValue function to merge a new value into the aggregation result.
-  * @param mergeCombiners function to merge outputs from multiple mergeValue function.
-  */
+/**
+ * A set of functions used to aggregate data.
+ *
+ * @param createCombiner function to create the initial value of the aggregation.
+ * @param mergeValue function to merge a new value into the aggregation result.
+ * @param mergeCombiners function to merge outputs from multiple mergeValue function.
+ */
 case class Aggregator[K, V, C] (
     createCombiner: V => C,
     mergeValue: (C, V) => C,
     mergeCombiners: (C, C) => C) {
 
   def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = {
-    val combiners = new JHashMap[K, C]
-    for (kv <- iter) {
-      val oldC = combiners.get(kv._1)
-      if (oldC == null) {
-        combiners.put(kv._1, createCombiner(kv._2))
-      } else {
-        combiners.put(kv._1, mergeValue(oldC, kv._2))
-      }
+    val combiners = new AppendOnlyMap[K, C]
+    var kv: Product2[K, V] = null
+    val update = (hadValue: Boolean, oldValue: C) => {
+      if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
+    }
+    while (iter.hasNext) {
+      kv = iter.next()
+      combiners.changeValue(kv._1, update)
     }
     combiners.iterator
   }
 
   def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
-    val combiners = new JHashMap[K, C]
-    iter.foreach { case(k, c) =>
-      val oldC = combiners.get(k)
-      if (oldC == null) {
-        combiners.put(k, c)
-      } else {
-        combiners.put(k, mergeCombiners(oldC, c))
-      }
+    val combiners = new AppendOnlyMap[K, C]
+    var kc: (K, C) = null
+    val update = (hadValue: Boolean, oldValue: C) => {
+      if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2
+    }
+    while (iter.hasNext) {
+      kc = iter.next()
+      combiners.changeValue(kc._1, update)
     }
     combiners.iterator
   }
diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
index 908ff56a6bbdb8bcd00a6ab671ee298ea8c67ed8..d9ed572da6dba07f37695b1f7c9406aed1f1b5d0 100644
--- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
@@ -22,13 +22,17 @@ import scala.collection.mutable.HashMap
 
 import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics}
 import org.apache.spark.serializer.Serializer
-import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId}
 import org.apache.spark.util.CompletionIterator
 
 
 private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
 
-  override def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer)
+  override def fetch[T](
+      shuffleId: Int,
+      reduceId: Int,
+      context: TaskContext,
+      serializer: Serializer)
     : Iterator[T] =
   {
 
@@ -45,12 +49,12 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
       splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size))
     }
 
-    val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] = splitsByAddress.toSeq.map {
+    val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map {
       case (address, splits) =>
-        (address, splits.map(s => ("shuffle_%d_%d_%d".format(shuffleId, s._1, reduceId), s._2)))
+        (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2)))
     }
 
-    def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[T] = {
+    def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = {
       val blockId = blockPair._1
       val blockOption = blockPair._2
       blockOption match {
@@ -58,9 +62,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
           block.asInstanceOf[Iterator[T]]
         }
         case None => {
-          val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r
           blockId match {
-            case regex(shufId, mapId, _) =>
+            case ShuffleBlockId(shufId, mapId, _) =>
               val address = statuses(mapId.toInt)._1
               throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, null)
             case _ =>
@@ -74,7 +77,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
     val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer)
     val itr = blockFetcherItr.flatMap(unpackBlock)
 
-    CompletionIterator[T, Iterator[T]](itr, {
+    val completionIter = CompletionIterator[T, Iterator[T]](itr, {
       val shuffleMetrics = new ShuffleReadMetrics
       shuffleMetrics.shuffleFinishTime = System.currentTimeMillis
       shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime
@@ -83,7 +86,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
       shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks
       shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks
       shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks
-      metrics.shuffleReadMetrics = Some(shuffleMetrics)
+      context.taskMetrics.shuffleReadMetrics = Some(shuffleMetrics)
     })
+
+    new InterruptibleIterator[T](context, completionIter)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala
index 4cf7eb96da25ee283df9bc97a72a540542293cba..519ecde50a163507c97340c6de3b308cd6277324 100644
--- a/core/src/main/scala/org/apache/spark/CacheManager.scala
+++ b/core/src/main/scala/org/apache/spark/CacheManager.scala
@@ -18,7 +18,7 @@
 package org.apache.spark
 
 import scala.collection.mutable.{ArrayBuffer, HashSet}
-import org.apache.spark.storage.{BlockManager, StorageLevel}
+import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, RDDBlockId}
 import org.apache.spark.rdd.RDD
 
 
@@ -28,17 +28,17 @@ import org.apache.spark.rdd.RDD
 private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
 
   /** Keys of RDD splits that are being computed/loaded. */
-  private val loading = new HashSet[String]()
+  private val loading = new HashSet[RDDBlockId]()
 
   /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */
   def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel)
       : Iterator[T] = {
-    val key = "rdd_%d_%d".format(rdd.id, split.index)
+    val key = RDDBlockId(rdd.id, split.index)
     logDebug("Looking for partition " + key)
     blockManager.get(key) match {
       case Some(values) =>
         // Partition is already materialized, so just return its values
-        return values.asInstanceOf[Iterator[T]]
+        return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
 
       case None =>
         // Mark the split as loading (unless someone else marks it first)
@@ -56,7 +56,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             // downside of the current code is that threads wait serially if this does happen.
             blockManager.get(key) match {
               case Some(values) =>
-                return values.asInstanceOf[Iterator[T]]
+                return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
               case None =>
                 logInfo("Whoever was loading %s failed; we'll try it ourselves".format(key))
                 loading.add(key)
@@ -73,7 +73,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
           if (context.runningLocally) { return computedValues }
           val elements = new ArrayBuffer[Any]
           elements ++= computedValues
-          blockManager.put(key, elements, storageLevel, true)
+          blockManager.put(key, elements, storageLevel, tellMaster = true)
           return elements.iterator.asInstanceOf[Iterator[T]]
         } finally {
           loading.synchronized {
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c6b4ac5192d144db22cee3c6cbba2bd38a6e9acc
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -0,0 +1,250 @@
+/*
+ * 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
+
+import scala.concurrent._
+import scala.concurrent.duration.Duration
+import scala.util.Try
+
+import org.apache.spark.scheduler.{JobSucceeded, JobWaiter}
+import org.apache.spark.scheduler.JobFailed
+import org.apache.spark.rdd.RDD
+
+
+/**
+ * A future for the result of an action. This is an extension of the Scala Future interface to
+ * support cancellation.
+ */
+trait FutureAction[T] extends Future[T] {
+  // Note that we redefine methods of the Future trait here explicitly so we can specify a different
+  // documentation (with reference to the word "action").
+
+  /**
+   * Cancels the execution of this action.
+   */
+  def cancel()
+
+  /**
+   * Blocks until this action completes.
+   * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf
+   *               for unbounded waiting, or a finite positive duration
+   * @return this FutureAction
+   */
+  override def ready(atMost: Duration)(implicit permit: CanAwait): FutureAction.this.type
+
+  /**
+   * Awaits and returns the result (of type T) of this action.
+   * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf
+   *               for unbounded waiting, or a finite positive duration
+   * @throws Exception exception during action execution
+   * @return the result value if the action is completed within the specific maximum wait time
+   */
+  @throws(classOf[Exception])
+  override def result(atMost: Duration)(implicit permit: CanAwait): T
+
+  /**
+   * When this action is completed, either through an exception, or a value, applies the provided
+   * function.
+   */
+  def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext)
+
+  /**
+   * Returns whether the action has already been completed with a value or an exception.
+   */
+  override def isCompleted: Boolean
+
+  /**
+   * The value of this Future.
+   *
+   * If the future is not completed the returned value will be None. If the future is completed
+   * the value will be Some(Success(t)) if it contains a valid result, or Some(Failure(error)) if
+   * it contains an exception.
+   */
+  override def value: Option[Try[T]]
+
+  /**
+   * Blocks and returns the result of this job.
+   */
+  @throws(classOf[Exception])
+  def get(): T = Await.result(this, Duration.Inf)
+}
+
+
+/**
+ * The future holding the result of an action that triggers a single job. Examples include
+ * count, collect, reduce.
+ */
+class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T)
+  extends FutureAction[T] {
+
+  override def cancel() {
+    jobWaiter.cancel()
+  }
+
+  override def ready(atMost: Duration)(implicit permit: CanAwait): SimpleFutureAction.this.type = {
+    if (!atMost.isFinite()) {
+      awaitResult()
+    } else jobWaiter.synchronized {
+      val finishTime = System.currentTimeMillis() + atMost.toMillis
+      while (!isCompleted) {
+        val time = System.currentTimeMillis()
+        if (time >= finishTime) {
+          throw new TimeoutException
+        } else {
+          jobWaiter.wait(finishTime - time)
+        }
+      }
+    }
+    this
+  }
+
+  @throws(classOf[Exception])
+  override def result(atMost: Duration)(implicit permit: CanAwait): T = {
+    ready(atMost)(permit)
+    awaitResult() match {
+      case scala.util.Success(res) => res
+      case scala.util.Failure(e) => throw e
+    }
+  }
+
+  override def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext) {
+    executor.execute(new Runnable {
+      override def run() {
+        func(awaitResult())
+      }
+    })
+  }
+
+  override def isCompleted: Boolean = jobWaiter.jobFinished
+
+  override def value: Option[Try[T]] = {
+    if (jobWaiter.jobFinished) {
+      Some(awaitResult())
+    } else {
+      None
+    }
+  }
+
+  private def awaitResult(): Try[T] = {
+    jobWaiter.awaitResult() match {
+      case JobSucceeded => scala.util.Success(resultFunc)
+      case JobFailed(e: Exception, _) => scala.util.Failure(e)
+    }
+  }
+}
+
+
+/**
+ * A FutureAction for actions that could trigger multiple Spark jobs. Examples include take,
+ * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the
+ * action thread if it is being blocked by a job.
+ */
+class ComplexFutureAction[T] extends FutureAction[T] {
+
+  // Pointer to the thread that is executing the action. It is set when the action is run.
+  @volatile private var thread: Thread = _
+
+  // A flag indicating whether the future has been cancelled. This is used in case the future
+  // is cancelled before the action was even run (and thus we have no thread to interrupt).
+  @volatile private var _cancelled: Boolean = false
+
+  // A promise used to signal the future.
+  private val p = promise[T]()
+
+  override def cancel(): Unit = this.synchronized {
+    _cancelled = true
+    if (thread != null) {
+      thread.interrupt()
+    }
+  }
+
+  /**
+   * Executes some action enclosed in the closure. To properly enable cancellation, the closure
+   * should use runJob implementation in this promise. See takeAsync for example.
+   */
+  def run(func: => T)(implicit executor: ExecutionContext): this.type = {
+    scala.concurrent.future {
+      thread = Thread.currentThread
+      try {
+        p.success(func)
+      } catch {
+        case e: Exception => p.failure(e)
+      } finally {
+        thread = null
+      }
+    }
+    this
+  }
+
+  /**
+   * Runs a Spark job. This is a wrapper around the same functionality provided by SparkContext
+   * to enable cancellation.
+   */
+  def runJob[T, U, R](
+      rdd: RDD[T],
+      processPartition: Iterator[T] => U,
+      partitions: Seq[Int],
+      resultHandler: (Int, U) => Unit,
+      resultFunc: => R) {
+    // If the action hasn't been cancelled yet, submit the job. The check and the submitJob
+    // command need to be in an atomic block.
+    val job = this.synchronized {
+      if (!cancelled) {
+        rdd.context.submitJob(rdd, processPartition, partitions, resultHandler, resultFunc)
+      } else {
+        throw new SparkException("Action has been cancelled")
+      }
+    }
+
+    // Wait for the job to complete. If the action is cancelled (with an interrupt),
+    // cancel the job and stop the execution. This is not in a synchronized block because
+    // Await.ready eventually waits on the monitor in FutureJob.jobWaiter.
+    try {
+      Await.ready(job, Duration.Inf)
+    } catch {
+      case e: InterruptedException =>
+        job.cancel()
+        throw new SparkException("Action has been cancelled")
+    }
+  }
+
+  /**
+   * Returns whether the promise has been cancelled.
+   */
+  def cancelled: Boolean = _cancelled
+
+  @throws(classOf[InterruptedException])
+  @throws(classOf[scala.concurrent.TimeoutException])
+  override def ready(atMost: Duration)(implicit permit: CanAwait): this.type = {
+    p.future.ready(atMost)(permit)
+    this
+  }
+
+  @throws(classOf[Exception])
+  override def result(atMost: Duration)(implicit permit: CanAwait): T = {
+    p.future.result(atMost)(permit)
+  }
+
+  override def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext): Unit = {
+    p.future.onComplete(func)(executor)
+  }
+
+  override def isCompleted: Boolean = p.isCompleted
+
+  override def value: Option[Try[T]] = p.future.value
+}
diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..56e0b8d2c0b9bb48766400e11446c0772b78c8f3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala
@@ -0,0 +1,30 @@
+/*
+ * 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
+
+/**
+ * An iterator that wraps around an existing iterator to provide task killing functionality.
+ * It works by checking the interrupted flag in TaskContext.
+ */
+class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T])
+  extends Iterator[T] {
+
+  def hasNext: Boolean = !context.interrupted && delegate.hasNext
+
+  def next(): T = delegate.next()
+}
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 1afb1870f1617924791426fa32ddb958cb903b06..fbda11f5787ec075a7bbc5b1a4123debd56cb977 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -20,21 +20,16 @@ package org.apache.spark
 import java.io._
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 
-import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
+import scala.concurrent.Await
+import scala.concurrent.duration._
 
 import akka.actor._
-import scala.concurrent.Await
 import akka.pattern.ask
-import akka.remote._
-
-import scala.concurrent.duration.Duration
-import akka.util.Timeout
-import scala.concurrent.duration._
 
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util.{MetadataCleanerType, Utils, MetadataCleaner, TimeStampedHashMap}
 
 
 private[spark] sealed trait MapOutputTrackerMessage
@@ -42,11 +37,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,27 +56,27 @@ private[spark] class MapOutputTracker extends Logging {
   private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
 
   // Set to the MapOutputTrackerActor living on the driver
-  var trackerActor: ActorRef = _
+  var trackerActor: Either[ActorRef, ActorSelection] = _
 
-  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
-
-  // 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]]
+  protected var epoch: Long = 0
+  protected val epochLock = new java.lang.Object
 
-  val metadataCleaner = new MetadataCleaner("MapOutputTracker", 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)
+      val future = trackerActor match {
+        case Left(a: ActorRef) => a.ask(message)(timeout)
+        case Right(b: ActorSelection) => b.ask(message)(timeout)
+      }
+      Await.result(future, timeout)
     } catch {
       case e: Exception =>
         throw new SparkException("Error communicating with MapOutputTracker", e)
@@ -88,50 +84,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]
 
@@ -170,7 +128,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 {
@@ -196,9 +154,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() {
@@ -208,15 +165,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
@@ -230,14 +179,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 incrementEpoch() {
+    epochLock.synchronized {
+      epoch += 1
+      logDebug("Increasing epoch to " + epoch)
     }
   }
 
-  def getSerializedLocations(shuffleId: Int): Array[Byte] = {
+  def getSerializedMapOutputStatuses(shuffleId: Int): Array[Byte] = {
     var statuses: Array[MapStatus] = null
     var epochGotten: Long = -1
     epochLock.synchronized {
@@ -249,13 +246,13 @@ private[spark] class MapOutputTracker extends Logging {
         case Some(bytes) =>
           return bytes
         case None =>
-          statuses = mapStatuses(shuffleId)
+          statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]())
           epochGotten = epoch
       }
     }
     // 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)
+    // out a snapshot of the locations as "statuses"; let's serialize and return that
+    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 {
@@ -263,13 +260,35 @@ 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.
+  }
+
+  def has(shuffleId: Int): Boolean = {
+    cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId)
+  }
+}
+
+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
@@ -280,18 +299,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/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 62b608c0885aca603e7660fd420a1e68c4211bed..bcec41c439afb6c3634450049f4cfef164d67026 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -17,11 +17,11 @@
 
 package org.apache.spark
 
-import org.apache.spark.util.Utils
-import org.apache.spark.rdd.RDD
-
 import scala.reflect.ClassTag
 
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.Utils
+
 /**
  * An object that defines how the elements in a key-value pair RDD are partitioned by key.
  * Maps each key to a partition ID, from 0 to `numPartitions - 1`.
diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
index 307c383a89c72d1af1670b4176551719b226ba2e..a85aa50a9b94bb9fcdbc2b27868cfde20e9c6733 100644
--- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
@@ -27,7 +27,10 @@ private[spark] abstract class ShuffleFetcher {
    * Fetch the shuffle outputs for a given ShuffleDependency.
    * @return An iterator over the elements of the fetched shuffle outputs.
    */
-  def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics,
+  def fetch[T](
+      shuffleId: Int,
+      reduceId: Int,
+      context: TaskContext,
       serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T]
 
   /** Stop the fetcher */
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 1b003cc68541d1de64bbfaf0d635ef55f93465fc..f3ce4c879df29eb0367f1206fafe4b0794b69670 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.Map
 import scala.collection.generic.Growable
-import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 import scala.reflect.{ ClassTag, classTag}
@@ -53,21 +52,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.{ClosureCleaner, Utils, MetadataCleaner, TimeStampedHashMap}
+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
@@ -86,7 +83,7 @@ class SparkContext(
     val sparkHome: String = null,
     val jars: Seq[String] = Nil,
     val environment: Map[String, String] = Map(),
-    // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc)
+    // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc)
     // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set
     // of data-local splits on host
     val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
@@ -121,9 +118,9 @@ class SparkContext(
 
   // Keeps track of all persisted RDDs
   private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
-  private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
+  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()
 
@@ -149,88 +146,16 @@ class SparkContext(
     executorEnvs ++= environment
   }
 
-  // Create and start the scheduler
-  private[spark] var taskScheduler: TaskScheduler = {
-    // Regular expression used for local[N] master format
-    val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
-    // Regular expression for local[N, maxRetries], used in tests with failing tasks
-    val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r
-    // Regular expression for simulating a Spark cluster of [N, cores, memory] locally
-    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
-
-    master match {
-      case "local" =>
-        new LocalScheduler(1, 0, this)
-
-      case LOCAL_N_REGEX(threads) =>
-        new LocalScheduler(threads.toInt, 0, this)
-
-      case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
-        new LocalScheduler(threads.toInt, maxFailures.toInt, this)
-
-      case SPARK_REGEX(sparkUrl) =>
-        val scheduler = new ClusterScheduler(this)
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
-        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
-        if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
-          throw new SparkException(
-            "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
-              memoryPerSlaveInt, SparkContext.executorMemoryRequested))
-        }
-
-        val scheduler = new ClusterScheduler(this)
-        val localCluster = new LocalSparkCluster(
-          numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
-        val sparkUrl = localCluster.start()
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
-        scheduler.initialize(backend)
-        backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
-          localCluster.stop()
-        }
-        scheduler
-
-      case "yarn-standalone" =>
-        val scheduler = try {
-          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
-          val cons = clazz.getConstructor(classOf[SparkContext])
-          cons.newInstance(this).asInstanceOf[ClusterScheduler]
-        } catch {
-          // TODO: Enumerate the exact reasons why it can fail
-          // But irrespective of it, it means we cannot proceed !
-          case th: Throwable => {
-            throw new SparkException("YARN mode not available ?", th)
-          }
-        }
-        val backend = new StandaloneSchedulerBackend(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))
-        }
-        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)
-        } else {
-          new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
-        }
-        scheduler.initialize(backend)
-        scheduler
-    }
+  // Set SPARK_USER for user who is running SparkContext.
+  val sparkUser = Option {
+    Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER"))
+  }.getOrElse {
+    SparkContext.SPARK_UNKNOWN_USER
   }
+  executorEnvs("SPARK_USER") = sparkUser
+
+  // Create and start the scheduler
+  private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName)
   taskScheduler.start()
 
   @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
@@ -241,7 +166,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) {
@@ -251,8 +176,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)
@@ -266,6 +193,12 @@ class SparkContext(
     override protected def childValue(parent: Properties): Properties = new Properties(parent)
   }
 
+  private[spark] def getLocalProperties(): Properties = localProperties.get()
+
+  private[spark] def setLocalProperties(props: Properties) {
+    localProperties.set(props)
+  }
+
   def initLocalProperties() {
     localProperties.set(new Properties())
   }
@@ -285,15 +218,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)
   }
 
+  /**
+   * 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)
@@ -332,7 +296,7 @@ class SparkContext(
   }
 
   /**
-   * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any
+   * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and any
    * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
    * etc).
    */
@@ -344,7 +308,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)
   }
 
@@ -358,24 +322,15 @@ class SparkContext(
       ): RDD[(K, V)] = {
     // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it.
     val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration))
-    hadoopFile(path, confBroadcast, inputFormatClass, keyClass, valueClass, minSplits)
-  }
-
-  /**
-   * Get an RDD for a Hadoop file with an arbitray InputFormat. Accept a Hadoop Configuration
-   * that has already been broadcast, assuming that it's safe to use it to construct a
-   * HadoopFileRDD (i.e., except for file 'path', all other configuration properties can be resued).
-   */
-  def hadoopFile[K, V](
-      path: String,
-      confBroadcast: Broadcast[SerializableWritable[Configuration]],
-      inputFormatClass: Class[_ <: InputFormat[K, V]],
-      keyClass: Class[K],
-      valueClass: Class[V],
-      minSplits: Int
-      ): RDD[(K, V)] = {
-    new HadoopFileRDD(
-      this, path, confBroadcast, inputFormatClass, keyClass, valueClass, minSplits)
+    val setInputPathsFunc = (jobConf: JobConf) => FileInputFormat.setInputPaths(jobConf, path)
+    new HadoopRDD(
+      this,
+      confBroadcast,
+      Some(setInputPathsFunc),
+      inputFormatClass,
+      keyClass,
+      valueClass,
+      minSplits)
   }
 
   /**
@@ -563,7 +518,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
 
@@ -657,12 +613,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("\\")) {
@@ -671,12 +626,27 @@ 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()) {
-              logWarning("local jar specified as parameter to addJar under Yarn mode")
-              return
+            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.
+              val fileName = new Path(uri.getPath).getName()
+              try {
+                env.httpFileServer.addJar(new File(fileName))
+              } catch {
+                case e: Exception => {
+                  logError("Error adding jar (" + e + "), was the --addJars option used?")
+                  throw e
+                }
+              }
+            } else {
+              env.httpFileServer.addJar(new File(uri.getPath))
             }
-            env.httpFileServer.addJar(new File(uri.getPath))
+          // A JAR file which exists locally on every worker node
+          case "local" =>
+            "file:" + uri.getPath
           case _ =>
             path
         }
@@ -750,13 +720,13 @@ class SparkContext(
       allowLocal: Boolean,
       resultHandler: (Int, U) => Unit) {
     val callSite = Utils.formatSparkCallSite
+    val cleanedFunc = clean(func)
     logInfo("Starting job: " + callSite)
     val start = System.nanoTime
-    val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler,
-      localProperties.get)
+    dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
+      resultHandler, localProperties.get)
     logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
     rdd.doCheckpoint()
-    result
   }
 
   /**
@@ -842,6 +812,42 @@ class SparkContext(
     result
   }
 
+  /**
+   * Submit a job for execution and return a FutureJob holding the result.
+   */
+  def submitJob[T, U, R](
+      rdd: RDD[T],
+      processPartition: Iterator[T] => U,
+      partitions: Seq[Int],
+      resultHandler: (Int, U) => Unit,
+      resultFunc: => R): SimpleFutureAction[R] =
+  {
+    val cleanF = clean(processPartition)
+    val callSite = Utils.formatSparkCallSite
+    val waiter = dagScheduler.submitJob(
+      rdd,
+      (context: TaskContext, iter: Iterator[T]) => cleanF(iter),
+      partitions,
+      callSite,
+      allowLocal = false,
+      resultHandler,
+      localProperties.get)
+    new SimpleFutureAction(waiter, resultFunc)
+  }
+
+  /**
+   * 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()
+  }
+
   /**
    * Clean a closure to make it ready to serialized and send to tasks
    * (removes unreferenced variables in $outer's, updates REPL variables)
@@ -859,9 +865,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.")
@@ -898,7 +903,12 @@ 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"
+
+  private[spark] val SPARK_UNKNOWN_USER = "<unknown>"
 
   implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
     def addInPlace(t1: Double, t2: Double): Double = t1 + t2
@@ -925,6 +935,8 @@ object SparkContext {
   implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
     new PairRDDFunctions(rdd)
 
+  implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd)
+
   implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
       rdd: RDD[(K, V)]) =
     new SequenceFileRDDFunctions(rdd)
@@ -1014,6 +1026,124 @@ object SparkContext {
       .map(Utils.memoryStringToMb)
       .getOrElse(512)
   }
+
+  // Creates a task scheduler based on a given master URL. Extracted for testing.
+  private
+  def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = {
+    // Regular expression used for local[N] master format
+    val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
+    // Regular expression for local[N, maxRetries], used in tests with failing tasks
+    val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r
+    // Regular expression for simulating a Spark cluster of [N, cores, memory] locally
+    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 by mesos:// or zk:// url
+    val MESOS_REGEX = """(mesos|zk)://.*""".r
+    // Regular expression for connection to Simr cluster
+    val SIMR_REGEX = """simr://(.*)""".r
+
+    master match {
+      case "local" =>
+        new LocalScheduler(1, 0, sc)
+
+      case LOCAL_N_REGEX(threads) =>
+        new LocalScheduler(threads.toInt, 0, sc)
+
+      case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
+        new LocalScheduler(threads.toInt, maxFailures.toInt, sc)
+
+      case SPARK_REGEX(sparkUrl) =>
+        val scheduler = new ClusterScheduler(sc)
+        val masterUrls = sparkUrl.split(",").map("spark://" + _)
+        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+        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
+        if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
+          throw new SparkException(
+            "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
+              memoryPerSlaveInt, SparkContext.executorMemoryRequested))
+        }
+
+        val scheduler = new ClusterScheduler(sc)
+        val localCluster = new LocalSparkCluster(
+          numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
+        val masterUrls = localCluster.start()
+        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+        scheduler.initialize(backend)
+        backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
+          localCluster.stop()
+        }
+        scheduler
+
+      case "yarn-standalone" =>
+        val scheduler = try {
+          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
+          val cons = clazz.getConstructor(classOf[SparkContext])
+          cons.newInstance(sc).asInstanceOf[ClusterScheduler]
+        } catch {
+          // TODO: Enumerate the exact reasons why it can fail
+          // But irrespective of it, it means we cannot proceed !
+          case th: Throwable => {
+            throw new SparkException("YARN mode not available ?", th)
+          }
+        }
+        val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+        scheduler.initialize(backend)
+        scheduler
+
+      case "yarn-client" =>
+        val scheduler = try {
+          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
+          val cons = clazz.getConstructor(classOf[SparkContext])
+          cons.newInstance(sc).asInstanceOf[ClusterScheduler]
+
+        } catch {
+          case th: Throwable => {
+            throw new SparkException("YARN mode not available ?", th)
+          }
+        }
+
+        val backend = try {
+          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
+          val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext])
+          cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
+        } catch {
+          case th: Throwable => {
+            throw new SparkException("YARN mode not available ?", th)
+          }
+        }
+
+        scheduler.initialize(backend)
+        scheduler
+
+      case mesosUrl @ MESOS_REGEX(_) =>
+        MesosNativeLibrary.load()
+        val scheduler = new ClusterScheduler(sc)
+        val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
+        val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs
+        val backend = if (coarseGrained) {
+          new CoarseMesosSchedulerBackend(scheduler, sc, url, appName)
+        } else {
+          new MesosSchedulerBackend(scheduler, sc, url, appName)
+        }
+        scheduler.initialize(backend)
+        scheduler
+
+      case SIMR_REGEX(simrUrl) =>
+        val scheduler = new ClusterScheduler(sc)
+        val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl)
+        scheduler.initialize(backend)
+        scheduler
+
+      case _ =>
+        throw new SparkException("Could not parse Master URL: '" + master + "'")
+    }
+  }
 }
 
 /**
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index a267407c673f0b54bb74ecb7fe9ea6bb2a76155c..826f5c2d8c6bde5ec81905906931556e216da1b6 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -20,18 +20,18 @@ package org.apache.spark
 import collection.mutable
 import serializer.Serializer
 
-import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
+import akka.actor._
 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() }
@@ -161,17 +152,17 @@ object SparkEnv extends Logging {
     val closureSerializer = serializerManager.get(
       System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"))
 
-    def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
+    def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = {
       if (isDriver) {
         logInfo("Registering " + name)
-        actorSystem.actorOf(Props(newActor), name = name)
+        Left(actorSystem.actorOf(Props(newActor), name = name))
       } else {
         val driverHost: String = System.getProperty("spark.driver.host", "localhost")
         val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
         Utils.checkHost(driverHost, "Expected hostname")
         val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
         logInfo("Connecting to " + name + ": " + url)
-        actorSystem.actorFor(url)
+        Right(actorSystem.actorSelection(url))
       }
     }
 
@@ -188,10 +179,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 c2c358c7ad6062826d44f7003c928d6168b7f793..cae983ed4c652cc936f6df4f704a04cd9b1cd2b6 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -17,21 +17,30 @@
 
 package org.apache.spark
 
-import executor.TaskMetrics
 import scala.collection.mutable.ArrayBuffer
 
+import org.apache.spark.executor.TaskMetrics
+
 class TaskContext(
   val stageId: Int,
-  val splitId: Int,
+  val partitionId: Int,
   val attemptId: Long,
   val runningLocally: Boolean = false,
-  val taskMetrics: TaskMetrics = TaskMetrics.empty()
+  @volatile var interrupted: Boolean = false,
+  private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty()
 ) extends Serializable {
 
-  @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit]
+  @deprecated("use partitionId", "0.8.1")
+  def splitId = partitionId
+
+  // List of callback functions to execute when the task completes.
+  @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit]
 
-  // Add a callback function to be executed on task completion. An example use
-  // is for HadoopRDD to register a callback to close the input stream.
+  /**
+   * Add a callback function to be executed on task completion. An example use
+   * is for HadoopRDD to register a callback to close the input stream.
+   * @param f Callback function.
+   */
   def addOnCompleteCallback(f: () => Unit) {
     onCompleteCallbacks += f
   }
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 8466c2a0044062e9ef260e6d10dff33bfdc52d6d..c1e5e04b31e6035193828c06a9b664f9baf87c65 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -52,4 +52,6 @@ private[spark] case class ExceptionFailure(
  */
 private[spark] case object TaskResultLost extends TaskEndReason
 
+private[spark] case object TaskKilled extends TaskEndReason
+
 private[spark] case class OtherFailure(message: String) extends TaskEndReason
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 f0a1960a1bce6d48899f6fc230a6081e4c47bf7a..da30cf619a1d0ecfabf501faecc0e2b0f0a64738 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
@@ -29,6 +29,8 @@ import org.apache.spark.storage.StorageLevel
 import java.lang.Double
 import org.apache.spark.Partitioner
 
+import scala.collection.JavaConverters._
+
 class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
 
   override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]]
@@ -51,6 +53,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()
 
@@ -83,6 +98,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`.
    *
@@ -161,6 +187,44 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
 
   /** (Experimental) Approximate operation to return the sum within a timeout. */
   def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout)
+
+  /**
+   * Compute a histogram of the data using bucketCount number of buckets evenly
+   *  spaced between the minimum and maximum of the RDD. For example if the min
+   *  value is 0 and the max is 100 and there are two buckets the resulting
+   *  buckets will be [0,50) [50,100]. bucketCount must be at least 1
+   * If the RDD contains infinity, NaN throws an exception
+   * If the elements in RDD do not vary (max == min) always returns a single bucket.
+   */
+  def histogram(bucketCount: Int): Pair[Array[scala.Double], Array[Long]] = {
+    val result = srdd.histogram(bucketCount)
+    (result._1, result._2)
+  }
+
+  /**
+   * Compute a histogram using the provided buckets. The buckets are all open
+   * to the left except for the last which is closed
+   *  e.g. for the array
+   *  [1,10,20,50] the buckets are [1,10) [10,20) [20,50]
+   *  e.g 1<=x<10 , 10<=x<20, 20<=x<50
+   *  And on the input of 1 and 50 we would have a histogram of 1,0,0 
+   * 
+   * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
+   * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
+   * to true.
+   * buckets must be sorted and not contain any duplicates.
+   * buckets array must be at least two elements 
+   * All NaN entries are treated the same. If you have a NaN bucket it must be
+   * the maximum value of the last position and all NaN entries will be counted
+   * in that bucket.
+   */
+  def histogram(buckets: Array[scala.Double]): Array[Long] = {
+    srdd.histogram(buckets, false)
+  }
+
+  def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = {
+    srdd.histogram(buckets.map(_.toDouble), evenBuckets)
+  }
 }
 
 object JavaDoubleRDD {
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 899e17d4fabeca6315f7603da69b8184cd577723..eeea0eddb11b6fb56ca4fb7c20f3c4ea7111a49c 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
@@ -66,6 +66,19 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K
   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)
 
   /**
@@ -95,6 +108,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K
   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.
    */
@@ -599,4 +623,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: ClassTag[K] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val cmv: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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 9968bc8e5f6d9d09eaab6793bc93481f7c88b91e..c47657f5125d0e89bab326994c6eb75c96e03b08 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
@@ -43,9 +43,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)
 
   /**
@@ -75,6 +83,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 ed92d31af59c304e307554b287ec5e658320ba81..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
@@ -27,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 b7c0d78e33357a964d02044a06f7591ce42b5dba..ed8fea97fc20506d085f99fe3f42cb6a3cda0832 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
@@ -23,8 +23,5 @@ import scala.reflect.ClassTag
  * 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() :  ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[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 7a505df4be8cf66b9adb76782aa6c4acb2766059..aae1349c5e17c05771b6ceb4178d40175026b207 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
@@ -23,8 +23,5 @@ import scala.reflect.ClassTag
  * 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() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[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 e97116986f77ac09f2c64fe6f21f9eb67624fa28..537439ef53888e9fc22d5399aa956b44d5fa7d7b 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
@@ -29,10 +29,8 @@ 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 ClassTag<R> returnType() {
-    return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
+    return ClassTag$.MODULE$.apply(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 cf77bb6b738c0a91ccc747b3ca64d0bb87dae1f1..a2d1214fb46929fc819f54e28a6454e6012d44fe 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
@@ -28,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 ClassTag<R> returnType() {
     return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
   }
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
similarity index 50%
rename from core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/Function3.java
index 3cf22851ddb546d11521b996774cb28fb2fc5f1a..fb1deceab5b562b0c0d54b49d79052540dd921ef 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
@@ -15,28 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.spark.rdd
+package org.apache.spark.api.java.function;
 
-import org.apache.spark.{Partition, TaskContext}
-import scala.reflect.ClassTag
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
+import scala.runtime.AbstractFunction2;
 
+import java.io.Serializable;
 
 /**
- * A variant of the MapPartitionsRDD that passes the partition index into the
- * closure. This can be used to generate or collect partition specific
- * information such as the number of tuples in a partition.
+ * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R.
  */
-private[spark]
-class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag](
-    prev: RDD[T],
-    f: (Int, Iterator[T]) => Iterator[U],
-    preservesPartitioning: Boolean
-  ) extends RDD[U](prev) {
+public abstract class Function3<T1, T2, T3, R> extends WrappedFunction3<T1, T2, T3, R>
+        implements Serializable {
 
-  override def getPartitions: Array[Partition] = firstParent[T].partitions
-
-  override val partitioner = if (preservesPartitioning) prev.partitioner else None
-
-  override def compute(split: Partition, context: TaskContext) =
-    f(split.index, firstParent[T].iterator(split, context))
+    public ClassTag<R> returnType() {
+        return (ClassTag<R>) ClassTag$.MODULE$.apply(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 fbd0cdabe06d7d3c3c196e4a1c8b75c19ab8f7b6..ca485b3cc2d9348484d27a88f2e8c43be73959bb 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
@@ -33,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 ClassTag<K> keyType() {
     return (ClassTag<K>) ClassTag$.MODULE$.apply(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 f09559627dabfd64b88e84f20aad5fdefcfedb76..cbe2306026a61fd51b0fc713ec2ca38126a3f972 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
@@ -28,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 ClassTag<K> keyType() {
     return (ClassTag<K>) ClassTag$.MODULE$.apply(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 4d887cf195265a5eeb7182d7b92880a82203925a..2bf7ac256eb92243a69746be876bc1bf4a448060 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
@@ -28,12 +28,11 @@ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.PipedRDD
 import org.apache.spark.util.Utils
 
 private[spark] class PythonRDD[T: ClassTag](
     parent: RDD[T],
-    command: Seq[String],
+    command: Array[Byte],
     envVars: JMap[String, String],
     pythonIncludes: JList[String],
     preservePartitoning: Boolean,
@@ -44,21 +43,10 @@ private[spark] class PythonRDD[T: ClassTag](
 
   val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
 
-  // Similar to Runtime.exec(), if we are given a single string, split it into words
-  // using a standard StringTokenizer (i.e. by spaces)
-  def this(parent: RDD[T], command: String, envVars: JMap[String, String],
-      pythonIncludes: JList[String],
-      preservePartitoning: Boolean, pythonExec: String,
-      broadcastVars: JList[Broadcast[Array[Byte]]],
-      accumulator: Accumulator[JList[Array[Byte]]]) =
-    this(parent, PipedRDD.tokenize(command), envVars, pythonIncludes, preservePartitoning, pythonExec,
-      broadcastVars, accumulator)
-
   override def getPartitions = parent.partitions
 
   override val partitioner = if (preservePartitoning) parent.partitioner else None
 
-
   override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
     val startTime = System.currentTimeMillis
     val env = SparkEnv.get
@@ -71,11 +59,10 @@ private[spark] class PythonRDD[T: ClassTag](
           SparkEnv.set(env)
           val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize)
           val dataOut = new DataOutputStream(stream)
-          val printOut = new PrintWriter(stream)
           // Partition index
           dataOut.writeInt(split.index)
           // sparkFilesDir
-          PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut)
+          dataOut.writeUTF(SparkFiles.getRootDirectory)
           // Broadcast variables
           dataOut.writeInt(broadcastVars.length)
           for (broadcast <- broadcastVars) {
@@ -85,21 +72,16 @@ private[spark] class PythonRDD[T: ClassTag](
           }
           // Python includes (*.zip and *.egg files)
           dataOut.writeInt(pythonIncludes.length)
-          for (f <- pythonIncludes) {
-            PythonRDD.writeAsPickle(f, dataOut)
-          }
+          pythonIncludes.foreach(dataOut.writeUTF)
           dataOut.flush()
-          // Serialized user code
-          for (elem <- command) {
-            printOut.println(elem)
-          }
-          printOut.flush()
+          // Serialized command:
+          dataOut.writeInt(command.length)
+          dataOut.write(command)
           // Data values
           for (elem <- parent.iterator(split, context)) {
-            PythonRDD.writeAsPickle(elem, dataOut)
+            PythonRDD.writeToStream(elem, dataOut)
           }
           dataOut.flush()
-          printOut.flush()
           worker.shutdownOutput()
         } catch {
           case e: IOException =>
@@ -132,7 +114,7 @@ private[spark] class PythonRDD[T: ClassTag](
               val obj = new Array[Byte](length)
               stream.readFully(obj)
               obj
-            case -3 =>
+            case SpecialLengths.TIMING_DATA =>
               // Timing data from worker
               val bootTime = stream.readLong()
               val initTime = stream.readLong()
@@ -143,24 +125,24 @@ private[spark] class PythonRDD[T: ClassTag](
               val total = finishTime - startTime
               logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish))
               read
-            case -2 =>
+            case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
               // Signals that an exception has been thrown in python
               val exLength = stream.readInt()
               val obj = new Array[Byte](exLength)
               stream.readFully(obj)
               throw new PythonException(new String(obj))
-            case -1 =>
+            case SpecialLengths.END_OF_DATA_SECTION =>
               // We've finished the data section of the output, but we can still
-              // read some accumulator updates; let's do that, breaking when we
-              // get a negative length record.
-              var len2 = stream.readInt()
-              while (len2 >= 0) {
-                val update = new Array[Byte](len2)
+              // read some accumulator updates:
+              val numAccumulatorUpdates = stream.readInt()
+              (1 to numAccumulatorUpdates).foreach { _ =>
+                val updateLen = stream.readInt()
+                val update = new Array[Byte](updateLen)
                 stream.readFully(update)
                 accumulator += Collections.singletonList(update)
-                len2 = stream.readInt()
+
               }
-              new Array[Byte](0)
+              Array.empty[Byte]
           }
         } catch {
           case eof: EOFException => {
@@ -197,62 +179,15 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
   val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this)
 }
 
-private[spark] object PythonRDD {
-
-  /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */
-  def stripPickle(arr: Array[Byte]) : Array[Byte] = {
-    arr.slice(2, arr.length - 1)
-  }
+private object SpecialLengths {
+  val END_OF_DATA_SECTION = -1
+  val PYTHON_EXCEPTION_THROWN = -2
+  val TIMING_DATA = -3
+}
 
-  /**
-   * Write strings, pickled Python objects, or pairs of pickled objects to a data output stream.
-   * The data format is a 32-bit integer representing the pickled object's length (in bytes),
-   * followed by the pickled data.
-   *
-   * Pickle module:
-   *
-   *    http://docs.python.org/2/library/pickle.html
-   *
-   * The pickle protocol is documented in the source of the `pickle` and `pickletools` modules:
-   *
-   *    http://hg.python.org/cpython/file/2.6/Lib/pickle.py
-   *    http://hg.python.org/cpython/file/2.6/Lib/pickletools.py
-   *
-   * @param elem the object to write
-   * @param dOut a data output stream
-   */
-  def writeAsPickle(elem: Any, dOut: DataOutputStream) {
-    if (elem.isInstanceOf[Array[Byte]]) {
-      val arr = elem.asInstanceOf[Array[Byte]]
-      dOut.writeInt(arr.length)
-      dOut.write(arr)
-    } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) {
-      val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]
-      val length = t._1.length + t._2.length - 3 - 3 + 4  // stripPickle() removes 3 bytes
-      dOut.writeInt(length)
-      dOut.writeByte(Pickle.PROTO)
-      dOut.writeByte(Pickle.TWO)
-      dOut.write(PythonRDD.stripPickle(t._1))
-      dOut.write(PythonRDD.stripPickle(t._2))
-      dOut.writeByte(Pickle.TUPLE2)
-      dOut.writeByte(Pickle.STOP)
-    } else if (elem.isInstanceOf[String]) {
-      // For uniformity, strings are wrapped into Pickles.
-      val s = elem.asInstanceOf[String].getBytes("UTF-8")
-      val length = 2 + 1 + 4 + s.length + 1
-      dOut.writeInt(length)
-      dOut.writeByte(Pickle.PROTO)
-      dOut.writeByte(Pickle.TWO)
-      dOut.write(Pickle.BINUNICODE)
-      dOut.writeInt(Integer.reverseBytes(s.length))
-      dOut.write(s)
-      dOut.writeByte(Pickle.STOP)
-    } else {
-      throw new SparkException("Unexpected RDD type")
-    }
-  }
+private[spark] object PythonRDD {
 
-  def readRDDFromPickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) :
+  def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int):
   JavaRDD[Array[Byte]] = {
     val file = new DataInputStream(new FileInputStream(filename))
     val objs = new collection.mutable.ArrayBuffer[Array[Byte]]
@@ -270,15 +205,32 @@ private[spark] object PythonRDD {
     JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
   }
 
-  def writeIteratorToPickleFile[T](items: java.util.Iterator[T], filename: String) {
+  def writeToStream(elem: Any, dataOut: DataOutputStream) {
+    elem match {
+      case bytes: Array[Byte] =>
+        dataOut.writeInt(bytes.length)
+        dataOut.write(bytes)
+      case pair: (Array[Byte], Array[Byte]) =>
+        dataOut.writeInt(pair._1.length)
+        dataOut.write(pair._1)
+        dataOut.writeInt(pair._2.length)
+        dataOut.write(pair._2)
+      case str: String =>
+        dataOut.writeUTF(str)
+      case other =>
+        throw new SparkException("Unexpected element type " + other.getClass)
+    }
+  }
+
+  def writeToFile[T](items: java.util.Iterator[T], filename: String) {
     import scala.collection.JavaConverters._
-    writeIteratorToPickleFile(items.asScala, filename)
+    writeToFile(items.asScala, filename)
   }
 
-  def writeIteratorToPickleFile[T](items: Iterator[T], filename: String) {
+  def writeToFile[T](items: Iterator[T], filename: String) {
     val file = new DataOutputStream(new FileOutputStream(filename))
     for (item <- items) {
-      writeAsPickle(item, file)
+      writeToStream(item, file)
     }
     file.close()
   }
@@ -289,17 +241,6 @@ private[spark] object PythonRDD {
   }
 }
 
-private object Pickle {
-  val PROTO: Byte = 0x80.toByte
-  val TWO: Byte = 0x02.toByte
-  val BINUNICODE: Byte = 'X'
-  val STOP: Byte = '.'
-  val TUPLE2: Byte = 0x86.toByte
-  val EMPTY_LIST: Byte = ']'
-  val MARK: Byte = '('
-  val APPENDS: Byte = 'e'
-}
-
 private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] {
   override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8")
 }
@@ -308,7 +249,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 93e7815ab515a29846f5fd939f02560417d0cb82..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.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: String = "broadcast_" + 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/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 9db26ae6de681d902248a13ebb9a8e2247ad33cf..47db720416447700dc34fabcfc272dc921e73675 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -19,22 +19,22 @@ package org.apache.spark.broadcast
 
 import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream}
 import java.net.URL
+import java.util.concurrent.TimeUnit
 
 import it.unimi.dsi.fastutil.io.FastBufferedInputStream
 import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
 import org.apache.spark.{HttpServer, Logging, SparkEnv}
 import org.apache.spark.io.CompressionCodec
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashSet}
-
+import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
+import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils}
 
 private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
   extends Broadcast[T](id) with Logging with Serializable {
   
   def value = value_
 
-  def blockId: String = "broadcast_" + id
+  def blockId = BroadcastBlockId(id)
 
   HttpBroadcast.synchronized {
     SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
@@ -82,7 +82,9 @@ private object HttpBroadcast extends Logging {
   private var server: HttpServer = null
 
   private val files = new TimeStampedHashSet[String]
-  private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup)
+  private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup)
+
+  private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt
 
   private lazy val compressionCodec = CompressionCodec.createCodec()
 
@@ -121,7 +123,7 @@ private object HttpBroadcast extends Logging {
   }
 
   def write(id: Long, value: Any) {
-    val file = new File(broadcastDir, "broadcast-" + id)
+    val file = new File(broadcastDir, BroadcastBlockId(id).name)
     val out: OutputStream = {
       if (compress) {
         compressionCodec.compressedOutputStream(new FileOutputStream(file))
@@ -137,12 +139,15 @@ private object HttpBroadcast extends Logging {
   }
 
   def read[T](id: Long): T = {
-    val url = serverUri + "/broadcast-" + id
+    val url = serverUri + "/" + BroadcastBlockId(id).name
     val in = {
+      val httpConnection = new URL(url).openConnection()
+      httpConnection.setReadTimeout(httpReadTimeout)
+      val inputStream = httpConnection.getInputStream()
       if (compress) {
-        compressionCodec.compressedInputStream(new URL(url).openStream())
+        compressionCodec.compressedInputStream(inputStream)
       } else {
-        new FastBufferedInputStream(new URL(url).openStream(), bufferSize)
+        new FastBufferedInputStream(inputStream, bufferSize)
       }
     }
     val ser = SparkEnv.get.serializer.newInstance()
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 80c97ca073ecc6c944f4861fb34156a5fab199dd..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
+++ /dev/null
@@ -1,603 +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.{Comparator, Random, UUID}
-
-import scala.collection.mutable.{ListBuffer, Map, Set}
-import scala.math
-
-import org.apache.spark._
-import org.apache.spark.storage.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 = "broadcast_" + 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/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 1cfff5e565d173c730e65ff301b26b59d7c31085..275331724afba010988baf082c6364fbcce9b5bf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -21,12 +21,14 @@ import scala.collection.immutable.List
 
 import org.apache.spark.deploy.ExecutorState.ExecutorState
 import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.master.RecoveryState.MasterState
 import org.apache.spark.deploy.worker.ExecutorRunner
 import org.apache.spark.util.Utils
 
 
 private[deploy] sealed trait DeployMessage extends Serializable
 
+/** Contains messages sent between Scheduler actor nodes. */
 private[deploy] object DeployMessages {
 
   // Worker to Master
@@ -52,17 +54,20 @@ private[deploy] object DeployMessages {
       exitStatus: Option[Int])
     extends DeployMessage
 
+  case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription])
+
   case class Heartbeat(workerId: String) extends DeployMessage
 
   // Master to Worker
 
-  case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
+  case class RegisteredWorker(masterUrl: String, masterWebUiUrl: String) extends DeployMessage
 
   case class RegisterWorkerFailed(message: String) extends DeployMessage
 
-  case class KillExecutor(appId: String, execId: Int) extends DeployMessage
+  case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage
 
   case class LaunchExecutor(
+      masterUrl: String,
       appId: String,
       execId: Int,
       appDesc: ApplicationDescription,
@@ -76,9 +81,11 @@ private[deploy] object DeployMessages {
   case class RegisterApplication(appDescription: ApplicationDescription)
     extends DeployMessage
 
+  case class MasterChangeAcknowledged(appId: String)
+
   // Master to Client
 
-  case class RegisteredApplication(appId: String) extends DeployMessage
+  case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage
 
   // TODO(matei): replace hostPort with host
   case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
@@ -94,6 +101,10 @@ private[deploy] object DeployMessages {
 
   case object StopClient
 
+  // Master to Worker & Client
+
+  case class MasterChanged(masterUrl: String, masterWebUiUrl: String)
+
   // MasterWebUI To Master
 
   case object RequestMasterState
@@ -101,7 +112,8 @@ private[deploy] object DeployMessages {
   // Master to MasterWebUI
 
   case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
-    activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+    activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
+    status: MasterState) {
 
     Utils.checkHost(host, "Required hostname")
     assert (port > 0)
@@ -123,12 +135,7 @@ private[deploy] object DeployMessages {
     assert (port > 0)
   }
 
-  // Actor System to Master
-
-  case object CheckForWorkerTimeOut
-
-  case object RequestWebUIPort
-
-  case class WebUIPortResponse(webUIBoundPort: Int)
+  // Actor System to Worker
 
+  case object SendHeartbeat
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2abf0b69dddb329140b622be0278630834ff6092
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.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.deploy
+
+/**
+ * Used to send state on-the-wire about Executors from Worker to Master.
+ * This state is sufficient for the Master to reconstruct its internal data structures during
+ * failover.
+ */
+private[spark] class ExecutorDescription(
+    val appId: String,
+    val execId: Int,
+    val cores: Int,
+    val state: ExecutorState.Value)
+  extends Serializable {
+
+  override def toString: String =
+    "ExecutorState(appId=%s, execId=%d, cores=%d, state=%s)".format(appId, execId, cores, state)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
new file mode 100644
index 0000000000000000000000000000000000000000..0aa8852649e058adf514224fd07ba4b78b7c92e8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -0,0 +1,420 @@
+/*
+ *
+ *  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.deploy
+
+import java.io._
+import java.net.URL
+import java.util.concurrent.TimeoutException
+
+import scala.concurrent.{Await, future, promise}
+import scala.concurrent.duration._
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.collection.mutable.ListBuffer
+import scala.sys.process._
+
+import net.liftweb.json.JsonParser
+
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.deploy.master.RecoveryState
+
+/**
+ * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master.
+ * In order to mimic a real distributed cluster more closely, Docker is used.
+ * Execute using
+ * ./spark-class org.apache.spark.deploy.FaultToleranceTest
+ *
+ * Make sure that that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS:
+ *   - spark.deploy.recoveryMode=ZOOKEEPER
+ *   - spark.deploy.zookeeper.url=172.17.42.1:2181
+ * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port.
+ *
+ * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a
+ * working installation of Docker. In addition to having Docker, the following are assumed:
+ *   - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/)
+ *   - The docker images tagged spark-test-master and spark-test-worker are built from the
+ *     docker/ directory. Run 'docker/spark-test/build' to generate these.
+ */
+private[spark] object FaultToleranceTest extends App with Logging {
+  val masters = ListBuffer[TestMasterInfo]()
+  val workers = ListBuffer[TestWorkerInfo]()
+  var sc: SparkContext = _
+
+  var numPassed = 0
+  var numFailed = 0
+
+  val sparkHome = System.getenv("SPARK_HOME")
+  assertTrue(sparkHome != null, "Run with a valid SPARK_HOME")
+
+  val containerSparkHome = "/opt/spark"
+  val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome)
+
+  System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip
+
+  def afterEach() {
+    if (sc != null) {
+      sc.stop()
+      sc = null
+    }
+    terminateCluster()
+  }
+
+  test("sanity-basic") {
+    addMasters(1)
+    addWorkers(1)
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("sanity-many-masters") {
+    addMasters(3)
+    addWorkers(3)
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("single-master-halt") {
+    addMasters(3)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    delay(30 seconds)
+    assertValidClusterState()
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("single-master-restart") {
+    addMasters(1)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    addMasters(1)
+    delay(30 seconds)
+    assertValidClusterState()
+
+    killLeader()
+    addMasters(1)
+    delay(30 seconds)
+    assertValidClusterState()
+  }
+
+  test("cluster-failure") {
+    addMasters(2)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    terminateCluster()
+    addMasters(2)
+    addWorkers(2)
+    assertValidClusterState()
+  }
+
+  test("all-but-standby-failure") {
+    addMasters(2)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    workers.foreach(_.kill())
+    workers.clear()
+    delay(30 seconds)
+    addWorkers(2)
+    assertValidClusterState()
+  }
+
+  test("rolling-outage") {
+    addMasters(1)
+    delay()
+    addMasters(1)
+    delay()
+    addMasters(1)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+    assertTrue(getLeader == masters.head)
+
+    (1 to 3).foreach { _ =>
+      killLeader()
+      delay(30 seconds)
+      assertValidClusterState()
+      assertTrue(getLeader == masters.head)
+      addMasters(1)
+    }
+  }
+
+  def test(name: String)(fn: => Unit) {
+    try {
+      fn
+      numPassed += 1
+      logInfo("Passed: " + name)
+    } catch {
+      case e: Exception =>
+        numFailed += 1
+        logError("FAILED: " + name, e)
+    }
+    afterEach()
+  }
+
+  def addMasters(num: Int) {
+    (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) }
+  }
+
+  def addWorkers(num: Int) {
+    val masterUrls = getMasterUrls(masters)
+    (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) }
+  }
+
+  /** Creates a SparkContext, which constructs a Client to interact with our cluster. */
+  def createClient() = {
+    if (sc != null) { sc.stop() }
+    // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this
+    // property, we need to reset it.
+    System.setProperty("spark.driver.port", "0")
+    sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
+  }
+
+  def getMasterUrls(masters: Seq[TestMasterInfo]): String = {
+    "spark://" + masters.map(master => master.ip + ":7077").mkString(",")
+  }
+
+  def getLeader: TestMasterInfo = {
+    val leaders = masters.filter(_.state == RecoveryState.ALIVE)
+    assertTrue(leaders.size == 1)
+    leaders(0)
+  }
+
+  def killLeader(): Unit = {
+    masters.foreach(_.readState())
+    val leader = getLeader
+    masters -= leader
+    leader.kill()
+  }
+
+  def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis)
+
+  def terminateCluster() {
+    masters.foreach(_.kill())
+    workers.foreach(_.kill())
+    masters.clear()
+    workers.clear()
+  }
+
+  /** This includes Client retry logic, so it may take a while if the cluster is recovering. */
+  def assertUsable() = {
+    val f = future {
+      try {
+        val res = sc.parallelize(0 until 10).collect()
+        assertTrue(res.toList == (0 until 10))
+        true
+      } catch {
+        case e: Exception =>
+          logError("assertUsable() had exception", e)
+          e.printStackTrace()
+          false
+      }
+    }
+
+    // Avoid waiting indefinitely (e.g., we could register but get no executors).
+    assertTrue(Await.result(f, 120 seconds))
+  }
+
+  /**
+   * Asserts that the cluster is usable and that the expected masters and workers
+   * are all alive in a proper configuration (e.g., only one leader).
+   */
+  def assertValidClusterState() = {
+    assertUsable()
+    var numAlive = 0
+    var numStandby = 0
+    var numLiveApps = 0
+    var liveWorkerIPs: Seq[String] = List()
+
+    def stateValid(): Boolean = {
+      (workers.map(_.ip) -- liveWorkerIPs).isEmpty &&
+        numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1
+    }
+
+    val f = future {
+      try {
+        while (!stateValid()) {
+          Thread.sleep(1000)
+
+          numAlive = 0
+          numStandby = 0
+          numLiveApps = 0
+
+          masters.foreach(_.readState())
+
+          for (master <- masters) {
+            master.state match {
+              case RecoveryState.ALIVE =>
+                numAlive += 1
+                liveWorkerIPs = master.liveWorkerIPs
+              case RecoveryState.STANDBY =>
+                numStandby += 1
+              case _ => // ignore
+            }
+
+            numLiveApps += master.numLiveApps
+          }
+        }
+        true
+      } catch {
+        case e: Exception =>
+          logError("assertValidClusterState() had exception", e)
+          false
+      }
+    }
+
+    try {
+      assertTrue(Await.result(f, 120 seconds))
+    } catch {
+      case e: TimeoutException =>
+        logError("Master states: " + masters.map(_.state))
+        logError("Num apps: " + numLiveApps)
+        logError("IPs expected: " + workers.map(_.ip) + " / found: " + liveWorkerIPs)
+        throw new RuntimeException("Failed to get into acceptable cluster state after 2 min.", e)
+    }
+  }
+
+  def assertTrue(bool: Boolean, message: String = "") {
+    if (!bool) {
+      throw new IllegalStateException("Assertion failed: " + message)
+    }
+  }
+
+  logInfo("Ran %s tests, %s passed and %s failed".format(numPassed+numFailed, numPassed, numFailed))
+}
+
+private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+  extends Logging  {
+
+  implicit val formats = net.liftweb.json.DefaultFormats
+  var state: RecoveryState.Value = _
+  var liveWorkerIPs: List[String] = _
+  var numLiveApps = 0
+
+  logDebug("Created master: " + this)
+
+  def readState() {
+    try {
+      val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream)
+      val json = JsonParser.parse(masterStream, closeAutomatically = true)
+
+      val workers = json \ "workers"
+      val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE")
+      liveWorkerIPs = liveWorkers.map(w => (w \ "host").extract[String])
+
+      numLiveApps = (json \ "activeapps").children.size
+
+      val status = json \\ "status"
+      val stateString = status.extract[String]
+      state = RecoveryState.values.filter(state => state.toString == stateString).head
+    } catch {
+      case e: Exception =>
+        // ignore, no state update
+        logWarning("Exception", e)
+    }
+  }
+
+  def kill() { Docker.kill(dockerId) }
+
+  override def toString: String =
+    "[ip=%s, id=%s, logFile=%s, state=%s]".
+      format(ip, dockerId.id, logFile.getAbsolutePath, state)
+}
+
+private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+  extends Logging {
+
+  implicit val formats = net.liftweb.json.DefaultFormats
+
+  logDebug("Created worker: " + this)
+
+  def kill() { Docker.kill(dockerId) }
+
+  override def toString: String =
+    "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath)
+}
+
+private[spark] object SparkDocker {
+  def startMaster(mountDir: String): TestMasterInfo = {
+    val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir)
+    val (ip, id, outFile) = startNode(cmd)
+    new TestMasterInfo(ip, id, outFile)
+  }
+
+  def startWorker(mountDir: String, masters: String): TestWorkerInfo = {
+    val cmd = Docker.makeRunCmd("spark-test-worker", args = masters, mountDir = mountDir)
+    val (ip, id, outFile) = startNode(cmd)
+    new TestWorkerInfo(ip, id, outFile)
+  }
+
+  private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = {
+    val ipPromise = promise[String]()
+    val outFile = File.createTempFile("fault-tolerance-test", "")
+    outFile.deleteOnExit()
+    val outStream: FileWriter = new FileWriter(outFile)
+    def findIpAndLog(line: String): Unit = {
+      if (line.startsWith("CONTAINER_IP=")) {
+        val ip = line.split("=")(1)
+        ipPromise.success(ip)
+      }
+
+      outStream.write(line + "\n")
+      outStream.flush()
+    }
+
+    dockerCmd.run(ProcessLogger(findIpAndLog _))
+    val ip = Await.result(ipPromise.future, 30 seconds)
+    val dockerId = Docker.getLastProcessId
+    (ip, dockerId, outFile)
+  }
+}
+
+private[spark] class DockerId(val id: String) {
+  override def toString = id
+}
+
+private[spark] object Docker extends Logging {
+  def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = {
+    val mountCmd = if (mountDir != "") { " -v " + mountDir } else ""
+
+    val cmd = "docker run %s %s %s".format(mountCmd, imageTag, args)
+    logDebug("Run command: " + cmd)
+    cmd
+  }
+
+  def kill(dockerId: DockerId) : Unit = {
+    "docker kill %s".format(dockerId.id).!
+  }
+
+  def getLastProcessId: DockerId = {
+    var id: String = null
+    "docker ps -l -q".!(ProcessLogger(line => id = line))
+    new DockerId(id)
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index 04d01c169d3e5d36d303599bf5cb150235546549..e607b8c6f4a361d5590dd11adc3519bed45518fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -72,7 +72,8 @@ private[spark] object JsonProtocol {
     ("memory" -> obj.workers.map(_.memory).sum) ~
     ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~
     ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~
-    ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo))
+    ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~
+    ("status" -> obj.status.toString)
   }
 
   def writeWorkerState(obj: WorkerStateResponse) = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 6a7d5a85babcdd0a1b9f3150cee72ac22a240882..59d12a3e6f842904f17caee6a45d255db2f5a95a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -17,12 +17,12 @@
 
 package org.apache.spark.deploy
 
-import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.actor.ActorSystem
 
 import org.apache.spark.deploy.worker.Worker
 import org.apache.spark.deploy.master.Master
-import org.apache.spark.util.{Utils, AkkaUtils}
-import org.apache.spark.{Logging}
+import org.apache.spark.util.Utils
+import org.apache.spark.Logging
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -39,22 +39,23 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
   private val masterActorSystems = ArrayBuffer[ActorSystem]()
   private val workerActorSystems = ArrayBuffer[ActorSystem]()
 
-  def start(): String = {
+  def start(): Array[String] = {
     logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
     val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0)
     masterActorSystems += masterSystem
     val masterUrl = "spark://" + localHostname + ":" + masterPort
+    val masters = Array(masterUrl)
 
     /* Start the Workers */
     for (workerNum <- 1 to numWorkers) {
       val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker,
-        memoryPerWorker, masterUrl, null, Some(workerNum))
+        memoryPerWorker, masters, null, Some(workerNum))
       workerActorSystems += workerSystem
     }
 
-    return masterUrl
+    return masters
   }
 
   def 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..fc1537f7963c44b4adb8e2e72c8786b92e6aaf70 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,70 @@
 
 package org.apache.spark.deploy
 
-import com.google.common.collect.MapMaker
+import java.security.PrivilegedExceptionAction
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.security.UserGroupInformation
 
+import org.apache.spark.{SparkContext, 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]()
+  val conf = newConfiguration()
+  UserGroupInformation.setConfiguration(conf)
 
-  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop
-  // subsystems
+  def runAsUser(user: String)(func: () => Unit) {
+    // if we are already running as the user intended there is no reason to do the doAs. It 
+    // will actually break secure HDFS access as it doesn't fill in the credentials. Also if
+    // the user is UNKNOWN then we shouldn't be creating a remote unknown user 
+    // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only 
+    // in SparkContext.
+    val currentUser = Option(System.getProperty("user.name")).
+      getOrElse(SparkContext.SPARK_UNKNOWN_USER)
+    if (user != SparkContext.SPARK_UNKNOWN_USER && currentUser != user) {
+      val ugi = UserGroupInformation.createRemoteUser(user)
+      ugi.doAs(new PrivilegedExceptionAction[Unit] {
+        def run: Unit = func()
+      })
+    } else {
+      func()
+    }
+  }
+
+  /**
+   * 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/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
index 164386782ccb99a059fcd5859385e875af1440aa..d0d65ca72ad7c1117ab0d07dd3c85c89a8fb9898 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
@@ -23,12 +23,11 @@ import scala.concurrent.duration._
 import scala.concurrent.Await
 
 import akka.actor._
-import akka.actor.Terminated
 import akka.pattern.AskTimeoutException
 import akka.pattern.ask
 import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkException, Logging}
 import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
@@ -37,41 +36,86 @@ import org.apache.spark.deploy.master.Master
 /**
  * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
  * and a listener for cluster events, and calls back the listener when various events occur.
+ *
+ * @param masterUrls Each url should look like spark://host:port.
  */
 private[spark] class Client(
     actorSystem: ActorSystem,
-    masterUrl: String,
+    masterUrls: Array[String],
     appDescription: ApplicationDescription,
     listener: ClientListener)
   extends Logging {
 
+  val REGISTRATION_TIMEOUT = 20.seconds
+  val REGISTRATION_RETRIES = 3
+
+  var masterAddress: Address = null
   var actor: ActorRef = null
   var appId: String = null
+  var registered = false
+  var activeMasterUrl: String = null
 
   class ClientActor extends Actor with Logging {
-    var master: ActorRef = null
-    var masterAddress: Address = null
+    var master: ActorSelection = null
     var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
+    var alreadyDead = false  // To avoid calling listener.dead() multiple times
 
     override def preStart() {
-      logInfo("Connecting to master " + masterUrl)
+      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
       try {
-        master = context.actorFor(Master.toAkkaUrl(masterUrl))
-        masterAddress = master.path.address
-        master ! RegisterApplication(appDescription)
-        context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
-        context.watch(master)  // Doesn't work with remote actors, but useful for testing
+        registerWithMaster()
       } catch {
         case e: Exception =>
-          logError("Failed to connect to master", e)
+          logWarning("Failed to connect to master", e)
           markDisconnected()
           context.stop(self)
       }
     }
 
+    def tryRegisterAllMasters() {
+      for (masterUrl <- masterUrls) {
+        logInfo("Connecting to master " + masterUrl + "...")
+        val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
+        actor ! RegisterApplication(appDescription)
+      }
+    }
+
+    def registerWithMaster() {
+      tryRegisterAllMasters()
+
+      import context.dispatcher
+      var retries = 0
+      lazy val retryTimer: Cancellable =
+        context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+          retries += 1
+          if (registered) {
+            retryTimer.cancel()
+          } else if (retries >= REGISTRATION_RETRIES) {
+            logError("All masters are unresponsive! Giving up.")
+            markDead()
+          } else {
+            tryRegisterAllMasters()
+          }
+        }
+      retryTimer // start timer
+    }
+
+    def changeMaster(url: String) {
+      activeMasterUrl = url
+      master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+      masterAddress = activeMasterUrl match {
+        case Master.sparkUrlRegex(host, port) =>
+          Address("akka.tcp", Master.systemName, host, port.toInt)
+        case x =>
+          throw new SparkException("Invalid spark URL: " + x)
+      }
+    }
+
     override def receive = {
-      case RegisteredApplication(appId_) =>
+      case RegisteredApplication(appId_, masterUrl) =>
         appId = appId_
+        registered = true
+        changeMaster(masterUrl)
         listener.connected(appId)
 
       case ApplicationRemoved(message) =>
@@ -83,7 +127,7 @@ private[spark] class Client(
         val fullId = appId + "/" + id
         logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores))
         listener.executorAdded(fullId, workerId, hostPort, cores, memory)
-
+ 
       case ExecutorUpdated(id, state, message, exitStatus) =>
         val fullId = appId + "/" + id
         val messageText = message.map(s => " (" + s + ")").getOrElse("")
@@ -92,23 +136,18 @@ private[spark] class Client(
           listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
         }
 
-      case Terminated(actor_) if actor_ == master =>
-        logError("Connection to master failed; stopping client")
-        markDisconnected()
-        context.stop(self)
+      case MasterChanged(masterUrl, masterWebUiUrl) =>
+        logInfo("Master has changed, new master is at " + masterUrl)
+        changeMaster(masterUrl)
+        alreadyDisconnected = false
+        sender ! MasterChangeAcknowledged(appId)
 
       case DisassociatedEvent(_, address, _) if address == masterAddress =>
-        logError("Connection to master failed; stopping client")
+        logWarning(s"Connection to $address failed; waiting for master to reconnect...")
         markDisconnected()
-        context.stop(self)
-
-      case AssociationErrorEvent(_, _, address, _) if address == masterAddress =>
-        logError("Connection to master failed; stopping client")
-        markDisconnected()
-        context.stop(self)
 
       case StopClient =>
-        markDisconnected()
+        markDead()
         sender ! true
         context.stop(self)
     }
@@ -122,6 +161,13 @@ private[spark] class Client(
         alreadyDisconnected = true
       }
     }
+
+    def markDead() {
+      if (!alreadyDead) {
+        listener.dead()
+        alreadyDead = true
+      }
+    }
   }
 
   def start() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
index 4605368c1177f56639356639f3aae9951aaee06e..be7a11bd1553724376d334245bf14a30d6a06a35 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
@@ -27,8 +27,12 @@ package org.apache.spark.deploy.client
 private[spark] trait ClientListener {
   def connected(appId: String): Unit
 
+  /** Disconnection may be a temporary state, as we fail over to a new Master. */
   def disconnected(): Unit
 
+  /** Dead means that we couldn't find any Masters to connect to, and have given up. */
+  def dead(): Unit
+
   def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
 
   def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index d5e9a0e09575844dc9138f1dd43dc507275529ad..5b62d3ba6c0e0ac8bb456ea8a8d39d69f3117165 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -33,6 +33,11 @@ private[spark] object TestClient {
       System.exit(0)
     }
 
+    def dead() {
+      logInfo("Could not connect to master")
+      System.exit(0)
+    }
+
     def executorAdded(id: String, workerId: String, hostPort: String, cores: Int, memory: Int) {}
 
     def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {}
@@ -44,7 +49,7 @@ private[spark] object TestClient {
     val desc = new ApplicationDescription(
       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
     val listener = new TestListener
-    val client = new Client(actorSystem, url, desc, listener)
+    val client = new Client(actorSystem, Array(url), desc, listener)
     client.start()
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index bd5327627a832fc3b28202920f889bbd421d7ec5..5150b7c7dec6ab135ba65b1d7c992901e164f5e0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -29,23 +29,46 @@ private[spark] class ApplicationInfo(
     val submitDate: Date,
     val driver: ActorRef,
     val appUiUrl: String)
-{
-  var state = ApplicationState.WAITING
-  var executors = new mutable.HashMap[Int, ExecutorInfo]
-  var coresGranted = 0
-  var endTime = -1L
-  val appSource = new ApplicationSource(this)
-
-  private var nextExecutorId = 0
-
-  def newExecutorId(): Int = {
-    val id = nextExecutorId
-    nextExecutorId += 1
-    id
+  extends Serializable {
+
+  @transient var state: ApplicationState.Value = _
+  @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _
+  @transient var coresGranted: Int = _
+  @transient var endTime: Long = _
+  @transient var appSource: ApplicationSource = _
+
+  @transient private var nextExecutorId: Int = _
+
+  init()
+
+  private def readObject(in: java.io.ObjectInputStream) : Unit = {
+    in.defaultReadObject()
+    init()
+  }
+
+  private def init() {
+    state = ApplicationState.WAITING
+    executors = new mutable.HashMap[Int, ExecutorInfo]
+    coresGranted = 0
+    endTime = -1L
+    appSource = new ApplicationSource(this)
+    nextExecutorId = 0
+  }
+
+  private def newExecutorId(useID: Option[Int] = None): Int = {
+    useID match {
+      case Some(id) =>
+        nextExecutorId = math.max(nextExecutorId, id + 1)
+        id
+      case None =>
+        val id = nextExecutorId
+        nextExecutorId += 1
+        id
+    }
   }
 
-  def addExecutor(worker: WorkerInfo, cores: Int): ExecutorInfo = {
-    val exec = new ExecutorInfo(newExecutorId(), this, worker, cores, desc.memoryPerSlave)
+  def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = {
+    val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave)
     executors(exec.id) = exec
     coresGranted += cores
     exec
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
index 39ef090ddf110224195999a5d3c8904fbc996f3d..67e6c5d66af0eceea4a47dc0effc73ce736d072f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -17,12 +17,11 @@
 
 package org.apache.spark.deploy.master
 
-private[spark] object ApplicationState
-  extends Enumeration {
+private[spark] object ApplicationState extends Enumeration {
 
   type ApplicationState = Value
 
-  val WAITING, RUNNING, FINISHED, FAILED = Value
+  val WAITING, RUNNING, FINISHED, FAILED, UNKNOWN = Value
 
   val MAX_NUM_RETRY = 10
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
index cf384a985e90ede61afa0eb7267d636af0eada5e..76db61dd619c698089832d788ce8b8a1438866d1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.deploy.master
 
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 
 private[spark] class ExecutorInfo(
     val id: Int,
@@ -28,5 +28,10 @@ private[spark] class ExecutorInfo(
 
   var state = ExecutorState.LAUNCHING
 
+  /** Copy all state (non-val) variables from the given on-the-wire ExecutorDescription. */
+  def copyState(execDesc: ExecutorDescription) {
+    state = execDesc.state
+  }
+
   def fullId: String = application.id + "/" + id
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..043945a211f26948d9ad7a3d99b80951acb5b23a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -0,0 +1,90 @@
+/*
+ * 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.deploy.master
+
+import java.io._
+
+import scala.Serializable
+
+import akka.serialization.Serialization
+import org.apache.spark.Logging
+
+/**
+ * Stores data in a single on-disk directory with one file per application and worker.
+ * Files are deleted when applications and workers are removed.
+ *
+ * @param dir Directory to store files. Created if non-existent (but not recursively).
+ * @param serialization Used to serialize our objects.
+ */
+private[spark] class FileSystemPersistenceEngine(
+    val dir: String,
+    val serialization: Serialization)
+  extends PersistenceEngine with Logging {
+
+  new File(dir).mkdir()
+
+  override def addApplication(app: ApplicationInfo) {
+    val appFile = new File(dir + File.separator + "app_" + app.id)
+    serializeIntoFile(appFile, app)
+  }
+
+  override def removeApplication(app: ApplicationInfo) {
+    new File(dir + File.separator + "app_" + app.id).delete()
+  }
+
+  override def addWorker(worker: WorkerInfo) {
+    val workerFile = new File(dir + File.separator + "worker_" + worker.id)
+    serializeIntoFile(workerFile, worker)
+  }
+
+  override def removeWorker(worker: WorkerInfo) {
+    new File(dir + File.separator + "worker_" + worker.id).delete()
+  }
+
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+    val sortedFiles = new File(dir).listFiles().sortBy(_.getName)
+    val appFiles = sortedFiles.filter(_.getName.startsWith("app_"))
+    val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_"))
+    val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+    (apps, workers)
+  }
+
+  private def serializeIntoFile(file: File, value: AnyRef) {
+    val created = file.createNewFile()
+    if (!created) { throw new IllegalStateException("Could not create file: " + file) }
+
+    val serializer = serialization.findSerializerFor(value)
+    val serialized = serializer.toBinary(value)
+
+    val out = new FileOutputStream(file)
+    out.write(serialized)
+    out.close()
+  }
+
+  def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = {
+    val fileData = new Array[Byte](file.length().asInstanceOf[Int])
+    val dis = new DataInputStream(new FileInputStream(file))
+    dis.readFully(fileData)
+    dis.close()
+
+    val clazz = m.runtimeClass.asInstanceOf[Class[T]]
+    val serializer = serialization.serializerFor(clazz)
+    serializer.fromBinary(fileData).asInstanceOf[T]
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f25a1ad3bf92afbd22539ba552b76bd60382ec78
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.deploy.master
+
+import akka.actor.{Actor, ActorRef}
+
+import org.apache.spark.deploy.master.MasterMessages.ElectedLeader
+
+/**
+ * A LeaderElectionAgent keeps track of whether the current Master is the leader, meaning it
+ * is the only Master serving requests.
+ * In addition to the API provided, the LeaderElectionAgent will use of the following messages
+ * to inform the Master of leader changes:
+ * [[org.apache.spark.deploy.master.MasterMessages.ElectedLeader ElectedLeader]]
+ * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]]
+ */
+private[spark] trait LeaderElectionAgent extends Actor {
+  val masterActor: ActorRef
+}
+
+/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */
+private[spark] class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent {
+  override def preStart() {
+    masterActor ! ElectedLeader
+  }
+
+  override def receive = {
+    case _ =>
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index cb0fe6a8500f8b070d4fbd3fcf7cec48f97e7c9b..c627dd380607e9acab7180e1d0488c98d5c668e0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -17,48 +17,39 @@
 
 package org.apache.spark.deploy.master
 
-import java.util.Date
 import java.text.SimpleDateFormat
+import java.util.concurrent.TimeUnit
+import java.util.Date
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.concurrent.Await
 import scala.concurrent.duration._
+import scala.concurrent.duration.{Duration, FiniteDuration}
 
 import akka.actor._
 import akka.pattern.ask
 import akka.remote._
+import akka.serialization.SerializationExtension
+import akka.util.Timeout
 
 import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.MasterMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{Utils, AkkaUtils}
-import akka.util.Timeout
-import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed
-import org.apache.spark.deploy.DeployMessages.KillExecutor
-import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
-import scala.Some
-import org.apache.spark.deploy.DeployMessages.WebUIPortResponse
-import org.apache.spark.deploy.DeployMessages.LaunchExecutor
-import org.apache.spark.deploy.DeployMessages.RegisteredApplication
-import org.apache.spark.deploy.DeployMessages.RegisterWorker
-import org.apache.spark.deploy.DeployMessages.ExecutorUpdated
-import org.apache.spark.deploy.DeployMessages.MasterStateResponse
-import org.apache.spark.deploy.DeployMessages.ExecutorAdded
-import org.apache.spark.deploy.DeployMessages.RegisterApplication
-import org.apache.spark.deploy.DeployMessages.ApplicationRemoved
-import org.apache.spark.deploy.DeployMessages.Heartbeat
-import org.apache.spark.deploy.DeployMessages.RegisteredWorker
-import akka.actor.Terminated
-
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
+  import context.dispatcher
+
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
   val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
   val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt
   val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt
- 
+  val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
+  val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
+
   var nextAppNumber = 0
   val workers = new HashSet[WorkerInfo]
   val idToWorker = new HashMap[String, WorkerInfo]
@@ -88,52 +79,112 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     if (envVar != null) envVar else host
   }
 
+  val masterUrl = "spark://" + host + ":" + port
+  var masterWebUiUrl: String = _
+
+  var state = RecoveryState.STANDBY
+
+  var persistenceEngine: PersistenceEngine = _
+
+  var leaderElectionAgent: ActorRef = _
+
   // As a temporary workaround before better ways of configuring memory, we allow users to set
   // a flag that will perform round-robin scheduling across the nodes (spreading out each app
   // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
   val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
-    logInfo("Starting Spark master at spark://" + host + ":" + port)
+    logInfo("Starting Spark master at " + masterUrl)
     // Listen for remote client disconnection events, since they don't go through Akka's watch()
     context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     webUi.start()
-    import context.dispatcher
+    masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get
     context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
 
     masterMetricsSystem.registerSource(masterSource)
     masterMetricsSystem.start()
     applicationMetricsSystem.start()
+
+    persistenceEngine = RECOVERY_MODE match {
+      case "ZOOKEEPER" =>
+        logInfo("Persisting recovery state to ZooKeeper")
+        new ZooKeeperPersistenceEngine(SerializationExtension(context.system))
+      case "FILESYSTEM" =>
+        logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
+        new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system))
+      case _ =>
+        new BlackHolePersistenceEngine()
+    }
+
+    leaderElectionAgent = RECOVERY_MODE match {
+        case "ZOOKEEPER" =>
+          context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl))
+        case _ =>
+          context.actorOf(Props(classOf[MonarchyLeaderAgent], self))
+      }
+  }
+
+  override def preRestart(reason: Throwable, message: Option[Any]) {
+    super.preRestart(reason, message) // calls postStop()!
+    logError("Master actor restarted due to exception", reason)
   }
 
   override def postStop() {
     webUi.stop()
     masterMetricsSystem.stop()
     applicationMetricsSystem.stop()
+    persistenceEngine.close()
+    context.stop(leaderElectionAgent)
   }
 
   override def receive = {
-    case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => {
+    case ElectedLeader => {
+      val (storedApps, storedWorkers) = persistenceEngine.readPersistedData()
+      state = if (storedApps.isEmpty && storedWorkers.isEmpty)
+        RecoveryState.ALIVE
+      else
+        RecoveryState.RECOVERING
+      logInfo("I have been elected leader! New state: " + state)
+      if (state == RecoveryState.RECOVERING) {
+        beginRecovery(storedApps, storedWorkers)
+        context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
+      }
+    }
+
+    case RevokedLeadership => {
+      logError("Leadership has been revoked -- master shutting down.")
+      System.exit(0)
+    }
+
+    case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => {
       logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
         host, workerPort, cores, Utils.megabytesToString(memory)))
-      if (idToWorker.contains(id)) {
+      if (state == RecoveryState.STANDBY) {
+        // ignore, don't send response
+      } else if (idToWorker.contains(id)) {
         sender ! RegisterWorkerFailed("Duplicate worker ID")
       } else {
-        addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress)
-        context.watch(sender)  // This doesn't work with remote actors but helps for testing
-        sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get)
+        val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory,
+          sender, workerWebUiPort, publicAddress)
+        registerWorker(worker)
+        persistenceEngine.addWorker(worker)
+        sender ! RegisteredWorker(masterUrl, masterWebUiUrl)
         schedule()
       }
     }
 
     case RegisterApplication(description) => {
-      logInfo("Registering app " + description.name)
-      val app = addApplication(description, sender)
-      logInfo("Registered app " + description.name + " with ID " + app.id)
-      waitingApps += app
-      context.watch(sender)  // This doesn't work with remote actors but helps for testing
-      sender ! RegisteredApplication(app.id)
-      schedule()
+      if (state == RecoveryState.STANDBY) {
+        // ignore, don't send response
+      } else {
+        logInfo("Registering app " + description.name)
+        val app = createApplication(description, sender)
+        registerApplication(app)
+        logInfo("Registered app " + description.name + " with ID " + app.id)
+        persistenceEngine.addApplication(app)
+        sender ! RegisteredApplication(app.id, masterUrl)
+        schedule()
+      }
     }
 
     case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
@@ -173,27 +224,49 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
 
-    case Terminated(actor) => {
-      // The disconnected actor could've been either a worker or an app; remove whichever of
-      // those we have an entry for in the corresponding actor hashmap
-      actorToWorker.get(actor).foreach(removeWorker)
-      actorToApp.get(actor).foreach(finishApplication)
+    case MasterChangeAcknowledged(appId) => {
+      idToApp.get(appId) match {
+        case Some(app) =>
+          logInfo("Application has been re-registered: " + appId)
+          app.state = ApplicationState.WAITING
+        case None =>
+          logWarning("Master change ack from unknown app: " + appId)
+      }
+
+      if (canCompleteRecovery) { completeRecovery() }
     }
 
-    case DisassociatedEvent(_, address, _) => {
-      // The disconnected client could've been either a worker or an app; remove whichever it was
-      addressToWorker.get(address).foreach(removeWorker)
-      addressToApp.get(address).foreach(finishApplication)
+    case WorkerSchedulerStateResponse(workerId, executors) => {
+      idToWorker.get(workerId) match {
+        case Some(worker) =>
+          logInfo("Worker has been re-registered: " + workerId)
+          worker.state = WorkerState.ALIVE
+
+          val validExecutors = executors.filter(exec => idToApp.get(exec.appId).isDefined)
+          for (exec <- validExecutors) {
+            val app = idToApp.get(exec.appId).get
+            val execInfo = app.addExecutor(worker, exec.cores, Some(exec.execId))
+            worker.addExecutor(execInfo)
+            execInfo.copyState(exec)
+          }
+        case None =>
+          logWarning("Scheduler state from unknown worker: " + workerId)
+      }
+
+      if (canCompleteRecovery) { completeRecovery() }
     }
 
-    case AssociationErrorEvent(_, _, address, _) => {
+    case DisassociatedEvent(_, address, _) => {
       // The disconnected client could've been either a worker or an app; remove whichever it was
+      logInfo(s"$address got disassociated, removing it.")
       addressToWorker.get(address).foreach(removeWorker)
       addressToApp.get(address).foreach(finishApplication)
+      if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
     }
 
     case RequestMasterState => {
-      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray)
+      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
+        state)
     }
 
     case CheckForWorkerTimeOut => {
@@ -205,6 +278,50 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
   }
 
+  def canCompleteRecovery =
+    workers.count(_.state == WorkerState.UNKNOWN) == 0 &&
+      apps.count(_.state == ApplicationState.UNKNOWN) == 0
+
+  def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) {
+    for (app <- storedApps) {
+      logInfo("Trying to recover app: " + app.id)
+      try {
+        registerApplication(app)
+        app.state = ApplicationState.UNKNOWN
+        app.driver ! MasterChanged(masterUrl, masterWebUiUrl)
+      } catch {
+        case e: Exception => logInfo("App " + app.id + " had exception on reconnect")
+      }
+    }
+
+    for (worker <- storedWorkers) {
+      logInfo("Trying to recover worker: " + worker.id)
+      try {
+        registerWorker(worker)
+        worker.state = WorkerState.UNKNOWN
+        worker.actor ! MasterChanged(masterUrl, masterWebUiUrl)
+      } catch {
+        case e: Exception => logInfo("Worker " + worker.id + " had exception on reconnect")
+      }
+    }
+  }
+
+  def completeRecovery() {
+    // Ensure "only-once" recovery semantics using a short synchronization period.
+    synchronized {
+      if (state != RecoveryState.RECOVERING) { return }
+      state = RecoveryState.COMPLETING_RECOVERY
+    }
+
+    // Kill off any workers and apps that didn't respond to us.
+    workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker)
+    apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication)
+
+    state = RecoveryState.ALIVE
+    schedule()
+    logInfo("Recovery complete - resuming operations!")
+  }
+
   /**
    * Can an app use the given worker? True if the worker has enough memory and we haven't already
    * launched an executor for the app on it (right now the standalone backend doesn't like having
@@ -219,6 +336,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
    * every time a new app joins or resource availability changes.
    */
   def schedule() {
+    if (state != RecoveryState.ALIVE) { return }
     // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
     // in the queue, then the second app, etc.
     if (spreadOutApps) {
@@ -266,14 +384,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
     logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
     worker.addExecutor(exec)
-    worker.actor ! LaunchExecutor(
+    worker.actor ! LaunchExecutor(masterUrl,
       exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
     exec.application.driver ! ExecutorAdded(
       exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)
   }
 
-  def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
-    publicAddress: String): WorkerInfo = {
+  def registerWorker(worker: WorkerInfo): Unit = {
     // There may be one or more refs to dead workers on this same node (w/ different ID's),
     // remove them.
     workers.filter { w =>
@@ -281,12 +398,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }.foreach { w =>
       workers -= w
     }
-    val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+
+    val workerAddress = worker.actor.path.address
+    if (addressToWorker.contains(workerAddress)) {
+      logInfo("Attempted to re-register worker at same address: " + workerAddress)
+      return
+    }
+
     workers += worker
     idToWorker(worker.id) = worker
-    actorToWorker(sender) = worker
-    addressToWorker(sender.path.address) = worker
-    worker
+    actorToWorker(worker.actor) = worker
+    addressToWorker(workerAddress) = worker
   }
 
   def removeWorker(worker: WorkerInfo) {
@@ -301,25 +423,36 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         exec.id, ExecutorState.LOST, Some("worker lost"), None)
       exec.application.removeExecutor(exec)
     }
+    persistenceEngine.removeWorker(worker)
   }
 
-  def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
+  def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
-    val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+    new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+  }
+
+  def registerApplication(app: ApplicationInfo): Unit = {
+    val appAddress = app.driver.path.address
+    if (addressToWorker.contains(appAddress)) {
+      logInfo("Attempted to re-register application at same address: " + appAddress)
+      return
+    }
+
     applicationMetricsSystem.registerSource(app.appSource)
     apps += app
     idToApp(app.id) = app
-    actorToApp(driver) = app
-    addressToApp(driver.path.address) = app
+    actorToApp(app.driver) = app
+    addressToApp(appAddress) = app
     if (firstApp == None) {
       firstApp = Some(app)
     }
+    // TODO: What is firstApp?? Can we remove it?
     val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray
-    if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) {
+    if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) {
       logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
     }
-    app
+    waitingApps += app
   }
 
   def finishApplication(app: ApplicationInfo) {
@@ -344,13 +477,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       waitingApps -= app
       for (exec <- app.executors.values) {
         exec.worker.removeExecutor(exec)
-        exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
+        exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id)
         exec.state = ExecutorState.KILLED
       }
       app.markFinished(state)
       if (state != ApplicationState.FINISHED) {
         app.driver ! ApplicationRemoved(state.toString)
       }
+      persistenceEngine.removeApplication(app)
       schedule()
     }
   }
@@ -381,9 +515,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 }
 
 private[spark] object Master {
-  private val systemName = "sparkMaster"
+  val systemName = "sparkMaster"
   private val actorName = "Master"
-  private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
+  val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
 
   def main(argStrings: Array[String]) {
     val args = new MasterArguments(argStrings)
@@ -404,8 +538,8 @@ private[spark] object Master {
   def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
     val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName)
-    val timeoutDuration = Duration.create(
-      System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+    val timeoutDuration: FiniteDuration = Duration.create(
+      System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS)
     implicit val timeout = Timeout(timeoutDuration)
     val respFuture = actor ? RequestWebUIPort   // ask pattern
     val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse]
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
new file mode 100644
index 0000000000000000000000000000000000000000..74a9f8cd824fb7bda55d4e97ada511c9aeb821fd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.deploy.master
+
+sealed trait MasterMessages extends Serializable
+
+/** Contains messages seen only by the Master and its associated entities. */
+private[master] object MasterMessages {
+
+  // LeaderElectionAgent to Master
+
+  case object ElectedLeader
+
+  case object RevokedLeadership
+
+  // Actor System to LeaderElectionAgent
+
+  case object CheckLeader
+
+  // Actor System to Master
+
+  case object CheckForWorkerTimeOut
+
+  case class BeginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo])
+
+  case object CompleteRecovery
+
+  case object RequestWebUIPort
+
+  case class WebUIPortResponse(webUIBoundPort: Int)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..94b986caf283518e2e809d5b6742541bc481ead7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.deploy.master
+
+/**
+ * Allows Master to persist any state that is necessary in order to recover from a failure.
+ * The following semantics are required:
+ *   - addApplication and addWorker are called before completing registration of a new app/worker.
+ *   - removeApplication and removeWorker are called at any time.
+ * Given these two requirements, we will have all apps and workers persisted, but
+ * we might not have yet deleted apps or workers that finished (so their liveness must be verified
+ * during recovery).
+ */
+private[spark] trait PersistenceEngine {
+  def addApplication(app: ApplicationInfo)
+
+  def removeApplication(app: ApplicationInfo)
+
+  def addWorker(worker: WorkerInfo)
+
+  def removeWorker(worker: WorkerInfo)
+
+  /**
+   * Returns the persisted data sorted by their respective ids (which implies that they're
+   * sorted by time of creation).
+   */
+  def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo])
+
+  def close() {}
+}
+
+private[spark] class BlackHolePersistenceEngine extends PersistenceEngine {
+  override def addApplication(app: ApplicationInfo) {}
+  override def removeApplication(app: ApplicationInfo) {}
+  override def addWorker(worker: WorkerInfo) {}
+  override def removeWorker(worker: WorkerInfo) {}
+  override def readPersistedData() = (Nil, Nil)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
new file mode 100644
index 0000000000000000000000000000000000000000..256a5a7c28e47ea0cf22edaa84da7262ad2839fc
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.deploy.master
+
+private[spark] object RecoveryState extends Enumeration {
+  type MasterState = Value
+
+  val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
new file mode 100644
index 0000000000000000000000000000000000000000..81e15c534fc31e1e8b8b4914ff9c24a1149014a2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
@@ -0,0 +1,203 @@
+/*
+ * 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.deploy.master
+
+import scala.collection.JavaConversions._
+import scala.concurrent.ops._
+
+import org.apache.spark.Logging
+import org.apache.zookeeper._
+import org.apache.zookeeper.data.Stat
+import org.apache.zookeeper.Watcher.Event.KeeperState
+
+/**
+ * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry
+ * logic. If the ZooKeeper session expires or otherwise dies, a new ZooKeeper session will be
+ * created. If ZooKeeper remains down after several retries, the given
+ * [[org.apache.spark.deploy.master.SparkZooKeeperWatcher SparkZooKeeperWatcher]] will be
+ * informed via zkDown().
+ *
+ * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many
+ * times or a semantic exception is thrown (e.g.., "node already exists").
+ */
+private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging {
+  val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "")
+
+  val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
+  val ZK_TIMEOUT_MILLIS = 30000
+  val RETRY_WAIT_MILLIS = 5000
+  val ZK_CHECK_PERIOD_MILLIS = 10000
+  val MAX_RECONNECT_ATTEMPTS = 3
+
+  private var zk: ZooKeeper = _
+
+  private val watcher = new ZooKeeperWatcher()
+  private var reconnectAttempts = 0
+  private var closed = false
+
+  /** Connect to ZooKeeper to start the session. Must be called before anything else. */
+  def connect() {
+    connectToZooKeeper()
+
+    new Thread() {
+      override def run() = sessionMonitorThread()
+    }.start()
+  }
+
+  def sessionMonitorThread(): Unit = {
+    while (!closed) {
+      Thread.sleep(ZK_CHECK_PERIOD_MILLIS)
+      if (zk.getState != ZooKeeper.States.CONNECTED) {
+        reconnectAttempts += 1
+        val attemptsLeft = MAX_RECONNECT_ATTEMPTS - reconnectAttempts
+        if (attemptsLeft <= 0) {
+          logError("Could not connect to ZooKeeper: system failure")
+          zkWatcher.zkDown()
+          close()
+        } else {
+          logWarning("ZooKeeper connection failed, retrying " + attemptsLeft + " more times...")
+          connectToZooKeeper()
+        }
+      }
+    }
+  }
+
+  def close() {
+    if (!closed && zk != null) { zk.close() }
+    closed = true
+  }
+
+  private def connectToZooKeeper() {
+    if (zk != null) zk.close()
+    zk = new ZooKeeper(ZK_URL, ZK_TIMEOUT_MILLIS, watcher)
+  }
+
+  /**
+   * Attempts to maintain a live ZooKeeper exception despite (very) transient failures.
+   * Mainly useful for handling the natural ZooKeeper session expiration.
+   */
+  private class ZooKeeperWatcher extends Watcher {
+    def process(event: WatchedEvent) {
+      if (closed) { return }
+
+      event.getState match {
+        case KeeperState.SyncConnected =>
+          reconnectAttempts = 0
+          zkWatcher.zkSessionCreated()
+        case KeeperState.Expired =>
+          connectToZooKeeper()
+        case KeeperState.Disconnected =>
+          logWarning("ZooKeeper disconnected, will retry...")
+      }
+    }
+  }
+
+  def create(path: String, bytes: Array[Byte], createMode: CreateMode): String = {
+    retry {
+      zk.create(path, bytes, ZK_ACL, createMode)
+    }
+  }
+
+  def exists(path: String, watcher: Watcher = null): Stat = {
+    retry {
+      zk.exists(path, watcher)
+    }
+  }
+
+  def getChildren(path: String, watcher: Watcher = null): List[String] = {
+    retry {
+      zk.getChildren(path, watcher).toList
+    }
+  }
+
+  def getData(path: String): Array[Byte] = {
+    retry {
+      zk.getData(path, false, null)
+    }
+  }
+
+  def delete(path: String, version: Int = -1): Unit = {
+    retry {
+      zk.delete(path, version)
+    }
+  }
+
+  /**
+   * Creates the given directory (non-recursively) if it doesn't exist.
+   * All znodes are created in PERSISTENT mode with no data.
+   */
+  def mkdir(path: String) {
+    if (exists(path) == null) {
+      try {
+        create(path, "".getBytes, CreateMode.PERSISTENT)
+      } catch {
+        case e: Exception =>
+          // If the exception caused the directory not to be created, bubble it up,
+          // otherwise ignore it.
+          if (exists(path) == null) { throw e }
+      }
+    }
+  }
+
+  /**
+   * Recursively creates all directories up to the given one.
+   * All znodes are created in PERSISTENT mode with no data.
+   */
+  def mkdirRecursive(path: String) {
+    var fullDir = ""
+    for (dentry <- path.split("/").tail) {
+      fullDir += "/" + dentry
+      mkdir(fullDir)
+    }
+  }
+
+  /**
+   * Retries the given function up to 3 times. The assumption is that failure is transient,
+   * UNLESS it is a semantic exception (i.e., trying to get data from a node that doesn't exist),
+   * in which case the exception will be thrown without retries.
+   *
+   * @param fn Block to execute, possibly multiple times.
+   */
+  def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = {
+    try {
+      fn
+    } catch {
+      case e: KeeperException.NoNodeException => throw e
+      case e: KeeperException.NodeExistsException => throw e
+      case e if n > 0 =>
+        logError("ZooKeeper exception, " + n + " more retries...", e)
+        Thread.sleep(RETRY_WAIT_MILLIS)
+        retry(fn, n-1)
+    }
+  }
+}
+
+trait SparkZooKeeperWatcher {
+  /**
+   * Called whenever a ZK session is created --
+   * this will occur when we create our first session as well as each time
+   * the session expires or errors out.
+   */
+  def zkSessionCreated()
+
+  /**
+   * Called if ZK appears to be completely down (i.e., not just a transient error).
+   * We will no longer attempt to reconnect to ZK, and the SparkZooKeeperSession is considered dead.
+   */
+  def zkDown()
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 6219f11f2a2b2c3b7b02bc7a56b7643ca8b1020f..e05f587b58c6437ce869fd26d7021c8dc20338b0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -22,28 +22,44 @@ import scala.collection.mutable
 import org.apache.spark.util.Utils
 
 private[spark] class WorkerInfo(
-  val id: String,
-  val host: String,
-  val port: Int,
-  val cores: Int,
-  val memory: Int,
-  val actor: ActorRef,
-  val webUiPort: Int,
-  val publicAddress: String) {
+    val id: String,
+    val host: String,
+    val port: Int,
+    val cores: Int,
+    val memory: Int,
+    val actor: ActorRef,
+    val webUiPort: Int,
+    val publicAddress: String)
+  extends Serializable {
 
   Utils.checkHost(host, "Expected hostname")
   assert (port > 0)
 
-  var executors = new mutable.HashMap[String, ExecutorInfo]  // fullId => info
-  var state: WorkerState.Value = WorkerState.ALIVE
-  var coresUsed = 0
-  var memoryUsed = 0
+  @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
+  @transient var state: WorkerState.Value = _
+  @transient var coresUsed: Int = _
+  @transient var memoryUsed: Int = _
 
-  var lastHeartbeat = System.currentTimeMillis()
+  @transient var lastHeartbeat: Long = _
+
+  init()
 
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
 
+  private def readObject(in: java.io.ObjectInputStream) : Unit = {
+    in.defaultReadObject()
+    init()
+  }
+
+  private def init() {
+    executors = new mutable.HashMap
+    state = WorkerState.ALIVE
+    coresUsed = 0
+    memoryUsed = 0
+    lastHeartbeat = System.currentTimeMillis()
+  }
+
   def hostPort: String = {
     assert (port > 0)
     host + ":" + port
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
index fb3fe88d92a4fdee9db15b3babe86c40c29bd78e..0b36ef60051fcacca0cfe6bd294ad8655f3cd24f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
@@ -20,5 +20,5 @@ package org.apache.spark.deploy.master
 private[spark] object WorkerState extends Enumeration {
   type WorkerState = Value
 
-  val ALIVE, DEAD, DECOMMISSIONED = Value
+  val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7809013e8383b2b65532d7cb1c1df738ac68c9a8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.deploy.master
+
+import akka.actor.ActorRef
+import org.apache.zookeeper._
+import org.apache.zookeeper.Watcher.Event.EventType
+
+import org.apache.spark.deploy.master.MasterMessages._
+import org.apache.spark.Logging
+
+private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String)
+  extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging  {
+
+  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+
+  private val watcher = new ZooKeeperWatcher()
+  private val zk = new SparkZooKeeperSession(this)
+  private var status = LeadershipStatus.NOT_LEADER
+  private var myLeaderFile: String = _
+  private var leaderUrl: String = _
+
+  override def preStart() {
+    logInfo("Starting ZooKeeper LeaderElection agent")
+    zk.connect()
+  }
+
+  override def zkSessionCreated() {
+    synchronized {
+      zk.mkdirRecursive(WORKING_DIR)
+      myLeaderFile =
+        zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL)
+      self ! CheckLeader
+    }
+  }
+
+  override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) {
+    logError("LeaderElectionAgent failed, waiting " + zk.ZK_TIMEOUT_MILLIS + "...", reason)
+    Thread.sleep(zk.ZK_TIMEOUT_MILLIS)
+    super.preRestart(reason, message)
+  }
+
+  override def zkDown() {
+    logError("ZooKeeper down! LeaderElectionAgent shutting down Master.")
+    System.exit(1)
+  }
+
+  override def postStop() {
+    zk.close()
+  }
+
+  override def receive = {
+    case CheckLeader => checkLeader()
+  }
+
+  private class ZooKeeperWatcher extends Watcher {
+    def process(event: WatchedEvent) {
+      if (event.getType == EventType.NodeDeleted) {
+        logInfo("Leader file disappeared, a master is down!")
+        self ! CheckLeader
+      }
+    }
+  }
+
+  /** Uses ZK leader election. Navigates several ZK potholes along the way. */
+  def checkLeader() {
+    val masters = zk.getChildren(WORKING_DIR).toList
+    val leader = masters.sorted.head
+    val leaderFile = WORKING_DIR + "/" + leader
+
+    // Setup a watch for the current leader.
+    zk.exists(leaderFile, watcher)
+
+    try {
+      leaderUrl = new String(zk.getData(leaderFile))
+    } catch {
+      // A NoNodeException may be thrown if old leader died since the start of this method call.
+      // This is fine -- just check again, since we're guaranteed to see the new values.
+      case e: KeeperException.NoNodeException =>
+        logInfo("Leader disappeared while reading it -- finding next leader")
+        checkLeader()
+        return
+    }
+
+    // Synchronization used to ensure no interleaving between the creation of a new session and the
+    // checking of a leader, which could cause us to delete our real leader file erroneously.
+    synchronized {
+      val isLeader = myLeaderFile == leaderFile
+      if (!isLeader && leaderUrl == masterUrl) {
+        // We found a different master file pointing to this process.
+        // This can happen in the following two cases:
+        // (1) The master process was restarted on the same node.
+        // (2) The ZK server died between creating the node and returning the name of the node.
+        //     For this case, we will end up creating a second file, and MUST explicitly delete the
+        //     first one, since our ZK session is still open.
+        // Note that this deletion will cause a NodeDeleted event to be fired so we check again for
+        // leader changes.
+        assert(leaderFile < myLeaderFile)
+        logWarning("Cleaning up old ZK master election file that points to this master.")
+        zk.delete(leaderFile)
+      } else {
+        updateLeadershipStatus(isLeader)
+      }
+    }
+  }
+
+  def updateLeadershipStatus(isLeader: Boolean) {
+    if (isLeader && status == LeadershipStatus.NOT_LEADER) {
+      status = LeadershipStatus.LEADER
+      masterActor ! ElectedLeader
+    } else if (!isLeader && status == LeadershipStatus.LEADER) {
+      status = LeadershipStatus.NOT_LEADER
+      masterActor ! RevokedLeadership
+    }
+  }
+
+  private object LeadershipStatus extends Enumeration {
+    type LeadershipStatus = Value
+    val LEADER, NOT_LEADER = Value
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..825344b3bb534ca0b6334e2599876d930079f8fd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -0,0 +1,85 @@
+/*
+ * 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.deploy.master
+
+import org.apache.spark.Logging
+import org.apache.zookeeper._
+
+import akka.serialization.Serialization
+
+class ZooKeeperPersistenceEngine(serialization: Serialization)
+  extends PersistenceEngine
+  with SparkZooKeeperWatcher
+  with Logging
+{
+  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+
+  val zk = new SparkZooKeeperSession(this)
+
+  zk.connect()
+
+  override def zkSessionCreated() {
+    zk.mkdirRecursive(WORKING_DIR)
+  }
+
+  override def zkDown() {
+    logError("PersistenceEngine disconnected from ZooKeeper -- ZK looks down.")
+  }
+
+  override def addApplication(app: ApplicationInfo) {
+    serializeIntoFile(WORKING_DIR + "/app_" + app.id, app)
+  }
+
+  override def removeApplication(app: ApplicationInfo) {
+    zk.delete(WORKING_DIR + "/app_" + app.id)
+  }
+
+  override def addWorker(worker: WorkerInfo) {
+    serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker)
+  }
+
+  override def removeWorker(worker: WorkerInfo) {
+    zk.delete(WORKING_DIR + "/worker_" + worker.id)
+  }
+
+  override def close() {
+    zk.close()
+  }
+
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+    val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted
+    val appFiles = sortedFiles.filter(_.startsWith("app_"))
+    val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val workerFiles = sortedFiles.filter(_.startsWith("worker_"))
+    val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+    (apps, workers)
+  }
+
+  private def serializeIntoFile(path: String, value: AnyRef) {
+    val serializer = serialization.findSerializerFor(value)
+    val serialized = serializer.toBinary(value)
+    zk.create(path, serialized, CreateMode.PERSISTENT)
+  }
+
+  def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = {
+    val fileData = zk.getData("/spark/master_status/" + filename)
+    val clazz = m.runtimeClass.asInstanceOf[Class[T]]
+    val serializer = serialization.serializerFor(clazz)
+    serializer.fromBinary(fileData).asInstanceOf[T]
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index e3dc30eefc56f951e69aa186982c0ddfe03573d2..fff9cb60c78498b2643af10a311c63b3b85607bb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -43,7 +43,8 @@ private[spark] class ExecutorRunner(
     val workerId: String,
     val host: String,
     val sparkHome: File,
-    val workDir: File)
+    val workDir: File,
+    var state: ExecutorState.Value)
   extends Logging {
 
   val fullId = appId + "/" + execId
@@ -83,7 +84,8 @@ private[spark] class ExecutorRunner(
         process.destroy()
         process.waitFor()
       }
-      worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None)
+      state = ExecutorState.KILLED
+      worker ! ExecutorStateChanged(appId, execId, state, None, None)
       Runtime.getRuntime.removeShutdownHook(shutdownHook)
     }
   }
@@ -102,7 +104,7 @@ private[spark] class ExecutorRunner(
     // SPARK-698: do not call the run.cmd script, as process.destroy()
     // fails to kill a process tree on Windows
     Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++
-      command.arguments.map(substituteVariables)
+      (command.arguments ++ Seq(appId)).map(substituteVariables)
   }
 
   /**
@@ -180,9 +182,9 @@ private[spark] class ExecutorRunner(
       // long-lived processes only. However, in the future, we might restart the executor a few
       // times on the same machine.
       val exitCode = process.waitFor()
+      state = ExecutorState.FAILED
       val message = "Command exited with code " + exitCode
-      worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message),
-                                    Some(exitCode))
+      worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))
     } catch {
       case interrupted: InterruptedException =>
         logInfo("Runner thread for executor " + fullId + " interrupted")
@@ -192,8 +194,9 @@ private[spark] class ExecutorRunner(
         if (process != null) {
           process.destroy()
         }
+        state = ExecutorState.FAILED
         val message = e.getClass + ": " + e.getMessage
-        worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None)
+        worker ! ExecutorStateChanged(appId, execId, state, Some(message), None)
       }
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 3904b701b2f2848243da8ef5505529f9f8fb068a..87531b6719e1b108ece3586e6c0ffe9d86fe6d2c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -17,34 +17,45 @@
 
 package org.apache.spark.deploy.worker
 
+import java.io.File
 import java.text.SimpleDateFormat
 import java.util.Date
-import java.io.File
 
 import scala.collection.mutable.HashMap
 import scala.concurrent.duration._
 
-import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
-import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent}
+import akka.actor._
+import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent}
 
-import org.apache.spark.Logging
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{Utils, AkkaUtils}
-
-
+import org.apache.spark.deploy.DeployMessages.WorkerStateResponse
+import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed
+import org.apache.spark.deploy.DeployMessages.KillExecutor
+import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
+import org.apache.spark.deploy.DeployMessages.Heartbeat
+import org.apache.spark.deploy.DeployMessages.RegisteredWorker
+import org.apache.spark.deploy.DeployMessages.LaunchExecutor
+import org.apache.spark.deploy.DeployMessages.RegisterWorker
+
+/**
+  * @param masterUrls Each url should look like spark://host:port.
+  */
 private[spark] class Worker(
     host: String,
     port: Int,
     webUiPort: Int,
     cores: Int,
     memory: Int,
-    masterUrl: String,
+    masterUrls: Array[String],
     workDirPath: String = null)
   extends Actor with Logging {
+  import context.dispatcher
 
   Utils.checkHost(host, "Expected hostname")
   assert (port > 0)
@@ -54,8 +65,19 @@ private[spark] class Worker(
   // Send a heartbeat every (heartbeat timeout) / 4 milliseconds
   val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
 
-  var master: ActorRef = null
-  var masterWebUiUrl : String = ""
+  val REGISTRATION_TIMEOUT = 20.seconds
+  val REGISTRATION_RETRIES = 3
+
+  // Index into masterUrls that we're currently trying to register with.
+  var masterIndex = 0
+
+  val masterLock: Object = new Object()
+  var master: ActorSelection = null
+  var masterAddress: Address = null
+  var activeMasterUrl: String = ""
+  var activeMasterWebUiUrl : String = ""
+  @volatile var registered = false
+  @volatile var connected = false
   val workerId = generateWorkerId()
   var sparkHome: File = null
   var workDir: File = null
@@ -95,54 +117,110 @@ private[spark] class Worker(
   }
 
   override def preStart() {
+    assert(!registered)
     logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format(
       host, port, cores, Utils.megabytesToString(memory)))
     sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
     logInfo("Spark home: " + sparkHome)
     createWorkDir()
     webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
-
+    context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     webUi.start()
-    connectToMaster()
+    registerWithMaster()
 
     metricsSystem.registerSource(workerSource)
     metricsSystem.start()
   }
 
-  def connectToMaster() {
-    logInfo("Connecting to master " + masterUrl)
-    master = context.actorFor(Master.toAkkaUrl(masterUrl))
-    master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress)
-    context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
-    context.watch(master) // Doesn't work with remote actors, but useful for testing
+  def changeMaster(url: String, uiUrl: String) {
+    masterLock.synchronized {
+      activeMasterUrl = url
+      activeMasterWebUiUrl = uiUrl
+      master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+      masterAddress = activeMasterUrl match {
+        case Master.sparkUrlRegex(_host, _port) =>
+          Address("akka.tcp", Master.systemName, _host, _port.toInt)
+        case x =>
+          throw new SparkException("Invalid spark URL: " + x)
+      }
+      connected = true
+    }
   }
 
-  import context.dispatcher
+  def tryRegisterAllMasters() {
+    for (masterUrl <- masterUrls) {
+      logInfo("Connecting to master " + masterUrl + "...")
+      val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
+      actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get,
+        publicAddress)
+    }
+  }
+
+  def registerWithMaster() {
+    tryRegisterAllMasters()
+
+    var retries = 0
+    lazy val retryTimer: Cancellable =
+      context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+        retries += 1
+        if (registered) {
+          retryTimer.cancel()
+        } else if (retries >= REGISTRATION_RETRIES) {
+          logError("All masters are unresponsive! Giving up.")
+          System.exit(1)
+        } else {
+          tryRegisterAllMasters()
+        }
+      }
+    retryTimer // start timer
+  }
 
   override def receive = {
-    case RegisteredWorker(url) =>
-      masterWebUiUrl = url
-      logInfo("Successfully registered with master")
-        context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
-        master ! Heartbeat(workerId)
+    case RegisteredWorker(masterUrl, masterWebUiUrl) =>
+      logInfo("Successfully registered with master " + masterUrl)
+      registered = true
+      changeMaster(masterUrl, masterWebUiUrl)
+      context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat)
+
+    case SendHeartbeat =>
+      masterLock.synchronized {
+        if (connected) { master ! Heartbeat(workerId) }
       }
 
+    case MasterChanged(masterUrl, masterWebUiUrl) =>
+      logInfo("Master has changed, new master is at " + masterUrl)
+      changeMaster(masterUrl, masterWebUiUrl)
+
+      val execs = executors.values.
+        map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
+      sender ! WorkerSchedulerStateResponse(workerId, execs.toList)
+
     case RegisterWorkerFailed(message) =>
-      logError("Worker registration failed: " + message)
-      System.exit(1)
-
-    case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
-      logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
-      val manager = new ExecutorRunner(
-        appId, execId, appDesc, cores_, memory_, self, workerId, host, new File(execSparkHome_), workDir)
-      executors(appId + "/" + execId) = manager
-      manager.start()
-      coresUsed += cores_
-      memoryUsed += memory_
-      master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)
+      if (!registered) {
+        logError("Worker registration failed: " + message)
+        System.exit(1)
+      }
+
+    case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
+      if (masterUrl != activeMasterUrl) {
+        logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.")
+      } else {
+        logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
+        val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
+          self, workerId, host, new File(execSparkHome_), workDir, ExecutorState.RUNNING)
+        executors(appId + "/" + execId) = manager
+        manager.start()
+        coresUsed += cores_
+        memoryUsed += memory_
+        masterLock.synchronized {
+          master ! ExecutorStateChanged(appId, execId, manager.state, None, None)
+        }
+      }
 
     case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
-      master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+      masterLock.synchronized {
+        master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+      }
       val fullId = appId + "/" + execId
       if (ExecutorState.isFinished(state)) {
         val executor = executors(fullId)
@@ -155,32 +233,34 @@ private[spark] class Worker(
         memoryUsed -= executor.memory
       }
 
-    case KillExecutor(appId, execId) =>
-      val fullId = appId + "/" + execId
-      executors.get(fullId) match {
-        case Some(executor) =>
-          logInfo("Asked to kill executor " + fullId)
-          executor.kill()
-        case None =>
-          logInfo("Asked to kill unknown executor " + fullId)
+    case KillExecutor(masterUrl, appId, execId) =>
+      if (masterUrl != activeMasterUrl) {
+        logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor " + execId)
+      } else {
+        val fullId = appId + "/" + execId
+        executors.get(fullId) match {
+          case Some(executor) =>
+            logInfo("Asked to kill executor " + fullId)
+            executor.kill()
+          case None =>
+            logInfo("Asked to kill unknown executor " + fullId)
+        }
       }
 
-    case DisassociatedEvent(_, _, _) =>
+    case x: DisassociatedEvent if x.remoteAddress == masterAddress =>
+      logInfo(s"$x Disassociated !")
       masterDisconnected()
 
     case RequestWorkerState => {
       sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
-        finishedExecutors.values.toList, masterUrl, cores, memory,
-        coresUsed, memoryUsed, masterWebUiUrl)
+        finishedExecutors.values.toList, activeMasterUrl, cores, memory,
+        coresUsed, memoryUsed, activeMasterWebUiUrl)
     }
   }
 
   def masterDisconnected() {
-    // TODO: It would be nice to try to reconnect to the master, but just shut down for now.
-    // (Note that if reconnecting we would also need to assign IDs differently.)
-    logError("Connection to master failed! Shutting down.")
-    executors.values.foreach(_.kill())
-    System.exit(1)
+    logError("Connection to master failed! Waiting for master to reconnect...")
+    connected = false
   }
 
   def generateWorkerId(): String = {
@@ -198,17 +278,18 @@ private[spark] object Worker {
   def main(argStrings: Array[String]) {
     val args = new WorkerArguments(argStrings)
     val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
-      args.memory, args.master, args.workDir)
+      args.memory, args.masters, args.workDir)
     actorSystem.awaitTermination()
   }
 
   def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
-    masterUrl: String, workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = {
+    masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+    : (ActorSystem, Int) = {
     // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
     val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
     actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-      masterUrl, workDir), name = "Worker")
+      masterUrls, workDir), name = "Worker")
     (actorSystem, boundPort)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 0ae89a864fd57a48694137e2bd6554c0a16f16fc..3ed528e6b3773cf7bc3bf903d24adc7970f13752 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -29,7 +29,7 @@ private[spark] class WorkerArguments(args: Array[String]) {
   var webUiPort = 8081
   var cores = inferDefaultCores()
   var memory = inferDefaultMemory()
-  var master: String = null
+  var masters: Array[String] = null
   var workDir: String = null
   
   // Check for settings in environment variables 
@@ -86,14 +86,14 @@ private[spark] class WorkerArguments(args: Array[String]) {
       printUsageAndExit(0)
 
     case value :: tail =>
-      if (master != null) {  // Two positional arguments were given
+      if (masters != null) {  // Two positional arguments were given
         printUsageAndExit(1)
       }
-      master = value
+      masters = value.stripPrefix("spark://").split(",").map("spark://" + _)
       parse(tail)
 
     case Nil =>
-      if (master == null) {  // No positional argument was given
+      if (masters == null) {  // No positional argument was given
         printUsageAndExit(1)
       }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 07bc479c8394eaa62fcf31690ac0fd6e2bb4cd5b..6c18a3c245f318df73bfbc92bc0cf314df9ed0be 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -17,23 +17,19 @@
 
 package org.apache.spark.deploy.worker.ui
 
-import akka.actor.ActorRef
-import akka.util.Timeout
+import java.io.File
 
 import scala.concurrent.duration._
 
-import java.io.{FileInputStream, File}
-
+import akka.util.Timeout
 import javax.servlet.http.HttpServletRequest
 
-import org.eclipse.jetty.server.{Handler, Server}
-
+import org.apache.spark.Logging
 import org.apache.spark.deploy.worker.Worker
-import org.apache.spark.{Logging}
-import org.apache.spark.ui.JettyUtils
+import org.apache.spark.ui.{JettyUtils, UIUtils}
 import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
+import org.eclipse.jetty.server.{Handler, Server}
 
 /**
  * Web UI server for the standalone worker.
@@ -108,7 +104,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 
     val logText = <node>{Utils.offsetBytes(path, startByte, endByte)}</node>
 
-    val linkToMaster = <p><a href={worker.masterWebUiUrl}>Back to Master</a></p>
+    val linkToMaster = <p><a href={worker.activeMasterWebUiUrl}>Back to Master</a></p>
 
     val range = <span>Bytes {startByte.toString} - {endByte.toString} of {logLength}</span>
 
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 65%
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 f705a5631a5c3fafd1def44c345e8cd4e1190ee5..debbdd4c444cc0095be31a33859fe47326bd160e 100644
--- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -22,25 +22,12 @@ import java.nio.ByteBuffer
 import akka.actor._
 import akka.remote._
 
-import org.apache.spark.{Logging, SparkEnv}
+import org.apache.spark.Logging
 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}
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask
-import akka.remote.DisassociatedEvent
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask
-import akka.remote.AssociationErrorEvent
-import akka.remote.DisassociatedEvent
-import akka.actor.Terminated
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor
-import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed
-
-
-private[spark] class StandaloneExecutorBackend(
+
+private[spark] class CoarseGrainedExecutorBackend(
     driverUrl: String,
     executorId: String,
     hostPort: String,
@@ -59,7 +46,6 @@ private[spark] class StandaloneExecutorBackend(
     driver = context.actorSelection(driverUrl)
     driver ! RegisterExecutor(executorId, hostPort, cores)
     context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
-   // context.watch(driver) // Doesn't work with remote actors, but useful for testing
   }
 
   override def receive = {
@@ -75,15 +61,28 @@ private[spark] class StandaloneExecutorBackend(
     case LaunchTask(taskDesc) =>
       logInfo("Got assigned task " + taskDesc.taskId)
       if (executor == null) {
-        logError("Received launchTask but executor was null")
+        logError("Received LaunchTask command but executor was null")
         System.exit(1)
       } else {
         executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
       }
 
-    case DisassociatedEvent(_, _, _) =>
-      logError("Driver terminated or disconnected! Shutting down.")
+    case KillTask(taskId, _) =>
+      if (executor == null) {
+        logError("Received KillTask command but executor was null")
+        System.exit(1)
+      } else {
+        executor.killTask(taskId)
+      }
+
+    case x: DisassociatedEvent =>
+      logError(s"Driver $x disassociated! 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) {
@@ -91,27 +90,30 @@ 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)
 
     // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor
     // before getting started with all our system properties, etc
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0,
+      indestructible = true)
     // set it
     val sparkHostPort = hostname + ":" + boundPort
     System.setProperty("spark.hostPort", sparkHostPort)
     actorSystem.actorOf(
-      Props(classOf[StandaloneExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
+      Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
       name = "Executor")
     actorSystem.awaitTermination()
   }
 
   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>]")
+      //the reason we allow the last appid argument is to make it easy to kill rogue executors
+      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 38000632341322c45937ad4254e15e975fc140c7..0b0a60ee607d12a0a8d45021b6036b1438afb01b 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -25,9 +25,10 @@ import java.util.concurrent._
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 
-import org.apache.spark.scheduler._
 import org.apache.spark._
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.scheduler._
+import org.apache.spark.storage.{StorageLevel, TaskResultBlockId}
 import org.apache.spark.util.Utils
 
 /**
@@ -36,7 +37,8 @@ import org.apache.spark.util.Utils
 private[spark] class Executor(
     executorId: String,
     slaveHostname: String,
-    properties: Seq[(String, String)])
+    properties: Seq[(String, String)],
+    isLocal: Boolean = false)
   extends Logging
 {
   // Application dependencies (added through SparkContext) that we've fetched so far on this node.
@@ -73,46 +75,79 @@ 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)
 
   // Initialize Spark environment (using system properties read above)
-  val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false)
-  SparkEnv.set(env)
-  env.metricsSystem.registerSource(executorSource)
+  private val env = {
+    if (!isLocal) {
+      val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0,
+        isDriver = false, isLocal = false)
+      SparkEnv.set(_env)
+      _env.metricsSystem.registerSource(executorSource)
+      _env
+    } else {
+      SparkEnv.get
+    }
+  }
 
-  private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size")
+  // Akka's message frame size. If task result is bigger than this, we use the block manager
+  // to send the result back.
+  private val akkaFrameSize = {
+    env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size")
+  }
 
   // Start worker thread pool
-  val threadPool = new ThreadPoolExecutor(
-    1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable])
+  val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker")
+
+  // Maintains the list of running tasks.
+  private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
+
+  val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER)
 
   def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) {
-    threadPool.execute(new TaskRunner(context, taskId, serializedTask))
+    val tr = new TaskRunner(context, taskId, serializedTask)
+    runningTasks.put(taskId, tr)
+    threadPool.execute(tr)
+  }
+
+  def killTask(taskId: Long) {
+    val tr = runningTasks.get(taskId)
+    if (tr != null) {
+      tr.kill()
+      // We remove the task also in the finally block in TaskRunner.run.
+      // The reason we need to remove it here is because killTask might be called before the task
+      // is even launched, and never reaching that finally block. ConcurrentHashMap's remove is
+      // idempotent.
+      runningTasks.remove(taskId)
+    }
   }
 
   /** Get the Yarn approved local directories. */
@@ -124,56 +159,87 @@ private[spark] class Executor(
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
       .getOrElse(""))
 
-    if (localDirs.isEmpty()) {
+    if (localDirs.isEmpty) {
       throw new Exception("Yarn Local dirs can't be empty")
     }
-    return localDirs
+    localDirs
   }
 
-  class TaskRunner(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
+  class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
     extends Runnable {
 
-    override def run() {
+    @volatile private var killed = false
+    @volatile private var task: Task[Any] = _
+
+    def kill() {
+      logInfo("Executor is trying to kill task " + taskId)
+      killed = true
+      if (task != null) {
+        task.kill()
+      }
+    }
+
+    override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () =>
       val startTime = System.currentTimeMillis()
       SparkEnv.set(env)
       Thread.currentThread.setContextClassLoader(replClassLoader)
       val ser = SparkEnv.get.closureSerializer.newInstance()
       logInfo("Running task ID " + taskId)
-      context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
+      execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
       var attemptedTask: Option[Task[Any]] = None
       var taskStart: Long = 0
-      def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum
-      val startGCTime = getTotalGCTime
+      def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum
+      val startGCTime = gcTime
 
       try {
         SparkEnv.set(env)
         Accumulators.clear()
         val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask)
         updateDependencies(taskFiles, taskJars)
-        val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
+        task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
+
+        // If this task has been killed before we deserialized it, let's quit now. Otherwise,
+        // continue executing the task.
+        if (killed) {
+          logInfo("Executor killed task " + taskId)
+          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+          return
+        }
+
         attemptedTask = Some(task)
-        logInfo("Its epoch is " + task.epoch)
+        logDebug("Task " + taskId +"'s epoch is " + task.epoch)
         env.mapOutputTracker.updateEpoch(task.epoch)
+
+        // Run the actual task and measure its runtime.
         taskStart = System.currentTimeMillis()
         val value = task.run(taskId.toInt)
         val taskFinish = System.currentTimeMillis()
+
+        // If the task has been killed, let's fail it.
+        if (task.killed) {
+          logInfo("Executor killed task " + taskId)
+          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+          return
+        }
+
         for (m <- task.metrics) {
-          m.hostname = Utils.localHostName
+          m.hostname = Utils.localHostName()
           m.executorDeserializeTime = (taskStart - startTime).toInt
           m.executorRunTime = (taskFinish - taskStart).toInt
-          m.jvmGCTime = getTotalGCTime - startGCTime
+          m.jvmGCTime = gcTime - startGCTime
         }
-        //TODO I'd also like to track the time it takes to serialize the task results, but that is huge headache, b/c
-        // we need to serialize the task metrics first.  If TaskMetrics had a custom serialized format, we could
-        // just change the relevants bytes in the byte buffer
+        // TODO I'd also like to track the time it takes to serialize the task results, but that is
+        // huge headache, b/c we need to serialize the task metrics first.  If TaskMetrics had a
+        // custom serialized format, we could just change the relevants bytes in the byte buffer
         val accumUpdates = Accumulators.values
+
         val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null))
         val serializedDirectResult = ser.serialize(directResult)
         logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit)
         val serializedResult = {
           if (serializedDirectResult.limit >= akkaFrameSize - 1024) {
             logInfo("Storing result for " + taskId + " in local BlockManager")
-            val blockId = "taskresult_" + taskId
+            val blockId = TaskResultBlockId(taskId)
             env.blockManager.putBytes(
               blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER)
             ser.serialize(new IndirectTaskResult[Any](blockId))
@@ -182,12 +248,13 @@ private[spark] class Executor(
             serializedDirectResult
           }
         }
-        context.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
+
+        execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
         logInfo("Finished task ID " + taskId)
       } catch {
         case ffe: FetchFailedException => {
           val reason = ffe.toTaskEndReason
-          context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
+          execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
         }
 
         case t: Throwable => {
@@ -195,10 +262,10 @@ private[spark] class Executor(
           val metrics = attemptedTask.flatMap(t => t.metrics)
           for (m <- metrics) {
             m.executorRunTime = serviceTime
-            m.jvmGCTime = getTotalGCTime - startGCTime
+            m.jvmGCTime = gcTime - startGCTime
           }
           val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics)
-          context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
+          execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
 
           // TODO: Should we exit the whole executor here? On the one hand, the failed task may
           // have left some weird state around depending on when the exception was thrown, but on
@@ -206,6 +273,8 @@ private[spark] class Executor(
           logError("Exception in task ID " + taskId, t)
           //System.exit(1)
         }
+      } finally {
+        runningTasks.remove(taskId)
       }
     }
   }
@@ -215,7 +284,7 @@ private[spark] class Executor(
    * created by the interpreter to the search path
    */
   private def createClassLoader(): ExecutorURLClassLoader = {
-    var loader = this.getClass.getClassLoader
+    val loader = this.getClass.getClassLoader
 
     // For each of the jars in the jarSet, add them to the class loader.
     // We assume each of the files has already been fetched.
@@ -237,7 +306,7 @@ private[spark] class Executor(
         val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
           .asInstanceOf[Class[_ <: ClassLoader]]
         val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader])
-        return constructor.newInstance(classUri, parent)
+        constructor.newInstance(classUri, parent)
       } catch {
         case _: ClassNotFoundException =>
           logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!")
@@ -245,7 +314,7 @@ private[spark] class Executor(
           null
       }
     } else {
-      return parent
+      parent
     }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index 34ed9c8f73c01cdb5f8c5cd14e32b1df6b010b35..97176e4f5b727c78fbc1a72cdf949df1c8fe8290 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -20,8 +20,6 @@ package org.apache.spark.executor
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
 import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.hdfs.DistributedFileSystem
-import org.apache.hadoop.fs.LocalFileSystem
 
 import scala.collection.JavaConversions._
 
diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
index da620919801beb5d3f4773defd513d6dda94a362..b56d8c99124dfbc7ea7c136464c5c2f0832bf735 100644
--- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
@@ -18,14 +18,18 @@
 package org.apache.spark.executor
 
 import java.nio.ByteBuffer
-import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver}
-import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _}
-import org.apache.spark.TaskState.TaskState
+
 import com.google.protobuf.ByteString
-import org.apache.spark.{Logging}
+
+import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver}
+import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
+
+import org.apache.spark.Logging
 import org.apache.spark.TaskState
+import org.apache.spark.TaskState.TaskState
 import org.apache.spark.util.Utils
 
+
 private[spark] class MesosExecutorBackend
   extends MesosExecutor
   with ExecutorBackend
@@ -71,7 +75,11 @@ private[spark] class MesosExecutorBackend
   }
 
   override def killTask(d: ExecutorDriver, t: TaskID) {
-    logWarning("Mesos asked us to kill task " + t.getValue + "; ignoring (not yet implemented)")
+    if (executor == null) {
+      logError("Received KillTask but executor was null")
+    } else {
+      executor.killTask(t.getValue.toLong)
+    }
   }
 
   override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {}
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..c0ce46e379344ef24d9ee0cfc538cb4fdd6b9062 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -61,45 +61,53 @@ object TaskMetrics {
 
 class ShuffleReadMetrics extends Serializable {
   /**
-   * Time when shuffle finishs
+   * Absolute time when this task finished reading shuffle data
    */
   var shuffleFinishTime: Long = _
 
   /**
-   * Total number of blocks fetched in a shuffle (remote or local)
+   * Number of blocks fetched in this shuffle by this task (remote or local)
    */
   var totalBlocksFetched: Int = _
 
   /**
-   * Number of remote blocks fetched in a shuffle
+   * Number of remote blocks fetched in this shuffle by this task
    */
   var remoteBlocksFetched: Int = _
 
   /**
-   * Local blocks fetched in a shuffle
+   * Number of local blocks fetched in this shuffle by this task
    */
   var localBlocksFetched: Int = _
 
   /**
-   * Total time that is spent blocked waiting for shuffle to fetch data
+   * Time the task spent waiting for remote shuffle blocks. This only includes the time
+   * blocking on shuffle input data. For instance if block B is being fetched while the task is
+   * still not finished processing block A, it is not considered to be blocking on block B.
    */
   var fetchWaitTime: Long = _
 
   /**
-   * The total amount of time for all the shuffle fetches.  This adds up time from overlapping
-   *     shuffles, so can be longer than task time
+   * Total time spent fetching remote shuffle blocks. This aggregates the time spent fetching all
+   * input blocks. Since block fetches are both pipelined and parallelized, this can
+   * exceed fetchWaitTime and executorRunTime.
    */
   var remoteFetchTime: Long = _
 
   /**
-   * Total number of remote bytes read from a shuffle
+   * Total number of remote bytes read from the shuffle by this task
    */
   var remoteBytesRead: Long = _
 }
 
 class ShuffleWriteMetrics extends Serializable {
   /**
-   * Number of bytes written for a shuffle
+   * Number of bytes written for the shuffle by this task
    */
   var shuffleBytesWritten: Long = _
+
+  /**
+   * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds
+   */
+  var shuffleWriteTime: Long = _
 }
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cdcfec8ca785bcafc3574164654f699d12da4768
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.metrics.sink
+
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+import java.net.InetSocketAddress
+
+import com.codahale.metrics.MetricRegistry
+import com.codahale.metrics.graphite.{GraphiteReporter, Graphite}
+
+import org.apache.spark.metrics.MetricsSystem
+
+class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink {
+  val GRAPHITE_DEFAULT_PERIOD = 10
+  val GRAPHITE_DEFAULT_UNIT = "SECONDS"
+  val GRAPHITE_DEFAULT_PREFIX = ""
+
+  val GRAPHITE_KEY_HOST = "host"
+  val GRAPHITE_KEY_PORT = "port"
+  val GRAPHITE_KEY_PERIOD = "period"
+  val GRAPHITE_KEY_UNIT = "unit"
+  val GRAPHITE_KEY_PREFIX = "prefix"
+
+  def propertyToOption(prop: String) = Option(property.getProperty(prop))
+
+  if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) {
+    throw new Exception("Graphite sink requires 'host' property.")
+  }
+
+  if (!propertyToOption(GRAPHITE_KEY_PORT).isDefined) {
+    throw new Exception("Graphite sink requires 'port' property.")
+  }
+
+  val host = propertyToOption(GRAPHITE_KEY_HOST).get
+  val port = propertyToOption(GRAPHITE_KEY_PORT).get.toInt
+
+  val pollPeriod = propertyToOption(GRAPHITE_KEY_PERIOD) match {
+    case Some(s) => s.toInt
+    case None => GRAPHITE_DEFAULT_PERIOD
+  }
+
+  val pollUnit = propertyToOption(GRAPHITE_KEY_UNIT) match {
+    case Some(s) => TimeUnit.valueOf(s.toUpperCase())
+    case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT)
+  }
+
+  val prefix = propertyToOption(GRAPHITE_KEY_PREFIX).getOrElse(GRAPHITE_DEFAULT_PREFIX)
+
+  MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
+
+  val graphite: Graphite = new Graphite(new InetSocketAddress(host, port))
+
+  val reporter: GraphiteReporter = GraphiteReporter.forRegistry(registry)
+      .convertDurationsTo(TimeUnit.MILLISECONDS)
+      .convertRatesTo(TimeUnit.SECONDS)
+      .prefixedWith(prefix)
+      .build(graphite)
+
+  override def start() {
+    reporter.start(pollPeriod, pollUnit)
+  }
+
+  override def stop() {
+    reporter.stop()
+  }
+}
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 c24fd48c046748a50e1a42bab02efbc0caeb839d..703bc6a9ca9fa4a539644c6350fefc775808478a 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/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
index 3c29700920a341cd69fed842f12322098092989b..1b9fa1e53a9f75932884ee78357eb5561f781902 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
@@ -20,17 +20,18 @@ package org.apache.spark.network.netty
 import io.netty.buffer._
 
 import org.apache.spark.Logging
+import org.apache.spark.storage.{TestBlockId, BlockId}
 
 private[spark] class FileHeader (
   val fileLen: Int,
-  val blockId: String) extends Logging {
+  val blockId: BlockId) extends Logging {
 
   lazy val buffer = {
     val buf = Unpooled.buffer()
     buf.capacity(FileHeader.HEADER_SIZE)
     buf.writeInt(fileLen)
-    buf.writeInt(blockId.length)
-    blockId.foreach((x: Char) => buf.writeByte(x))
+    buf.writeInt(blockId.name.length)
+    blockId.name.foreach((x: Char) => buf.writeByte(x))
     //padding the rest of header
     if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) {
       buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes)
@@ -57,18 +58,15 @@ private[spark] object FileHeader {
     for (i <- 1 to idLength) {
       idBuilder += buf.readByte().asInstanceOf[Char]
     }
-    val blockId = idBuilder.toString()
+    val blockId = BlockId(idBuilder.toString())
     new FileHeader(length, blockId)
   }
 
-
-  def main (args:Array[String]){
-
-    val header = new FileHeader(25,"block_0");
-    val buf = header.buffer;
-    val newheader = FileHeader.create(buf);
-    System.out.println("id="+newheader.blockId+",size="+newheader.fileLen)
-
+  def main (args:Array[String]) {
+    val header = new FileHeader(25, TestBlockId("my_block"))
+    val buf = header.buffer
+    val newHeader = FileHeader.create(buf)
+    System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen)
   }
 }
 
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
index 9493ccffd99e52b230326bb29d5d24b8139f29fb..b1e1576dadc1a88c9b6de8949b0d41255c2a4055 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
@@ -27,12 +27,13 @@ import org.apache.spark.Logging
 import org.apache.spark.network.ConnectionManagerId
 
 import scala.collection.JavaConverters._
+import org.apache.spark.storage.BlockId
 
 
 private[spark] class ShuffleCopier extends Logging {
 
-  def getBlock(host: String, port: Int, blockId: String,
-      resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+  def getBlock(host: String, port: Int, blockId: BlockId,
+      resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
 
     val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
     val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt
@@ -41,7 +42,7 @@ private[spark] class ShuffleCopier extends Logging {
     try {
       fc.init()
       fc.connect(host, port)
-      fc.sendRequest(blockId)
+      fc.sendRequest(blockId.name)
       fc.waitForClose()
       fc.close()
     } catch {
@@ -53,14 +54,14 @@ private[spark] class ShuffleCopier extends Logging {
     }
   }
 
-  def getBlock(cmId: ConnectionManagerId, blockId: String,
-      resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+  def getBlock(cmId: ConnectionManagerId, blockId: BlockId,
+      resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
     getBlock(cmId.host, cmId.port, blockId, resultCollectCallback)
   }
 
   def getBlocks(cmId: ConnectionManagerId,
-    blocks: Seq[(String, Long)],
-    resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+    blocks: Seq[(BlockId, Long)],
+    resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
 
     for ((blockId, size) <- blocks) {
       getBlock(cmId, blockId, resultCollectCallback)
@@ -71,22 +72,22 @@ private[spark] class ShuffleCopier extends Logging {
 
 private[spark] object ShuffleCopier extends Logging {
 
-  private class ShuffleClientHandler(resultCollectCallBack: (String, Long, ByteBuf) => Unit)
+  private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit)
     extends FileClientHandler with Logging {
 
     override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) {
-      logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)");
+      logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)")
       resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen))
     }
 
-    override def handleError(blockId: String) {
+    override def handleError(blockId: BlockId) {
       if (!isComplete) {
         resultCollectCallBack(blockId, -1, null)
       }
     }
   }
 
-  def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) {
+  def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) {
     if (size != -1) {
       logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"")
     }
@@ -99,7 +100,7 @@ private[spark] object ShuffleCopier extends Logging {
     }
     val host = args(0)
     val port = args(1).toInt
-    val file = args(2)
+    val blockId = BlockId(args(2))
     val threads = if (args.length > 3) args(3).toInt else 10
 
     val copiers = Executors.newFixedThreadPool(80)
@@ -107,12 +108,12 @@ private[spark] object ShuffleCopier extends Logging {
       Executors.callable(new Runnable() {
         def run() {
           val copier = new ShuffleCopier()
-          copier.getBlock(host, port, file, echoResultCollectCallBack)
+          copier.getBlock(host, port, blockId, echoResultCollectCallBack)
         }
       })
     }).asJava
     copiers.invokeAll(tasks)
-    copiers.shutdown
+    copiers.shutdown()
     System.exit(0)
   }
 }
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 8afcbe190a31cddd66d75f0a9fae901e38f3018b..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,6 +21,7 @@ import java.io.File
 
 import org.apache.spark.Logging
 import org.apache.spark.util.Utils
+import org.apache.spark.storage.{BlockId, FileSegment}
 
 
 private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging {
@@ -53,8 +54,8 @@ private[spark] object ShuffleSender {
     val localDirs = args.drop(2).map(new File(_))
 
     val pResovler = new PathResolver {
-      override def getAbsolutePath(blockId: String): String = {
-        if (!blockId.startsWith("shuffle_")) {
+      override def getBlockLocation(blockId: BlockId): FileSegment = {
+        if (!blockId.isShuffle) {
           throw new Exception("Block " + blockId + " is not a shuffle block")
         }
         // Figure out which local directory it hashes to, and which subdirectory in that
@@ -62,8 +63,8 @@ private[spark] object ShuffleSender {
         val dirId = hash % localDirs.length
         val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
         val subDir = new File(localDirs(dirId), "%02x".format(subDirId))
-        val file = new File(subDir, blockId)
-        return file.getAbsolutePath
+        val file = new File(subDir, blockId.name)
+        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/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
new file mode 100644
index 0000000000000000000000000000000000000000..d1c74a50635102a69e09f084282932ecb77cf059
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -0,0 +1,123 @@
+/*
+ * 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.rdd
+
+import java.util.concurrent.atomic.AtomicLong
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.reflect.ClassTag
+
+import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
+
+/**
+ * A set of asynchronous RDD actions available through an implicit conversion.
+ * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
+ */
+class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging {
+
+  /**
+   * Returns a future for counting the number of elements in the RDD.
+   */
+  def countAsync(): FutureAction[Long] = {
+    val totalCount = new AtomicLong
+    self.context.submitJob(
+      self,
+      (iter: Iterator[T]) => {
+        var result = 0L
+        while (iter.hasNext) {
+          result += 1L
+          iter.next()
+        }
+        result
+      },
+      Range(0, self.partitions.size),
+      (index: Int, data: Long) => totalCount.addAndGet(data),
+      totalCount.get())
+  }
+
+  /**
+   * Returns a future for retrieving all elements of this RDD.
+   */
+  def collectAsync(): FutureAction[Seq[T]] = {
+    val results = new Array[Array[T]](self.partitions.size)
+    self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size),
+      (index, data) => results(index) = data, results.flatten.toSeq)
+  }
+
+  /**
+   * Returns a future for retrieving the first num elements of the RDD.
+   */
+  def takeAsync(num: Int): FutureAction[Seq[T]] = {
+    val f = new ComplexFutureAction[Seq[T]]
+
+    f.run {
+      val results = new ArrayBuffer[T](num)
+      val totalParts = self.partitions.length
+      var partsScanned = 0
+      while (results.size < num && partsScanned < totalParts) {
+        // The number of partitions to try in this iteration. It is ok for this number to be
+        // greater than totalParts because we actually cap it at totalParts in runJob.
+        var numPartsToTry = 1
+        if (partsScanned > 0) {
+          // If we didn't find any rows after the first iteration, just try all partitions next.
+          // Otherwise, interpolate the number of partitions we need to try, but overestimate it
+          // by 50%.
+          if (results.size == 0) {
+            numPartsToTry = totalParts - 1
+          } else {
+            numPartsToTry = (1.5 * num * partsScanned / results.size).toInt
+          }
+        }
+        numPartsToTry = math.max(0, numPartsToTry)  // guard against negative num of partitions
+
+        val left = num - results.size
+        val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
+
+        val buf = new Array[Array[T]](p.size)
+        f.runJob(self,
+          (it: Iterator[T]) => it.take(left).toArray,
+          p,
+          (index: Int, data: Array[T]) => buf(index) = data,
+          Unit)
+
+        buf.foreach(results ++= _.take(num - results.size))
+        partsScanned += numPartsToTry
+      }
+      results.toSeq
+    }
+
+    f
+  }
+
+  /**
+   * Applies a function f to all elements of this RDD.
+   */
+  def foreachAsync(f: T => Unit): FutureAction[Unit] = {
+    self.context.submitJob[T, Unit, Unit](self, _.foreach(f), Range(0, self.partitions.size),
+      (index, data) => Unit, Unit)
+  }
+
+  /**
+   * Applies a function f to each partition of this RDD.
+   */
+  def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = {
+    self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size),
+      (index, data) => Unit, Unit)
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index fe2946bcbe12019a91b4607c188529da18274cd7..424354ae165a021d15243eaa89ac157d060a6f83 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -17,16 +17,17 @@
 
 package org.apache.spark.rdd
 
-import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
-import org.apache.spark.storage.BlockManager
 import scala.reflect.ClassTag
 
-private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
+import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
+import org.apache.spark.storage.{BlockId, BlockManager}
+
+private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition {
   val index = idx
 }
 
 private[spark]
-class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String])
+class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId])
   extends RDD[T](sc, Nil) {
 
   @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 4fb7f3aace3a0dd22d0c0e420bc76a33280bbb68..d2a3d60965484bee75ce7cc3838bd22b7aa54c43 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -71,7 +71,7 @@ class CartesianRDD[T: ClassTag, U:ClassTag](
   override def compute(split: Partition, context: TaskContext) = {
     val currSplit = split.asInstanceOf[CartesianPartition]
     for (x <- rdd1.iterator(currSplit.s1, context);
-      y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
+         y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
   }
 
   override def getDependencies: Seq[Dependency[_]] = List(
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 3f4d4ad46a0452cd13f4513ef435805bb5a93f9b..a712ef1c27093b5c8e6b53fd0efa5083f0170886 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -17,15 +17,13 @@
 
 package org.apache.spark.rdd
 
+import java.io.IOException
+
 import scala.reflect.ClassTag
-import org.apache.spark._
-import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.{NullWritable, BytesWritable}
-import org.apache.hadoop.util.ReflectionUtils
+
 import org.apache.hadoop.fs.Path
-import java.io.{File, IOException, EOFException}
-import java.text.NumberFormat
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
 
 private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
 
@@ -84,9 +82,9 @@ 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.splitId)
+    val finalOutputName = splitIdToFile(ctx.partitionId)
     val finalOutputPath = new Path(outputDir, finalOutputName)
     val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId)
 
@@ -123,7 +121,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()
@@ -146,7 +144,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/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 0187256a8e3268ebed4b44a783cb939e39801506..911a002884d65db61efb9ebc040ec6324e66ee3e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -18,13 +18,12 @@
 package org.apache.spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}
-import java.util.{HashMap => JHashMap}
 
-import scala.collection.JavaConversions
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.spark.{Partition, Partitioner, SparkEnv, TaskContext}
+import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext}
 import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency}
+import org.apache.spark.util.AppendOnlyMap
 
 
 private[spark] sealed trait CoGroupSplitDep extends Serializable
@@ -105,17 +104,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
     val split = s.asInstanceOf[CoGroupPartition]
     val numRdds = split.deps.size
     // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
-    val map = new JHashMap[K, Seq[ArrayBuffer[Any]]]
+    val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]]
 
-    def getSeq(k: K): Seq[ArrayBuffer[Any]] = {
-      val seq = map.get(k)
-      if (seq != null) {
-        seq
-      } else {
-        val seq = Array.fill(numRdds)(new ArrayBuffer[Any])
-        map.put(k, seq)
-        seq
-      }
+    val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => {
+      if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any])
+    }
+
+    val getSeq = (k: K) => {
+      map.changeValue(k, update)
     }
 
     val ser = SparkEnv.get.serializerManager.get(serializerClass)
@@ -129,12 +125,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
       case ShuffleCoGroupSplitDep(shuffleId) => {
         // Read map outputs of shuffle
         val fetcher = SparkEnv.get.shuffleFetcher
-        fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context.taskMetrics, ser).foreach {
+        fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach {
           kv => getSeq(kv._1)(depNum) += kv._2
         }
       }
     }
-    JavaConversions.mapAsScalaMap(map).iterator
+    new InterruptibleIterator(context, map.iterator)
   }
 
   override def clearDependencies() {
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index a4bec417529fc9022b9da540993bc235cae67b81..688c310ee9caf37f117c728b3b9a27adb778ef7f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -24,6 +24,8 @@ import org.apache.spark.partial.SumEvaluator
 import org.apache.spark.util.StatCounter
 import org.apache.spark.{TaskContext, Logging}
 
+import scala.collection.immutable.NumericRange
+
 /**
  * Extra functions available on RDDs of Doubles through an implicit conversion.
  * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
@@ -76,4 +78,129 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
     val evaluator = new SumEvaluator(self.partitions.size, confidence)
     self.context.runApproximateJob(self, processPartition, evaluator, timeout)
   }
+
+  /**
+   * Compute a histogram of the data using bucketCount number of buckets evenly
+   *  spaced between the minimum and maximum of the RDD. For example if the min
+   *  value is 0 and the max is 100 and there are two buckets the resulting
+   *  buckets will be [0, 50) [50, 100]. bucketCount must be at least 1
+   * If the RDD contains infinity, NaN throws an exception
+   * If the elements in RDD do not vary (max == min) always returns a single bucket.
+   */
+  def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = {
+    // Compute the minimum and the maxium
+    val (max: Double, min: Double) = self.mapPartitions { items =>
+      Iterator(items.foldRight(Double.NegativeInfinity,
+        Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) =>
+        (x._1.max(e), x._2.min(e))))
+    }.reduce { (maxmin1, maxmin2) =>
+      (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2))
+    }
+    if (min.isNaN || max.isNaN || max.isInfinity || min.isInfinity ) {
+      throw new UnsupportedOperationException(
+        "Histogram on either an empty RDD or RDD containing +/-infinity or NaN")
+    }
+    val increment = (max-min)/bucketCount.toDouble
+    val range = if (increment != 0) {
+      Range.Double.inclusive(min, max, increment)
+    } else {
+      List(min, min)
+    }
+    val buckets = range.toArray
+    (buckets, histogram(buckets, true))
+  }
+
+  /**
+   * Compute a histogram using the provided buckets. The buckets are all open
+   * to the left except for the last which is closed
+   *  e.g. for the array
+   *  [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50]
+   *  e.g 1<=x<10 , 10<=x<20, 20<=x<50
+   *  And on the input of 1 and 50 we would have a histogram of 1, 0, 0 
+   * 
+   * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
+   * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
+   * to true.
+   * buckets must be sorted and not contain any duplicates.
+   * buckets array must be at least two elements 
+   * All NaN entries are treated the same. If you have a NaN bucket it must be
+   * the maximum value of the last position and all NaN entries will be counted
+   * in that bucket.
+   */
+  def histogram(buckets: Array[Double], evenBuckets: Boolean = false): Array[Long] = {
+    if (buckets.length < 2) {
+      throw new IllegalArgumentException("buckets array must have at least two elements")
+    }
+    // The histogramPartition function computes the partail histogram for a given
+    // partition. The provided bucketFunction determines which bucket in the array
+    // to increment or returns None if there is no bucket. This is done so we can
+    // specialize for uniformly distributed buckets and save the O(log n) binary
+    // search cost.
+    def histogramPartition(bucketFunction: (Double) => Option[Int])(iter: Iterator[Double]):
+        Iterator[Array[Long]] = {
+      val counters = new Array[Long](buckets.length - 1)
+      while (iter.hasNext) {
+        bucketFunction(iter.next()) match {
+          case Some(x: Int) => {counters(x) += 1}
+          case _ => {}
+        }
+      }
+      Iterator(counters)
+    }
+    // Merge the counters.
+    def mergeCounters(a1: Array[Long], a2: Array[Long]): Array[Long] = {
+      a1.indices.foreach(i => a1(i) += a2(i))
+      a1
+    }
+    // Basic bucket function. This works using Java's built in Array
+    // binary search. Takes log(size(buckets))
+    def basicBucketFunction(e: Double): Option[Int] = {
+      val location = java.util.Arrays.binarySearch(buckets, e)
+      if (location < 0) {
+        // If the location is less than 0 then the insertion point in the array
+        // to keep it sorted is -location-1
+        val insertionPoint = -location-1
+        // If we have to insert before the first element or after the last one
+        // its out of bounds.
+        // We do this rather than buckets.lengthCompare(insertionPoint)
+        // because Array[Double] fails to override it (for now).
+        if (insertionPoint > 0 && insertionPoint < buckets.length) {
+          Some(insertionPoint-1)
+        } else {
+          None
+        }
+      } else if (location < buckets.length - 1) {
+        // Exact match, just insert here
+        Some(location)
+      } else {
+        // Exact match to the last element
+        Some(location - 1)
+      }
+    }
+    // Determine the bucket function in constant time. Requires that buckets are evenly spaced
+    def fastBucketFunction(min: Double, increment: Double, count: Int)(e: Double): Option[Int] = {
+      // If our input is not a number unless the increment is also NaN then we fail fast
+      if (e.isNaN()) {
+        return None
+      }
+      val bucketNumber = (e - min)/(increment)
+      // We do this rather than buckets.lengthCompare(bucketNumber)
+      // because Array[Double] fails to override it (for now).
+      if (bucketNumber > count || bucketNumber < 0) {
+        None
+      } else {
+        Some(bucketNumber.toInt.min(count - 1))
+      }
+    }
+    // Decide which bucket function to pass to histogramPartition. We decide here
+    // rather than having a general function so that the decission need only be made
+    // once rather than once per shard
+    val bucketFunction = if (evenBuckets) {
+      fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _
+    } else {
+      basicBucketFunction _
+    }
+    self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters)
+  }
+
 }
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 d3b3fffd409d17dfda16aca529596b064b9b1058..53f77a38f55f685e215c5c70b467a17db4a1116c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -27,54 +27,19 @@ import org.apache.hadoop.mapred.RecordReader
 import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
-import org.apache.spark.{Logging, Partition, SerializableWritable, SparkContext, SparkEnv,
-  TaskContext}
+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}
 
-/**
- * An RDD that reads a file (or multiple files) from Hadoop (e.g. files in HDFS, the local file
- * system, or S3).
- * This accepts a general, broadcasted Hadoop Configuration because those tend to remain the same
- * across multiple reads; the 'path' is the only variable that is different across new JobConfs
- * created from the Configuration.
- */
-class HadoopFileRDD[K, V](
-    sc: SparkContext,
-    path: String,
-    broadcastedConf: Broadcast[SerializableWritable[Configuration]],
-    inputFormatClass: Class[_ <: InputFormat[K, V]],
-    keyClass: Class[K],
-    valueClass: Class[V],
-    minSplits: Int)
-  extends HadoopRDD[K, V](sc, broadcastedConf, inputFormatClass, keyClass, valueClass, minSplits) {
-
-  override def getJobConf(): JobConf = {
-    if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) {
-      // getJobConf() has been called previously, so there is already a local cache of the JobConf
-      // needed by this RDD.
-      return HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf]
-    } else {
-      // Create a new JobConf, set the input file/directory paths to read from, and cache the
-      // JobConf (i.e., in a shared hash map in the slave's JVM process that's accessible through
-      // HadoopRDD.putCachedMetadata()), so that we only create one copy across multiple
-      // getJobConf() calls for this RDD in the local process.
-      // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects.
-      val newJobConf = new JobConf(broadcastedConf.value.value)
-      FileInputFormat.setInputPaths(newJobConf, path)
-      HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
-      return newJobConf
-    }
-  }
-}
 
 /**
  * A Spark split class that wraps around a Hadoop InputSplit.
  */
 private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSplit)
   extends Partition {
-  
+
   val inputSplit = new SerializableWritable[InputSplit](s)
 
   override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
@@ -83,11 +48,24 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
 }
 
 /**
- * A base class that provides core functionality for reading data partitions stored in Hadoop.
+ * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
+ * sources in HBase, or S3).
+ *
+ * @param sc The SparkContext to associate the RDD with.
+ * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed
+ *     variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job.
+ *     Otherwise, a new JobConf will be created on each slave using the enclosed Configuration.
+ * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD
+ *     creates.
+ * @param inputFormatClass Storage format of the data to be read.
+ * @param keyClass Class of the key associated with the inputFormatClass.
+ * @param valueClass Class of the value associated with the inputFormatClass.
+ * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate.
  */
 class HadoopRDD[K, V](
     sc: SparkContext,
     broadcastedConf: Broadcast[SerializableWritable[Configuration]],
+    initLocalJobConfFuncOpt: Option[JobConf => Unit],
     inputFormatClass: Class[_ <: InputFormat[K, V]],
     keyClass: Class[K],
     valueClass: Class[V],
@@ -105,6 +83,7 @@ class HadoopRDD[K, V](
       sc,
       sc.broadcast(new SerializableWritable(conf))
         .asInstanceOf[Broadcast[SerializableWritable[Configuration]]],
+      None /* initLocalJobConfFuncOpt */,
       inputFormatClass,
       keyClass,
       valueClass,
@@ -130,6 +109,7 @@ class HadoopRDD[K, V](
       // local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
       // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects.
       val newJobConf = new JobConf(broadcastedConf.value.value)
+      initLocalJobConfFuncOpt.map(f => f(newJobConf))
       HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
       return newJobConf
     }
@@ -152,6 +132,8 @@ class HadoopRDD[K, V](
 
   override def getPartitions: Array[Partition] = {
     val jobConf = getJobConf()
+    // add the credentials here as this can be called before SparkContext initialized
+    SparkHadoopUtil.get.addCredentials(jobConf)
     val inputFormat = getInputFormat(jobConf)
     if (inputFormat.isInstanceOf[Configurable]) {
       inputFormat.asInstanceOf[Configurable].setConf(jobConf)
@@ -164,38 +146,41 @@ class HadoopRDD[K, V](
     array
   }
 
-  override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] {
-    val split = theSplit.asInstanceOf[HadoopPartition]
-    logInfo("Input split: " + split.inputSplit)
-    var reader: RecordReader[K, V] = null
-
-    val jobConf = getJobConf()
-    val inputFormat = getInputFormat(jobConf)
-    reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
-
-    // Register an on-task-completion callback to close the input stream.
-    context.addOnCompleteCallback{ () => closeIfNeeded() }
-
-    val key: K = reader.createKey()
-    val value: V = reader.createValue()
-
-    override def getNext() = {
-      try {
-        finished = !reader.next(key, value)
-      } catch {
-        case eof: EOFException =>
-          finished = true
+  override def compute(theSplit: Partition, context: TaskContext) = {
+    val iter = new NextIterator[(K, V)] {
+      val split = theSplit.asInstanceOf[HadoopPartition]
+      logInfo("Input split: " + split.inputSplit)
+      var reader: RecordReader[K, V] = null
+
+      val jobConf = getJobConf()
+      val inputFormat = getInputFormat(jobConf)
+      reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
+
+      // Register an on-task-completion callback to close the input stream.
+      context.addOnCompleteCallback{ () => closeIfNeeded() }
+
+      val key: K = reader.createKey()
+      val value: V = reader.createValue()
+
+      override def getNext() = {
+        try {
+          finished = !reader.next(key, value)
+        } catch {
+          case eof: EOFException =>
+            finished = true
+        }
+        (key, value)
       }
-      (key, value)
-    }
 
-    override def close() {
-      try {
-        reader.close()
-      } catch {
-        case e: Exception => logWarning("Exception in RecordReader.close()", e)
+      override def close() {
+        try {
+          reader.close()
+        } catch {
+          case e: Exception => logWarning("Exception in RecordReader.close()", e)
+        }
       }
     }
+    new InterruptibleIterator[(K, V)](context, iter)
   }
 
   override def getPreferredLocations(split: Partition): Seq[String] = {
@@ -216,10 +201,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/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index e72f86fb13d253b2dbbabd0df460aa7279739e73..8df8718f3b65b75f81fcb448b56c8c669768d93c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -20,6 +20,7 @@ package org.apache.spark.rdd
 import java.sql.{Connection, ResultSet}
 
 import scala.reflect.ClassTag
+
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.util.NextIterator
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index cdb5946b49366da715d71287c8c67c9c02799703..db15baf503ad61e1253e988f1b22dd5084a3b094 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -20,19 +20,16 @@ package org.apache.spark.rdd
 import org.apache.spark.{Partition, TaskContext}
 import scala.reflect.ClassTag
 
-
-private[spark]
-class MapPartitionsRDD[U: ClassTag, T: ClassTag](
+private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
     prev: RDD[T],
-    f: Iterator[T] => Iterator[U],
+    f: (TaskContext, Int, Iterator[T]) => Iterator[U],  // (TaskContext, partition index, iterator)
     preservesPartitioning: Boolean = false)
   extends RDD[U](prev) {
 
-  override val partitioner =
-    if (preservesPartitioning) firstParent[T].partitioner else None
+  override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None
 
   override def getPartitions: Array[Partition] = firstParent[T].partitions
 
   override def compute(split: Partition, context: TaskContext) =
-    f(firstParent[T].iterator(split, context))
+    f(context, split.index, firstParent[T].iterator(split, context))
 }
diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
index eb3b19907d77469e91fdf89ddd23a1700185a5cd..8d7c2885936656ee03ed0d47571991a12881edc6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
@@ -17,9 +17,10 @@
 
 package org.apache.spark.rdd
 
-import org.apache.spark.{Partition, TaskContext}
 import scala.reflect.ClassTag
 
+import org.apache.spark.{Partition, TaskContext}
+
 private[spark]
 class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U)
   extends RDD[U](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 7b3a89f7e006a56a8f4e7b3bc7183f5d87379d82..2662d48c84a2eb60b83ae495633226cb17a7446f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce._
 
-import org.apache.spark.{Dependency, Logging, Partition, SerializableWritable, SparkContext, TaskContext}
+import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext}
 
 
 private[spark]
@@ -71,49 +71,52 @@ class NewHadoopRDD[K, V](
     result
   }
 
-  override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
-    val split = theSplit.asInstanceOf[NewHadoopPartition]
-    logInfo("Input split: " + split.serializableHadoopSplit)
-    val conf = confBroadcast.value.value
-    val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0)
-    val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
-    val format = inputFormatClass.newInstance
-    if (format.isInstanceOf[Configurable]) {
-      format.asInstanceOf[Configurable].setConf(conf)
-    }
-    val reader = format.createRecordReader(
-      split.serializableHadoopSplit.value, hadoopAttemptContext)
-    reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
-
-    // Register an on-task-completion callback to close the input stream.
-    context.addOnCompleteCallback(() => close())
-
-    var havePair = false
-    var finished = false
-
-    override def hasNext: Boolean = {
-      if (!finished && !havePair) {
-        finished = !reader.nextKeyValue
-        havePair = !finished
+  override def compute(theSplit: Partition, context: TaskContext) = {
+    val iter = new Iterator[(K, V)] {
+      val split = theSplit.asInstanceOf[NewHadoopPartition]
+      logInfo("Input split: " + split.serializableHadoopSplit)
+      val conf = confBroadcast.value.value
+      val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0)
+      val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
+      val format = inputFormatClass.newInstance
+      if (format.isInstanceOf[Configurable]) {
+        format.asInstanceOf[Configurable].setConf(conf)
+      }
+      val reader = format.createRecordReader(
+        split.serializableHadoopSplit.value, hadoopAttemptContext)
+      reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
+
+      // Register an on-task-completion callback to close the input stream.
+      context.addOnCompleteCallback(() => close())
+
+      var havePair = false
+      var finished = false
+
+      override def hasNext: Boolean = {
+        if (!finished && !havePair) {
+          finished = !reader.nextKeyValue
+          havePair = !finished
+        }
+        !finished
       }
-      !finished
-    }
 
-    override def next: (K, V) = {
-      if (!hasNext) {
-        throw new java.util.NoSuchElementException("End of stream")
+      override def next(): (K, V) = {
+        if (!hasNext) {
+          throw new java.util.NoSuchElementException("End of stream")
+        }
+        havePair = false
+        (reader.getCurrentKey, reader.getCurrentValue)
       }
-      havePair = false
-      return (reader.getCurrentKey, reader.getCurrentValue)
-    }
 
-    private def close() {
-      try {
-        reader.close()
-      } catch {
-        case e: Exception => logWarning("Exception in RecordReader.close()", e)
+      private def close() {
+        try {
+          reader.close()
+        } catch {
+          case e: Exception => logWarning("Exception in RecordReader.close()", e)
+        }
       }
     }
+    new InterruptibleIterator(context, iter)
   }
 
   override def getPreferredLocations(split: Partition): Seq[String] = {
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 4a465840c6f7444cf0cfa02f0d3679cb807e5691..d5691f2267bfaa313d0e06ea1d2622c025818bb6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -17,9 +17,10 @@
 
 package org.apache.spark.rdd
 
-import org.apache.spark.{RangePartitioner, Logging}
 import scala.reflect.ClassTag
 
+import org.apache.spark.{Logging, RangePartitioner}
+
 /**
  * Extra functions available on RDDs of (key, value) pairs where the key is sortable through
  * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index c8e623081ae17744950efebe9410443d5d5e3b25..0c2a051a42078c7fbdc0ecf855e9abae11c161c5 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -85,18 +85,24 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
     }
     val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
     if (self.partitioner == Some(partitioner)) {
-      self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
+      self.mapPartitionsWithContext((context, iter) => {
+        new InterruptibleIterator(context, aggregator.combineValuesByKey(iter))
+      }, preservesPartitioning = true)
     } else if (mapSideCombine) {
       val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
       val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner)
         .setSerializer(serializerClass)
-      partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = true)
+      partitioned.mapPartitionsWithContext((context, iter) => {
+        new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter))
+      }, preservesPartitioning = true)
     } else {
       // Don't apply map-side combiner.
       // A sanity check to make sure mergeCombiners is not defined.
       assert(mergeCombiners == null)
       val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass)
-      values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
+      values.mapPartitionsWithContext((context, iter) => {
+        new InterruptibleIterator(context, aggregator.combineValuesByKey(iter))
+      }, preservesPartitioning = true)
     }
   }
 
@@ -565,7 +571,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
       // around by taking a mod. We expect that no task will be attempted 2 billion times.
       val attemptNumber = (context.attemptId % Int.MaxValue).toInt
       /* "reduce task" <split #> <attempt # = spark task #> */
-      val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber)
+      val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.partitionId, attemptNumber)
       val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
       val format = outputFormatClass.newInstance
       val committer = format.getOutputCommitter(hadoopContext)
@@ -664,7 +670,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
       // around by taking a mod. We expect that no task will be attempted 2 billion times.
       val attemptNumber = (context.attemptId % Int.MaxValue).toInt
 
-      writer.setup(context.stageId, context.splitId, attemptNumber)
+      writer.setup(context.stageId, context.partitionId, attemptNumber)
       writer.open()
 
       var count = 0
diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index 78fe0cdcdbef843aaae110e098bc322010f623f6..09d0a8189d25c2c9cc8cb08737e3e9a3740ac1c6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -96,8 +96,9 @@ private[spark] class ParallelCollectionRDD[T: ClassTag](
     slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray
   }
 
-  override def compute(s: Partition, context: TaskContext) =
-    s.asInstanceOf[ParallelCollectionPartition[T]].iterator
+  override def compute(s: Partition, context: TaskContext) = {
+    new InterruptibleIterator(context, s.asInstanceOf[ParallelCollectionPartition[T]].iterator)
+  }
 
   override def getPreferredLocations(s: Partition): Seq[String] = {
     locationPrefs.getOrElse(s.index, Nil)
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index bb9b309a70734dd3e7363476a9cfee906352417b..ea8885b36e3aecd1cab9f81a64b6989547538843 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -35,11 +35,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo
   extends NarrowDependency[T](rdd) {
 
   @transient
-  val partitions: Array[Partition] = rdd.partitions.zipWithIndex
-    .filter(s => partitionFilterFunc(s._2))
+  val partitions: Array[Partition] = rdd.partitions
+    .filter(s => partitionFilterFunc(s.index)).zipWithIndex
     .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition }
 
-  override def getParents(partitionId: Int) = List(partitions(partitionId).index)
+  override def getParents(partitionId: Int) = {
+    List(partitions(partitionId).asInstanceOf[PartitionPruningRDDPartition].parentSplit.index)
+  }
 }
 
 
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 731ef90c90ea39b21deb3704fa589e10f9091207..ea45566ad1da96610f13d3250e6c66a5630040c5 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -104,7 +104,7 @@ abstract class RDD[T: ClassTag](
   protected def getPreferredLocations(split: Partition): Seq[String] = Nil
 
   /** Optionally overridden by subclasses to specify how they are partitioned. */
-  val partitioner: Option[Partitioner] = None
+  @transient val partitioner: Option[Partitioner] = None
 
   // =======================================================================
   // Methods and fields available on all RDDs
@@ -117,7 +117,7 @@ abstract class RDD[T: ClassTag](
   val id: Int = sc.newRddId()
 
   /** A friendly name for this RDD */
-  var name: String = null
+  @transient var name: String = null
 
   /** Assign a name to this RDD */
   def setName(_name: String) = {
@@ -126,7 +126,7 @@ abstract class RDD[T: ClassTag](
   }
 
   /** User-defined generator of this RDD*/
-  var generator = Utils.getCallSiteInfo.firstUserClass
+  @transient var generator = Utils.getCallSiteInfo.firstUserClass
 
   /** Reset generator*/
   def setGenerator(_generator: String) = {
@@ -268,6 +268,19 @@ abstract class RDD[T: ClassTag](
 
   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.
    *
@@ -398,7 +411,6 @@ abstract class RDD[T: ClassTag](
   def pipe(command: String, env: Map[String, String]): RDD[String] =
     new PipedRDD(this, command, env)
 
-
   /**
    * Return an RDD created by piping elements to a forked external process.
    * The print behavior can be customized by providing two functions.
@@ -421,26 +433,41 @@ abstract class RDD[T: ClassTag](
       command: Seq[String],
       env: Map[String, String] = Map(),
       printPipeContext: (String => Unit) => Unit = null,
-      printRDDElement: (T, String => Unit) => Unit = null): RDD[String] =
+      printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = {
     new PipedRDD(this, command, env,
       if (printPipeContext ne null) sc.clean(printPipeContext) else null,
       if (printRDDElement ne null) sc.clean(printRDDElement) else null)
+  }
 
   /**
    * Return a new RDD by applying a function to each partition of this RDD.
    */
-  def mapPartitions[U: ClassTag](f: Iterator[T] => Iterator[U],
-    preservesPartitioning: Boolean = false): RDD[U] =
-    new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
+  def mapPartitions[U: ClassTag](
+      f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+    val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter)
+    new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
+  }
 
   /**
    * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
    * of the original partition.
    */
   def mapPartitionsWithIndex[U: ClassTag](
-    f: (Int, Iterator[T]) => Iterator[U],
-    preservesPartitioning: Boolean = false): RDD[U] =
-    new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+      f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+    val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter)
+    new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
+  }
+
+  /**
+   * Return a new RDD by applying a function to each partition of this RDD. This is a variant of
+   * mapPartitions that also passes the TaskContext into the closure.
+   */
+  def mapPartitionsWithContext[U: ClassTag](
+      f: (TaskContext, Iterator[T]) => Iterator[U],
+      preservesPartitioning: Boolean = false): RDD[U] = {
+    val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter)
+    new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
+  }
 
   /**
    * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
@@ -448,22 +475,22 @@ abstract class RDD[T: ClassTag](
    */
   @deprecated("use mapPartitionsWithIndex", "0.7.0")
   def mapPartitionsWithSplit[U: ClassTag](
-    f: (Int, Iterator[T]) => Iterator[U],
-    preservesPartitioning: Boolean = false): RDD[U] =
-    new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+      f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+    mapPartitionsWithIndex(f, preservesPartitioning)
+  }
 
   /**
    * Maps f over this RDD, where f takes an additional parameter of type A.  This
    * additional parameter is produced by constructA, which is called in each
    * partition with the index of that partition.
    */
-  def mapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false)
-    (f:(T, A) => U): RDD[U] = {
-      def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
-        val a = constructA(index)
-        iter.map(t => f(t, a))
-      }
-    new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning)
+  def mapWith[A: ClassTag, U: ClassTag]
+      (constructA: Int => A, preservesPartitioning: Boolean = false)
+      (f: (T, A) => U): RDD[U] = {
+    mapPartitionsWithIndex((index, iter) => {
+      val a = constructA(index)
+      iter.map(t => f(t, a))
+    }, preservesPartitioning)
   }
 
   /**
@@ -471,13 +498,13 @@ abstract class RDD[T: ClassTag](
    * additional parameter is produced by constructA, which is called in each
    * partition with the index of that partition.
    */
-  def flatMapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false)
-    (f:(T, A) => Seq[U]): RDD[U] = {
-      def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
-        val a = constructA(index)
-        iter.flatMap(t => f(t, a))
-      }
-    new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning)
+  def flatMapWith[A: ClassTag, U: ClassTag]
+      (constructA: Int => A, preservesPartitioning: Boolean = false)
+      (f: (T, A) => Seq[U]): RDD[U] = {
+    mapPartitionsWithIndex((index, iter) => {
+      val a = constructA(index)
+      iter.flatMap(t => f(t, a))
+    }, preservesPartitioning)
   }
 
   /**
@@ -485,13 +512,11 @@ abstract class RDD[T: ClassTag](
    * This additional parameter is produced by constructA, which is called in each
    * partition with the index of that partition.
    */
-  def foreachWith[A: ClassTag](constructA: Int => A)
-    (f:(T, A) => Unit) {
-      def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
-        val a = constructA(index)
-        iter.map(t => {f(t, a); t})
-      }
-    (new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)).foreach(_ => {})
+  def foreachWith[A: ClassTag](constructA: Int => A)(f: (T, A) => Unit) {
+    mapPartitionsWithIndex { (index, iter) =>
+      val a = constructA(index)
+      iter.map(t => {f(t, a); t})
+    }.foreach(_ => {})
   }
 
   /**
@@ -499,13 +524,11 @@ abstract class RDD[T: ClassTag](
    * additional parameter is produced by constructA, which is called in each
    * partition with the index of that partition.
    */
-  def filterWith[A: ClassTag](constructA: Int => A)
-    (p:(T, A) => Boolean): RDD[T] = {
-      def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
-        val a = constructA(index)
-        iter.filter(t => p(t, a))
-      }
-    new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)
+  def filterWith[A: ClassTag](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
+    mapPartitionsWithIndex((index, iter) => {
+      val a = constructA(index)
+      iter.filter(t => p(t, a))
+    }, preservesPartitioning = true)
   }
 
   /**
@@ -525,17 +548,27 @@ abstract class RDD[T: ClassTag](
   def zipPartitions[B: ClassTag, V: ClassTag]
       (rdd2: RDD[B])
       (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
-    new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2)
+    new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false)
+
+  def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
+      (rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)
+      (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
+    new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning)
 
   def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
       (rdd2: RDD[B], rdd3: RDD[C])
       (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
-    new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3)
+    new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false)
+
+  def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
+      (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean)
+      (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
+    new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning)
 
   def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
       (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D])
       (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
-    new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4)
+    new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, false)
 
 
   // Actions (launch a job to return a value to the user program)
@@ -544,16 +577,14 @@ abstract class RDD[T: ClassTag](
    * Applies a function f to all elements of this RDD.
    */
   def foreach(f: T => Unit) {
-    val cleanF = sc.clean(f)
-    sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF))
+    sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f))
   }
 
   /**
    * Applies a function f to each partition of this RDD.
    */
   def foreachPartition(f: Iterator[T] => Unit) {
-    val cleanF = sc.clean(f)
-    sc.runJob(this, (iter: Iterator[T]) => cleanF(iter))
+    sc.runJob(this, (iter: Iterator[T]) => f(iter))
   }
 
   /**
@@ -678,6 +709,8 @@ abstract class RDD[T: ClassTag](
    */
   def count(): Long = {
     sc.runJob(this, (iter: Iterator[T]) => {
+      // Use a while loop to count the number of elements rather than iter.size because
+      // iter.size uses a for loop, which is slightly slower in current version of Scala.
       var result = 0L
       while (iter.hasNext) {
         result += 1L
@@ -905,7 +938,7 @@ abstract class RDD[T: ClassTag](
   private var storageLevel: StorageLevel = StorageLevel.NONE
 
   /** Record user function generating this RDD. */
-  private[spark] val origin = Utils.formatSparkCallSite
+  @transient private[spark] val origin = Utils.formatSparkCallSite
 
   private[spark] def elementClassTag: ClassTag[T] = classTag[T]
 
@@ -920,7 +953,7 @@ abstract class RDD[T: ClassTag](
   def context = sc
 
   // Avoid handling doCheckpoint multiple times to prevent excessive recursion
-  private var doCheckpointCalled = false
+  @transient private var doCheckpointCalled = false
 
   /**
    * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index b7205865cf09434552049329c2989d19cb725e69..3682c84598e730a659f6fd1c639f53afb7403892 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -17,9 +17,10 @@
 
 package org.apache.spark.rdd
 
-import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext}
 import scala.reflect.ClassTag
 
+import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency,
+  SparkEnv, TaskContext}
 
 private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
   override val index = idx
@@ -57,7 +58,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag](
 
   override def compute(split: Partition, context: TaskContext): Iterator[P] = {
     val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
-    SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context.taskMetrics,
+    SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context,
       SparkEnv.get.serializerManager.get(serializerClass))
   }
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index 85c512f3de6d76be78a496fa9199897aba9c0ce1..aab30b1bb49048ee90a513adece417e28ed64372 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -111,7 +111,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
       }
       case ShuffleCoGroupSplitDep(shuffleId) => {
         val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index,
-          context.taskMetrics, serializer)
+          context, serializer)
         iter.foreach(op)
       }
     }
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
index e02c17bf45514c0cc1e5750763d599864fc2a807..83be3c6eb40c480171c1b13369eec1ee71ebaba1 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
@@ -23,7 +23,8 @@ import scala.reflect.ClassTag
 
 private[spark] class ZippedPartitionsPartition(
     idx: Int,
-    @transient rdds: Seq[RDD[_]])
+    @transient rdds: Seq[RDD[_]],
+    @transient val preferredLocations: Seq[String])
   extends Partition {
 
   override val index: Int = idx
@@ -40,31 +41,29 @@ private[spark] class ZippedPartitionsPartition(
 
 abstract class ZippedPartitionsBaseRDD[V: ClassTag](
     sc: SparkContext,
-    var rdds: Seq[RDD[_]])
+    var rdds: Seq[RDD[_]],
+    preservesPartitioning: Boolean = false)
   extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) {
 
+  override val partitioner =
+    if (preservesPartitioning) firstParent[Any].partitioner else None
+
   override def getPartitions: Array[Partition] = {
-    val sizes = rdds.map(x => x.partitions.size)
-    if (!sizes.forall(x => x == sizes(0))) {
+    val numParts = rdds.head.partitions.size
+    if (!rdds.forall(rdd => rdd.partitions.size == numParts)) {
       throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions")
     }
-    val array = new Array[Partition](sizes(0))
-    for (i <- 0 until sizes(0)) {
-      array(i) = new ZippedPartitionsPartition(i, rdds)
+    Array.tabulate[Partition](numParts) { i =>
+      val prefs = rdds.map(rdd => rdd.preferredLocations(rdd.partitions(i)))
+      // Check whether there are any hosts that match all RDDs; otherwise return the union
+      val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y))
+      val locs = if (!exactMatchLocations.isEmpty) exactMatchLocations else prefs.flatten.distinct
+      new ZippedPartitionsPartition(i, rdds, locs)
     }
-    array
   }
 
   override def getPreferredLocations(s: Partition): Seq[String] = {
-    val parts = s.asInstanceOf[ZippedPartitionsPartition].partitions
-    val prefs = rdds.zip(parts).map { case (rdd, p) => rdd.preferredLocations(p) }
-    // Check whether there are any hosts that match all RDDs; otherwise return the union
-    val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y))
-    if (!exactMatchLocations.isEmpty) {
-      exactMatchLocations
-    } else {
-      prefs.flatten.distinct
-    }
+    s.asInstanceOf[ZippedPartitionsPartition].preferredLocations
   }
 
   override def clearDependencies() {
@@ -77,8 +76,9 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag](
     sc: SparkContext,
     f: (Iterator[A], Iterator[B]) => Iterator[V],
     var rdd1: RDD[A],
-    var rdd2: RDD[B])
-  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) {
+    var rdd2: RDD[B],
+    preservesPartitioning: Boolean = false)
+  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2), preservesPartitioning) {
 
   override def compute(s: Partition, context: TaskContext): Iterator[V] = {
     val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions
@@ -98,8 +98,9 @@ class ZippedPartitionsRDD3
     f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V],
     var rdd1: RDD[A],
     var rdd2: RDD[B],
-    var rdd3: RDD[C])
-  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) {
+    var rdd3: RDD[C],
+    preservesPartitioning: Boolean = false)
+  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3), preservesPartitioning) {
 
   override def compute(s: Partition, context: TaskContext): Iterator[V] = {
     val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions
@@ -123,8 +124,9 @@ class ZippedPartitionsRDD4
     var rdd1: RDD[A],
     var rdd2: RDD[B],
     var rdd3: RDD[C],
-    var rdd4: RDD[D])
-  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) {
+    var rdd4: RDD[D],
+    preservesPartitioning: Boolean = false)
+  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4), preservesPartitioning) {
 
   override def compute(s: Partition, context: TaskContext): Iterator[V] = {
     val partitions = s.asInstanceOf[ZippedPartitionsPartition].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 03fe0e00f9d5527a47e836ff7e527478932baf2d..963d15b76d4c4d7495eed8e7fd01d86588c67334 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -19,18 +19,20 @@ package org.apache.spark.scheduler
 
 import java.io.NotSerializableException
 import java.util.Properties
-import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
 import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
+import scala.concurrent.duration._
 import scala.reflect.ClassTag
 
+import akka.actor._
+
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
-import org.apache.spark.storage.{BlockManager, BlockManagerMaster}
-import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId}
+import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap}
 
 /**
  * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of
@@ -42,76 +44,92 @@ import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
  * locations to run each task on, based on the current cache status, and passes these to the
  * low-level TaskScheduler. Furthermore, it handles failures due to shuffle output files being
  * lost, in which case old stages may need to be resubmitted. Failures *within* a stage that are
- * not caused by shuffie file loss are handled by the TaskScheduler, which will retry each task
+ * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task
  * a small number of times before cancelling the whole stage.
  *
  * THREADING: This class runs all its logic in a single thread executing the run() method, to which
- * events are submitted using a synchonized queue (eventQueue). The public API methods, such as
+ * events are submitted using a synchronized queue (eventQueue). The public API methods, such as
  * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods
  * should be private.
  */
 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) {
-    eventQueue.put(BeginEvent(task, taskInfo))
+  def taskStarted(task: Task[_], taskInfo: TaskInfo) {
+    eventProcessActor ! BeginEvent(task, taskInfo)
+  }
+
+  // Called to report that a task has completed and results are being fetched remotely.
+  def taskGettingResult(task: Task[_], taskInfo: TaskInfo) {
+    eventProcessActor ! GettingResultEvent(task, taskInfo)
   }
 
   // Called by TaskScheduler to report task completions or failures.
-  override def taskEnded(
+  def taskEnded(
       task: Task[_],
       reason: TaskEndReason,
       result: Any,
       accumUpdates: Map[Long, Any],
       taskInfo: TaskInfo,
       taskMetrics: TaskMetrics) {
-    eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics))
+    eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)
   }
 
   // Called by TaskScheduler when an executor fails.
-  override def executorLost(execId: String) {
-    eventQueue.put(ExecutorLost(execId))
+  def executorLost(execId: String) {
+    eventProcessActor ! ExecutorLost(execId)
   }
 
   // Called by TaskScheduler when a host is added
-  override def executorGained(execId: String, host: String) {
-    eventQueue.put(ExecutorGained(execId, host))
+  def executorGained(execId: String, host: String) {
+    eventProcessActor ! ExecutorGained(execId, host)
   }
 
-  // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures.
-  override def taskSetFailed(taskSet: TaskSet, reason: String) {
-    eventQueue.put(TaskSetFailed(taskSet, reason))
+  // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or
+  // cancellation of the job itself.
+  def taskSetFailed(taskSet: TaskSet, reason: String) {
+    eventProcessActor ! TaskSetFailed(taskSet, reason)
   }
 
   // The time, in millis, to wait for fetch failure events to stop coming in after one is detected;
   // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one
   // as more failure events come in
-  val RESUBMIT_TIMEOUT = 50L
+  val RESUBMIT_TIMEOUT = 50.milliseconds
 
   // The time, in millis, to wake up between polls of the completion queue in order to potentially
   // resubmit failed stages
   val POLL_TIMEOUT = 10L
 
-  private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent]
+  // Warns the user if a stage contains a task with size greater than this value (in KB)
+  val TASK_SIZE_TO_WARN = 100
+
+  private var eventProcessActor: ActorRef = _
+
+  private[scheduler] val nextJobId = new AtomicInteger(0)
 
-  val nextJobId = new AtomicInteger(0)
+  def numTotalJobs: Int = nextJobId.get()
 
-  val nextStageId = new AtomicInteger(0)
+  private val nextStageId = new AtomicInteger(0)
 
-  val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+  private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]]
 
-  val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
+  private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]]
+
+  private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+
+  private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
 
   private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
 
@@ -128,6 +146,7 @@ class DAGScheduler(
   //       stray messages to detect.
   val failedEpoch = new HashMap[String, Long]
 
+  // stage id to the active job
   val idToActiveJob = new HashMap[Int, ActiveJob]
 
   val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
@@ -139,16 +158,57 @@ class DAGScheduler(
   val activeJobs = new HashSet[ActiveJob]
   val resultStageToJob = new HashMap[Stage, ActiveJob]
 
-  val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup)
+  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup)
 
-  // Start a thread to run the DAGScheduler event loop
+  /**
+   * Starts the event processing actor.  The actor has two responsibilities:
+   *
+   * 1. Waits for events like job submission, task finished, task failure etc., and calls
+   *    [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them.
+   * 2. Schedules a periodical task to resubmit failed stages.
+   *
+   * NOTE: the actor cannot be started in the constructor, because the periodical task references
+   * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus
+   * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed.
+   */
   def start() {
-    new Thread("DAGScheduler") {
-      setDaemon(true)
-      override def run() {
-        DAGScheduler.this.run()
+    eventProcessActor = env.actorSystem.actorOf(Props(new Actor {
+      /**
+       * A handle to the periodical task, used to cancel the task when the actor is stopped.
+       */
+      var resubmissionTask: Cancellable = _
+
+      override def preStart() {
+        import context.dispatcher
+        /**
+         * A message is sent to the actor itself periodically to remind the actor to resubmit failed
+         * stages.  In this way, stage resubmission can be done within the same thread context of
+         * other event processing logic to avoid unnecessary synchronization overhead.
+         */
+        resubmissionTask = context.system.scheduler.schedule(
+          RESUBMIT_TIMEOUT, RESUBMIT_TIMEOUT, self, ResubmitFailedStages)
       }
-    }.start()
+
+      /**
+       * The main event loop of the DAG scheduler.
+       */
+      def receive = {
+        case event: DAGSchedulerEvent =>
+          logDebug("Got event of type " + event.getClass.getName)
+
+          /**
+           * All events are forwarded to `processEvent()`, so that the event processing logic can
+           * easily tested without starting a dedicated actor.  Please refer to `DAGSchedulerSuite`
+           * for details.
+           */
+          if (!processEvent(event)) {
+            submitWaitingStages()
+          } else {
+            resubmissionTask.cancel()
+            context.stop(self)
+          }
+      }
+    }))
   }
 
   def addSparkListener(listener: SparkListener) {
@@ -157,7 +217,7 @@ class DAGScheduler(
 
   private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = {
     if (!cacheLocs.contains(rdd.id)) {
-      val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
+      val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId]
       val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster)
       cacheLocs(rdd.id) = blockIds.map { id =>
         locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId))
@@ -179,34 +239,60 @@ class DAGScheduler(
     shuffleToMapStage.get(shuffleDep.shuffleId) match {
       case Some(stage) => stage
       case None =>
-        val stage = newStage(shuffleDep.rdd, Some(shuffleDep), jobId)
+        val stage = newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId)
         shuffleToMapStage(shuffleDep.shuffleId) = stage
         stage
     }
   }
 
   /**
-   * Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or
-   * as a result stage for the final RDD used directly in an action. The stage will also be
-   * associated with the provided jobId.
+   * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation
+   * of a shuffle map stage in newOrUsedStage.  The stage will be associated with the provided
+   * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage directly.
    */
   private def newStage(
       rdd: RDD[_],
+      numTasks: Int,
       shuffleDep: Option[ShuffleDependency[_,_]],
       jobId: Int,
       callSite: Option[String] = None)
     : Stage =
   {
-    if (shuffleDep != None) {
+    val id = nextStageId.getAndIncrement()
+    val stage =
+      new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
+    stageIdToStage(id) = stage
+    updateJobIdStageIdMaps(jobId, stage)
+    stageToInfos(stage) = new StageInfo(stage)
+    stage
+  }
+
+  /**
+   * Create a shuffle map Stage for the given RDD.  The stage will also be associated with the
+   * provided jobId.  If a stage for the shuffleId existed previously so that the shuffleId is
+   * present in the MapOutputTracker, then the number and location of available outputs are
+   * recovered from the MapOutputTracker
+   */
+  private def newOrUsedStage(
+      rdd: RDD[_],
+      numTasks: Int,
+      shuffleDep: ShuffleDependency[_,_],
+      jobId: Int,
+      callSite: Option[String] = None)
+    : Stage =
+  {
+    val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite)
+    if (mapOutputTracker.has(shuffleDep.shuffleId)) {
+      val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId)
+      val locs = MapOutputTracker.deserializeMapStatuses(serLocs)
+      for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i))
+      stage.numAvailableOutputs = locs.size
+    } else {
       // Kind of ugly: need to register RDDs with the cache and map output tracker here
       // since we can't do it in the RDD constructor because # of partitions is unknown
       logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
-      mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
+      mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.size)
     }
-    val id = nextStageId.getAndIncrement()
-    val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
-    stageIdToStage(id) = stage
-    stageToInfos(stage) = StageInfo(stage)
     stage
   }
 
@@ -262,32 +348,123 @@ class DAGScheduler(
   }
 
   /**
-   * Returns (and does not submit) a JobSubmitted event suitable to run a given job, and a
-   * JobWaiter whose getResult() method will return the result of the job when it is complete.
-   *
-   * The job is assumed to have at least one partition; zero partition jobs should be handled
-   * without a JobSubmitted event.
+   * Registers the given jobId among the jobs that need the given stage and
+   * all of that stage's ancestors.
+   */
+  private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) {
+    def updateJobIdStageIdMapsList(stages: List[Stage]) {
+      if (!stages.isEmpty) {
+        val s = stages.head
+        stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId
+        jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id
+        val parents = getParentStages(s.rdd, jobId)
+        val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId)))
+        updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail)
+      }
+    }
+    updateJobIdStageIdMapsList(List(stage))
+  }
+
+  /**
+   * Removes job and any stages that are not needed by any other job.  Returns the set of ids for stages that
+   * were removed.  The associated tasks for those stages need to be cancelled if we got here via job cancellation.
+   */
+  private def removeJobAndIndependentStages(jobId: Int): Set[Int] = {
+    val registeredStages = jobIdToStageIds(jobId)
+    val independentStages = new HashSet[Int]()
+    if (registeredStages.isEmpty) {
+      logError("No stages registered for job " + jobId)
+    } else {
+      stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach {
+        case (stageId, jobSet) =>
+          if (!jobSet.contains(jobId)) {
+            logError("Job %d not registered for stage %d even though that stage was registered for the job"
+              .format(jobId, stageId))
+          } else {
+            def removeStage(stageId: Int) {
+              // data structures based on Stage
+              stageIdToStage.get(stageId).foreach { s =>
+                if (running.contains(s)) {
+                  logDebug("Removing running stage %d".format(stageId))
+                  running -= s
+                }
+                stageToInfos -= s
+                shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove)
+                if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) {
+                  logDebug("Removing pending status for stage %d".format(stageId))
+                }
+                pendingTasks -= s
+                if (waiting.contains(s)) {
+                  logDebug("Removing stage %d from waiting set.".format(stageId))
+                  waiting -= s
+                }
+                if (failed.contains(s)) {
+                  logDebug("Removing stage %d from failed set.".format(stageId))
+                  failed -= s
+                }
+              }
+              // data structures based on StageId
+              stageIdToStage -= stageId
+              stageIdToJobIds -= stageId
+
+              logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size))
+            }
+
+            jobSet -= jobId
+            if (jobSet.isEmpty) { // no other job needs this stage
+              independentStages += stageId
+              removeStage(stageId)
+            }
+          }
+      }
+    }
+    independentStages.toSet
+  }
+
+  private def jobIdToStageIdsRemove(jobId: Int) {
+    if (!jobIdToStageIds.contains(jobId)) {
+      logDebug("Trying to remove unregistered job " + jobId)
+    } else {
+      removeJobAndIndependentStages(jobId)
+      jobIdToStageIds -= jobId
+    }
+  }
+
+  /**
+   * Submit a job to the job scheduler and get a JobWaiter object back. The JobWaiter object
+   * can be used to block until the the job finishes executing or can be used to cancel the job.
    */
-  private[scheduler] def prepareJob[T, U: ClassTag](
-      finalRdd: RDD[T],
+  def submitJob[T, U](
+      rdd: RDD[T],
       func: (TaskContext, Iterator[T]) => U,
       partitions: Seq[Int],
       callSite: String,
       allowLocal: Boolean,
       resultHandler: (Int, U) => Unit,
-      properties: Properties = null)
-    : (JobSubmitted, JobWaiter[U]) =
+      properties: Properties = null): JobWaiter[U] =
   {
+    // 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 =>
+      throw new IllegalArgumentException(
+        "Attempting to access a non-existent partition: " + p + ". " +
+          "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 waiter = new JobWaiter(partitions.size, resultHandler)
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
-    val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter,
-      properties)
-    (toSubmit, waiter)
+    val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
+    eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
+    waiter
   }
 
   def runJob[T, U: ClassTag](
-      finalRdd: RDD[T],
+      rdd: RDD[T],
       func: (TaskContext, Iterator[T]) => U,
       partitions: Seq[Int],
       callSite: String,
@@ -295,21 +472,7 @@ class DAGScheduler(
       resultHandler: (Int, U) => Unit,
       properties: Properties = null)
   {
-    if (partitions.size == 0) {
-      return
-    }
-
-    // Check to make sure we are not launching a task on a partition that does not exist.
-    val maxPartitions = finalRdd.partitions.length
-    partitions.find(p => p >= maxPartitions).foreach { p =>
-      throw new IllegalArgumentException(
-        "Attempting to access a non-existent partition: " + p + ". " +
-        "Total number of partitions: " + maxPartitions)
-    }
-
-    val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob(
-        finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties)
-    eventQueue.put(toSubmit)
+    val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties)
     waiter.awaitResult() match {
       case JobSucceeded => {}
       case JobFailed(exception: Exception, _) =>
@@ -330,19 +493,51 @@ class DAGScheduler(
     val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
     val partitions = (0 until rdd.partitions.size).toArray
-    eventQueue.put(JobSubmitted(rdd, func2, partitions, allowLocal = false, callSite, listener, properties))
+    val jobId = nextJobId.getAndIncrement()
+    eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)
     listener.awaitResult()    // Will throw an exception if the job fails
   }
 
   /**
-   * Process one event retrieved from the event queue.
-   * Returns true if we should stop the event loop.
+   * Cancel a job that is running or waiting in the queue.
+   */
+  def cancelJob(jobId: Int) {
+    logInfo("Asked to cancel job " + jobId)
+    eventProcessActor ! JobCancelled(jobId)
+  }
+
+  def cancelJobGroup(groupId: String) {
+    logInfo("Asked to cancel job group " + groupId)
+    eventProcessActor ! JobGroupCancelled(groupId)
+  }
+
+  /**
+   * Cancel all jobs that are running or waiting in the queue.
+   */
+  def cancelAllJobs() {
+    eventProcessActor ! AllJobsCancelled
+  }
+
+  /**
+   * Process one event retrieved from the event processing actor.
+   *
+   * @param event The event to be processed.
+   * @return `true` if we should stop the event loop.
    */
   private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
     event match {
-      case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) =>
-        val jobId = nextJobId.getAndIncrement()
-        val finalStage = newStage(finalRDD, None, jobId, Some(callSite))
+      case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
+        var finalStage: Stage = null
+        try {
+          // New stage creation at times and if its not protected, the scheduler thread is killed. 
+          // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted
+          finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
+        } catch {
+          case e: Exception =>
+            logWarning("Creating new stage failed due to exception - job: " + jobId, e)
+            listener.jobFailed(e)
+            return false
+        }
         val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
         clearCacheLocs()
         logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length +
@@ -352,31 +547,67 @@ class DAGScheduler(
         logInfo("Missing parents: " + getMissingParentStages(finalStage))
         if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
           // Compute very short actions like first() or take() with no parent stages locally.
+          listenerBus.post(SparkListenerJobStart(job, Array(), properties))
           runLocally(job)
         } else {
-          listenerBus.post(SparkListenerJobStart(job, properties))
           idToActiveJob(jobId) = job
           activeJobs += job
           resultStageToJob(finalStage) = job
+          listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties))
           submitStage(finalStage)
         }
 
+      case JobCancelled(jobId) =>
+        handleJobCancellation(jobId)
+
+      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 activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
+        val jobIds = activeInGroup.map(_.jobId)
+        jobIds.foreach { handleJobCancellation }
+
+      case AllJobsCancelled =>
+        // Cancel all running jobs.
+        running.map(_.jobId).foreach { handleJobCancellation }
+        activeJobs.clear()      // These should already be empty by this point,
+        idToActiveJob.clear()   // but just in case we lost track of some jobs...
+
       case ExecutorGained(execId, host) =>
         handleExecutorGained(execId, host)
 
       case ExecutorLost(execId) =>
         handleExecutorLost(execId)
 
-      case begin: BeginEvent =>
-        listenerBus.post(SparkListenerTaskStart(begin.task, begin.taskInfo))
+      case BeginEvent(task, taskInfo) =>
+        for (
+          job <- idToActiveJob.get(task.stageId);
+          stage <- stageIdToStage.get(task.stageId);
+          stageInfo <- stageToInfos.get(stage)
+        ) {
+          if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && !stageInfo.emittedTaskSizeWarning) {
+            stageInfo.emittedTaskSizeWarning = true
+            logWarning(("Stage %d (%s) contains a task of very large " +
+              "size (%d KB). The maximum recommended task size is %d KB.").format(
+              task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN))
+          }
+        }
+        listenerBus.post(SparkListenerTaskStart(task, taskInfo))
+
+      case GettingResultEvent(task, taskInfo) =>
+        listenerBus.post(SparkListenerTaskGettingResult(task, taskInfo))
 
-      case completion: CompletionEvent =>
-        listenerBus.post(SparkListenerTaskEnd(
-          completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))
+      case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) =>
+        listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics))
         handleTaskCompletion(completion)
 
       case TaskSetFailed(taskSet, reason) =>
-        abortStage(stageIdToStage(taskSet.stageId), reason)
+        stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) }
+
+      case ResubmitFailedStages =>
+        if (failed.size > 0) {
+          resubmitFailedStages()
+        }
 
       case StopDAGScheduler =>
         // Cancel any active jobs
@@ -422,42 +653,6 @@ class DAGScheduler(
     }
   }
 
-
-  /**
-   * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure
-   * events and responds by launching tasks. This runs in a dedicated thread and receives events
-   * via the eventQueue.
-   */
-  private def run() {
-    SparkEnv.set(env)
-
-    while (true) {
-      val event = eventQueue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)
-      if (event != null) {
-        logDebug("Got event of type " + event.getClass.getName)
-      }
-      this.synchronized { // needed in case other threads makes calls into methods of this class
-        if (event != null) {
-          if (processEvent(event)) {
-            return
-          }
-        }
-
-        val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability
-        // Periodically resubmit failed stages if some map output fetches have failed and we have
-        // waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails,
-        // tasks on many other nodes are bound to get a fetch failure, and they won't all get it at
-        // the same time, so we want to make sure we've identified all the reduce tasks that depend
-        // on the failed node.
-        if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) {
-          resubmitFailedStages()
-        } else {
-          submitWaitingStages()
-        }
-      }
-    }
-  }
-
   /**
    * Run a job on an RDD locally, assuming it has only a single partition and no dependencies.
    * We run the operation in a separate thread just in case it takes a bunch of time, so that we
@@ -474,6 +669,7 @@ class DAGScheduler(
 
   // Broken out for easier testing in DAGSchedulerSuite.
   protected def runLocallyWithinThread(job: ActiveJob) {
+    var jobResult: JobResult = JobSucceeded
     try {
       SparkEnv.set(env)
       val rdd = job.finalStage.rdd
@@ -488,31 +684,59 @@ class DAGScheduler(
       }
     } catch {
       case e: Exception =>
+        jobResult = JobFailed(e, Some(job.finalStage))
         job.listener.jobFailed(e)
+    } finally {
+      val s = job.finalStage
+      stageIdToJobIds -= s.id    // clean up data structures that were populated for a local job,
+      stageIdToStage -= s.id     // but that won't get cleaned up via the normal paths through
+      stageToInfos -= s          // completion events or stage abort
+      jobIdToStageIds -= job.jobId
+      listenerBus.post(SparkListenerJobEnd(job, jobResult))
+    }
+  }
+
+  /** Finds the earliest-created active job that needs the stage */
+  // TODO: Probably should actually find among the active jobs that need this
+  // stage the one with the highest priority (highest-priority pool, earliest created).
+  // That should take care of at least part of the priority inversion problem with
+  // cross-job dependencies.
+  private def activeJobForStage(stage: Stage): Option[Int] = {
+    if (stageIdToJobIds.contains(stage.id)) {
+      val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted
+      jobsThatUseStage.find(idToActiveJob.contains(_))
+    } else {
+      None
     }
   }
 
   /** Submits stage, but first recursively submits any missing parents. */
   private def submitStage(stage: Stage) {
-    logDebug("submitStage(" + stage + ")")
-    if (!waiting(stage) && !running(stage) && !failed(stage)) {
-      val missing = getMissingParentStages(stage).sortBy(_.id)
-      logDebug("missing: " + missing)
-      if (missing == Nil) {
-        logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
-        submitMissingTasks(stage)
-        running += stage
-      } else {
-        for (parent <- missing) {
-          submitStage(parent)
+    val jobId = activeJobForStage(stage)
+    if (jobId.isDefined) {
+      logDebug("submitStage(" + stage + ")")
+      if (!waiting(stage) && !running(stage) && !failed(stage)) {
+        val missing = getMissingParentStages(stage).sortBy(_.id)
+        logDebug("missing: " + missing)
+        if (missing == Nil) {
+          logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
+          submitMissingTasks(stage, jobId.get)
+          running += stage
+        } else {
+          for (parent <- missing) {
+            submitStage(parent)
+          }
+          waiting += stage
         }
-        waiting += stage
       }
+    } else {
+      abortStage(stage, "No active job for stage " + stage.id)
     }
   }
 
+
   /** Called when stage's parents are available and we can now do its task. */
-  private def submitMissingTasks(stage: Stage) {
+  private def submitMissingTasks(stage: Stage, jobId: Int) {
     logDebug("submitMissingTasks(" + stage + ")")
     // Get our pending tasks and remember them in our pendingTasks entry
     val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet)
@@ -533,7 +757,7 @@ class DAGScheduler(
       }
     }
 
-    val properties = if (idToActiveJob.contains(stage.jobId)) {
+    val properties = if (idToActiveJob.contains(jobId)) {
       idToActiveJob(stage.jobId).properties
     } else {
       //this stage will be assigned to "default" pool
@@ -542,7 +766,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
@@ -563,9 +787,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))
@@ -579,15 +801,20 @@ class DAGScheduler(
    */
   private def handleTaskCompletion(event: CompletionEvent) {
     val task = event.task
+
+    if (!stageIdToStage.contains(task.stageId)) {
+      // Skip all the actions if the stage has been cancelled.
+      return
+    }
     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
     }
@@ -612,6 +839,7 @@ class DAGScheduler(
                     activeJobs -= job
                     resultStageToJob -= stage
                     markStageAsFinished(stage)
+                    jobIdToStageIdsRemove(job.jobId)
                     listenerBus.post(SparkListenerJobEnd(job, JobSucceeded))
                   }
                   job.listener.taskSucceeded(rt.outputId, event.result)
@@ -627,7 +855,7 @@ class DAGScheduler(
             if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) {
               logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId)
             } else {
-              stage.addOutputLoc(smt.partition, status)
+              stage.addOutputLoc(smt.partitionId, status)
             }
             if (running.contains(stage) && pendingTasks(stage).isEmpty) {
               markStageAsFinished(stage)
@@ -648,7 +876,7 @@ class DAGScheduler(
                   changeEpoch = true)
               }
               clearCacheLocs()
-              if (stage.outputLocs.count(_ == Nil) != 0) {
+              if (stage.outputLocs.exists(_ == Nil)) {
                 // Some tasks had failed; let's resubmit this stage
                 // TODO: Lower-level scheduler should also deal with this
                 logInfo("Resubmitting " + stage + " (" + stage.name +
@@ -665,9 +893,12 @@ class DAGScheduler(
                 }
                 waiting --= newlyRunnable
                 running ++= newlyRunnable
-                for (stage <- newlyRunnable.sortBy(_.id)) {
+                for {
+                  stage <- newlyRunnable.sortBy(_.id)
+                  jobId <- activeJobForStage(stage)
+                } {
                   logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable")
-                  submitMissingTasks(stage)
+                  submitMissingTasks(stage, jobId)
                 }
               }
             }
@@ -751,21 +982,42 @@ class DAGScheduler(
     }
   }
 
+  private def handleJobCancellation(jobId: Int) {
+    if (!jobIdToStageIds.contains(jobId)) {
+      logDebug("Trying to cancel unregistered job " + jobId)
+    } else {
+      val independentStages = removeJobAndIndependentStages(jobId)
+      independentStages.foreach { taskSched.cancelTasks }
+      val error = new SparkException("Job %d cancelled".format(jobId))
+      val job = idToActiveJob(jobId)
+      job.listener.jobFailed(error)
+      jobIdToStageIds -= jobId
+      activeJobs -= job
+      idToActiveJob -= jobId
+      listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage))))
+    }
+  }
+
   /**
    * Aborts all jobs depending on a particular Stage. This is called in response to a task set
-   * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
+   * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
    */
   private def abortStage(failedStage: Stage, reason: String) {
+    if (!stageIdToStage.contains(failedStage.id)) {
+      // Skip all the actions if the stage has been removed.
+      return
+    }
     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 failed: " + reason)
+      val error = new SparkException("Job aborted: " + reason)
       job.listener.jobFailed(error)
-      listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
+      jobIdToStageIdsRemove(job.jobId)
       idToActiveJob -= resultStage.jobId
       activeJobs -= job
       resultStageToJob -= resultStage
+      listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
     }
     if (dependentStages.isEmpty) {
       logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done")
@@ -823,7 +1075,7 @@ class DAGScheduler(
     // If the RDD has narrow dependencies, pick the first partition of the first narrow dep
     // that has any placement preferences. Ideally we would choose based on transfer sizes,
     // but this will do for now.
-    rdd.dependencies.foreach(_ match {
+    rdd.dependencies.foreach {
       case n: NarrowDependency[_] =>
         for (inPart <- n.getParents(partition)) {
           val locs = getPreferredLocs(n.rdd, inPart)
@@ -831,30 +1083,29 @@ class DAGScheduler(
             return locs
         }
       case _ =>
-    })
+    }
     Nil
   }
 
   private def cleanup(cleanupTime: Long) {
-    var sizeBefore = stageIdToStage.size
-    stageIdToStage.clearOldValues(cleanupTime)
-    logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size)
-
-    sizeBefore = shuffleToMapStage.size
-    shuffleToMapStage.clearOldValues(cleanupTime)
-    logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size)
-
-    sizeBefore = pendingTasks.size
-    pendingTasks.clearOldValues(cleanupTime)
-    logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size)
-
-    sizeBefore = stageToInfos.size
-    stageToInfos.clearOldValues(cleanupTime)
-    logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size)
+    Map(
+      "stageIdToStage" -> stageIdToStage,
+      "shuffleToMapStage" -> shuffleToMapStage,
+      "pendingTasks" -> pendingTasks,
+      "stageToInfos" -> stageToInfos,
+      "jobIdToStageIds" -> jobIdToStageIds,
+      "stageIdToJobIds" -> stageIdToJobIds).
+      foreach { case(s, t) => {
+      val sizeBefore = t.size
+      t.clearOldValues(cleanupTime)
+      logInfo("%s %d --> %d".format(s, sizeBefore, t.size))
+    }}
   }
 
   def stop() {
-    eventQueue.put(StopDAGScheduler)
+    if (eventProcessActor != null) {
+      eventProcessActor ! StopDAGScheduler
+    }
     metadataCleaner.cancel()
     taskSched.stop()
   }
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 10ff1b4376268dbc96a8b1115df36eea5337cb0b..add11876130b18eabf6cc8404f223cef31bafa9b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -31,9 +31,10 @@ import org.apache.spark.executor.TaskMetrics
  * submitted) but there is a single "logic" thread that reads these events and takes decisions.
  * This greatly simplifies synchronization.
  */
-private[spark] sealed trait DAGSchedulerEvent
+private[scheduler] sealed trait DAGSchedulerEvent
 
-private[spark] case class JobSubmitted(
+private[scheduler] case class JobSubmitted(
+    jobId: Int,
     finalRDD: RDD[_],
     func: (TaskContext, Iterator[_]) => _,
     partitions: Array[Int],
@@ -43,9 +44,19 @@ private[spark] case class JobSubmitted(
     properties: Properties = null)
   extends DAGSchedulerEvent
 
-private[spark] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent
+private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent
 
-private[spark] case class CompletionEvent(
+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,
     result: Any,
@@ -54,10 +65,13 @@ private[spark] case class CompletionEvent(
     taskMetrics: TaskMetrics)
   extends DAGSchedulerEvent
 
-private[spark] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
+private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
+
+private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
 
-private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
+private[scheduler]
+case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
 
-private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
+private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent
 
-private[spark] case object StopDAGScheduler extends DAGSchedulerEvent
+private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
index 151514896f6fa2289bbae8a878772a333e9f55ec..7b5c0e29ad84043474e443db395c6d98b92536f0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
@@ -40,7 +40,7 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: Spar
   })
 
   metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] {
-    override def getValue: Int = dagScheduler.nextJobId.get()
+    override def getValue: Int = dagScheduler.numTotalJobs
   })
 
   metricRegistry.register(MetricRegistry.name("job", "activeJobs"), new Gauge[Int] {
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..60927831a159a7d4e2b092b97775ac3005ed1c1e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -1,292 +1,384 @@
-/*
- * 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 java.io.PrintWriter
-import java.io.File
-import java.io.FileNotFoundException
-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 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
-
-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 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
-  
-  // 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 + "/")
-    }
-  }
-  
-  // Create a log file for one job, the file name is the jobID
-  protected def createLogWriter(jobID: Int) {
-    try{
-      val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID)
-      jobIDToPrintWriter += (jobID -> fileWriter)
-      } catch {
-        case e: FileNotFoundException => e.printStackTrace()
-      }
-  }
-  
-  // Close log file, and clean the stage relationship in stageIDToJobID 
-  protected def closeLogWriter(jobID: Int) = 
-    jobIDToPrintWriter.get(jobID).foreach { fileWriter => 
-      fileWriter.close()
-      jobIDToStages.get(jobID).foreach(_.foreach{ stage => 
-        stageIDToJobID -= stage.id
-      })
-      jobIDToPrintWriter -= jobID
-      jobIDToStages -= jobID
-    }
-  
-  // Write log information to log file, withTime parameter controls whether to recored 
-  // time stamp for the information
-  protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) {
-    var writeInfo = info
-    if (withTime) {
-      val date = new Date(System.currentTimeMillis())
-      writeInfo = DATE_FORMAT.format(date) + ": " +info
-    }
-    jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo))
-  }
-  
-  protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) = 
-    stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime))
-
-  protected def buildJobDep(jobID: Int, stage: Stage) {
-    if (stage.jobId == jobID) {
-      jobIDToStages.get(jobID) match {
-        case Some(stageList) => stageList += stage
-        case None => val stageList = new  ListBuffer[Stage]
-                     stageList += stage
-                     jobIDToStages += (jobID -> stageList)
-      }
-      stageIDToJobID += (stage.id -> jobID)
-      stage.parents.foreach(buildJobDep(jobID, _))
-    }
-  }
-
-  protected def recordStageDep(jobID: Int) {
-    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)
-        }
-      }
-      rddList
-    }
-    jobIDToStages.get(jobID).foreach {_.foreach { stage => 
-        var depRddDesc: String = ""
-        getRddsInStage(stage.rdd).foreach { rdd => 
-          depRddDesc += rdd.id + ","
-        }
-        var depStageDesc: String = ""
-        stage.parents.foreach { stage => 
-          depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")"
-        }
-        jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + 
-                   depRddDesc.substring(0, depRddDesc.length - 1) + ")" + 
-                   " STAGE_DEP=" + depStageDesc, false)
-      }
-    }
-  }
-  
-  // Generate indents and convert to String
-  protected def indentString(indent: Int) = {
-    val sb = new StringBuilder()
-    for (i <- 1 to indent) {
-      sb.append(" ")
-    }
-    sb.toString()
-  }
-  
-  protected def getRddName(rdd: RDD[_]) = {
-    var rddName = rdd.getClass.getName
-    if (rdd.name != null) {
-      rddName = rdd.name 
-    }
-    rddName
-  }
-  
-  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)
-      }
-    }
-  }
-  
-  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"
-    }
-    if (stage.jobId == jobID) {
-      jobLogInfo(jobID, indentString(indent) + stageInfo, false)
-      recordRddInStageGraph(jobID, stage.rdd, indent)
-      stage.parents.foreach(recordStageDepGraph(jobID, _, indent + 2))
-    } else
-      jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false)
-  }
-  
-  // Record task metrics into job log files
-  protected def recordTaskMetrics(stageID: Int, status: String, 
-                                taskInfo: TaskInfo, taskMetrics: TaskMetrics) {
-    val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + 
-               " 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 => ""
-      }
-    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))
-  }
-  
-  override def onStageCompleted(stageCompleted: StageCompleted) {
-    stageLogInfo(
-      stageCompleted.stageInfo.stage.id,
-      "STAGE_ID=%d STATUS=COMPLETED".format(stageCompleted.stageInfo.stage.id))
-    
-  }
-
-  override def onTaskStart(taskStart: SparkListenerTaskStart) { }
-
-  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
-    val task = taskEnd.task
-    val taskInfo = taskEnd.taskInfo
-    var taskStatus = ""
-    task match {
-      case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK"
-      case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK"
-    }
-    taskEnd.reason match {
-      case Success => taskStatus += " STATUS=SUCCESS"
-        recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskEnd.taskMetrics)
-      case Resubmitted => 
-        taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + 
-                      " STAGE_ID=" + task.stageId
-        stageLogInfo(task.stageId, taskStatus)
-      case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => 
-        taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + 
-                      task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + 
-                      mapId + " REDUCE_ID=" + reduceId
-        stageLogInfo(task.stageId, taskStatus)
-      case OtherFailure(message) => 
-        taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId + 
-                      " STAGE_ID=" + task.stageId + " INFO=" + message
-        stageLogInfo(task.stageId, taskStatus)
-      case _ =>
-    }
-  }
-  
-  override def onJobEnd(jobEnd: SparkListenerJobEnd) {
-    val job = jobEnd.job
-    var info = "JOB_ID=" + job.jobId
-    jobEnd.jobResult match {
-      case JobSucceeded => info += " STATUS=SUCCESS"
-      case JobFailed(exception, _) =>
-        info += " STATUS=FAILED REASON="
-        exception.getMessage.split("\\s+").foreach(info += _ + "_")
-      case _ =>
-    }
-    jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase)
-    closeLogWriter(job.jobId)
-  }
-
-  protected def recordJobProperties(jobID: Int, properties: Properties) {
-    if(properties != null) {
-      val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "")
-      jobLogInfo(jobID, description, false)
-    }
-  }
-
-  override def onJobStart(jobStart: SparkListenerJobStart) {
-    val job = jobStart.job
-    val properties = jobStart.properties
-    createLogWriter(job.jobId)
-    recordJobProperties(job.jobId, properties)
-    buildJobDep(job.jobId, job.finalStage)
-    recordStageDep(job.jobId)
-    recordStageDepGraph(job.jobId, job.finalStage)
-    jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED")
-  }
-}
+/*
+ * 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 java.io.{IOException, File, FileNotFoundException, PrintWriter}
+import java.text.SimpleDateFormat
+import java.util.{Date, Properties}
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.mutable.{HashMap, HashSet, ListBuffer}
+
+import org.apache.spark._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * A logger class to record runtime information for jobs in Spark. This class outputs one log file
+ * for each Spark job, containing RDD graph, tasks start/stop, shuffle information.
+ * JobLogger is a subclass of SparkListener, use addSparkListener to add JobLogger to a SparkContext
+ * after the SparkContext is created.
+ * Note that each JobLogger only works for one SparkContext
+ * @param logDirName The base directory for the log files.
+ */
+
+class JobLogger(val user: String, val logDirName: String)
+  extends SparkListener with Logging {
+
+  def this() = this(System.getProperty("user.name", "<unknown>"),
+    String.valueOf(System.currentTimeMillis()))
+
+  private val logDir =
+    if (System.getenv("SPARK_LOG_DIR") != null)
+      System.getenv("SPARK_LOG_DIR")
+    else
+      "/tmp/spark-%s".format(user)
+
+  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()
+
+  // 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 jobLogger */
+  protected def createLogDir() {
+    val dir = new File(logDir + "/" + logDirName + "/")
+    if (dir.exists()) {
+      return
+    }
+    if (dir.mkdirs() == false) {
+      // JobLogger should throw a exception rather than continue to construct this object.
+      throw new IOException("create log directory error:" + logDir + "/" + logDirName + "/")
+    }
+  }
+
+  /**
+   * Create a log file for one job
+   * @param jobID ID of the job
+   * @exception FileNotFoundException Fail to create log file
+   */
+  protected def createLogWriter(jobID: Int) {
+    try {
+      val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID)
+      jobIDToPrintWriter += (jobID -> fileWriter)
+    } catch {
+      case e: FileNotFoundException => e.printStackTrace()
+    }
+  }
+
+  /**
+   * Close log file, and clean the stage relationship in stageIDToJobID
+   * @param jobID ID of the job
+   */
+  protected def closeLogWriter(jobID: Int) {
+    jobIDToPrintWriter.get(jobID).foreach { fileWriter =>
+      fileWriter.close()
+      jobIDToStages.get(jobID).foreach(_.foreach{ stage =>
+        stageIDToJobID -= stage.id
+      })
+      jobIDToPrintWriter -= jobID
+      jobIDToStages -= jobID
+    }
+  }
+
+  /**
+   * Write info into log file
+   * @param jobID ID of the job
+   * @param info Info to be recorded
+   * @param withTime Controls whether to record time stamp before the info, default is true
+   */
+  protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) {
+    var writeInfo = info
+    if (withTime) {
+      val date = new Date(System.currentTimeMillis())
+      writeInfo = DATE_FORMAT.format(date) + ": " +info
+    }
+    jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo))
+  }
+
+  /**
+   * Write info into log file
+   * @param stageID ID of the stage
+   * @param info Info to be recorded
+   * @param withTime Controls whether to record time stamp before the info, default is true
+   */
+  protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) {
+    stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime))
+  }
+
+  /**
+   * Build stage dependency for a job
+   * @param jobID ID of the job
+   * @param stage Root stage of the job
+   */
+  protected def buildJobDep(jobID: Int, stage: Stage) {
+    if (stage.jobId == jobID) {
+      jobIDToStages.get(jobID) match {
+        case Some(stageList) => stageList += stage
+        case None => val stageList = new  ListBuffer[Stage]
+                     stageList += stage
+                     jobIDToStages += (jobID -> stageList)
+      }
+      stageIDToJobID += (stage.id -> jobID)
+      stage.parents.foreach(buildJobDep(jobID, _))
+    }
+  }
+
+  /**
+   * Record stage dependency and RDD dependency for a stage
+   * @param jobID Job ID of the stage
+   */
+  protected def recordStageDep(jobID: Int) {
+    def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = {
+      var rddList = new ListBuffer[RDD[_]]
+      rddList += rdd
+      rdd.dependencies.foreach {
+        case shufDep: ShuffleDependency[_, _] =>
+        case dep: Dependency[_] => rddList ++= getRddsInStage(dep.rdd)
+      }
+      rddList
+    }
+    jobIDToStages.get(jobID).foreach {_.foreach { stage =>
+        var depRddDesc: String = ""
+        getRddsInStage(stage.rdd).foreach { rdd =>
+          depRddDesc += rdd.id + ","
+        }
+        var depStageDesc: String = ""
+        stage.parents.foreach { stage =>
+          depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")"
+        }
+        jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" +
+                   depRddDesc.substring(0, depRddDesc.length - 1) + ")" +
+                   " STAGE_DEP=" + depStageDesc, false)
+      }
+    }
+  }
+
+  /**
+   * Generate indents and convert to String
+   * @param indent Number of indents
+   * @return string of indents
+   */
+  protected def indentString(indent: Int): String = {
+    val sb = new StringBuilder()
+    for (i <- 1 to indent) {
+      sb.append(" ")
+    }
+    sb.toString()
+  }
+
+  /**
+   * Get RDD's name
+   * @param rdd Input RDD
+   * @return String of RDD's name
+   */
+  protected def getRddName(rdd: RDD[_]): String = {
+    var rddName = rdd.getClass.getSimpleName
+    if (rdd.name != null) {
+      rddName = rdd.name
+    }
+    rddName
+  }
+
+  /**
+   * Record RDD dependency graph in a stage
+   * @param jobID Job ID of the stage
+   * @param rdd Root RDD of the stage
+   * @param indent Indent number before info
+   */
+  protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) {
+    val rddInfo =
+      if (rdd.getStorageLevel != StorageLevel.NONE) {
+        "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " CACHED" + " " +
+                rdd.origin + " " + rdd.generator
+      } else {
+        "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " NONE" + " " +
+                rdd.origin + " " + rdd.generator
+      }
+    jobLogInfo(jobID, indentString(indent) + rddInfo, false)
+    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)
+    }
+  }
+
+  /**
+   * Record stage dependency graph of a job
+   * @param jobID Job ID of the stage
+   * @param stage Root stage of the job
+   * @param indent Indent number before info, default is 0
+   */
+  protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) {
+    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)
+      if (!idSet.contains(stage.id)) {
+        idSet += stage.id
+        recordRddInStageGraph(jobID, stage.rdd, indent)
+        stage.parents.foreach(recordStageDepGraph(jobID, _, idSet, indent + 2))
+      }
+    } else {
+      jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false)
+    }
+  }
+
+  /**
+   * Record task metrics into job log files, including execution info and shuffle metrics
+   * @param stageID Stage ID of the task
+   * @param status Status info of the task
+   * @param taskInfo Task description info
+   * @param taskMetrics Task running metrics
+   */
+  protected def recordTaskMetrics(stageID: Int, status: String,
+                                taskInfo: TaskInfo, taskMetrics: TaskMetrics) {
+    val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID +
+               " 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 => ""
+    }
+    stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics)
+  }
+
+  /**
+   * When stage is submitted, record stage submit info
+   * @param stageSubmitted Stage submitted event
+   */
+  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) {
+    stageLogInfo(stageSubmitted.stage.stageId,"STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format(
+        stageSubmitted.stage.stageId, stageSubmitted.stage.numTasks))
+  }
+
+  /**
+   * When stage is completed, record stage completion status
+   * @param stageCompleted Stage completed event
+   */
+  override def onStageCompleted(stageCompleted: StageCompleted) {
+    stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format(
+        stageCompleted.stage.stageId))
+  }
+
+  override def onTaskStart(taskStart: SparkListenerTaskStart) { }
+
+  /**
+   * When task ends, record task completion status and metrics
+   * @param taskEnd Task end event
+   */
+  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
+    val task = taskEnd.task
+    val taskInfo = taskEnd.taskInfo
+    var taskStatus = ""
+    task match {
+      case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK"
+      case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK"
+    }
+    taskEnd.reason match {
+      case Success => taskStatus += " STATUS=SUCCESS"
+        recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskEnd.taskMetrics)
+      case Resubmitted =>
+        taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId +
+                      " STAGE_ID=" + task.stageId
+        stageLogInfo(task.stageId, taskStatus)
+      case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
+        taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" +
+                      task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" +
+                      mapId + " REDUCE_ID=" + reduceId
+        stageLogInfo(task.stageId, taskStatus)
+      case OtherFailure(message) =>
+        taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId +
+                      " STAGE_ID=" + task.stageId + " INFO=" + message
+        stageLogInfo(task.stageId, taskStatus)
+      case _ =>
+    }
+  }
+
+  /**
+   * When job ends, recording job completion status and close log file
+   * @param jobEnd Job end event
+   */
+  override def onJobEnd(jobEnd: SparkListenerJobEnd) {
+    val job = jobEnd.job
+    var info = "JOB_ID=" + job.jobId
+    jobEnd.jobResult match {
+      case JobSucceeded => info += " STATUS=SUCCESS"
+      case JobFailed(exception, _) =>
+        info += " STATUS=FAILED REASON="
+        exception.getMessage.split("\\s+").foreach(info += _ + "_")
+      case _ =>
+    }
+    jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase)
+    closeLogWriter(job.jobId)
+  }
+
+  /**
+   * Record job properties into job log file
+   * @param jobID ID of the job
+   * @param properties Properties of the job
+   */
+  protected def recordJobProperties(jobID: Int, properties: Properties) {
+    if(properties != null) {
+      val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "")
+      jobLogInfo(jobID, description, false)
+    }
+  }
+
+  /**
+   * When job starts, record job property and stage graph
+   * @param jobStart Job start event
+   */
+  override def onJobStart(jobStart: SparkListenerJobStart) {
+    val job = jobStart.job
+    val properties = jobStart.properties
+    createLogWriter(job.jobId)
+    recordJobProperties(job.jobId, properties)
+    buildJobDep(job.jobId, job.finalStage)
+    recordStageDep(job.jobId)
+    recordStageDepGraph(job.jobId, job.finalStage, new HashSet[Int])
+    jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED")
+  }
+}
+
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
index 200d8817990a42a7de74245a4d66a7b148778941..b026f860a8cd82ca6f61da52a68cd753c84ffd9d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
@@ -17,48 +17,59 @@
 
 package org.apache.spark.scheduler
 
-import scala.collection.mutable.ArrayBuffer
-
 /**
  * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their
  * results to the given handler function.
  */
-private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Unit)
+private[spark] class JobWaiter[T](
+    dagScheduler: DAGScheduler,
+    jobId: Int,
+    totalTasks: Int,
+    resultHandler: (Int, T) => Unit)
   extends JobListener {
 
   private var finishedTasks = 0
 
-  private var jobFinished = false          // Is the job as a whole finished (succeeded or failed)?
-  private var jobResult: JobResult = null  // If the job is finished, this will be its result
+  // Is the job as a whole finished (succeeded or failed)?
+  @volatile
+  private var _jobFinished = totalTasks == 0
 
-  override def taskSucceeded(index: Int, result: Any) {
-    synchronized {
-      if (jobFinished) {
-        throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter")
-      }
-      resultHandler(index, result.asInstanceOf[T])
-      finishedTasks += 1
-      if (finishedTasks == totalTasks) {
-        jobFinished = true
-        jobResult = JobSucceeded
-        this.notifyAll()
-      }
-    }
+  def jobFinished = _jobFinished
+
+  // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero
+  // partition RDDs), we set the jobResult directly to JobSucceeded.
+  private var jobResult: JobResult = if (jobFinished) JobSucceeded else null
+
+  /**
+   * Sends a signal to the DAGScheduler to cancel the job. The cancellation itself is handled
+   * asynchronously. After the low level scheduler cancels all the tasks belonging to this job, it
+   * will fail this job with a SparkException.
+   */
+  def cancel() {
+    dagScheduler.cancelJob(jobId)
   }
 
-  override def jobFailed(exception: Exception) {
-    synchronized {
-      if (jobFinished) {
-        throw new UnsupportedOperationException("jobFailed() called on a finished JobWaiter")
-      }
-      jobFinished = true
-      jobResult = JobFailed(exception, None)
+  override def taskSucceeded(index: Int, result: Any): Unit = synchronized {
+    if (_jobFinished) {
+      throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter")
+    }
+    resultHandler(index, result.asInstanceOf[T])
+    finishedTasks += 1
+    if (finishedTasks == totalTasks) {
+      _jobFinished = true
+      jobResult = JobSucceeded
       this.notifyAll()
     }
   }
 
+  override def jobFailed(exception: Exception): Unit = synchronized {
+    _jobFinished = true
+    jobResult = JobFailed(exception, None)
+    this.notifyAll()
+  }
+
   def awaitResult(): JobResult = synchronized {
-    while (!jobFinished) {
+    while (!_jobFinished) {
       this.wait()
     }
     return jobResult
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
index 9eb8d4850169677492bd37d00cc1ab1057822ae3..596f9adde94938ad6a9c092aeb0d34cb937b665e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
@@ -43,7 +43,10 @@ private[spark] class Pool(
   var runningTasks = 0
 
   var priority = 0
-  var stageId = 0
+
+  // A pool's stage id is used to break the tie in scheduling.
+  var stageId = -1
+
   var name = poolName
   var parent: Pool = null
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 07e8317e3aed83a819f3ccf113e809847ede7185..310ec62ca8afe3a25264bb1d3368c06e86ea4f1e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -23,7 +23,7 @@ import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.RDDCheckpointData
-import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap}
 
 private[spark] object ResultTask {
 
@@ -32,23 +32,23 @@ private[spark] object ResultTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner("ResultTask", serializedInfoCache.clearOldValues)
+  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
     synchronized {
       val old = serializedInfoCache.get(stageId).orNull
       if (old != null) {
-        return old
+        old
       } else {
         val out = new ByteArrayOutputStream
-        val ser = SparkEnv.get.closureSerializer.newInstance
+        val ser = SparkEnv.get.closureSerializer.newInstance()
         val objOut = ser.serializeStream(new GZIPOutputStream(out))
         objOut.writeObject(rdd)
         objOut.writeObject(func)
         objOut.close()
         val bytes = out.toByteArray
         serializedInfoCache.put(stageId, bytes)
-        return bytes
+        bytes
       }
     }
   }
@@ -56,11 +56,11 @@ private[spark] object ResultTask {
   def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = {
     val loader = Thread.currentThread.getContextClassLoader
     val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
-    val ser = SparkEnv.get.closureSerializer.newInstance
+    val ser = SparkEnv.get.closureSerializer.newInstance()
     val objIn = ser.deserializeStream(in)
     val rdd = objIn.readObject().asInstanceOf[RDD[_]]
     val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _]
-    return (rdd, func)
+    (rdd, func)
   }
 
   def clearCache() {
@@ -71,29 +71,37 @@ private[spark] object ResultTask {
 }
 
 
+/**
+ * A task that sends back the output to the driver application.
+ *
+ * See [[org.apache.spark.scheduler.Task]] for more information.
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param rdd input to func
+ * @param func a function to apply on a partition of the RDD
+ * @param _partitionId index of the number in the RDD
+ * @param locs preferred task execution locations for locality scheduling
+ * @param outputId index of the task in this job (a job can launch tasks on only a subset of the
+ *                 input RDD's partitions).
+ */
 private[spark] class ResultTask[T, U](
     stageId: Int,
     var rdd: RDD[T],
     var func: (TaskContext, Iterator[T]) => U,
-    var partition: Int,
+    _partitionId: Int,
     @transient locs: Seq[TaskLocation],
     var outputId: Int)
-  extends Task[U](stageId) with Externalizable {
+  extends Task[U](stageId, _partitionId) with Externalizable {
 
   def this() = this(0, null, null, 0, null, 0)
 
-  var split = if (rdd == null) {
-    null
-  } else {
-    rdd.partitions(partition)
-  }
+  var split = if (rdd == null) null else rdd.partitions(partitionId)
 
   @transient private val preferredLocs: Seq[TaskLocation] = {
     if (locs == null) Nil else locs.toSet.toSeq
   }
 
-  override def run(attemptId: Long): U = {
-    val context = new TaskContext(stageId, partition, attemptId, runningLocally = false)
+  override def runTask(context: TaskContext): U = {
     metrics = Some(context.taskMetrics)
     try {
       func(context, rdd.iterator(split, context))
@@ -104,17 +112,17 @@ private[spark] class ResultTask[T, U](
 
   override def preferredLocations: Seq[TaskLocation] = preferredLocs
 
-  override def toString = "ResultTask(" + stageId + ", " + partition + ")"
+  override def toString = "ResultTask(" + stageId + ", " + partitionId + ")"
 
   override def writeExternal(out: ObjectOutput) {
     RDDCheckpointData.synchronized {
-      split = rdd.partitions(partition)
+      split = rdd.partitions(partitionId)
       out.writeInt(stageId)
       val bytes = ResultTask.serializeInfo(
         stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
       out.writeInt(bytes.length)
       out.write(bytes)
-      out.writeInt(partition)
+      out.writeInt(partitionId)
       out.writeInt(outputId)
       out.writeLong(epoch)
       out.writeObject(split)
@@ -129,7 +137,7 @@ private[spark] class ResultTask[T, U](
     val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes)
     rdd = rdd_.asInstanceOf[RDD[T]]
     func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
-    partition = in.readInt()
+    partitionId = in.readInt()
     outputId = in.readInt()
     epoch = in.readLong()
     split = in.readObject().asInstanceOf[Partition]
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
index 4e25086ec91e03e7fa0f6e25d4c5e9e55fecca23..356fe56bf32c73c70f2f67ad2c5502957577fd80 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -30,7 +30,10 @@ import scala.xml.XML
  * addTaskSetManager: build the leaf nodes(TaskSetManagers)
  */
 private[spark] trait SchedulableBuilder {
+  def rootPool: Pool
+
   def buildPools()
+
   def addTaskSetManager(manager: Schedulable, properties: Properties)
 }
 
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 d23df0dd2b0f198680f7ef7c1857845c10c405b5..0f2deb4bcbbb207b7757cd183896ceb3dc7b2ff9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap
 import org.apache.spark._
 import org.apache.spark.executor.ShuffleWriteMetrics
 import org.apache.spark.storage._
-import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner}
+import org.apache.spark.util.{MetadataCleanerType, TimeStampedHashMap, MetadataCleaner}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.RDDCheckpointData
 
@@ -37,7 +37,7 @@ private[spark] object ShuffleMapTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner("ShuffleMapTask", serializedInfoCache.clearOldValues)
+  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
     synchronized {
@@ -53,7 +53,7 @@ private[spark] object ShuffleMapTask {
         objOut.close()
         val bytes = out.toByteArray
         serializedInfoCache.put(stageId, bytes)
-        return bytes
+        bytes
       }
     }
   }
@@ -66,7 +66,7 @@ private[spark] object ShuffleMapTask {
       val objIn = ser.deserializeStream(in)
       val rdd = objIn.readObject().asInstanceOf[RDD[_]]
       val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]]
-      return (rdd, dep)
+      (rdd, dep)
     }
   }
 
@@ -75,7 +75,7 @@ private[spark] object ShuffleMapTask {
     val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
     val objIn = new ObjectInputStream(in)
     val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap
-    return (HashMap(set.toSeq: _*))
+    HashMap(set.toSeq: _*)
   }
 
   def clearCache() {
@@ -85,13 +85,25 @@ private[spark] object ShuffleMapTask {
   }
 }
 
+/**
+ * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner
+ * specified in the ShuffleDependency).
+ *
+ * See [[org.apache.spark.scheduler.Task]] for more information.
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param rdd the final RDD in this stage
+ * @param dep the ShuffleDependency
+ * @param _partitionId index of the number in the RDD
+ * @param locs preferred task execution locations for locality scheduling
+ */
 private[spark] class ShuffleMapTask(
     stageId: Int,
     var rdd: RDD[_],
     var dep: ShuffleDependency[_,_],
-    var partition: Int,
+    _partitionId: Int,
     @transient private var locs: Seq[TaskLocation])
-  extends Task[MapStatus](stageId)
+  extends Task[MapStatus](stageId, _partitionId)
   with Externalizable
   with Logging {
 
@@ -101,16 +113,16 @@ private[spark] class ShuffleMapTask(
     if (locs == null) Nil else locs.toSet.toSeq
   }
 
-  var split = if (rdd == null) null else rdd.partitions(partition)
+  var split = if (rdd == null) null else rdd.partitions(partitionId)
 
   override def writeExternal(out: ObjectOutput) {
     RDDCheckpointData.synchronized {
-      split = rdd.partitions(partition)
+      split = rdd.partitions(partitionId)
       out.writeInt(stageId)
       val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
       out.writeInt(bytes.length)
       out.write(bytes)
-      out.writeInt(partition)
+      out.writeInt(partitionId)
       out.writeLong(epoch)
       out.writeObject(split)
     }
@@ -124,68 +136,73 @@ private[spark] class ShuffleMapTask(
     val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes)
     rdd = rdd_
     dep = dep_
-    partition = in.readInt()
+    partitionId = in.readInt()
     epoch = in.readLong()
     split = in.readObject().asInstanceOf[Partition]
   }
 
-  override def run(attemptId: Long): MapStatus = {
+  override def runTask(context: TaskContext): MapStatus = {
     val numOutputSplits = dep.partitioner.numPartitions
-
-    val taskContext = new TaskContext(stageId, partition, attemptId, runningLocally = false)
-    metrics = Some(taskContext.taskMetrics)
+    metrics = Some(context.taskMetrics)
 
     val blockManager = SparkEnv.get.blockManager
-    var shuffle: ShuffleBlocks = null
-    var buckets: ShuffleWriterGroup = null
+    val shuffleBlockManager = blockManager.shuffleBlockManager
+    var shuffle: ShuffleWriterGroup = null
+    var success = false
 
     try {
       // Obtain all the block writers for shuffle blocks.
       val ser = SparkEnv.get.serializerManager.get(dep.serializerClass)
-      shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser)
-      buckets = shuffle.acquireWriters(partition)
+      shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser)
 
       // Write the map output to its associated buckets.
-      for (elem <- rdd.iterator(split, taskContext)) {
+      for (elem <- rdd.iterator(split, context)) {
         val pair = elem.asInstanceOf[Product2[Any, Any]]
         val bucketId = dep.partitioner.getPartition(pair._1)
-        buckets.writers(bucketId).write(pair)
+        shuffle.writers(bucketId).write(pair)
       }
 
       // Commit the writes. Get the size of each bucket block (total block size).
       var totalBytes = 0L
-      val compressedSizes: Array[Byte] = buckets.writers.map { writer: BlockObjectWriter =>
+      var totalTime = 0L
+      val compressedSizes: Array[Byte] = shuffle.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)
 
-      return new MapStatus(blockManager.blockManagerId, compressedSizes)
+      success = true
+      new MapStatus(blockManager.blockManagerId, compressedSizes)
     } catch { case e: Exception =>
       // If there is an exception from running the task, revert the partial writes
       // and throw the exception upstream to Spark.
-      if (buckets != null) {
-        buckets.writers.foreach(_.revertPartialWrites())
+      if (shuffle != null && shuffle.writers != null) {
+        for (writer <- shuffle.writers) {
+          writer.revertPartialWrites()
+          writer.close()
+        }
       }
       throw e
     } finally {
       // Release the writers back to the shuffle block manager.
-      if (shuffle != null && buckets != null) {
-        shuffle.releaseWriters(buckets)
+      if (shuffle != null && shuffle.writers != null) {
+        shuffle.releaseWriters(success)
       }
       // Execute the callbacks on task completion.
-      taskContext.executeOnCompleteCallbacks()
+      context.executeOnCompleteCallbacks()
     }
   }
 
   override def preferredLocations: Seq[TaskLocation] = preferredLocs
 
-  override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition)
+  override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partitionId)
 }
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 62b521ad45d39c6854f2eae64c4971e02f9de350..3841b5616dca24471a5d3e85baab617308a24f0a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -24,17 +24,20 @@ 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
 
-case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null)
+case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null)
      extends SparkListenerEvents
 
 case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult)
@@ -54,7 +57,13 @@ trait SparkListener {
   /**
    * Called when a task starts
    */
-  def onTaskStart(taskEnd: SparkListenerTaskStart) { }
+  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..e9f2198a007e526a237f7190e542761bc191c8af 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,17 @@ 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
+  var emittedTaskSizeWarning = false
 }
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 598d91752a31e81b715cb324c0e348e8aedd518f..69b42e86eae3e3dd4c737355d143b10bb18185c3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -17,25 +17,74 @@
 
 package org.apache.spark.scheduler
 
-import org.apache.spark.serializer.SerializerInstance
 import java.io.{DataInputStream, DataOutputStream}
 import java.nio.ByteBuffer
-import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-import org.apache.spark.util.ByteBufferInputStream
+
 import scala.collection.mutable.HashMap
+
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+
+import org.apache.spark.TaskContext
 import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.serializer.SerializerInstance
+import org.apache.spark.util.ByteBufferInputStream
+
 
 /**
- * A task to execute on a worker node.
+ * A unit of execution. We have two kinds of Task's in Spark:
+ * - [[org.apache.spark.scheduler.ShuffleMapTask]]
+ * - [[org.apache.spark.scheduler.ResultTask]]
+ *
+ * A Spark job consists of one or more stages. The very last stage in a job consists of multiple
+ * ResultTask's, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task
+ * and sends the task output back to the driver application. A ShuffleMapTask executes the task
+ * and divides the task output to multiple buckets (based on the task's partitioner).
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param partitionId index of the number in the RDD
  */
-private[spark] abstract class Task[T](val stageId: Int) extends Serializable {
-  def run(attemptId: Long): T
+private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable {
+
+  final def run(attemptId: Long): T = {
+    context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false)
+    if (_killed) {
+      kill()
+    }
+    runTask(context)
+  }
+
+  def runTask(context: TaskContext): T
+
   def preferredLocations: Seq[TaskLocation] = Nil
 
-  var epoch: Long = -1   // Map output tracker epoch. Will be set by TaskScheduler.
+  // Map output tracker epoch. Will be set by TaskScheduler.
+  var epoch: Long = -1
 
   var metrics: Option[TaskMetrics] = None
 
+  // Task context, to be initialized in run().
+  @transient protected var context: TaskContext = _
+
+  // A flag to indicate whether the task is killed. This is used in case context is not yet
+  // initialized when kill() is invoked.
+  @volatile @transient private var _killed = false
+
+  /**
+   * Whether the task has been killed.
+   */
+  def killed: Boolean = _killed
+
+  /**
+   * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark
+   * code and user code to properly handle the flag. This function should be idempotent so it can
+   * be called multiple times.
+   */
+  def kill() {
+    _killed = true
+    if (context != null) {
+      context.interrupted = true
+    }
+  }
 }
 
 /**
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..3c22edd5248f403190a2f543597728d08dba92a2 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,27 @@ 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
 
+  var serializedSize: Int = 0
+
+  def markGettingResult(time: Long = System.currentTimeMillis) {
+    gettingResultTime = time
+  }
+
   def markSuccessful(time: Long = System.currentTimeMillis) {
     finishTime = time
   }
@@ -43,6 +61,8 @@ class TaskInfo(
     failed = true
   }
 
+  def gettingResult: Boolean = gettingResultTime != 0
+
   def finished: Boolean = finishTime != 0
 
   def successful: Boolean = finished && !failed
@@ -52,6 +72,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/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
index db3954a9d3ef0a205ea8f0bee7acf98927a3b985..7e468d0d678081c3c0ab3350c22f51993e14afe1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
@@ -24,13 +24,14 @@ import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.{SparkEnv}
 import java.nio.ByteBuffer
 import org.apache.spark.util.Utils
+import org.apache.spark.storage.BlockId
 
 // Task result. Also contains updates to accumulator variables.
 private[spark] sealed trait TaskResult[T]
 
 /** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */
 private[spark]
-case class IndirectTaskResult[T](val blockId: String) extends TaskResult[T] with Serializable
+case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Serializable
 
 /** A TaskResult that contains the task's return value and accumulator updates. */
 private[spark]
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 7c2a9f03d7ad894fcda8050da685c816738285a3..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 {
 
@@ -45,8 +44,11 @@ private[spark] trait TaskScheduler {
   // Submit a sequence of tasks to run.
   def submitTasks(taskSet: TaskSet): Unit
 
-  // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called.
-  def setListener(listener: TaskSchedulerListener): Unit
+  // Cancel a stage.
+  def cancelTasks(stageId: Int)
+
+  // 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/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
index c3ad325156f53786224a58fd48e50697c9c94344..03bf76083761f063ad0a015d9dd6173195e8f323 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
@@ -31,5 +31,9 @@ private[spark] class TaskSet(
     val properties: Properties) {
     val id: String = stageId + "." + attempt
 
+  def kill() {
+    tasks.foreach(_.kill())
+  }
+
   override def toString: String = "TaskSet " + id
 }
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 1a844b7e7ee4b1d356dfd515dec31726bc7d86ac..66ab8ea4cdf57a084e308554e2d870f8cb759c8c 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
@@ -17,7 +17,6 @@
 
 package org.apache.spark.scheduler.cluster
 
-import java.lang.{Boolean => JBoolean}
 import java.nio.ByteBuffer
 import java.util.concurrent.atomic.AtomicLong
 import java.util.{TimerTask, Timer}
@@ -25,6 +24,7 @@ import java.util.{TimerTask, Timer}
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
+import scala.concurrent.duration._
 
 import org.apache.spark._
 import org.apache.spark.TaskState.TaskState
@@ -79,14 +79,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
 
   private val executorIdToHost = new HashMap[String, String]
 
-  // JAR server, if any JARs were added by the user to the SparkContext
-  var jarServer: HttpServer = null
-
-  // URIs of JARs to pass to executor
-  var jarUris: String = ""
-
   // Listener object to pass upcalls into
-  var listener: TaskSchedulerListener = null
+  var dagScheduler: DAGScheduler = null
 
   var backend: SchedulerBackend = null
 
@@ -101,12 +95,12 @@ 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) {
-    backend = context
+  def initialize(backend: SchedulerBackend) {
+    this.backend = backend
     // temporarily set rootPool name to empty
     rootPool = new Pool("", schedulingMode, 0, 0)
     schedulableBuilder = {
@@ -126,21 +120,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     backend.start()
 
     if (System.getProperty("spark.speculation", "false").toBoolean) {
-      new Thread("ClusterScheduler speculation check") {
-        setDaemon(true)
-
-        override def run() {
-          logInfo("Starting speculative execution thread")
-          while (true) {
-            try {
-              Thread.sleep(SPECULATION_INTERVAL)
-            } catch {
-              case e: InterruptedException => {}
-            }
-            checkSpeculatableTasks()
-          }
-        }
-      }.start()
+      logInfo("Starting speculative execution thread")
+      import sc.env.actorSystem.dispatcher
+      sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
+            SPECULATION_INTERVAL milliseconds) {
+        checkSpeculatableTasks()
+      }
     }
   }
 
@@ -171,8 +156,33 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     backend.reviveOffers()
   }
 
-  def taskSetFinished(manager: TaskSetManager) {
-    this.synchronized {
+  override def cancelTasks(stageId: Int): Unit = synchronized {
+    logInfo("Cancelling stage " + stageId)
+    activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
+      // There are two possible cases here:
+      // 1. The task set manager has been created and some tasks have been scheduled.
+      //    In this case, send a kill signal to the executors to kill the task and then abort
+      //    the stage.
+      // 2. The task set manager has been created but no tasks has been scheduled. In this case,
+      //    simply abort the stage.
+      val taskIds = taskSetTaskIds(tsm.taskSet.id)
+      if (taskIds.size > 0) {
+        taskIds.foreach { tid =>
+          val execId = taskIdToExecutorId(tid)
+          backend.killTask(tid, execId)
+        }
+      }
+      logInfo("Stage %d was cancelled".format(stageId))
+      tsm.removeAllRunningTasks()
+      taskSetFinished(tsm)
+    }
+  }
+
+  def taskSetFinished(manager: TaskSetManager): Unit = synchronized {
+    // Check to see if the given task set has been removed. This is possible in the case of
+    // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has
+    // more than one running tasks).
+    if (activeTaskSets.contains(manager.taskSet.id)) {
       activeTaskSets -= manager.taskSet.id
       manager.parent.removeSchedulable(manager)
       logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name))
@@ -240,7 +250,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
 
   def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
     var failedExecutor: Option[String] = None
-    var taskFailed = false
     synchronized {
       try {
         if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) {
@@ -260,9 +269,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
               }
               taskIdToExecutorId.remove(tid)
             }
-            if (state == TaskState.FAILED) {
-              taskFailed = true
-            }
             activeTaskSets.get(taskSetId).foreach { taskSet =>
               if (state == TaskState.FINISHED) {
                 taskSet.removeRunningTask(tid)
@@ -281,15 +287,15 @@ 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)
-      backend.reviveOffers()
-    }
-    if (taskFailed) {
-      // Also revive offers if a task had failed for some reason other than host lost
+      dagScheduler.executorLost(failedExecutor.get)
       backend.reviveOffers()
     }
   }
 
+  def handleTaskGettingResult(taskSetManager: ClusterTaskSetManager, tid: Long) {
+    taskSetManager.handleTaskGettingResult(tid)
+  }
+
   def handleSuccessfulTask(
     taskSetManager: ClusterTaskSetManager,
     tid: Long,
@@ -303,8 +309,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     taskState: TaskState,
     reason: Option[TaskEndReason]) = synchronized {
     taskSetManager.handleFailedTask(tid, taskState, reason)
-    if (taskState == TaskState.FINISHED) {
-      // The task finished successfully but the result was lost, so we should revive offers.
+    if (taskState != TaskState.KILLED) {
+      // Need to revive offers again now that the task set manager state has been updated to
+      // reflect failed tasks that need to be re-run.
       backend.reviveOffers()
     }
   }
@@ -334,9 +341,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     if (backend != null) {
       backend.stop()
     }
-    if (jarServer != null) {
-      jarServer.stop()
-    }
     if (taskResultGetter != null) {
       taskResultGetter.stop()
     }
@@ -384,9 +388,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()
     }
   }
@@ -405,7 +409,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 194ab55102406fcefb321b288e6a593b3b584733..94961790dfe40e8a39e0f4bb114db5a0a75cc54c 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
@@ -17,18 +17,17 @@
 
 package org.apache.spark.scheduler.cluster
 
-import java.nio.ByteBuffer
-import java.util.{Arrays, NoSuchElementException}
+import java.io.NotSerializableException
+import java.util.Arrays
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 import scala.math.max
 import scala.math.min
-import scala.Some
 
 import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv,
-  SparkException, Success, TaskEndReason, TaskResultLost, TaskState}
+  Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState}
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.scheduler._
 import org.apache.spark.util.{SystemClock, Clock}
@@ -378,6 +377,7 @@ private[spark] class ClusterTaskSetManager(
           logInfo("Serialized task %s:%d as %d bytes in %d ms".format(
             taskSet.id, index, serializedTask.limit, timeTaken))
           val taskName = "task %s:%d".format(taskSet.id, index)
+          info.serializedSize = serializedTask.limit
           if (taskAttempts(index).size == 1)
             taskStarted(task,info)
           return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask))
@@ -417,11 +417,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)
@@ -431,7 +437,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.
@@ -447,7 +453,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)
@@ -458,54 +465,65 @@ private[spark] class ClusterTaskSetManager(
     val index = info.index
     info.markFailed()
     if (!successful(index)) {
-      logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index))
+      logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index))
       copiesRunning(index) -= 1
       // Check if the problem is a map output fetch failure. In that case, this
       // task will never succeed on any node, so tell the scheduler about it.
       reason.foreach {
-        _ match {
-          case fetchFailed: FetchFailed =>
-            logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress)
-            sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null)
-            successful(index) = true
-            tasksSuccessful += 1
-            sched.taskSetFinished(this)
-            removeAllRunningTasks()
+        case fetchFailed: FetchFailed =>
+          logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress)
+          sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null)
+          successful(index) = true
+          tasksSuccessful += 1
+          sched.taskSetFinished(this)
+          removeAllRunningTasks()
+          return
+
+        case TaskKilled =>
+          logWarning("Task %d was killed.".format(tid))
+          sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null)
+          return
+
+        case ef: ExceptionFailure =>
+          sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
+          if (ef.className == classOf[NotSerializableException].getName()) {
+            // If the task result wasn't serializable, there's no point in trying to re-execute it.
+            logError("Task %s:%s had a not serializable result: %s; not retrying".format(
+              taskSet.id, index, ef.description))
+            abort("Task %s:%s had a not serializable result: %s".format(
+              taskSet.id, index, ef.description))
             return
-
-          case ef: ExceptionFailure =>
-            sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
-            val key = ef.description
-            val now = clock.getTime()
-            val (printFull, dupCount) = {
-              if (recentExceptions.contains(key)) {
-                val (dupCount, printTime) = recentExceptions(key)
-                if (now - printTime > EXCEPTION_PRINT_INTERVAL) {
-                  recentExceptions(key) = (0, now)
-                  (true, 0)
-                } else {
-                  recentExceptions(key) = (dupCount + 1, printTime)
-                  (false, dupCount + 1)
-                }
-              } else {
+          }
+          val key = ef.description
+          val now = clock.getTime()
+          val (printFull, dupCount) = {
+            if (recentExceptions.contains(key)) {
+              val (dupCount, printTime) = recentExceptions(key)
+              if (now - printTime > EXCEPTION_PRINT_INTERVAL) {
                 recentExceptions(key) = (0, now)
                 (true, 0)
+              } else {
+                recentExceptions(key) = (dupCount + 1, printTime)
+                (false, dupCount + 1)
               }
-            }
-            if (printFull) {
-              val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
-              logInfo("Loss was due to %s\n%s\n%s".format(
-                ef.className, ef.description, locs.mkString("\n")))
             } else {
-              logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount))
+              recentExceptions(key) = (0, now)
+              (true, 0)
             }
+          }
+          if (printFull) {
+            val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
+            logWarning("Loss was due to %s\n%s\n%s".format(
+              ef.className, ef.description, locs.mkString("\n")))
+          } else {
+            logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount))
+          }
 
-          case TaskResultLost =>
-            logInfo("Lost result for TID %s on host %s".format(tid, info.host))
-            sched.listener.taskEnded(tasks(index), TaskResultLost, null, null, info, null)
+        case TaskResultLost =>
+          logWarning("Lost result for TID %s on host %s".format(tid, info.host))
+          sched.dagScheduler.taskEnded(tasks(index), TaskResultLost, null, null, info, null)
 
-          case _ => {}
-        }
+        case _ => {}
       }
       // On non-fetch failures, re-enqueue the task as pending for a max number of retries
       addPendingTask(index)
@@ -532,7 +550,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)
   }
@@ -556,7 +574,7 @@ private[spark] class ClusterTaskSetManager(
     runningTasks = runningTasksSet.size
   }
 
-  private def removeAllRunningTasks() {
+  private[cluster] def removeAllRunningTasks() {
     val numRunningTasks = runningTasksSet.size
     runningTasksSet.clear()
     if (parent != null) {
@@ -605,7 +623,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)
         }
       }
     }
@@ -630,11 +648,11 @@ private[spark] class ClusterTaskSetManager(
     var foundTasks = false
     val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
     logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)
-    if (tasksSuccessful >= minFinishedForSpeculation) {
+    if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) {
       val time = clock.getTime()
       val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
       Arrays.sort(durations)
-      val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1))
+      val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1))
       val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100)
       // TODO: Threshold should also look at standard deviation of task durations and have a lower
       // bound based on that.
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 c0b836bf1a021737174323b65f24ff71048541bc..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,26 +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 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 */
@@ -54,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 CoarseGrainedClusterMessage
+
+  case object StopExecutor extends CoarseGrainedClusterMessage
 
-  case object StopDriver extends StandaloneClusterMessage
+  case object StopExecutors extends CoarseGrainedClusterMessage
 
-  case class RemoveExecutor(executorId: String, reason: String) extends StandaloneClusterMessage
+  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 71%
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 b6f0ec961a69c14d19c208e124274eaabd7e7986..f5e8766f6d7ad3c4a143aa5a60a151ed9821ab36 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
@@ -25,20 +25,23 @@ import scala.concurrent.duration._
 
 import akka.actor._
 import akka.pattern.ask
-import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
+import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 
 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
@@ -49,7 +52,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     private val executorAddress = new HashMap[String, Address]
     private val executorHost = new HashMap[String, String]
     private val freeCores = new HashMap[String, Int]
-    private val actorToExecutorId = new HashMap[ActorRef, String]
     private val addressToExecutorId = new HashMap[Address, String]
 
     override def preStart() {
@@ -70,12 +72,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
         } else {
           logInfo("Registered executor: " + sender + " with ID " + executorId)
           sender ! RegisteredExecutor(sparkProperties)
-          context.watch(sender)
           executorActor(executorId) = sender
           executorHost(executorId) = Utils.parseHostPort(hostPort)._1
           freeCores(executorId) = cores
           executorAddress(executorId) = sender.path.address
-          actorToExecutorId(sender) = executorId
           addressToExecutorId(sender.path.address) = executorId
           totalCoreCount.addAndGet(cores)
           makeOffers()
@@ -84,29 +84,40 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
       case StatusUpdate(executorId, taskId, state, data) =>
         scheduler.statusUpdate(taskId, state, data.value)
         if (TaskState.isFinished(state)) {
-          freeCores(executorId) += 1
-          makeOffers(executorId)
+          if (executorActor.contains(executorId)) {
+            freeCores(executorId) += 1
+            makeOffers(executorId)
+          } else {
+            // Ignoring the update since we don't know about the executor.
+            val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s"
+            logWarning(msg.format(taskId, state, sender, executorId))
+          }
         }
 
       case ReviveOffers =>
         makeOffers()
 
+      case KillTask(taskId, executorId) =>
+        executorActor(executorId) ! KillTask(taskId, executorId)
+
       case StopDriver =>
         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
 
-      case Terminated(actor) =>
-        actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
+      case DisassociatedEvent(_, address, _) => 
+        addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated"))
 
-      case DisassociatedEvent(_, remoteAddress, _) =>
-        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
-
-      case AssociationErrorEvent(_, _, remoteAddress, _) =>
-        addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown"))
     }
 
     // Make fake resource offers on all executors
@@ -134,7 +145,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
       if (executorActor.contains(executorId)) {
         logInfo("Executor " + executorId + " disconnected, so removing it")
         val numCores = freeCores(executorId)
-        actorToExecutorId -= executorActor(executorId)
         addressToExecutorId -= executorAddress(executorId)
         executorActor -= executorId
         executorHost -= executorId
@@ -159,16 +169,32 @@ 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")
   }
 
-  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() {
+    stopExecutors()
     try {
       if (driverActor != null) {
         val future = driverActor.ask(StopDriver)(timeout)
-        Await.result(future, timeout)
+        Await.ready(future, timeout)
       }
     } catch {
       case e: Exception =>
@@ -180,6 +206,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     driverActor ! ReviveOffers
   }
 
+  override def killTask(taskId: Long, executorId: String) {
+    driverActor ! KillTask(taskId, executorId)
+  }
+
   override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
       .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
 
@@ -187,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)
@@ -195,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/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
index d57eb3276f7a186a80b0fe40fe2296b86f5eb0ab..5367218faa685cdbe15c79c6f00de8b7fbf35c21 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.scheduler.cluster
 
-import org.apache.spark.{SparkContext}
+import org.apache.spark.SparkContext
 
 /**
  * A backend interface for cluster scheduling systems that allows plugging in different ones under
@@ -30,8 +30,8 @@ private[spark] trait SchedulerBackend {
   def reviveOffers(): Unit
   def defaultParallelism(): Int
 
+  def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException
+
   // Memory used by each executor (in megabytes)
   protected val executorMemory: Int = SparkContext.executorMemoryRequested
-
-  // TODO: Probably want to add a killTask too
 }
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..e8fecec4a64f30cb2e78d49ced075f4a30dd28d7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.tcp://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)
+    logInfo("Writing Spark UI Address: " + sc.ui.appUIAddress)
+
+    // 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.writeUTF(sc.ui.appUIAddress)
+    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.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 fa83ae19d6108c9acd73f0fc41adc0f4ff20c7e2..7127a72d6d1e5ac6a3ce16e86a97a5a8e63099c6 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
@@ -26,9 +26,9 @@ import org.apache.spark.util.Utils
 private[spark] class SparkDeploySchedulerBackend(
     scheduler: ClusterScheduler,
     sc: SparkContext,
-    master: String,
+    masters: Array[String],
     appName: String)
-  extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
   with ClientListener
   with Logging {
 
@@ -44,15 +44,15 @@ private[spark] class SparkDeploySchedulerBackend(
     // The endpoint for executors to talk to us
     val driverUrl = "akka.tcp://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)
 
-    client = new Client(sc.env.actorSystem, master, appDesc, this)
+    client = new Client(sc.env.actorSystem, masters, appDesc, this)
     client.start()
   }
 
@@ -71,8 +71,14 @@ private[spark] class SparkDeploySchedulerBackend(
 
   override def disconnected() {
     if (!stopping) {
-      logError("Disconnected from Spark cluster!")
-      scheduler.error("Disconnected from Spark cluster")
+      logWarning("Disconnected from Spark cluster! Waiting for reconnection...")
+    }
+  }
+
+  override def dead() {
+    if (!stopping) {
+      logError("Spark cluster looks dead, giving up.")
+      scheduler.error("Spark cluster looks down")
     }
   }
 
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 b2a8f0647217b10e5a3440809e4c54ea93d20b1c..e68c5277135f697bef29b24719f236aa904d7f04 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 bf4040fafce46fe88b8b8cb2ff1819afe9e67551..84fe3094cc7bab94b50f45bd69f22cc54ab6138f 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.tcp://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()
@@ -180,6 +181,7 @@ private[spark] class CoarseMesosSchedulerBackend(
             !slaveIdsWithExecutors.contains(slaveId)) {
           // Launch an executor on the slave
           val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired)
+          totalCoresAcquired += cpusToUse
           val taskId = newMesosTaskId()
           taskIdToSlaveId(taskId) = slaveId
           slaveIdsWithExecutors += slaveId
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 4d1bb1c63973c26069809f0ca7ea044eb673c4f1..01e95162c0f70b2d5fd9305799c6ff774d998258 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
@@ -17,23 +17,19 @@
 
 package org.apache.spark.scheduler.local
 
-import java.io.File
-import java.lang.management.ManagementFactory
-import java.util.concurrent.atomic.AtomicInteger
 import java.nio.ByteBuffer
+import java.util.concurrent.atomic.AtomicInteger
 
-import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import akka.actor._
 
 import org.apache.spark._
 import org.apache.spark.TaskState.TaskState
-import org.apache.spark.executor.ExecutorURLClassLoader
+import org.apache.spark.executor.{Executor, ExecutorBackend}
 import org.apache.spark.scheduler._
 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
-import akka.actor._
-import org.apache.spark.util.Utils
+
 
 /**
  * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally
@@ -41,52 +37,57 @@ import org.apache.spark.util.Utils
  * testing fault recovery.
  */
 
-private[spark]
+private[local]
 case class LocalReviveOffers()
 
-private[spark]
+private[local]
 case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer)
 
+private[local]
+case class KillTask(taskId: Long)
+
 private[spark]
-class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging {
+class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int)
+  extends Actor with Logging {
+
+  val executor = new Executor("localhost", "localhost", Seq.empty, isLocal = true)
 
   def receive = {
     case LocalReviveOffers =>
       launchTask(localScheduler.resourceOffer(freeCores))
+
     case LocalStatusUpdate(taskId, state, serializeData) =>
-      freeCores += 1
-      localScheduler.statusUpdate(taskId, state, serializeData)
-      launchTask(localScheduler.resourceOffer(freeCores))
+      if (TaskState.isFinished(state)) {
+        freeCores += 1
+        launchTask(localScheduler.resourceOffer(freeCores))
+      }
+
+    case KillTask(taskId) =>
+      executor.killTask(taskId)
   }
 
-  def launchTask(tasks : Seq[TaskDescription]) {
+  private def launchTask(tasks: Seq[TaskDescription]) {
     for (task <- tasks) {
       freeCores -= 1
-      localScheduler.threadPool.submit(new Runnable {
-        def run() {
-          localScheduler.runTask(task.taskId, task.serializedTask)
-        }
-      })
+      executor.launchTask(localScheduler, task.taskId, task.serializedTask)
     }
   }
 }
 
-private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext)
+private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val sc: SparkContext)
   extends TaskScheduler
+  with ExecutorBackend
   with Logging {
 
-  var attemptId = new AtomicInteger(0)
-  var threadPool = Utils.newDaemonFixedThreadPool(threads)
   val env = SparkEnv.get
-  var listener: TaskSchedulerListener = null
+  val attemptId = new AtomicInteger
+  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.
   val currentFiles: HashMap[String, Long] = new HashMap[String, Long]()
   val currentJars: HashMap[String, Long] = new HashMap[String, Long]()
 
-  val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader)
-
   var schedulableBuilder: SchedulableBuilder = null
   var rootPool: Pool = null
   val schedulingMode: SchedulingMode = SchedulingMode.withName(
@@ -113,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) {
@@ -127,6 +128,27 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
     }
   }
 
+  override def cancelTasks(stageId: Int): Unit = synchronized {
+    logInfo("Cancelling stage " + stageId)
+    logInfo("Cancelling stage " + activeTaskSets.map(_._2.stageId))
+    activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
+      // There are two possible cases here:
+      // 1. The task set manager has been created and some tasks have been scheduled.
+      //    In this case, send a kill signal to the executors to kill the task and then abort
+      //    the stage.
+      // 2. The task set manager has been created but no tasks has been scheduled. In this case,
+      //    simply abort the stage.
+      val taskIds = taskSetTaskIds(tsm.taskSet.id)
+      if (taskIds.size > 0) {
+        taskIds.foreach { tid =>
+          localActor ! KillTask(tid)
+        }
+      }
+      logInfo("Stage %d was cancelled".format(stageId))
+      taskSetFinished(tsm)
+    }
+  }
+
   def resourceOffer(freeCores: Int): Seq[TaskDescription] = {
     synchronized {
       var freeCpuCores = freeCores
@@ -166,107 +188,34 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
     }
   }
 
-  def runTask(taskId: Long, bytes: ByteBuffer) {
-    logInfo("Running " + taskId)
-    val info = new TaskInfo(taskId, 0, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL)
-    // Set the Spark execution environment for the worker thread
-    SparkEnv.set(env)
-    val ser = SparkEnv.get.closureSerializer.newInstance()
-    val objectSer = SparkEnv.get.serializer.newInstance()
-    var attemptedTask: Option[Task[_]] = None   
-    val start = System.currentTimeMillis()
-    var taskStart: Long = 0
-    def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum
-    val startGCTime = getTotalGCTime
-
-    try {
-      Accumulators.clear()
-      Thread.currentThread().setContextClassLoader(classLoader)
-
-      // Serialize and deserialize the task so that accumulators are changed to thread-local ones;
-      // this adds a bit of unnecessary overhead but matches how the Mesos Executor works.
-      val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes)
-      updateDependencies(taskFiles, taskJars)   // Download any files added with addFile
-      val deserializedTask = ser.deserialize[Task[_]](
-        taskBytes, Thread.currentThread.getContextClassLoader)
-      attemptedTask = Some(deserializedTask)
-      val deserTime = System.currentTimeMillis() - start
-      taskStart = System.currentTimeMillis()
-
-      // Run it
-      val result: Any = deserializedTask.run(taskId)
-
-      // Serialize and deserialize the result to emulate what the Mesos
-      // executor does. This is useful to catch serialization errors early
-      // on in development (so when users move their local Spark programs
-      // to the cluster, they don't get surprised by serialization errors).
-      val serResult = objectSer.serialize(result)
-      deserializedTask.metrics.get.resultSize = serResult.limit()
-      val resultToReturn = objectSer.deserialize[Any](serResult)
-      val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
-        ser.serialize(Accumulators.values))
-      val serviceTime = System.currentTimeMillis() - taskStart
-      logInfo("Finished " + taskId)
-      deserializedTask.metrics.get.executorRunTime = serviceTime.toInt
-      deserializedTask.metrics.get.jvmGCTime = getTotalGCTime - startGCTime
-      deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt
-      val taskResult = new DirectTaskResult(
-        result, accumUpdates, deserializedTask.metrics.getOrElse(null))
-      val serializedResult = ser.serialize(taskResult)
-      localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult)
-    } catch {
-      case t: Throwable => {
-        val serviceTime = System.currentTimeMillis() - taskStart
-        val metrics = attemptedTask.flatMap(t => t.metrics)
-        for (m <- metrics) {
-          m.executorRunTime = serviceTime.toInt
-          m.jvmGCTime = getTotalGCTime - startGCTime
-        }
-        val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics)
-        localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure))
-      }
-    }
-  }
-
-  /**
-   * Download any missing dependencies if we receive a new set of files and JARs from the
-   * SparkContext. Also adds any new JARs we fetched to the class loader.
-   */
-  private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
-    synchronized {
-      // Fetch missing dependencies
-      for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
-        logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
-        currentFiles(name) = timestamp
-      }
-
-      for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
-        logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
-        currentJars(name) = timestamp
-        // Add it to our class loader
-        val localName = name.split("/").last
-        val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
-        if (!classLoader.getURLs.contains(url)) {
-          logInfo("Adding " + url + " to class loader")
-          classLoader.addURL(url)
+  override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) {
+    if (TaskState.isFinished(state)) {
+      synchronized {
+        taskIdToTaskSetId.get(taskId) match {
+          case Some(taskSetId) =>
+            val taskSetManager = activeTaskSets.get(taskSetId)
+            taskSetManager.foreach { tsm =>
+              taskSetTaskIds(taskSetId) -= taskId
+
+              state match {
+                case TaskState.FINISHED =>
+                  tsm.taskEnded(taskId, state, serializedData)
+                case TaskState.FAILED =>
+                  tsm.taskFailed(taskId, state, serializedData)
+                case TaskState.KILLED =>
+                  tsm.error("Task %d was killed".format(taskId))
+                case _ => {}
+              }
+            }
+          case None =>
+            logInfo("Ignoring update from TID " + taskId + " because its task set is gone")
         }
       }
+      localActor ! LocalStatusUpdate(taskId, state, serializedData)
     }
   }
 
-  def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) {
-    synchronized {
-      val taskSetId = taskIdToTaskSetId(taskId)
-      val taskSetManager = activeTaskSets(taskSetId)
-      taskSetTaskIds(taskSetId) -= taskId
-      taskSetManager.statusUpdate(taskId, state, serializedData)
-    }
-  }
-
-   override def stop() {
-    threadPool.shutdownNow()
+  override def stop() {
   }
 
   override def defaultParallelism() = threads
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 c2e2399ccb818df13fb648b3e41df61eb7bbedeb..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
@@ -132,19 +132,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
     return None
   }
 
-  def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
-    SparkEnv.set(env)
-    state match {
-      case TaskState.FINISHED =>
-        taskEnded(tid, state, serializedData)
-      case TaskState.FAILED =>
-        taskFailed(tid, state, serializedData)
-      case _ => {}
-    }
-  }
-
   def taskStarted(task: Task[_], info: TaskInfo) {
-    sched.listener.taskStarted(task, info)
+    sched.dagScheduler.taskStarted(task, info)
   }
 
   def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) {
@@ -159,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
@@ -176,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
@@ -185,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)
       }
@@ -195,5 +185,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
   }
 
   override def error(message: String) {
+    sched.dagScheduler.taskSetFailed(taskSet, message)
+    sched.taskSetFinished(this)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index e936b1cfedd919faff00cc460a5c81a8a0d238d9..e748c2275d589c6702576b72ac49038e86033c93 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -26,15 +26,18 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
 import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar}
 
 import org.apache.spark.{SerializableWritable, Logging}
-import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock, StorageLevel}
-
 import org.apache.spark.broadcast.HttpBroadcast
+import org.apache.spark.scheduler.MapStatus
+import org.apache.spark.storage._
 
 /**
- * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]].
+ * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]].
  */
 class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging {
-  private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
+
+  private val bufferSize = {
+    System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
+  }
 
   def newKryoOutput() = new KryoOutput(bufferSize)
 
@@ -43,20 +46,11 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
     val kryo = instantiator.newKryo()
     val classLoader = Thread.currentThread.getContextClassLoader
 
-    // Register some commonly used classes
-    val toRegister: Seq[AnyRef] = Seq(
-      ByteBuffer.allocate(1),
-      StorageLevel.MEMORY_ONLY,
-      PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY),
-      GotBlock("1", ByteBuffer.allocate(1)),
-      GetBlock("1"),
-      1 to 10,
-      1 until 10,
-      1L to 10L,
-      1L until 10L
-    )
-
-    for (obj <- toRegister) kryo.register(obj.getClass)
+    // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops.
+    // Do this before we invoke the user registrator so the user registrator can override this.
+    kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
+
+    for (cls <- KryoSerializer.toRegister) kryo.register(cls)
 
     // Allow sending SerializableWritable
     kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer())
@@ -78,10 +72,6 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
     new AllScalaRegistrar().apply(kryo)
 
     kryo.setClassLoader(classLoader)
-
-    // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops
-    kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
-
     kryo
   }
 
@@ -165,3 +155,21 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ
 trait KryoRegistrator {
   def registerClasses(kryo: Kryo)
 }
+
+private[serializer] object KryoSerializer {
+  // Commonly used classes.
+  private val toRegister: Seq[Class[_]] = Seq(
+    ByteBuffer.allocate(1).getClass,
+    classOf[StorageLevel],
+    classOf[PutBlock],
+    classOf[GotBlock],
+    classOf[GetBlock],
+    classOf[MapStatus],
+    classOf[BlockManagerId],
+    classOf[Array[Byte]],
+    (1 to 10).getClass,
+    (1 until 10).getClass,
+    (1L to 10L).getClass,
+    (1L until 10L).getClass
+  )
+}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockException.scala b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
index 290dbce4f545a6d5ccf292529cec1f4eafc3a51e..0d0a2dadc72dad6e476e363e416407943c9a4770 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockException.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
@@ -18,5 +18,5 @@
 package org.apache.spark.storage
 
 private[spark]
-case class BlockException(blockId: String, message: String) extends Exception(message)
+case class BlockException(blockId: BlockId, message: String) extends Exception(message)
 
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
index 3aeda3879d1f15a7424c6470959eb56e7619ab46..e51c5b30a3daba19b3407c6a7ec1191f486747c6 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -47,7 +47,7 @@ import org.apache.spark.util.Utils
  */
 
 private[storage]
-trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])]
+trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])]
   with Logging with BlockFetchTracker {
   def initialize()
 }
@@ -57,20 +57,20 @@ private[storage]
 object BlockFetcherIterator {
 
   // A request to fetch one or more blocks, complete with their sizes
-  class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) {
+  class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) {
     val size = blocks.map(_._2).sum
   }
 
   // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize
   // the block (since we want all deserializaton to happen in the calling thread); can also
   // represent a fetch failure if size == -1.
-  class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) {
+  class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) {
     def failed: Boolean = size == -1
   }
 
   class BasicBlockFetcherIterator(
       private val blockManager: BlockManager,
-      val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+      val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
       serializer: Serializer)
     extends BlockFetcherIterator {
 
@@ -92,12 +92,12 @@ object BlockFetcherIterator {
     // This represents the number of local blocks, also counting zero-sized blocks
     private var numLocal = 0
     // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks
-    protected val localBlocksToFetch = new ArrayBuffer[String]()
+    protected val localBlocksToFetch = new ArrayBuffer[BlockId]()
 
     // This represents the number of remote blocks, also counting zero-sized blocks
     private var numRemote = 0
     // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks
-    protected val remoteBlocksToFetch = new HashSet[String]()
+    protected val remoteBlocksToFetch = new HashSet[BlockId]()
 
     // A queue to hold our results.
     protected val results = new LinkedBlockingQueue[FetchResult]
@@ -167,7 +167,7 @@ object BlockFetcherIterator {
           logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
           val iterator = blockInfos.iterator
           var curRequestSize = 0L
-          var curBlocks = new ArrayBuffer[(String, Long)]
+          var curBlocks = new ArrayBuffer[(BlockId, Long)]
           while (iterator.hasNext) {
             val (blockId, size) = iterator.next()
             // Skip empty blocks
@@ -183,7 +183,7 @@ object BlockFetcherIterator {
               // Add this FetchRequest
               remoteRequests += new FetchRequest(address, curBlocks)
               curRequestSize = 0
-              curBlocks = new ArrayBuffer[(String, Long)]
+              curBlocks = new ArrayBuffer[(BlockId, Long)]
             }
           }
           // Add in the final request
@@ -241,7 +241,7 @@ object BlockFetcherIterator {
 
     override def hasNext: Boolean = resultsGotten < _numBlocksToFetch
 
-    override def next(): (String, Option[Iterator[Any]]) = {
+    override def next(): (BlockId, Option[Iterator[Any]]) = {
       resultsGotten += 1
       val startFetchWait = System.currentTimeMillis()
       val result = results.take()
@@ -267,7 +267,7 @@ object BlockFetcherIterator {
 
   class NettyBlockFetcherIterator(
       blockManager: BlockManager,
-      blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+      blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
       serializer: Serializer)
     extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) {
 
@@ -303,7 +303,7 @@ object BlockFetcherIterator {
 
     override protected def sendRequest(req: FetchRequest) {
 
-      def putResult(blockId: String, blockSize: Long, blockData: ByteBuf) {
+      def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) {
         val fetchResult = new FetchResult(blockId, blockSize,
           () => dataDeserialize(blockId, blockData.nioBuffer, serializer))
         results.put(fetchResult)
@@ -337,7 +337,7 @@ object BlockFetcherIterator {
       logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
     }
 
-    override def next(): (String, Option[Iterator[Any]]) = {
+    override def next(): (BlockId, Option[Iterator[Any]]) = {
       resultsGotten += 1
       val result = results.take()
       // If all the results has been retrieved, copiers will exit automatically
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7156d855d873b3ec63710ca72204922aa564a5d8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -0,0 +1,103 @@
+/*
+ * 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
+
+/**
+ * Identifies a particular Block of data, usually associated with a single file.
+ * A Block can be uniquely identified by its filename, but each type of Block has a different
+ * set of keys which produce its unique name.
+ *
+ * If your BlockId should be serializable, be sure to add it to the BlockId.fromString() method.
+ */
+private[spark] sealed abstract class BlockId {
+  /** A globally unique identifier for this Block. Can be used for ser/de. */
+  def name: String
+
+  // convenience methods
+  def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None
+  def isRDD = isInstanceOf[RDDBlockId]
+  def isShuffle = isInstanceOf[ShuffleBlockId]
+  def isBroadcast = isInstanceOf[BroadcastBlockId] || isInstanceOf[BroadcastHelperBlockId]
+
+  override def toString = name
+  override def hashCode = name.hashCode
+  override def equals(other: Any): Boolean = other match {
+    case o: BlockId => getClass == o.getClass && name.equals(o.name)
+    case _ => false
+  }
+}
+
+private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId {
+  def name = "rdd_" + rddId + "_" + splitIndex
+}
+
+private[spark]
+case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId {
+  def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId
+}
+
+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
+}
+
+private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId {
+  def name = "input-" + streamId + "-" + uniqueId
+}
+
+// Intended only for testing purposes
+private[spark] case class TestBlockId(id: String) extends BlockId {
+  def name = "test_" + id
+}
+
+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
+
+  /** Converts a BlockId "name" String back into a BlockId. */
+  def apply(id: String) = id match {
+    case RDD(rddId, splitIndex) =>
+      RDDBlockId(rddId.toInt, splitIndex.toInt)
+    case SHUFFLE(shuffleId, mapId, reduceId) =>
+      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) =>
+      StreamBlockId(streamId.toInt, uniqueId.toLong)
+    case TEST(value) =>
+      TestBlockId(value)
+    case _ =>
+      throw new IllegalStateException("Unrecognized BlockId: " + id)
+  }
+}
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..c8f397609a0b473fc6c8557b3edf7f155509b2bb
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala
@@ -0,0 +1,81 @@
+/*
+ * 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] class BlockInfo(val level: StorageLevel, val 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
+}
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 7852849ce55f09184b04c0f91226bdb94e58db7b..7e721a49a5f62b95f664c824ca79069a56e3a3f2 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -17,17 +17,18 @@
 
 package org.apache.spark.storage
 
-import java.io.{InputStream, OutputStream}
+import java.io.{File, 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 scala.concurrent.{Await, Future}
 import scala.concurrent.duration.Duration
 import scala.concurrent.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
@@ -37,7 +38,6 @@ import org.apache.spark.util._
 
 import sun.nio.ch.DirectBuffer
 
-
 private[spark] class BlockManager(
     executorId: String,
     actorSystem: ActorSystem,
@@ -46,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(shuffleBlockManager,
+    System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
 
-  private val blockInfo = new TimeStampedHashMap[String, BlockInfo]
+  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)
@@ -154,7 +100,8 @@ private[spark] class BlockManager(
 
   var heartBeatTask: Cancellable = null
 
-  val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks)
+  private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks)
+  private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks)
   initialize()
 
   // The compression codec to use. Note that the "lazy" val is necessary because we want to delay
@@ -248,7 +195,7 @@ private[spark] class BlockManager(
   /**
    * Get storage level of local block. If no info exists for the block, then returns null.
    */
-  def getLevel(blockId: String): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
+  def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
 
   /**
    * Tell the master about the current storage status of a block. This will send a block update
@@ -258,7 +205,7 @@ private[spark] class BlockManager(
    * droppedMemorySize exists to account for when block is dropped from memory to disk (so it is still valid).
    * This ensures that update in master will compensate for the increase in memory on slave.
    */
-  def reportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L) {
+  def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) {
     val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize)
     if (needReregister) {
       logInfo("Got told to reregister updating block " + blockId)
@@ -269,11 +216,11 @@ 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.
    */
-  private def tryToReportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = {
+  private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = {
     val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized {
       info.level match {
         case null =>
@@ -298,7 +245,7 @@ private[spark] class BlockManager(
   /**
    * Get locations of an array of blocks.
    */
-  def getLocationBlockIds(blockIds: Array[String]): Array[Seq[BlockManagerId]] = {
+  def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = {
     val startTimeMs = System.currentTimeMillis
     val locations = master.getLocations(blockIds).toArray
     logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
@@ -310,7 +257,7 @@ private[spark] class BlockManager(
    * shuffle blocks. It is safe to do so without a lock on block info since disk store
    * never deletes (recent) items.
    */
-  def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = {
+  def getLocalFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = {
     diskStore.getValues(blockId, serializer).orElse(
       sys.error("Block " + blockId + " not found on disk, though it should be"))
   }
@@ -318,94 +265,19 @@ private[spark] class BlockManager(
   /**
    * Get block from local block manager.
    */
-  def getLocal(blockId: String): Option[Iterator[Any]] = {
+  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: String): Option[ByteBuffer] = {
-    // TODO: This whole thing is very similar to getLocal; we need to refactor it somehow
+  def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = {
     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 (ShuffleBlockManager.isShuffle(blockId)) {
+    if (blockId.isShuffle) {
       return diskStore.getBytes(blockId) match {
         case Some(bytes) =>
           Some(bytes)
@@ -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: String): Option[Iterator[Any]] = {
-    if (blockId == null) {
-      throw new IllegalArgumentException("Block Id is null")
-    }
+  def getRemote(blockId: BlockId): Option[Iterator[Any]] = {
     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 locations
+  /**
+   * 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]]
+   }
+
+  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,35 +411,10 @@ private[spark] class BlockManager(
     return None
   }
 
-  /**
-   * Get block from remote block managers as serialized bytes.
-   */
-   def getRemoteBytes(blockId: String): 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).
    */
-  def get(blockId: String): Option[Iterator[Any]] = {
+  def get(blockId: BlockId): Option[Iterator[Any]] = {
     val local = getLocal(blockId)
     if (local.isDefined) {
       logInfo("Found block %s locally".format(blockId))
@@ -543,7 +435,7 @@ private[spark] class BlockManager(
    * so that we can control the maxMegabytesInFlight for the fetch.
    */
   def getMultiple(
-    blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer)
+    blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer)
       : BlockFetcherIterator = {
 
     val iter =
@@ -557,7 +449,7 @@ private[spark] class BlockManager(
     iter
   }
 
-  def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
+  def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
     : Long = {
     val elements = new ArrayBuffer[Any]
     elements ++= values
@@ -566,35 +458,38 @@ 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: String, serializer: Serializer, bufferSize: Int)
+  def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
     : BlockObjectWriter = {
-    val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize)
-    writer.registerCloseEventHandler(() => {
-      val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false)
-      blockInfo.put(blockId, myInfo)
-      myInfo.markReady(writer.size())
-    })
-    writer
+    val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
+    new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream)
   }
 
   /**
    * Put a new block of values to the block manager. Returns its (estimated) size in bytes.
    */
-  def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel,
-    tellMaster: Boolean = true) : Long = {
+  def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel,
+          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
@@ -610,7 +505,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 +515,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 +527,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,132 +596,46 @@ 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: String, 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
   }
 
   /**
    * Replicate block to another node.
    */
   var cachedPeers: Seq[BlockManagerId] = null
-  private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) {
+  private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) {
     val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
     if (cachedPeers == null) {
       cachedPeers = master.getPeers(blockManagerId, level.replication - 1)
@@ -827,14 +658,14 @@ private[spark] class BlockManager(
   /**
    * Read a block consisting of a single object.
    */
-  def getSingle(blockId: String): Option[Any] = {
+  def getSingle(blockId: BlockId): Option[Any] = {
     get(blockId).map(_.next())
   }
 
   /**
    * Write a block consisting of a single object.
    */
-  def putSingle(blockId: String, value: Any, level: StorageLevel, tellMaster: Boolean = true) {
+  def putSingle(blockId: BlockId, value: Any, level: StorageLevel, tellMaster: Boolean = true) {
     put(blockId, Iterator(value), level, tellMaster)
   }
 
@@ -842,7 +673,7 @@ private[spark] class BlockManager(
    * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory
    * store reaches its limit and needs to free up space.
    */
-  def dropFromMemory(blockId: String, data: Either[ArrayBuffer[Any], ByteBuffer]) {
+  def dropFromMemory(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]) {
     logInfo("Dropping block " + blockId + " from memory")
     val info = blockInfo.get(blockId).orNull
     if (info != null)  {
@@ -891,16 +722,15 @@ private[spark] class BlockManager(
     // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps
     // from RDD.id to blocks.
     logInfo("Removing RDD " + rddId)
-    val rddPrefix = "rdd_" + rddId + "_"
-    val blocksToRemove = blockInfo.filter(_._1.startsWith(rddPrefix)).map(_._1)
-    blocksToRemove.foreach(blockId => removeBlock(blockId, false))
+    val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
+    blocksToRemove.foreach(blockId => removeBlock(blockId, tellMaster = false))
     blocksToRemove.size
   }
 
   /**
    * Remove a block from both memory and disk.
    */
-  def removeBlock(blockId: String, tellMaster: Boolean = true) {
+  def removeBlock(blockId: BlockId, tellMaster: Boolean = true) {
     logInfo("Removing block " + blockId)
     val info = blockInfo.get(blockId).orNull
     if (info != null) info.synchronized {
@@ -921,13 +751,22 @@ private[spark] class BlockManager(
     }
   }
 
-  def dropOldBlocks(cleanupTime: Long) {
-    logInfo("Dropping blocks older than " + cleanupTime)
+  private def dropOldNonBroadcastBlocks(cleanupTime: Long) {
+    logInfo("Dropping non broadcast blocks older than " + cleanupTime)
+    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) {
+      if (time < cleanupTime && shouldDrop(id)) {
         info.synchronized {
           val level = info.level
           if (level.useMemory) {
@@ -944,39 +783,45 @@ private[spark] class BlockManager(
     }
   }
 
-  def shouldCompress(blockId: String): Boolean = {
-    if (ShuffleBlockManager.isShuffle(blockId)) {
-      compressShuffle
-    } else if (blockId.startsWith("broadcast_")) {
-      compressBroadcast
-    } else if (blockId.startsWith("rdd_")) {
-      compressRdds
-    } else {
-      false    // Won't happen in a real cluster, but it can in tests
-    }
+  def shouldCompress(blockId: BlockId): Boolean = blockId match {
+    case ShuffleBlockId(_, _, _) => compressShuffle
+    case BroadcastBlockId(_) => compressBroadcast
+    case RDDBlockId(_, _) => compressRdds
+    case _ => false
   }
 
   /**
    * Wrap an output stream for compression if block compression is enabled for its block type
    */
-  def wrapForCompression(blockId: String, s: OutputStream): OutputStream = {
+  def wrapForCompression(blockId: BlockId, s: OutputStream): OutputStream = {
     if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s
   }
 
   /**
    * Wrap an input stream for compression if block compression is enabled for its block type
    */
-  def wrapForCompression(blockId: String, s: InputStream): InputStream = {
+  def wrapForCompression(blockId: BlockId, s: InputStream): InputStream = {
     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: String,
+      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)
   }
@@ -986,7 +831,7 @@ private[spark] class BlockManager(
    * the iterator is reached.
    */
   def dataDeserialize(
-      blockId: String,
+      blockId: BlockId,
       bytes: ByteBuffer,
       serializer: Serializer = defaultSerializer): Iterator[Any] = {
     bytes.rewind()
@@ -1004,6 +849,7 @@ private[spark] class BlockManager(
     memoryStore.clear()
     diskStore.clear()
     metadataCleaner.cancel()
+    broadcastCleaner.cancel()
     logInfo("BlockManager stopped")
   }
 }
@@ -1041,20 +887,20 @@ private[spark] object BlockManager extends Logging {
   }
 
   def blockIdsToBlockManagers(
-      blockIds: Array[String],
+      blockIds: Array[BlockId],
       env: SparkEnv,
       blockManagerMaster: BlockManagerMaster = null)
-  : Map[String, Seq[BlockManagerId]] =
+  : Map[BlockId, Seq[BlockManagerId]] =
   {
-    // env == null and blockManagerMaster != null is used in tests
+    // blockManagerMaster != null is used in tests
     assert (env != null || blockManagerMaster != null)
-    val blockLocations: Seq[Seq[BlockManagerId]] = if (env != null) {
+    val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) {
       env.blockManager.getLocationBlockIds(blockIds)
     } else {
       blockManagerMaster.getLocations(blockIds)
     }
 
-    val blockManagers = new HashMap[String, Seq[BlockManagerId]]
+    val blockManagers = new HashMap[BlockId, Seq[BlockManagerId]]
     for (i <- 0 until blockIds.length) {
       blockManagers(blockIds(i)) = blockLocations(i)
     }
@@ -1062,19 +908,19 @@ private[spark] object BlockManager extends Logging {
   }
 
   def blockIdsToExecutorIds(
-      blockIds: Array[String],
+      blockIds: Array[BlockId],
       env: SparkEnv,
       blockManagerMaster: BlockManagerMaster = null)
-    : Map[String, Seq[String]] =
+    : Map[BlockId, Seq[String]] =
   {
     blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId))
   }
 
   def blockIdsToHosts(
-      blockIds: Array[String],
+      blockIds: Array[BlockId],
       env: SparkEnv,
       blockManagerMaster: BlockManagerMaster = null)
-    : Map[String, Seq[String]] =
+    : Map[BlockId, Seq[String]] =
   {
     blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host))
   }
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index 0c977f05d1d22dcc730c438bdbee13c11167b62a..e05b842476d53b3dc4217a0bdda29a9058c4ac57 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -17,25 +17,17 @@
 
 package org.apache.spark.storage
 
-import java.io._
-import java.util.{HashMap => JHashMap}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import scala.util.Random
-
-import akka.actor.{Actor, ActorRef, ActorSystem, Props}
-import scala.concurrent.Await
-import scala.concurrent.Future
+import scala.concurrent.{Await, Future}
+import scala.concurrent.duration._
 import scala.concurrent.ExecutionContext.Implicits.global
 
+import akka.actor._
 import akka.pattern.ask
-import scala.concurrent.duration._
 
 import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.storage.BlockManagerMessages._
 
-private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
+private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging {
 
   val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
   val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
@@ -69,7 +61,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
 
   def updateBlockInfo(
       blockManagerId: BlockManagerId,
-      blockId: String,
+      blockId: BlockId,
       storageLevel: StorageLevel,
       memSize: Long,
       diskSize: Long): Boolean = {
@@ -80,12 +72,12 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
   }
 
   /** Get locations of the blockId from the driver */
-  def getLocations(blockId: String): Seq[BlockManagerId] = {
+  def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
     askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId))
   }
 
   /** Get locations of multiple blockIds from the driver */
-  def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
+  def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
     askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds))
   }
 
@@ -103,7 +95,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
    * Remove a block from the slaves that have it. This can only be used to remove
    * blocks that the driver knows about.
    */
-  def removeBlock(blockId: String) {
+  def removeBlock(blockId: BlockId) {
     askDriverWithReply(RemoveBlock(blockId))
   }
 
@@ -165,7 +157,10 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
     while (attempts < AKKA_RETRY_ATTEMPTS) {
       attempts += 1
       try {
-        val future = driverActor.ask(message)(timeout)
+        val future = driverActor match {
+          case Left(a: ActorRef) => a.ask(message)(timeout)
+          case Right(b: ActorSelection) => b.ask(message)(timeout)
+        }
         val result = Await.result(future, timeout)
         if (result == null) {
           throw new SparkException("BlockManagerMaster returned null")
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 3776951782ed2e21d85aedd4d7bc16c599cf9173..154a3980e9fa15800f4dc5c998bd05ea3646c9d2 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -48,7 +48,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId]
 
   // Mapping from block id to the set of block managers that have the block.
-  private val blockLocations = new JHashMap[String, mutable.HashSet[BlockManagerId]]
+  private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]
 
   val akkaTimeout = Duration.create(
     System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
@@ -130,10 +130,9 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
     // First remove the metadata for the given RDD, and then asynchronously remove the blocks
     // from the slaves.
 
-    val prefix = "rdd_" + rddId + "_"
     // Find all blocks for the given RDD, remove the block from both blockLocations and
     // the blockManagerInfo that is tracking the blocks.
-    val blocks = blockLocations.keySet().filter(_.startsWith(prefix))
+    val blocks = blockLocations.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
     blocks.foreach { blockId =>
       val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId)
       bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId)))
@@ -199,7 +198,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
 
   // Remove a block from the slaves that have it. This can only be used to remove
   // blocks that the master knows about.
-  private def removeBlockFromWorkers(blockId: String) {
+  private def removeBlockFromWorkers(blockId: BlockId) {
     val locations = blockLocations.get(blockId)
     if (locations != null) {
       locations.foreach { blockManagerId: BlockManagerId =>
@@ -229,9 +228,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.
@@ -248,7 +245,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
 
   private def updateBlockInfo(
       blockManagerId: BlockManagerId,
-      blockId: String,
+      blockId: BlockId,
       storageLevel: StorageLevel,
       memSize: Long,
       diskSize: Long) {
@@ -293,11 +290,11 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
     sender ! true
   }
 
-  private def getLocations(blockId: String): Seq[BlockManagerId] = {
+  private def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
     if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty
   }
 
-  private def getLocationsMultipleBlockIds(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
+  private def getLocationsMultipleBlockIds(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
     blockIds.map(blockId => getLocations(blockId))
   }
 
@@ -331,7 +328,7 @@ object BlockManagerMasterActor {
     private var _remainingMem: Long = maxMem
 
     // Mapping from block id to its status.
-    private val _blocks = new JHashMap[String, BlockStatus]
+    private val _blocks = new JHashMap[BlockId, BlockStatus]
 
     logInfo("Registering block manager %s with %s RAM".format(
       blockManagerId.hostPort, Utils.bytesToString(maxMem)))
@@ -340,7 +337,7 @@ object BlockManagerMasterActor {
       _lastSeenMs = System.currentTimeMillis()
     }
 
-    def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long,
+    def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long,
                         diskSize: Long) {
 
       updateLastSeenMs()
@@ -384,7 +381,7 @@ object BlockManagerMasterActor {
       }
     }
 
-    def removeBlock(blockId: String) {
+    def removeBlock(blockId: BlockId) {
       if (_blocks.containsKey(blockId)) {
         _remainingMem += _blocks.get(blockId).memSize
         _blocks.remove(blockId)
@@ -395,7 +392,7 @@ object BlockManagerMasterActor {
 
     def lastSeenMs: Long = _lastSeenMs
 
-    def blocks: JHashMap[String, BlockStatus] = _blocks
+    def blocks: JHashMap[BlockId, BlockStatus] = _blocks
 
     override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem
 
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 24333a179c96467aa3f42e0aea0d69d0bb7edc6c..45f51da2885481d3e4e23e97384734c6cd5245bc 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -30,7 +30,7 @@ private[storage] object BlockManagerMessages {
 
   // Remove a block from the slaves that have it. This can only be used to remove
   // blocks that the master knows about.
-  case class RemoveBlock(blockId: String) extends ToBlockManagerSlave
+  case class RemoveBlock(blockId: BlockId) extends ToBlockManagerSlave
 
   // Remove all blocks belonging to a specific RDD.
   case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave
@@ -51,7 +51,7 @@ private[storage] object BlockManagerMessages {
 
   class UpdateBlockInfo(
       var blockManagerId: BlockManagerId,
-      var blockId: String,
+      var blockId: BlockId,
       var storageLevel: StorageLevel,
       var memSize: Long,
       var diskSize: Long)
@@ -62,7 +62,7 @@ private[storage] object BlockManagerMessages {
 
     override def writeExternal(out: ObjectOutput) {
       blockManagerId.writeExternal(out)
-      out.writeUTF(blockId)
+      out.writeUTF(blockId.name)
       storageLevel.writeExternal(out)
       out.writeLong(memSize)
       out.writeLong(diskSize)
@@ -70,7 +70,7 @@ private[storage] object BlockManagerMessages {
 
     override def readExternal(in: ObjectInput) {
       blockManagerId = BlockManagerId(in)
-      blockId = in.readUTF()
+      blockId = BlockId(in.readUTF())
       storageLevel = StorageLevel(in)
       memSize = in.readLong()
       diskSize = in.readLong()
@@ -79,7 +79,7 @@ private[storage] object BlockManagerMessages {
 
   object UpdateBlockInfo {
     def apply(blockManagerId: BlockManagerId,
-        blockId: String,
+        blockId: BlockId,
         storageLevel: StorageLevel,
         memSize: Long,
         diskSize: Long): UpdateBlockInfo = {
@@ -87,14 +87,14 @@ private[storage] object BlockManagerMessages {
     }
 
     // For pattern-matching
-    def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = {
+    def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long)] = {
       Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize))
     }
   }
 
-  case class GetLocations(blockId: String) extends ToBlockManagerMaster
+  case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster
 
-  case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster
+  case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster
 
   case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster
 
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/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
index 678c38203c797a16bed558c85543c5a5be1fdeb4..0c66addf9def6f78628bfff2cec9b7c9e9a8a3b4 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
@@ -77,7 +77,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
     }
   }
 
-  private def putBlock(id: String, bytes: ByteBuffer, level: StorageLevel) {
+  private def putBlock(id: BlockId, bytes: ByteBuffer, level: StorageLevel) {
     val startTimeMs = System.currentTimeMillis()
     logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes)
     blockManager.putBytes(id, bytes, level)
@@ -85,7 +85,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
         + " with data size: " + bytes.limit)
   }
 
-  private def getBlock(id: String): ByteBuffer = {
+  private def getBlock(id: BlockId): ByteBuffer = {
     val startTimeMs = System.currentTimeMillis()
     logDebug("GetBlock " + id + " started from " + startTimeMs)
     val buffer = blockManager.getLocalBytes(id) match {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
index d8fa6a91d15f51d2e30e6a7f0c546c6e20970afa..80dcb5a2074d0a6c07c899db61564bd5c894657c 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
@@ -24,9 +24,9 @@ import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.network._
 
-private[spark] case class GetBlock(id: String)
-private[spark] case class GotBlock(id: String, data: ByteBuffer)
-private[spark] case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel) 
+private[spark] case class GetBlock(id: BlockId)
+private[spark] case class GotBlock(id: BlockId, data: ByteBuffer)
+private[spark] case class PutBlock(id: BlockId, data: ByteBuffer, level: StorageLevel)
 
 private[spark] class BlockMessage() {
   // Un-initialized: typ = 0
@@ -34,7 +34,7 @@ private[spark] class BlockMessage() {
   // GotBlock: typ = 2
   // PutBlock: typ = 3
   private var typ: Int = BlockMessage.TYPE_NON_INITIALIZED
-  private var id: String = null
+  private var id: BlockId = null
   private var data: ByteBuffer = null
   private var level: StorageLevel = null
  
@@ -74,7 +74,7 @@ private[spark] class BlockMessage() {
     for (i <- 1 to idLength) {
       idBuilder += buffer.getChar()
     }
-    id = idBuilder.toString()
+    id = BlockId(idBuilder.toString)
     
     if (typ == BlockMessage.TYPE_PUT_BLOCK) {
 
@@ -109,28 +109,17 @@ private[spark] class BlockMessage() {
     set(buffer)
   }
   
-  def getType: Int = {
-    return typ
-  }
-  
-  def getId: String = {
-    return id
-  }
-  
-  def getData: ByteBuffer = {
-    return data
-  }
-  
-  def getLevel: StorageLevel = {
-    return level
-  }
+  def getType: Int = typ
+  def getId: BlockId = id
+  def getData: ByteBuffer = data
+  def getLevel: StorageLevel =  level
   
   def toBufferMessage: BufferMessage = {
     val startTime = System.currentTimeMillis
     val buffers = new ArrayBuffer[ByteBuffer]()
-    var buffer = ByteBuffer.allocate(4 + 4 + id.length() * 2)
-    buffer.putInt(typ).putInt(id.length())
-    id.foreach((x: Char) => buffer.putChar(x))
+    var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2)
+    buffer.putInt(typ).putInt(id.name.length)
+    id.name.foreach((x: Char) => buffer.putChar(x))
     buffer.flip()
     buffers += buffer
 
@@ -212,7 +201,8 @@ private[spark] object BlockMessage {
 
   def main(args: Array[String]) {
     val B = new BlockMessage()
-    B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2))
+    val blockId = TestBlockId("ABC")
+    B.set(new PutBlock(blockId, ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2))
     val bMsg = B.toBufferMessage
     val C = new BlockMessage()
     C.set(bMsg)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
index 0aaf846b5b6606d3af2ba5fce6147f79050d5ea9..6ce9127c7450175d61ed66871678ea7b7fc62dc8 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
@@ -111,14 +111,15 @@ private[spark] object BlockMessageArray {
   }
   
   def main(args: Array[String]) {
-    val blockMessages = 
+    val blockMessages =
       (0 until 10).map { i =>
         if (i % 2 == 0) {
           val buffer =  ByteBuffer.allocate(100)
           buffer.clear
-          BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY_SER))
+          BlockMessage.fromPutBlock(PutBlock(TestBlockId(i.toString), buffer,
+            StorageLevel.MEMORY_ONLY_SER))
         } else {
-          BlockMessage.fromGetBlock(GetBlock(i.toString))
+          BlockMessage.fromGetBlock(GetBlock(TestBlockId(i.toString)))
         }
       }
     val blockMessageArray = new BlockMessageArray(blockMessages)
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 39f103297f90c86eb563da121c13797f9f76054b..b4451fc7b8e5666a9c93c8e89d27ebb08b224543 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
@@ -25,22 +32,14 @@ package org.apache.spark.storage
  *
  * This interface does not support concurrent writes.
  */
-abstract class BlockObjectWriter(val blockId: String) {
-
-  var closeEventHandler: () => Unit = _
+abstract class BlockObjectWriter(val blockId: BlockId) {
 
   def open(): BlockObjectWriter
 
-  def close() {
-    closeEventHandler()
-  }
+  def close()
 
   def isOpen: Boolean
 
-  def registerCloseEventHandler(handler: () => Unit) {
-    closeEventHandler = handler
-  }
-
   /**
    * Flush the partial writes and commit them as a single atomic block. Return the
    * number of bytes written for this commit.
@@ -59,7 +58,128 @@ abstract class BlockObjectWriter(val blockId: String) {
   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))
+    override def close() = out.close()
+    override def flush() = out.flush()
+  }
+
+  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 val initialPosition = file.length()
+  private var lastValidPosition = initialPosition
+  private var initialized = false
+  private var _timeWriting = 0L
+
+  override def open(): BlockObjectWriter = {
+    fos = new FileOutputStream(file, true)
+    ts = new TimeTrackingOutputStream(fos)
+    channel = fos.getChannel()
+    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
+    }
+  }
+
+  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/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
index fa834371f4800e82fb1f6541d0db88db60b962a5..ea426562402ae4ef4a16223f4c12ea862528ad9b 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
@@ -27,7 +27,7 @@ import org.apache.spark.Logging
  */
 private[spark]
 abstract class BlockStore(val blockManager: BlockManager) extends Logging {
-  def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel)
+  def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel)
 
   /**
    * Put in a block and, possibly, also return its content as either bytes or another Iterator.
@@ -36,26 +36,26 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging {
    * @return a PutResult that contains the size of the data, as well as the values put if
    *         returnValues is true (if not, the result's data field can be null)
    */
-  def putValues(blockId: String, values: ArrayBuffer[Any], level: StorageLevel, 
+  def putValues(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel,
     returnValues: Boolean) : PutResult
 
   /**
    * Return the size of a block in bytes.
    */
-  def getSize(blockId: String): Long
+  def getSize(blockId: BlockId): Long
 
-  def getBytes(blockId: String): Option[ByteBuffer]
+  def getBytes(blockId: BlockId): Option[ByteBuffer]
 
-  def getValues(blockId: String): Option[Iterator[Any]]
+  def getValues(blockId: BlockId): Option[Iterator[Any]]
 
   /**
    * Remove a block, if it exists.
    * @param blockId the block to remove.
    * @return True if the block was found and removed, False otherwise.
    */
-  def remove(blockId: String): Boolean
+  def remove(blockId: BlockId): Boolean
 
-  def contains(blockId: String): Boolean
+  def contains(blockId: BlockId): Boolean
 
   def clear() { }
 }
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..fcd2e9798295596b48966c4b4b2529f4730a7ea3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -0,0 +1,151 @@
+/*
+ * 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 org.apache.spark.Logging
+import org.apache.spark.executor.ExecutorExitCode
+import org.apache.spark.network.netty.{PathResolver, ShuffleSender}
+import org.apache.spark.util.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(shuffleManager: ShuffleBlockManager, 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
+
+  addShutdownHook()
+
+  /**
+   * 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 (blockId.isShuffle && shuffleManager.consolidateShuffleFiles) {
+      shuffleManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])
+    } else {
+      val file = getFile(blockId.name)
+      new FileSegment(file, 0, file.length())
+    }
+  }
+
+  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)
+  }
+
+  def getFile(blockId: BlockId): File = getFile(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 = {
+    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 63447baf8c6b402d3cd8afaf8bedcd7eabc27916..5a1e7b44440fdac533ae6256ba61c33d70552b7d 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -17,153 +17,46 @@
 
 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: String, 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: String, serializer: Serializer, bufferSize: Int)
-    : BlockObjectWriter = {
-    new DiskBlockObjectWriter(blockId, serializer, bufferSize)
+  override def getSize(blockId: BlockId): Long = {
+    diskManager.getBlockLocation(blockId).length
   }
 
-  override def getSize(blockId: String): Long = {
-    getFile(blockId).length()
-  }
-
-  override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) {
+  override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) {
     // So that we do not modify the input offsets !
     // duplicate does not copy buffer, so inexpensive
     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.getFile(blockId)
+    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(
-      blockId: String,
+      blockId: BlockId,
       values: ArrayBuffer[Any],
       level: StorageLevel,
       returnValues: Boolean)
@@ -171,159 +64,62 @@ 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.getFile(blockId)
+    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)
     }
   }
 
-  override def getBytes(blockId: String): Option[ByteBuffer] = {
-    val file = getFile(blockId)
-    val bytes = getFileBytes(file)
-    Some(bytes)
+  override def getBytes(blockId: BlockId): Option[ByteBuffer] = {
+    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: String): Option[Iterator[Any]] = {
-    getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes))
+  override def getValues(blockId: BlockId): Option[Iterator[Any]] = {
+    getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer))
   }
 
   /**
    * A version of getValues that allows a custom serializer. This is used as part of the
    * shuffle short-circuit code.
    */
-  def getValues(blockId: String, serializer: Serializer): Option[Iterator[Any]] = {
+  def getValues(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = {
     getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer))
   }
 
-  override def remove(blockId: String): Boolean = {
-    val file = getFile(blockId)
-    if (file.exists()) {
+  override def remove(blockId: BlockId): Boolean = {
+    val fileSegment = diskManager.getBlockLocation(blockId)
+    val file = fileSegment.file
+    if (file.exists() && file.length() == fileSegment.length) {
       file.delete()
     } else {
-      false
-    }
-  }
-
-  override def contains(blockId: String): Boolean = {
-    getFile(blockId).exists()
-  }
-
-  private def createFile(blockId: String, 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: String): 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)
-  }
-
-  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 (fileSegment.length < file.length()) {
+        logWarning("Could not delete block associated with only a part of a file: " + blockId)
       }
-      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
+      false
     }
   }
 
-  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(blockId: String): String = {
-        if (!blockId.startsWith("shuffle_")) {
-          return null
-        }
-        DiskStore.this.getFile(blockId).getAbsolutePath()
-      }
-    }
-    shuffleSender = new ShuffleSender(port, pResolver)
-    logInfo("Created ShuffleSender binding to port : "+ shuffleSender.port)
-    shuffleSender.port
+  override def contains(blockId: BlockId): Boolean = {
+    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/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
index 77a39c71ed2c1d22a18a3af76c4cf5b7ef96ca5b..05f676c6e224998550d1108d40e392b3104a6e2d 100644
--- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
@@ -32,7 +32,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
 
   case class Entry(value: Any, size: Long, deserialized: Boolean)
 
-  private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
+  private val entries = new LinkedHashMap[BlockId, Entry](32, 0.75f, true)
   @volatile private var currentMemory = 0L
   // Object used to ensure that only one thread is putting blocks and if necessary, dropping
   // blocks from the memory store.
@@ -42,13 +42,13 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
 
   def freeMemory: Long = maxMemory - currentMemory
 
-  override def getSize(blockId: String): Long = {
+  override def getSize(blockId: BlockId): Long = {
     entries.synchronized {
       entries.get(blockId).size
     }
   }
 
-  override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) {
+  override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) {
     // Work on a duplicate - since the original input might be used elsewhere.
     val bytes = _bytes.duplicate()
     bytes.rewind()
@@ -64,7 +64,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
   }
 
   override def putValues(
-      blockId: String,
+      blockId: BlockId,
       values: ArrayBuffer[Any],
       level: StorageLevel,
       returnValues: Boolean)
@@ -81,7 +81,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     }
   }
 
-  override def getBytes(blockId: String): Option[ByteBuffer] = {
+  override def getBytes(blockId: BlockId): Option[ByteBuffer] = {
     val entry = entries.synchronized {
       entries.get(blockId)
     }
@@ -94,7 +94,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     }
   }
 
-  override def getValues(blockId: String): Option[Iterator[Any]] = {
+  override def getValues(blockId: BlockId): Option[Iterator[Any]] = {
     val entry = entries.synchronized {
       entries.get(blockId)
     }
@@ -108,7 +108,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     }
   }
 
-  override def remove(blockId: String): Boolean = {
+  override def remove(blockId: BlockId): Boolean = {
     entries.synchronized {
       val entry = entries.remove(blockId)
       if (entry != null) {
@@ -131,14 +131,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
   }
 
   /**
-   * Return the RDD ID that a given block ID is from, or null if it is not an RDD block.
+   * Return the RDD ID that a given block ID is from, or None if it is not an RDD block.
    */
-  private def getRddId(blockId: String): String = {
-    if (blockId.startsWith("rdd_")) {
-      blockId.split('_')(1)
-    } else {
-      null
-    }
+  private def getRddId(blockId: BlockId): Option[Int] = {
+    blockId.asRDDId.map(_.rddId)
   }
 
   /**
@@ -151,7 +147,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
    * blocks to free memory for one block, another thread may use up the freed space for
    * another block.
    */
-  private def tryToPut(blockId: String, value: Any, size: Long, deserialized: Boolean): Boolean = {
+  private def tryToPut(blockId: BlockId, value: Any, size: Long, deserialized: Boolean): Boolean = {
     // TODO: Its possible to optimize the locking by locking entries only when selecting blocks
     // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has been
     // released, it must be ensured that those to-be-dropped blocks are not double counted for
@@ -195,7 +191,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
    * Assumes that a lock is held by the caller to ensure only one thread is dropping blocks.
    * Otherwise, the freed space may fill up before the caller puts in their new value.
    */
-  private def ensureFreeSpace(blockIdToAdd: String, space: Long): Boolean = {
+  private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): Boolean = {
 
     logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format(
       space, currentMemory, maxMemory))
@@ -207,7 +203,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
 
     if (maxMemory - currentMemory < space) {
       val rddToAdd = getRddId(blockIdToAdd)
-      val selectedBlocks = new ArrayBuffer[String]()
+      val selectedBlocks = new ArrayBuffer[BlockId]()
       var selectedMemory = 0L
 
       // This is synchronized to ensure that the set of entries is not changed
@@ -218,7 +214,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
         while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) {
           val pair = iterator.next()
           val blockId = pair.getKey
-          if (rddToAdd != null && rddToAdd == getRddId(blockId)) {
+          if (rddToAdd != None && rddToAdd == getRddId(blockId)) {
             logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " +
               "block from the same RDD")
             return false
@@ -252,7 +248,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     return true
   }
 
-  override def contains(blockId: String): Boolean = {
+  override def contains(blockId: BlockId): Boolean = {
     entries.synchronized { entries.containsKey(blockId) }
   }
 }
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 9da11efb570d8d4c218c5884665a4ba0d237a07e..e828e1d1c5e7bdf45296ab478b9806c0f0017e51 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -17,51 +17,199 @@
 
 package org.apache.spark.storage
 
-import org.apache.spark.serializer.Serializer
+import java.io.File
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.atomic.AtomicInteger
 
+import scala.collection.JavaConversions._
 
-private[spark]
-class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter])
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector}
+import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup
 
+/** A group of writers for a ShuffleMapTask, one writer per reducer. */
+private[spark] trait ShuffleWriterGroup {
+  val writers: Array[BlockObjectWriter]
 
-private[spark]
-trait ShuffleBlocks {
-  def acquireWriters(mapId: Int): ShuffleWriterGroup
-  def releaseWriters(group: ShuffleWriterGroup)
+  /** @param success Indicates all writes were successful. If false, no blocks will be recorded. */
+  def releaseWriters(success: Boolean)
 }
 
-
+/**
+ * Manages assigning disk-based block writers to shuffle tasks. Each shuffle task gets one file
+ * per reducer (this set of files is called a ShuffleFileGroup).
+ *
+ * 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.
+ * Each shuffle file is then mapped to a FileSegment, which is a 3-tuple (file, offset, length)
+ * that specifies where in a given file the actual block data is located.
+ *
+ * Shuffle file metadata is stored in a space-efficient manner. Rather than simply mapping
+ * ShuffleBlockIds directly to FileSegments, each ShuffleFileGroup maintains a list of offsets for
+ * each block stored in each file. In order to find the location of a shuffle block, we search the
+ * files within a ShuffleFileGroups associated with the block's reducer.
+ */
 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", "false").toBoolean
+
+  private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
+
+  /**
+   * Contains all the state related to a particular shuffle. This includes a pool of unused
+   * ShuffleFileGroups, as well as all ShuffleFileGroups that have been created for the shuffle.
+   */
+  private class ShuffleState() {
+    val nextFileId = new AtomicInteger(0)
+    val unusedFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]()
+    val allFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]()
+  }
 
-  def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): ShuffleBlocks = {
-    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 writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
-          val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId)
-          blockManager.getDiskBlockWriter(blockId, serializer, bufferSize)
+  type ShuffleId = Int
+  private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState]
+
+  private
+  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup)
+
+  def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = {
+    new ShuffleWriterGroup {
+      shuffleStates.putIfAbsent(shuffleId, new ShuffleState())
+      private val shuffleState = shuffleStates(shuffleId)
+      private var fileGroup: ShuffleFileGroup = null
+
+      val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) {
+        fileGroup = getUnusedFileGroup()
+        Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
+          val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
+          blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize)
+        }
+      } else {
+        Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
+          val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
+          val blockFile = blockManager.diskBlockManager.getFile(blockId)
+          blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize)
+        }
+      }
+
+      override def releaseWriters(success: Boolean) {
+        if (consolidateShuffleFiles) {
+          if (success) {
+            val offsets = writers.map(_.fileSegment().offset)
+            fileGroup.recordMapOutput(mapId, offsets)
+          }
+          recycleFileGroup(fileGroup)
         }
-        new ShuffleWriterGroup(mapId, writers)
       }
 
-      override def releaseWriters(group: ShuffleWriterGroup) = {
-        // Nothing really to release here.
+      private def getUnusedFileGroup(): ShuffleFileGroup = {
+        val fileGroup = shuffleState.unusedFileGroups.poll()
+        if (fileGroup != null) fileGroup else newFileGroup()
+      }
+
+      private def newFileGroup(): ShuffleFileGroup = {
+        val fileId = shuffleState.nextFileId.getAndIncrement()
+        val files = Array.tabulate[File](numBuckets) { bucketId =>
+          val filename = physicalFileName(shuffleId, bucketId, fileId)
+          blockManager.diskBlockManager.getFile(filename)
+        }
+        val fileGroup = new ShuffleFileGroup(fileId, shuffleId, files)
+        shuffleState.allFileGroups.add(fileGroup)
+        fileGroup
+      }
+
+      private def recycleFileGroup(group: ShuffleFileGroup) {
+        shuffleState.unusedFileGroups.add(group)
       }
     }
   }
-}
 
+  /**
+   * Returns the physical file segment in which the given BlockId is located.
+   * This function should only be called if shuffle file consolidation is enabled, as it is
+   * an error condition if we don't find the expected block.
+   */
+  def getBlockLocation(id: ShuffleBlockId): FileSegment = {
+    // Search all file groups associated with this shuffle.
+    val shuffleState = shuffleStates(id.shuffleId)
+    for (fileGroup <- shuffleState.allFileGroups) {
+      val segment = fileGroup.getFileSegmentFor(id.mapId, id.reduceId)
+      if (segment.isDefined) { return segment.get }
+    }
+    throw new IllegalStateException("Failed to find shuffle block: " + id)
+  }
+
+  private def physicalFileName(shuffleId: Int, bucketId: Int, fileId: Int) = {
+    "merged_shuffle_%d_%d_%d".format(shuffleId, bucketId, fileId)
+  }
+
+  private def cleanup(cleanupTime: Long) {
+    shuffleStates.clearOldValues(cleanupTime)
+  }
+}
 
 private[spark]
 object ShuffleBlockManager {
+  /**
+   * A group of shuffle files, one per reducer.
+   * A particular mapper will be assigned a single ShuffleFileGroup to write its output to.
+   */
+  private class ShuffleFileGroup(val shuffleId: Int, val fileId: Int, val files: Array[File]) {
+    /**
+     * Stores the absolute index of each mapId in the files of this group. For instance,
+     * if mapId 5 is the first block in each file, mapIdToIndex(5) = 0.
+     */
+    private val mapIdToIndex = new PrimitiveKeyOpenHashMap[Int, Int]()
 
-  // Returns the block id for a given shuffle block.
-  def blockId(shuffleId: Int, bucketId: Int, groupId: Int): String = {
-    "shuffle_" + shuffleId + "_" + groupId + "_" + bucketId
-  }
+    /**
+     * Stores consecutive offsets of blocks into each reducer file, ordered by position in the file.
+     * This ordering allows us to compute block lengths by examining the following block offset.
+     * Note: mapIdToIndex(mapId) returns the index of the mapper into the vector for every
+     * reducer.
+     */
+    private val blockOffsetsByReducer = Array.fill[PrimitiveVector[Long]](files.length) {
+      new PrimitiveVector[Long]()
+    }
+
+    def numBlocks = mapIdToIndex.size
+
+    def apply(bucketId: Int) = files(bucketId)
+
+    def recordMapOutput(mapId: Int, offsets: Array[Long]) {
+      mapIdToIndex(mapId) = numBlocks
+      for (i <- 0 until offsets.length) {
+        blockOffsetsByReducer(i) += offsets(i)
+      }
+    }
 
-  // Returns true if the block is a shuffle block.
-  def isShuffle(blockId: String): Boolean = blockId.startsWith("shuffle_")
+    /** Returns the FileSegment associated with the given map task, or None if no entry exists. */
+    def getFileSegmentFor(mapId: Int, reducerId: Int): Option[FileSegment] = {
+      val file = files(reducerId)
+      val blockOffsets = blockOffsetsByReducer(reducerId)
+      val index = mapIdToIndex.getOrElse(mapId, -1)
+      if (index >= 0) {
+        val offset = blockOffsets(index)
+        val length =
+          if (index + 1 < numBlocks) {
+            blockOffsets(index + 1) - offset
+          } else {
+            file.length() - offset
+          }
+        assert(length >= 0)
+        Some(new FileSegment(file, offset, length))
+      } else {
+        None
+      }
+    }
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index 632ff047d10428dbcc1ab055e2d23bf1a5753b66..b5596dffd3449afe8279f83fa528623ea9ba7a1e 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -101,7 +101,7 @@ class StorageLevel private(
     var result = ""
     result += (if (useDisk) "Disk " else "")
     result += (if (useMemory) "Memory " else "")
-    result += (if (deserialized) "Deserialized " else "Serialized")
+    result += (if (deserialized) "Deserialized " else "Serialized ")
     result += "%sx Replicated".format(replication)
     result
   }
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..d52b3d8284d1510b1ba6d2b3c7adf00865a6e740
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
@@ -0,0 +1,103 @@
+/*
+ * 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.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.forMapTask(1, mapId, numOutputSplits,
+        new KryoSerializer())
+      val writers = shuffle.writers
+      for (i <- 1 to recordsPerMap) {
+        writers(i % numOutputSplits).write(writeData)
+      }
+      writers.map {w =>
+        w.commit()
+        total.addAndGet(w.fileSegment().length)
+        w.close()
+      }
+
+      shuffle.releaseWriters(true)
+    }
+
+    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/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index 2bb7715696cc7b25ede77a4ebe405df5fb73016f..1720007e4e70b6aff4685aacc8459a007343afc1 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -23,20 +23,24 @@ import org.apache.spark.util.Utils
 
 private[spark]
 case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
-  blocks: Map[String, BlockStatus]) {
+  blocks: Map[BlockId, BlockStatus]) {
 
-  def memUsed(blockPrefix: String = "") = {
-    blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize).
-      reduceOption(_+_).getOrElse(0l)
-  }
+  def memUsed() = blocks.values.map(_.memSize).reduceOption(_+_).getOrElse(0L)
 
-  def diskUsed(blockPrefix: String = "") = {
-    blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.diskSize).
-      reduceOption(_+_).getOrElse(0l)
-  }
+  def memUsedByRDD(rddId: Int) =
+    rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_+_).getOrElse(0L)
+
+  def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_+_).getOrElse(0L)
+
+  def diskUsedByRDD(rddId: Int) =
+    rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_+_).getOrElse(0L)
 
   def memRemaining : Long = maxMem - memUsed()
 
+  def rddBlocks = blocks.flatMap {
+    case (rdd: RDDBlockId, status) => Some(rdd, status)
+    case _ => None
+  }
 }
 
 case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
@@ -60,7 +64,7 @@ object StorageUtils {
   /* Returns RDD-level information, compiled from a list of StorageStatus objects */
   def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus],
     sc: SparkContext) : Array[RDDInfo] = {
-    rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc)
+    rddInfoFromBlockStatusList(storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc)
   }
 
   /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */
@@ -71,26 +75,21 @@ object StorageUtils {
   }
 
   /* Given a list of BlockStatus objets, returns information for each RDD */
-  def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus],
+  def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus],
     sc: SparkContext) : Array[RDDInfo] = {
 
     // Group by rddId, ignore the partition name
-    val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) =>
-      k.substring(0,k.lastIndexOf('_'))
-    }.mapValues(_.values.toArray)
+    val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray)
 
     // For each RDD, generate an RDDInfo object
-    val rddInfos = groupedRddBlocks.map { case (rddKey, rddBlocks) =>
+    val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) =>
       // Add up memory and disk sizes
       val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
       val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _)
 
-      // Find the id of the RDD, e.g. rdd_1 => 1
-      val rddId = rddKey.split("_").last.toInt
-
       // Get the friendly name and storage level for the RDD, if available
       sc.persistentRdds.get(rddId).map { r =>
-        val rddName = Option(r.name).getOrElse(rddKey)
+        val rddName = Option(r.name).getOrElse(rddId.toString)
         val rddStorageLevel = r.getStorageLevel
         RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize)
       }
@@ -101,16 +100,14 @@ object StorageUtils {
     rddInfos
   }
 
-  /* Removes all BlockStatus object that are not part of a block prefix */
-  def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus],
-    prefix: String) : Array[StorageStatus] = {
+  /* Filters storage status by a given RDD id. */
+  def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int)
+    : Array[StorageStatus] = {
 
     storageStatusList.map { status =>
-      val newBlocks = status.blocks.filterKeys(_.startsWith(prefix))
+      val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus]
       //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _)
       StorageStatus(status.blockManagerId, status.maxMem, newBlocks)
     }
-
   }
-
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
index f2ae8dd97dab1758308e3b36ffd1db1eb03b0955..a8db37ded14331b2b2b45d3d7e8fb90be364a4d1 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -36,11 +36,11 @@ private[spark] object ThreadingTest {
   val numBlocksPerProducer = 20000
 
   private[spark] class ProducerThread(manager: BlockManager, id: Int) extends Thread {
-    val queue = new ArrayBlockingQueue[(String, Seq[Int])](100)
+    val queue = new ArrayBlockingQueue[(BlockId, Seq[Int])](100)
 
     override def run() {
       for (i <- 1 to numBlocksPerProducer) {
-        val blockId = "b-" + id + "-" + i
+        val blockId = TestBlockId("b-" + id + "-" + i)
         val blockSize = Random.nextInt(1000)
         val block = (1 to blockSize).map(_ => Random.nextInt())
         val level = randomLevel()
@@ -64,7 +64,7 @@ private[spark] object ThreadingTest {
 
   private[spark] class ConsumerThread(
       manager: BlockManager,
-      queue: ArrayBlockingQueue[(String, Seq[Int])]
+      queue: ArrayBlockingQueue[(BlockId, Seq[Int])]
     ) extends Thread {
     var numBlockConsumed = 0
 
@@ -93,7 +93,7 @@ private[spark] object ThreadingTest {
     val actorSystem = ActorSystem("test")
     val serializer = new KryoSerializer
     val blockManagerMaster = new BlockManagerMaster(
-      actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+      Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
     val blockManager = new BlockManager(
       "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
     val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i))
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 453394dfda1ba0a5f317d81896fd2663fffa9e59..fcd1b518d099b366e2532f700282fd1c658aee1a 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
     val master = args(0)
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
index 42e9be6e19254a77715d269dae4a584ba0691b26..e596690bc3df87475adac7f3abbf97e35d176634 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
@@ -76,7 +76,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
       </tr>
     }
 
-    val execInfo = for (b <- 0 until storageStatusList.size) yield getExecInfo(b)
+    val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId)
     val execTable = UIUtils.listingTable(execHead, execRow, execInfo)
 
     val content =
@@ -99,16 +99,17 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
     UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors)
   }
 
-  def getExecInfo(a: Int): Seq[String] = {
-    val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId
-    val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort
-    val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString
-    val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString
-    val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString
-    val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString
-    val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size).getOrElse(0)
-    val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0)
-    val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0)
+  def getExecInfo(statusId: Int): Seq[String] = {
+    val status = sc.getExecutorStorageStatus(statusId)
+    val execId = status.blockManagerId.executorId
+    val hostPort = status.blockManagerId.hostPort
+    val rddBlocks = status.blocks.size.toString
+    val memUsed = status.memUsed().toString
+    val maxMem = status.maxMem.toString
+    val diskUsed = status.diskUsed().toString
+    val activeTasks = listener.executorToTasksActive.getOrElse(execId, HashSet.empty[Long]).size
+    val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0)
+    val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0)
     val totalTasks = activeTasks + failedTasks + completedTasks
 
     Seq(
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..69f9446babd9511643079355beb51e3a683eabe0 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,25 @@ 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 finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished)
 
       val summary =
         <div>
           <ul class="unstyled">
             <li>
-              <strong>CPU time: </strong>
-              {parent.formatDuration(listener.stageToTime.getOrElse(stageId, 0L) + activeTime)}
+              <strong>Total duration across all tasks: </strong>
+              {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
             </li>
             {if (hasShuffleRead)
               <li>
@@ -83,10 +85,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)
@@ -104,6 +106,33 @@ private[spark] class StagePage(parent: JobProgressUI) {
           val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map(
             ms => parent.formatDuration(ms.toLong))
 
+          val gettingResultTimes = validTasks.map{case (info, metrics, exception) =>
+            if (info.gettingResultTime > 0) {
+              (info.finishTime - info.gettingResultTime).toDouble
+            } else {
+              0.0
+            }
+          }
+          val gettingResultQuantiles = ("Time spent fetching task results" +:
+            Distribution(gettingResultTimes).get.getQuantiles().map(
+              millis => parent.formatDuration(millis.toLong)))
+          // The scheduler delay includes the network delay to send the task to the worker
+          // machine and to send back the result (but not the time to fetch the task result,
+          // if it needed to be fetched from the block manager on the worker).
+          val schedulerDelays = validTasks.map{case (info, metrics, exception) =>
+            val totalExecutionTime = {
+              if (info.gettingResultTime > 0) {
+                (info.gettingResultTime - info.launchTime).toDouble
+              } else {
+                (info.finishTime - info.launchTime).toDouble
+              }
+            }
+            totalExecutionTime - metrics.get.executorRunTime
+          }
+          val schedulerDelayQuantiles = ("Scheduler delay" +:
+            Distribution(schedulerDelays).get.getQuantiles().map(
+              millis => parent.formatDuration(millis.toLong)))
+
           def getQuantileCols(data: Seq[Double]) =
             Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong))
 
@@ -119,7 +148,10 @@ private[spark] class StagePage(parent: JobProgressUI) {
           }
           val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes)
 
-          val listings: Seq[Seq[String]] = Seq(serviceQuantiles,
+          val listings: Seq[Seq[String]] = Seq(
+            serviceQuantiles,
+            gettingResultQuantiles,
+            schedulerDelayQuantiles,
             if (hasShuffleRead) shuffleReadQuantiles else Nil,
             if (hasShuffleWrite) shuffleWriteQuantiles else Nil)
 
@@ -133,7 +165,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
         summary ++
         <h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
         <div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
-        <h4>Tasks</h4> ++ taskTable;
+        <h4>Tasks</h4> ++ taskTable
 
       headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages)
     }
@@ -152,7 +184,21 @@ private[spark] class StagePage(parent: JobProgressUI) {
       else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("")
     val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L)
 
+    val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead}
+    val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("")
+    val shuffleReadReadable = maybeShuffleRead.map{Utils.bytesToString(_)}.getOrElse("")
+
+    val maybeShuffleWrite = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten}
+    val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("")
+    val shuffleWriteReadable = maybeShuffleWrite.map{Utils.bytesToString(_)}.getOrElse("")
+
+    val maybeWriteTime = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleWriteTime}
+    val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("")
+    val writeTimeReadable = maybeWriteTime.map{ t => t / (1000 * 1000)}.map{ ms =>
+      if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("")
+
     <tr>
+      <td>{info.index}</td>
       <td>{info.taskId}</td>
       <td>{info.status}</td>
       <td>{info.taskLocality}</td>
@@ -165,12 +211,17 @@ private[spark] class StagePage(parent: JobProgressUI) {
         {if (gcTime > 0) parent.formatDuration(gcTime) else ""}
       </td>
       {if (shuffleRead) {
-        <td>{metrics.flatMap{m => m.shuffleReadMetrics}.map{s =>
-          Utils.bytesToString(s.remoteBytesRead)}.getOrElse("")}</td>
+         <td sorttable_customkey={shuffleReadSortable}>
+           {shuffleReadReadable}
+         </td>
       }}
       {if (shuffleWrite) {
-        <td>{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s =>
-          Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("")}</td>
+         <td sorttable_customkey={writeTimeSortable}>
+           {writeTimeReadable}
+         </td>
+         <td sorttable_customkey={shuffleWriteSortable}>
+           {shuffleWriteReadable}
+         </td>
       }}
       <td>{exception.map(e =>
         <span>
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..9ad6de3c6d8de79c758f1d0764b1a171bd56012c 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,43 @@ 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 shuffleReadSortable = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L)
+    val shuffleRead = shuffleReadSortable match {
       case 0 => ""
       case b => Utils.bytesToString(b)
     }
-    val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match {
+
+    val shuffleWriteSortable = listener.stageIdToShuffleWrite.getOrElse(s.stageId, 0L)
+    val shuffleWrite = shuffleWriteSortable 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>}
@@ -119,8 +122,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
       <td class="progress-cell">
         {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)}
       </td>
-      <td>{shuffleRead}</td>
-      <td>{shuffleWrite}</td>
+      <td sorttable_customekey={shuffleReadSortable.toString}>{shuffleRead}</td>
+      <td sorttable_customekey={shuffleWriteSortable.toString}>{shuffleWrite}</td>
     </tr>
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 43c125767709fc5ff9bfe7983689265726eb0a58..b83cd54f3c39a844d1e209075b9fa8fe24ba083b 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import org.apache.spark.storage.{StorageStatus, StorageUtils}
+import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils}
 import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus
 import org.apache.spark.ui.UIUtils._
 import org.apache.spark.ui.Page._
@@ -33,21 +33,20 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
   val sc = parent.sc
 
   def render(request: HttpServletRequest): Seq[Node] = {
-    val id = request.getParameter("id")
-    val prefix = "rdd_" + id.toString
+    val id = request.getParameter("id").toInt
     val storageStatusList = sc.getExecutorStorageStatus
-    val filteredStorageStatusList = StorageUtils.
-      filterStorageStatusByPrefix(storageStatusList, prefix)
+    val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id)
     val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head
 
     val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage")
-    val workers = filteredStorageStatusList.map((prefix, _))
+    val workers = filteredStorageStatusList.map((id, _))
     val workerTable = listingTable(workerHeaders, workerRow, workers)
 
     val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk",
       "Executors")
 
-    val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1)
+    val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.
+      sortWith(_._1.name < _._1.name)
     val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList)
     val blocks = blockStatuses.map {
       case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN")))
@@ -99,7 +98,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
     headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage)
   }
 
-  def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = {
+  def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = {
     val (id, block, locations) = row
     <tr>
       <td>{id}</td>
@@ -118,15 +117,15 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
     </tr>
   }
 
-  def workerRow(worker: (String, StorageStatus)): Seq[Node] = {
-    val (prefix, status) = worker
+  def workerRow(worker: (Int, StorageStatus)): Seq[Node] = {
+    val (rddId, status) = worker
     <tr>
       <td>{status.blockManagerId.host + ":" + status.blockManagerId.port}</td>
       <td>
-        {Utils.bytesToString(status.memUsed(prefix))}
+        {Utils.bytesToString(status.memUsedByRDD(rddId))}
         ({Utils.bytesToString(status.memRemaining)} Remaining)
       </td>
-      <td>{Utils.bytesToString(status.diskUsed(prefix))}</td>
+      <td>{Utils.bytesToString(status.diskUsedByRDD(rddId))}</td>
     </tr>
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index 8daf50ab6902b6bb9f86d94347cfe53039fe55f6..74133cef6c132ff6ec5029864c8da8a3c30ece27 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -17,11 +17,8 @@
 
 package org.apache.spark.util
 
-import akka.actor.{ActorSystem, ExtendedActorSystem}
+import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem}
 import com.typesafe.config.ConfigFactory
-import scala.concurrent.duration._
-import scala.concurrent.Await
-import akka.remote.RemoteActorRefProvider
 
 /**
  * Various utility classes for working with Akka.
@@ -34,38 +31,54 @@ private[spark] object AkkaUtils {
    *
    * Note: the `name` parameter is important, as even if a client sends a message to right
    * host + port, if the system name is incorrect, Akka will drop the message.
+   *
+   * If indestructible is set to true, the Actor System will continue running in the event
+   * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]].
    */
-  def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = {
+  def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false)
+    : (ActorSystem, Int) = {
+
     val akkaThreads   = System.getProperty("spark.akka.threads", "4").toInt
     val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
 
-    val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt
+    val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt
 
     val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
-    val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
-    // 10 seconds is the default akka timeout, but in a cluster, we need higher by default.
-    val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt
+    val lifecycleEvents =
+      if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
+
+    val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt
+    val akkaFailureDetector =
+      System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble
+    val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt
 
-    val akkaConf = ConfigFactory.parseString("""
-      akka.daemonic = on
-      akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
-      akka.stdout-loglevel = "ERROR"
-      akka.actor.provider = "akka.remote.RemoteActorRefProvider"
-      akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
-      akka.remote.netty.tcp.hostname = "%s"
-      akka.remote.netty.tcp.port = %d
-      akka.remote.netty.tcp.connection-timeout = %d s
-      akka.remote.netty.tcp.maximum-frame-size = %dMiB
-      akka.remote.netty.tcp.execution-pool-size = %d
-      akka.actor.default-dispatcher.throughput = %d
-      akka.remote.log-remote-lifecycle-events = %s
-                                             """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
-        lifecycleEvents))
+    val akkaConf = ConfigFactory.parseString(
+      s"""
+      |akka.daemonic = on
+      |akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
+      |akka.stdout-loglevel = "ERROR"
+      |akka.jvm-exit-on-fatal-error = off
+      |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
+      |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
+      |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
+      |akka.actor.provider = "akka.remote.RemoteActorRefProvider"
+      |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
+      |akka.remote.netty.tcp.hostname = "$host"
+      |akka.remote.netty.tcp.port = $port
+      |akka.remote.netty.tcp.tcp-nodelay = on
+      |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s
+      |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
+      |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
+      |akka.actor.default-dispatcher.throughput = $akkaBatchSize
+      |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
+      """.stripMargin)
 
-    val actorSystem = ActorSystem(name, akkaConf)
+    val actorSystem = if (indestructible) {
+      IndestructibleActorSystem(name, akkaConf)
+    } else {
+      ActorSystem(name, akkaConf)
+    }
 
-    // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a
-    // hack because Akka doesn't let you figure out the port through the public API yet.
     val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider
     val boundPort = provider.getDefaultAddress.port.get
     (actorSystem, boundPort)
diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
new file mode 100644
index 0000000000000000000000000000000000000000..8bb4ee3bfa22e3ad1233d778e59feedc892115f7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
@@ -0,0 +1,237 @@
+/*
+ * 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.util
+
+/**
+ * A simple open hash table optimized for the append-only use case, where keys
+ * are never removed, but the value for each key may be changed.
+ *
+ * This implementation uses quadratic probing with a power-of-2 hash table
+ * size, which is guaranteed to explore all spaces for each key (see
+ * http://en.wikipedia.org/wiki/Quadratic_probing).
+ *
+ * TODO: Cache the hash values of each key? java.util.HashMap does that.
+ */
+private[spark]
+class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable {
+  require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements")
+  require(initialCapacity >= 1, "Invalid initial capacity")
+
+  private var capacity = nextPowerOf2(initialCapacity)
+  private var mask = capacity - 1
+  private var curSize = 0
+  private var growThreshold = LOAD_FACTOR * capacity
+
+  // Holds keys and values in the same array for memory locality; specifically, the order of
+  // elements is key0, value0, key1, value1, key2, value2, etc.
+  private var data = new Array[AnyRef](2 * capacity)
+
+  // Treat the null key differently so we can use nulls in "data" to represent empty items.
+  private var haveNullValue = false
+  private var nullValue: V = null.asInstanceOf[V]
+
+  private val LOAD_FACTOR = 0.7
+
+  /** Get the value for a given key */
+  def apply(key: K): V = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      return nullValue
+    }
+    var pos = rehash(k.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (k.eq(curKey) || k.equals(curKey)) {
+        return data(2 * pos + 1).asInstanceOf[V]
+      } else if (curKey.eq(null)) {
+        return null.asInstanceOf[V]
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+    return null.asInstanceOf[V]
+  }
+
+  /** Set the value for a key */
+  def update(key: K, value: V): Unit = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      if (!haveNullValue) {
+        incrementSize()
+      }
+      nullValue = value
+      haveNullValue = true
+      return
+    }
+    var pos = rehash(key.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (curKey.eq(null)) {
+        data(2 * pos) = k
+        data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+        incrementSize()  // Since we added a new key
+        return
+      } else if (k.eq(curKey) || k.equals(curKey)) {
+        data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+        return
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+  }
+
+  /**
+   * Set the value for key to updateFunc(hadValue, oldValue), where oldValue will be the old value
+   * for key, if any, or null otherwise. Returns the newly updated value.
+   */
+  def changeValue(key: K, updateFunc: (Boolean, V) => V): V = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      if (!haveNullValue) {
+        incrementSize()
+      }
+      nullValue = updateFunc(haveNullValue, nullValue)
+      haveNullValue = true
+      return nullValue
+    }
+    var pos = rehash(k.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (k.eq(curKey) || k.equals(curKey)) {
+        val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V])
+        data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+        return newValue
+      } else if (curKey.eq(null)) {
+        val newValue = updateFunc(false, null.asInstanceOf[V])
+        data(2 * pos) = k
+        data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+        incrementSize()
+        return newValue
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+    null.asInstanceOf[V] // Never reached but needed to keep compiler happy
+  }
+
+  /** Iterator method from Iterable */
+  override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] {
+    var pos = -1
+
+    /** Get the next value we should return from next(), or null if we're finished iterating */
+    def nextValue(): (K, V) = {
+      if (pos == -1) {    // Treat position -1 as looking at the null value
+        if (haveNullValue) {
+          return (null.asInstanceOf[K], nullValue)
+        }
+        pos += 1
+      }
+      while (pos < capacity) {
+        if (!data(2 * pos).eq(null)) {
+          return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V])
+        }
+        pos += 1
+      }
+      null
+    }
+
+    override def hasNext: Boolean = nextValue() != null
+
+    override def next(): (K, V) = {
+      val value = nextValue()
+      if (value == null) {
+        throw new NoSuchElementException("End of iterator")
+      }
+      pos += 1
+      value
+    }
+  }
+
+  override def size: Int = curSize
+
+  /** Increase table size by 1, rehashing if necessary */
+  private def incrementSize() {
+    curSize += 1
+    if (curSize > growThreshold) {
+      growTable()
+    }
+  }
+
+  /**
+   * Re-hash a value to deal better with hash functions that don't differ in the lower bits.
+   * We use the Murmur Hash 3 finalization step that's also used in fastutil.
+   */
+  private def rehash(h: Int): Int = {
+    it.unimi.dsi.fastutil.HashCommon.murmurHash3(h)
+  }
+
+  /** Double the table's size and re-hash everything */
+  private def growTable() {
+    val newCapacity = capacity * 2
+    if (newCapacity >= (1 << 30)) {
+      // We can't make the table this big because we want an array of 2x
+      // that size for our data, but array sizes are at most Int.MaxValue
+      throw new Exception("Can't make capacity bigger than 2^29 elements")
+    }
+    val newData = new Array[AnyRef](2 * newCapacity)
+    val newMask = newCapacity - 1
+    // Insert all our old values into the new array. Note that because our old keys are
+    // unique, there's no need to check for equality here when we insert.
+    var oldPos = 0
+    while (oldPos < capacity) {
+      if (!data(2 * oldPos).eq(null)) {
+        val key = data(2 * oldPos)
+        val value = data(2 * oldPos + 1)
+        var newPos = rehash(key.hashCode) & newMask
+        var i = 1
+        var keepGoing = true
+        while (keepGoing) {
+          val curKey = newData(2 * newPos)
+          if (curKey.eq(null)) {
+            newData(2 * newPos) = key
+            newData(2 * newPos + 1) = value
+            keepGoing = false
+          } else {
+            val delta = i
+            newPos = (newPos + delta) & newMask
+            i += 1
+          }
+        }
+      }
+      oldPos += 1
+    }
+    data = newData
+    capacity = newCapacity
+    mask = newMask
+    growThreshold = LOAD_FACTOR * newCapacity
+  }
+
+  private def nextPowerOf2(n: Int): Int = {
+    val highBit = Integer.highestOneBit(n)
+    if (highBit == n) n else highBit << 1
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
new file mode 100644
index 0000000000000000000000000000000000000000..bf71882ef770a89c1725f791da119154e7ff5344
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+// Must be in akka.actor package as ActorSystemImpl is protected[akka].
+package akka.actor
+
+import scala.util.control.{ControlThrowable, NonFatal}
+
+import com.typesafe.config.Config
+
+/**
+ * An [[akka.actor.ActorSystem]] which refuses to shut down in the event of a fatal exception.
+ * This is necessary as Spark Executors are allowed to recover from fatal exceptions
+ * (see [[org.apache.spark.executor.Executor]]).
+ */
+object IndestructibleActorSystem {
+  def apply(name: String, config: Config): ActorSystem =
+    apply(name, config, ActorSystem.findClassLoader())
+
+  def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem =
+    new IndestructibleActorSystemImpl(name, config, classLoader).start()
+}
+
+private[akka] class IndestructibleActorSystemImpl(
+    override val name: String,
+    applicationConfig: Config,
+    classLoader: ClassLoader)
+  extends ActorSystemImpl(name, applicationConfig, classLoader) {
+
+  protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = {
+    val fallbackHandler = super.uncaughtExceptionHandler
+
+    new Thread.UncaughtExceptionHandler() {
+      def uncaughtException(thread: Thread, cause: Throwable): Unit = {
+        if (isFatalError(cause) && !settings.JvmExitOnFatalError) {
+          log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " +
+            "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name)
+          //shutdown()                 //TODO make it configurable
+        } else {
+          fallbackHandler.uncaughtException(thread, cause)
+        }
+      }
+    }
+  }
+
+  def isFatalError(e: Throwable): Boolean = {
+    e match {
+      case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable =>
+        false
+      case _ =>
+        true
+    }
+  }
+}
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 a430a75451173b197378981c8836bb8904f340df..7b41ef89f1da86fc2d5237218e408d7487cdd8b9 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.util
 
-import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
 import java.util.{TimerTask, Timer}
 import org.apache.spark.Logging
 
@@ -25,11 +24,14 @@ import org.apache.spark.Logging
 /**
  * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
  */
-class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
+class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
+  val name = cleanerType.toString
+
   private val delaySeconds = MetadataCleaner.getDelaySeconds
   private val periodSeconds = math.max(10, delaySeconds / 10)
   private val timer = new Timer(name + " cleanup timer", true)
 
+
   private val task = new TimerTask {
     override def run() {
       try {
@@ -53,9 +55,37 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
   }
 }
 
+object MetadataCleanerType extends Enumeration {
+
+  val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK,
+    SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value
+
+  type MetadataCleanerType = Value
+
+  def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString
+}
 
 object MetadataCleaner {
+
+  // using only sys props for now : so that workers can also get to it while preserving earlier behavior.
   def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt
-  def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) }
+
+  def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = {
+    System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt
+  }
+
+  def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) {
+    System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString)
+  }
+
+  def setDelaySeconds(delay: Int, resetAll: Boolean = true) {
+    // override for all ?
+    System.setProperty("spark.cleaner.ttl", delay.toString)
+    if (resetAll) {
+      for (cleanerType <- MetadataCleanerType.values) {
+        System.clearProperty(MetadataCleanerType.systemProperty(cleanerType))
+      }
+    }
+  }
 }
 
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 94ce50e96463eed7cadc4edbff4d0e6c0910e8f0..3f7858d2dedc311cdbf70017d10d0c17cd173dbc 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -18,22 +18,15 @@
 package org.apache.spark.util
 
 import java.io._
-import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket}
+import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address}
 import java.util.{Locale, Random, UUID}
+import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor}
 
-import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor}
-import java.util.regex.Pattern
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
-
-import scala.collection.mutable.{ArrayBuffer, HashMap}
 import scala.collection.JavaConversions._
 import scala.collection.Map
+import scala.collection.mutable.ArrayBuffer
 import scala.io.Source
 import scala.reflect.ClassTag
-import scala.Some
-
 
 import com.google.common.io.Files
 import com.google.common.util.concurrent.ThreadFactoryBuilder
@@ -43,7 +36,7 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
 import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
 import org.apache.spark.deploy.SparkHadoopUtil
 import java.nio.ByteBuffer
-import org.apache.spark.{SparkEnv, SparkException, Logging}
+import org.apache.spark.{SparkException, Logging}
 
 
 /**
@@ -155,7 +148,7 @@ private[spark] object Utils extends Logging {
     return buf
   }
 
-  private val shutdownDeletePaths = new collection.mutable.HashSet[String]()
+  private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]()
 
   // Register the path to be deleted via shutdown hook
   def registerShutdownDeleteDir(file: File) {
@@ -287,9 +280,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)
@@ -454,14 +446,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
@@ -472,10 +467,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()
@@ -820,4 +818,34 @@ 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]
+  }
+
+  /**
+   * Method executed for repeating a task for side effects.
+   * Unlike a for comprehension, it permits JVM JIT optimization
+   */
+  def times(numIters: Int)(f: => Unit): Unit = {
+    var i = 0
+    while (i < numIters) {
+      f
+      i += 1
+    }
+  }
+
+  /** 
+   * Timing method based on iterations that permit JVM JIT optimization.
+   * @param numIters number of iterations
+   * @param f function to be executed
+   */
+  def timeIt(numIters: Int)(f: => Unit): Long = {
+    val start = System.currentTimeMillis
+    times(numIters)(f)
+    System.currentTimeMillis - start
+  }
+
 }
diff --git a/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e9907e6c855aea4cc945f98c48b59293a441a1d8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala
@@ -0,0 +1,94 @@
+/*
+ * 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.util
+
+import java.util.{Random => JavaRandom}
+import org.apache.spark.util.Utils.timeIt
+
+/**
+ * This class implements a XORShift random number generator algorithm 
+ * Source:
+ * Marsaglia, G. (2003). Xorshift RNGs. Journal of Statistical Software, Vol. 8, Issue 14.
+ * @see <a href="http://www.jstatsoft.org/v08/i14/paper">Paper</a>
+ * This implementation is approximately 3.5 times faster than
+ * {@link java.util.Random java.util.Random}, partly because of the algorithm, but also due
+ * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class 
+ * uses a regular Long. We can forgo thread safety since we use a new instance of the RNG
+ * for each thread.
+ */
+private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) {
+  
+  def this() = this(System.nanoTime)
+
+  private var seed = init
+  
+  // we need to just override next - this will be called by nextInt, nextDouble,
+  // nextGaussian, nextLong, etc.
+  override protected def next(bits: Int): Int = {    
+    var nextSeed = seed ^ (seed << 21)
+    nextSeed ^= (nextSeed >>> 35)
+    nextSeed ^= (nextSeed << 4)  
+    seed = nextSeed
+    (nextSeed & ((1L << bits) -1)).asInstanceOf[Int]
+  }
+}
+
+/** Contains benchmark method and main method to run benchmark of the RNG */
+private[spark] object XORShiftRandom {
+
+  /**
+   * Main method for running benchmark
+   * @param args takes one argument - the number of random numbers to generate
+   */
+  def main(args: Array[String]): Unit = {
+    if (args.length != 1) {
+      println("Benchmark of XORShiftRandom vis-a-vis java.util.Random")
+      println("Usage: XORShiftRandom number_of_random_numbers_to_generate")
+      System.exit(1)
+    }
+    println(benchmark(args(0).toInt))
+  }
+
+  /**
+   * @param numIters Number of random numbers to generate while running the benchmark
+   * @return Map of execution times for {@link java.util.Random java.util.Random}
+   * and XORShift
+   */
+  def benchmark(numIters: Int) = {
+
+    val seed = 1L
+    val million = 1e6.toInt
+    val javaRand = new JavaRandom(seed)
+    val xorRand = new XORShiftRandom(seed)
+    
+    // this is just to warm up the JIT - we're not timing anything
+    timeIt(1e6.toInt) {
+      javaRand.nextInt()
+      xorRand.nextInt()
+    }
+
+    val iters = timeIt(numIters)(_)
+    
+    /* Return results as a map instead of just printing to screen
+    in case the user wants to do something with them */ 
+    Map("javaTime" -> iters {javaRand.nextInt()},
+        "xorTime" -> iters {xorRand.nextInt()})
+
+  }
+
+}
\ No newline at end of file
diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a1a452315d1437d35ff674b496371224edd98dea
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
@@ -0,0 +1,103 @@
+/*
+ * 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.util.collection
+
+
+/**
+ * A simple, fixed-size bit set implementation. This implementation is fast because it avoids
+ * safety/bound checking.
+ */
+class BitSet(numBits: Int) {
+
+  private[this] val words = new Array[Long](bit2words(numBits))
+  private[this] val numWords = words.length
+
+  /**
+   * Sets the bit at the specified index to true.
+   * @param index the bit index
+   */
+  def set(index: Int) {
+    val bitmask = 1L << (index & 0x3f)  // mod 64 and shift
+    words(index >> 6) |= bitmask        // div by 64 and mask
+  }
+
+  /**
+   * Return the value of the bit with the specified index. The value is true if the bit with
+   * the index is currently set in this BitSet; otherwise, the result is false.
+   *
+   * @param index the bit index
+   * @return the value of the bit with the specified index
+   */
+  def get(index: Int): Boolean = {
+    val bitmask = 1L << (index & 0x3f)   // mod 64 and shift
+    (words(index >> 6) & bitmask) != 0  // div by 64 and mask
+  }
+
+  /** Return the number of bits set to true in this BitSet. */
+  def cardinality(): Int = {
+    var sum = 0
+    var i = 0
+    while (i < numWords) {
+      sum += java.lang.Long.bitCount(words(i))
+      i += 1
+    }
+    sum
+  }
+
+  /**
+   * Returns the index of the first bit that is set to true that occurs on or after the
+   * specified starting index. If no such bit exists then -1 is returned.
+   *
+   * To iterate over the true bits in a BitSet, use the following loop:
+   *
+   *  for (int i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i+1)) {
+   *    // operate on index i here
+   *  }
+   *
+   * @param fromIndex the index to start checking from (inclusive)
+   * @return the index of the next set bit, or -1 if there is no such bit
+   */
+  def nextSetBit(fromIndex: Int): Int = {
+    var wordIndex = fromIndex >> 6
+    if (wordIndex >= numWords) {
+      return -1
+    }
+
+    // Try to find the next set bit in the current word
+    val subIndex = fromIndex & 0x3f
+    var word = words(wordIndex) >> subIndex
+    if (word != 0) {
+      return (wordIndex << 6) + subIndex + java.lang.Long.numberOfTrailingZeros(word)
+    }
+
+    // Find the next set bit in the rest of the words
+    wordIndex += 1
+    while (wordIndex < numWords) {
+      word = words(wordIndex)
+      if (word != 0) {
+        return (wordIndex << 6) + java.lang.Long.numberOfTrailingZeros(word)
+      }
+      wordIndex += 1
+    }
+
+    -1
+  }
+
+  /** Return the number of longs it would take to hold numBits. */
+  private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1
+}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c26f23d50024a2acec987bffeb35fffa2ad6314c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.util.collection
+
+import scala.reflect.ClassTag
+
+/**
+ * A fast hash map implementation for nullable keys. This hash map supports insertions and updates,
+ * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less
+ * space overhead.
+ *
+ * Under the hood, it uses our OpenHashSet implementation.
+ */
+private[spark]
+class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag](
+    initialCapacity: Int)
+  extends Iterable[(K, V)]
+  with Serializable {
+
+  def this() = this(64)
+
+  protected var _keySet = new OpenHashSet[K](initialCapacity)
+
+  // Init in constructor (instead of in declaration) to work around a Scala compiler specialization
+  // bug that would generate two arrays (one for Object and one for specialized T).
+  private var _values: Array[V] = _
+  _values = new Array[V](_keySet.capacity)
+
+  @transient private var _oldValues: Array[V] = null
+
+  // Treat the null key differently so we can use nulls in "data" to represent empty items.
+  private var haveNullValue = false
+  private var nullValue: V = null.asInstanceOf[V]
+
+  override def size: Int = if (haveNullValue) _keySet.size + 1 else _keySet.size
+
+  /** Get the value for a given key */
+  def apply(k: K): V = {
+    if (k == null) {
+      nullValue
+    } else {
+      val pos = _keySet.getPos(k)
+      if (pos < 0) {
+        null.asInstanceOf[V]
+      } else {
+        _values(pos)
+      }
+    }
+  }
+
+  /** Set the value for a key */
+  def update(k: K, v: V) {
+    if (k == null) {
+      haveNullValue = true
+      nullValue = v
+    } else {
+      val pos = _keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK
+      _values(pos) = v
+      _keySet.rehashIfNeeded(k, grow, move)
+      _oldValues = null
+    }
+  }
+
+  /**
+   * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise,
+   * set its value to mergeValue(oldValue).
+   *
+   * @return the newly updated value.
+   */
+  def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = {
+    if (k == null) {
+      if (haveNullValue) {
+        nullValue = mergeValue(nullValue)
+      } else {
+        haveNullValue = true
+        nullValue = defaultValue
+      }
+      nullValue
+    } else {
+      val pos = _keySet.addWithoutResize(k)
+      if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) {
+        val newValue = defaultValue
+        _values(pos & OpenHashSet.POSITION_MASK) = newValue
+        _keySet.rehashIfNeeded(k, grow, move)
+        newValue
+      } else {
+        _values(pos) = mergeValue(_values(pos))
+        _values(pos)
+      }
+    }
+  }
+
+  override def iterator = new Iterator[(K, V)] {
+    var pos = -1
+    var nextPair: (K, V) = computeNextPair()
+
+    /** Get the next value we should return from next(), or null if we're finished iterating */
+    def computeNextPair(): (K, V) = {
+      if (pos == -1) {    // Treat position -1 as looking at the null value
+        if (haveNullValue) {
+          pos += 1
+          return (null.asInstanceOf[K], nullValue)
+        }
+        pos += 1
+      }
+      pos = _keySet.nextPos(pos)
+      if (pos >= 0) {
+        val ret = (_keySet.getValue(pos), _values(pos))
+        pos += 1
+        ret
+      } else {
+        null
+      }
+    }
+
+    def hasNext = nextPair != null
+
+    def next() = {
+      val pair = nextPair
+      nextPair = computeNextPair()
+      pair
+    }
+  }
+
+  // The following member variables are declared as protected instead of private for the
+  // specialization to work (specialized class extends the non-specialized one and needs access
+  // to the "private" variables).
+  // They also should have been val's. We use var's because there is a Scala compiler bug that
+  // would throw illegal access error at runtime if they are declared as val's.
+  protected var grow = (newCapacity: Int) => {
+    _oldValues = _values
+    _values = new Array[V](newCapacity)
+  }
+
+  protected var move = (oldPos: Int, newPos: Int) => {
+    _values(newPos) = _oldValues(oldPos)
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
new file mode 100644
index 0000000000000000000000000000000000000000..87e009a4de93d8ab71cd2a0fb36393c67ebc9888
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
@@ -0,0 +1,279 @@
+/*
+ * 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.util.collection
+
+import scala.reflect._
+
+/**
+ * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never
+ * removed.
+ *
+ * The underlying implementation uses Scala compiler's specialization to generate optimized
+ * storage for two primitive types (Long and Int). It is much faster than Java's standard HashSet
+ * while incurring much less memory overhead. This can serve as building blocks for higher level
+ * data structures such as an optimized HashMap.
+ *
+ * This OpenHashSet is designed to serve as building blocks for higher level data structures
+ * such as an optimized hash map. Compared with standard hash set implementations, this class
+ * provides its various callbacks interfaces (e.g. allocateFunc, moveFunc) and interfaces to
+ * retrieve the position of a key in the underlying array.
+ *
+ * It uses quadratic probing with a power-of-2 hash table size, which is guaranteed
+ * to explore all spaces for each key (see http://en.wikipedia.org/wiki/Quadratic_probing).
+ */
+private[spark]
+class OpenHashSet[@specialized(Long, Int) T: ClassTag](
+    initialCapacity: Int,
+    loadFactor: Double)
+  extends Serializable {
+
+  require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements")
+  require(initialCapacity >= 1, "Invalid initial capacity")
+  require(loadFactor < 1.0, "Load factor must be less than 1.0")
+  require(loadFactor > 0.0, "Load factor must be greater than 0.0")
+
+  import OpenHashSet._
+
+  def this(initialCapacity: Int) = this(initialCapacity, 0.7)
+
+  def this() = this(64)
+
+  // The following member variables are declared as protected instead of private for the
+  // specialization to work (specialized class extends the non-specialized one and needs access
+  // to the "private" variables).
+
+  protected val hasher: Hasher[T] = {
+    // It would've been more natural to write the following using pattern matching. But Scala 2.9.x
+    // compiler has a bug when specialization is used together with this pattern matching, and
+    // throws:
+    // scala.tools.nsc.symtab.Types$TypeError: type mismatch;
+    //  found   : scala.reflect.AnyValManifest[Long]
+    //  required: scala.reflect.ClassTag[Int]
+    //         at scala.tools.nsc.typechecker.Contexts$Context.error(Contexts.scala:298)
+    //         at scala.tools.nsc.typechecker.Infer$Inferencer.error(Infer.scala:207)
+    //         ...
+    val mt = classTag[T]
+    if (mt == ClassTag.Long) {
+      (new LongHasher).asInstanceOf[Hasher[T]]
+    } else if (mt == ClassTag.Int) {
+      (new IntHasher).asInstanceOf[Hasher[T]]
+    } else {
+      new Hasher[T]
+    }
+  }
+
+  protected var _capacity = nextPowerOf2(initialCapacity)
+  protected var _mask = _capacity - 1
+  protected var _size = 0
+  protected var _growThreshold = (loadFactor * _capacity).toInt
+
+  protected var _bitset = new BitSet(_capacity)
+
+  // Init of the array in constructor (instead of in declaration) to work around a Scala compiler
+  // specialization bug that would generate two arrays (one for Object and one for specialized T).
+  protected var _data: Array[T] = _
+  _data = new Array[T](_capacity)
+
+  /** Number of elements in the set. */
+  def size: Int = _size
+
+  /** The capacity of the set (i.e. size of the underlying array). */
+  def capacity: Int = _capacity
+
+  /** Return true if this set contains the specified element. */
+  def contains(k: T): Boolean = getPos(k) != INVALID_POS
+
+  /**
+   * Add an element to the set. If the set is over capacity after the insertion, grow the set
+   * and rehash all elements.
+   */
+  def add(k: T) {
+    addWithoutResize(k)
+    rehashIfNeeded(k, grow, move)
+  }
+
+  /**
+   * Add an element to the set. This one differs from add in that it doesn't trigger rehashing.
+   * The caller is responsible for calling rehashIfNeeded.
+   *
+   * Use (retval & POSITION_MASK) to get the actual position, and
+   * (retval & EXISTENCE_MASK) != 0 for prior existence.
+   *
+   * @return The position where the key is placed, plus the highest order bit is set if the key
+   *         exists previously.
+   */
+  def addWithoutResize(k: T): Int = {
+    var pos = hashcode(hasher.hash(k)) & _mask
+    var i = 1
+    while (true) {
+      if (!_bitset.get(pos)) {
+        // This is a new key.
+        _data(pos) = k
+        _bitset.set(pos)
+        _size += 1
+        return pos | NONEXISTENCE_MASK
+      } else if (_data(pos) == k) {
+        // Found an existing key.
+        return pos
+      } else {
+        val delta = i
+        pos = (pos + delta) & _mask
+        i += 1
+      }
+    }
+    // Never reached here
+    assert(INVALID_POS != INVALID_POS)
+    INVALID_POS
+  }
+
+  /**
+   * Rehash the set if it is overloaded.
+   * @param k A parameter unused in the function, but to force the Scala compiler to specialize
+   *          this method.
+   * @param allocateFunc Callback invoked when we are allocating a new, larger array.
+   * @param moveFunc Callback invoked when we move the key from one position (in the old data array)
+   *                 to a new position (in the new data array).
+   */
+  def rehashIfNeeded(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) {
+    if (_size > _growThreshold) {
+      rehash(k, allocateFunc, moveFunc)
+    }
+  }
+
+  /**
+   * Return the position of the element in the underlying array, or INVALID_POS if it is not found.
+   */
+  def getPos(k: T): Int = {
+    var pos = hashcode(hasher.hash(k)) & _mask
+    var i = 1
+    while (true) {
+      if (!_bitset.get(pos)) {
+        return INVALID_POS
+      } else if (k == _data(pos)) {
+        return pos
+      } else {
+        val delta = i
+        pos = (pos + delta) & _mask
+        i += 1
+      }
+    }
+    // Never reached here
+    INVALID_POS
+  }
+
+  /** Return the value at the specified position. */
+  def getValue(pos: Int): T = _data(pos)
+
+  /**
+   * Return the next position with an element stored, starting from the given position inclusively.
+   */
+  def nextPos(fromPos: Int): Int = _bitset.nextSetBit(fromPos)
+
+  /**
+   * Double the table's size and re-hash everything. We are not really using k, but it is declared
+   * so Scala compiler can specialize this method (which leads to calling the specialized version
+   * of putInto).
+   *
+   * @param k A parameter unused in the function, but to force the Scala compiler to specialize
+   *          this method.
+   * @param allocateFunc Callback invoked when we are allocating a new, larger array.
+   * @param moveFunc Callback invoked when we move the key from one position (in the old data array)
+   *                 to a new position (in the new data array).
+   */
+  private def rehash(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) {
+    val newCapacity = _capacity * 2
+    allocateFunc(newCapacity)
+    val newBitset = new BitSet(newCapacity)
+    val newData = new Array[T](newCapacity)
+    val newMask = newCapacity - 1
+
+    var oldPos = 0
+    while (oldPos < capacity) {
+      if (_bitset.get(oldPos)) {
+        val key = _data(oldPos)
+        var newPos = hashcode(hasher.hash(key)) & newMask
+        var i = 1
+        var keepGoing = true
+        // No need to check for equality here when we insert so this has one less if branch than
+        // the similar code path in addWithoutResize.
+        while (keepGoing) {
+          if (!newBitset.get(newPos)) {
+            // Inserting the key at newPos
+            newData(newPos) = key
+            newBitset.set(newPos)
+            moveFunc(oldPos, newPos)
+            keepGoing = false
+          } else {
+            val delta = i
+            newPos = (newPos + delta) & newMask
+            i += 1
+          }
+        }
+      }
+      oldPos += 1
+    }
+
+    _bitset = newBitset
+    _data = newData
+    _capacity = newCapacity
+    _mask = newMask
+    _growThreshold = (loadFactor * newCapacity).toInt
+  }
+
+  /**
+   * Re-hash a value to deal better with hash functions that don't differ in the lower bits.
+   * We use the Murmur Hash 3 finalization step that's also used in fastutil.
+   */
+  private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h)
+
+  private def nextPowerOf2(n: Int): Int = {
+    val highBit = Integer.highestOneBit(n)
+    if (highBit == n) n else highBit << 1
+  }
+}
+
+
+private[spark]
+object OpenHashSet {
+
+  val INVALID_POS = -1
+  val NONEXISTENCE_MASK = 0x80000000
+  val POSITION_MASK = 0xEFFFFFF
+
+  /**
+   * A set of specialized hash function implementation to avoid boxing hash code computation
+   * in the specialized implementation of OpenHashSet.
+   */
+  sealed class Hasher[@specialized(Long, Int) T] {
+    def hash(o: T): Int = o.hashCode()
+  }
+
+  class LongHasher extends Hasher[Long] {
+    override def hash(o: Long): Int = (o ^ (o >>> 32)).toInt
+  }
+
+  class IntHasher extends Hasher[Int] {
+    override def hash(o: Int): Int = o
+  }
+
+  private def grow1(newSize: Int) {}
+  private def move1(oldPos: Int, newPos: Int) { }
+
+  private val grow = grow1 _
+  private val move = move1 _
+}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2e1ef06cbc4e11df051a53f73e5c6facb9c5c5a3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
@@ -0,0 +1,128 @@
+/*
+ * 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.util.collection
+
+import scala.reflect._
+
+/**
+ * A fast hash map implementation for primitive, non-null keys. This hash map supports
+ * insertions and updates, but not deletions. This map is about an order of magnitude
+ * faster than java.util.HashMap, while using much less space overhead.
+ *
+ * Under the hood, it uses our OpenHashSet implementation.
+ */
+private[spark]
+class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
+                              @specialized(Long, Int, Double) V: ClassTag](
+    initialCapacity: Int)
+  extends Iterable[(K, V)]
+  with Serializable {
+
+  def this() = this(64)
+
+  require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int])
+
+  // Init in constructor (instead of in declaration) to work around a Scala compiler specialization
+  // bug that would generate two arrays (one for Object and one for specialized T).
+  protected var _keySet: OpenHashSet[K] = _
+  private var _values: Array[V] = _
+  _keySet = new OpenHashSet[K](initialCapacity)
+  _values = new Array[V](_keySet.capacity)
+
+  private var _oldValues: Array[V] = null
+
+  override def size = _keySet.size
+
+  /** Get the value for a given key */
+  def apply(k: K): V = {
+    val pos = _keySet.getPos(k)
+    _values(pos)
+  }
+
+  /** Get the value for a given key, or returns elseValue if it doesn't exist. */
+  def getOrElse(k: K, elseValue: V): V = {
+    val pos = _keySet.getPos(k)
+    if (pos >= 0) _values(pos) else elseValue
+  }
+
+  /** Set the value for a key */
+  def update(k: K, v: V) {
+    val pos = _keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK
+    _values(pos) = v
+    _keySet.rehashIfNeeded(k, grow, move)
+    _oldValues = null
+  }
+
+  /**
+   * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise,
+   * set its value to mergeValue(oldValue).
+   *
+   * @return the newly updated value.
+   */
+  def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = {
+    val pos = _keySet.addWithoutResize(k)
+    if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) {
+      val newValue = defaultValue
+      _values(pos & OpenHashSet.POSITION_MASK) = newValue
+      _keySet.rehashIfNeeded(k, grow, move)
+      newValue
+    } else {
+      _values(pos) = mergeValue(_values(pos))
+      _values(pos)
+    }
+  }
+
+  override def iterator = new Iterator[(K, V)] {
+    var pos = 0
+    var nextPair: (K, V) = computeNextPair()
+
+    /** Get the next value we should return from next(), or null if we're finished iterating */
+    def computeNextPair(): (K, V) = {
+      pos = _keySet.nextPos(pos)
+      if (pos >= 0) {
+        val ret = (_keySet.getValue(pos), _values(pos))
+        pos += 1
+        ret
+      } else {
+        null
+      }
+    }
+
+    def hasNext = nextPair != null
+
+    def next() = {
+      val pair = nextPair
+      nextPair = computeNextPair()
+      pair
+    }
+  }
+
+  // The following member variables are declared as protected instead of private for the
+  // specialization to work (specialized class extends the unspecialized one and needs access
+  // to the "private" variables).
+  // They also should have been val's. We use var's because there is a Scala compiler bug that
+  // would throw illegal access error at runtime if they are declared as val's.
+  protected var grow = (newCapacity: Int) => {
+    _oldValues = _values
+    _values = new Array[V](newCapacity)
+  }
+
+  protected var move = (oldPos: Int, newPos: Int) => {
+    _values(newPos) = _oldValues(oldPos)
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b84eb65c62bc7732fc410fed483b34a237c963b8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.util.collection
+
+import scala.reflect.ClassTag
+
+/**
+ * An append-only, non-threadsafe, array-backed vector that is optimized for primitive types.
+ */
+private[spark]
+class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: Int = 64) {
+  private var _numElements = 0
+  private var _array: Array[V] = _
+
+  // NB: This must be separate from the declaration, otherwise the specialized parent class
+  // will get its own array with the same initial size.
+  _array = new Array[V](initialSize)
+
+  def apply(index: Int): V = {
+    require(index < _numElements)
+    _array(index)
+  }
+
+  def +=(value: V) {
+    if (_numElements == _array.length) {
+      resize(_array.length * 2)
+    }
+    _array(_numElements) = value
+    _numElements += 1
+  }
+
+  def capacity: Int = _array.length
+
+  def length: Int = _numElements
+
+  def size: Int = _numElements
+
+  /** Gets the underlying array backing this vector. */
+  def array: Array[V] = _array
+
+  /** Trims this vector so that the capacity is equal to the size. */
+  def trim(): PrimitiveVector[V] = resize(size)
+
+  /** Resizes the array, dropping elements if the total length decreases. */
+  def resize(newLength: Int): PrimitiveVector[V] = {
+    val newArray = new Array[V](newLength)
+    _array.copyToArray(newArray)
+    _array = newArray
+    if (newLength < _numElements) {
+      _numElements = newLength
+    }
+    this
+  }
+}
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/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
index 3a7171c48822155be972d57ae8c1c9d0b3e2e7eb..ea936e815b24adf894da049466c7b9644d78dea5 100644
--- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
@@ -23,7 +23,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite}
 import org.scalatest.mock.EasyMockSugar
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.storage.{BlockManager, StorageLevel}
+import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel}
 
 // TODO: Test the CacheManager's thread-safety aspects
 class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar {
@@ -52,13 +52,14 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
 
   test("get uncached rdd") {
     expecting {
-      blockManager.get("rdd_0_0").andReturn(None)
-      blockManager.put("rdd_0_0", ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, true).
-        andReturn(0)
+      blockManager.get(RDDBlockId(0, 0)).andReturn(None)
+      blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY,
+        true).andReturn(0)
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, runningLocally = false, null)
+      val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
+        taskMetrics = null)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(1, 2, 3, 4))
     }
@@ -66,11 +67,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
 
   test("get cached rdd") {
     expecting {
-      blockManager.get("rdd_0_0").andReturn(Some(ArrayBuffer(5, 6, 7).iterator))
+      blockManager.get(RDDBlockId(0, 0)).andReturn(Some(ArrayBuffer(5, 6, 7).iterator))
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, runningLocally = false, null)
+      val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
+        taskMetrics = null)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(5, 6, 7))
     }
@@ -79,11 +81,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
   test("get uncached local rdd") {
     expecting {
       // Local computation should not persist the resulting value, so don't expect a put().
-      blockManager.get("rdd_0_0").andReturn(None)
+      blockManager.get(RDDBlockId(0, 0)).andReturn(None)
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, runningLocally = true, null)
+      val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false,
+        taskMetrics = null)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(1, 2, 3, 4))
     }
diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index 78e35732d2fd8b726e5f8ef0c931beda79799596..f25d921d3f87faa004975500e56c8b96416c3a13 100644
--- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -22,7 +22,7 @@ import org.scalatest.FunSuite
 import java.io.File
 import org.apache.spark.rdd._
 import org.apache.spark.SparkContext._
-import storage.StorageLevel
+import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId}
 import org.apache.spark.util.Utils
 
 class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
@@ -63,8 +63,6 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     testCheckpointing(_.sample(false, 0.5, 0))
     testCheckpointing(_.glom())
     testCheckpointing(_.mapPartitions(_.map(_.toString)))
-    testCheckpointing(r => new MapPartitionsWithIndexRDD(r,
-      (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
     testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
     testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
     testCheckpointing(_.pipe(Seq("cat")))
@@ -84,7 +82,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
   }
 
   test("BlockRDD") {
-    val blockId = "id"
+    val blockId = TestBlockId("id")
     val blockManager = SparkEnv.get.blockManager
     blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
     val blockRDD = new BlockRDD[String](sc, Array(blockId))
@@ -192,7 +190,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
   }
 
   test("CheckpointRDD with zero partitions") {
-    val rdd = new BlockRDD[Int](sc, Array[String]())
+    val rdd = new BlockRDD[Int](sc, Array[BlockId]())
     assert(rdd.partitions.size === 0)
     assert(rdd.isCheckpointed === false)
     rdd.checkpoint()
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 988ab1747d1e24789cd67800bb85d830efdec0e4..d9cffb74de963692c67191f1b50f27fe5a00bb1c 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -18,24 +18,14 @@
 package org.apache.spark
 
 import network.ConnectionManagerId
-import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 import org.scalatest.concurrent.Timeouts._
+import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.prop.Checkers
 import org.scalatest.time.{Span, Millis}
-import org.scalacheck.Arbitrary._
-import org.scalacheck.Gen
-import org.scalacheck.Prop._
-import org.eclipse.jetty.server.{Server, Request, Handler}
-
-import com.google.common.io.Files
-
-import scala.collection.mutable.ArrayBuffer
 
 import SparkContext._
-import storage.{GetBlock, BlockManagerWorker, StorageLevel}
-import ui.JettyUtils
+import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel}
 
 
 class NotSerializableClass
@@ -193,7 +183,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
 
     // Get all the locations of the first partition and try to fetch the partitions
     // from those locations.
-    val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray
+    val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray
     val blockId = blockIds(0)
     val blockManager = SparkEnv.get.blockManager
     blockManager.master.getLocations(blockId).foreach(id => {
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 591c1d498dc0555d66ccded041b7fcc05503403a..4234f6eac72f4cebd63232e7c2d33c2521805ce1 100644
--- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -364,6 +364,20 @@ public class JavaAPISuite implements Serializable {
     List<Double> take = rdd.take(5);
   }
 
+  @Test
+  public void javaDoubleRDDHistoGram() {
+   JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+   // Test using generated buckets
+   Tuple2<double[], long[]> results = rdd.histogram(2);
+   double[] expected_buckets = {1.0, 2.5, 4.0};
+   long[] expected_counts = {2, 2};
+   Assert.assertArrayEquals(expected_buckets, results._1, 0.1);
+   Assert.assertArrayEquals(expected_counts, results._2);
+   // Test with provided buckets
+   long[] histogram = rdd.histogram(expected_buckets);
+   Assert.assertArrayEquals(expected_counts, histogram);
+  }
+
   @Test
   public void map() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
@@ -472,6 +486,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));
@@ -495,7 +530,7 @@ public class JavaAPISuite implements Serializable {
   @Test
   public void iterator() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
-    TaskContext context = new TaskContext(0, 0, 0, false, null);
+    TaskContext context = new TaskContext(0, 0, 0, false, false, null);
     Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
   }
 
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1121e06e2e6cc5b630f7ad5fc9020b8bfd6853a2
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -0,0 +1,209 @@
+/*
+ * 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
+
+import java.util.concurrent.Semaphore
+
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
+import scala.concurrent.future
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener}
+
+
+/**
+ * Test suite for cancelling running jobs. We run the cancellation tasks for single job action
+ * (e.g. count) as well as multi-job action (e.g. take). We test the local and cluster schedulers
+ * in both FIFO and fair scheduling modes.
+ */
+class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
+  with LocalSparkContext {
+
+  override def afterEach() {
+    super.afterEach()
+    resetSparkContext()
+    System.clearProperty("spark.scheduler.mode")
+  }
+
+  test("local mode, FIFO scheduler") {
+    System.setProperty("spark.scheduler.mode", "FIFO")
+    sc = new SparkContext("local[2]", "test")
+    testCount()
+    testTake()
+    // Make sure we can still launch tasks.
+    assert(sc.parallelize(1 to 10, 2).count === 10)
+  }
+
+  test("local mode, fair scheduler") {
+    System.setProperty("spark.scheduler.mode", "FAIR")
+    val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
+    System.setProperty("spark.scheduler.allocation.file", xmlPath)
+    sc = new SparkContext("local[2]", "test")
+    testCount()
+    testTake()
+    // Make sure we can still launch tasks.
+    assert(sc.parallelize(1 to 10, 2).count === 10)
+  }
+
+  test("cluster mode, FIFO scheduler") {
+    System.setProperty("spark.scheduler.mode", "FIFO")
+    sc = new SparkContext("local-cluster[2,1,512]", "test")
+    testCount()
+    testTake()
+    // Make sure we can still launch tasks.
+    assert(sc.parallelize(1 to 10, 2).count === 10)
+  }
+
+  test("cluster mode, fair scheduler") {
+    System.setProperty("spark.scheduler.mode", "FAIR")
+    val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
+    System.setProperty("spark.scheduler.allocation.file", xmlPath)
+    sc = new SparkContext("local-cluster[2,1,512]", "test")
+    testCount()
+    testTake()
+    // Make sure we can still launch tasks.
+    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
+    val sem1 = new Semaphore(0)
+    val sem2 = new Semaphore(0)
+
+    sc = new SparkContext("local[2]", "test")
+    sc.dagScheduler.addSparkListener(new SparkListener {
+      override def onTaskStart(taskStart: SparkListenerTaskStart) {
+        sem1.release()
+      }
+    })
+
+    // Create two actions that would share the some stages.
+    val rdd = sc.parallelize(1 to 10, 2).map { i =>
+      sem2.acquire()
+      (i, i)
+    }.reduceByKey(_+_)
+    val f1 = rdd.collectAsync()
+    val f2 = rdd.countAsync()
+
+    // Kill one of the action.
+    future {
+      sem1.acquire()
+      f1.cancel()
+      sem2.release(10)
+    }
+
+    // Expect both to fail now.
+    // TODO: update this test when we change Spark so cancelling f1 wouldn't affect f2.
+    intercept[SparkException] { f1.get() }
+    intercept[SparkException] { f2.get() }
+  }
+ */
+  def testCount() {
+    // Cancel before launching any tasks
+    {
+      val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync()
+      future { f.cancel() }
+      val e = intercept[SparkException] { f.get() }
+      assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+    }
+
+    // Cancel after some tasks have been launched
+    {
+      // 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()
+        }
+      })
+
+      val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync()
+      future {
+        // Wait until some tasks were launched before we cancel the job.
+        sem.acquire()
+        f.cancel()
+      }
+      val e = intercept[SparkException] { f.get() }
+      assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+    }
+  }
+
+  def testTake() {
+    // Cancel before launching any tasks
+    {
+      val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000)
+      future { f.cancel() }
+      val e = intercept[SparkException] { f.get() }
+      assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+    }
+
+    // Cancel after some tasks have been launched
+    {
+      // 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()
+        }
+      })
+      val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000)
+      future {
+        sem.acquire()
+        f.cancel()
+      }
+      val e = intercept[SparkException] { f.get() }
+      assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
index 459e257d79a36a4a12d3e5c1f0de6490e09106ca..8dd5786da6ff5c93f3bee3b3287baf89c7551220 100644
--- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
@@ -30,7 +30,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self
   @transient var sc: SparkContext = _
 
   override def beforeAll() {
-    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
     super.beforeAll()
   }
 
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 18fb1bf590ab4639ec0328a144d641da894f3c1c..271dc905bc386d6dcc2313c759d934bdf6f5ab2e 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 = Left(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 = Left(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 = Left(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,14 +100,14 @@ 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()
-    masterTracker.trackerActor = actorSystem.actorOf(
-        Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")
+    val masterTracker = new MapOutputTrackerMaster()
+    masterTracker.trackerActor = Left(actorSystem.actorOf(
+        Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker"))
 
     val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
     val slaveTracker = new MapOutputTracker()
-    slaveTracker.trackerActor = slaveSystem.actorFor(
-        "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")
+    slaveTracker.trackerActor = Right(slaveSystem.actorSelection(
+        "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker"))
 
     masterTracker.registerShuffle(10, 1)
     masterTracker.incrementEpoch()
diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
index 7d938917f2650754f93ed019ace4d02a4cf6d349..1374d01774693bfb1c71122a1dfb33c2b9b1a098 100644
--- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
@@ -142,11 +142,11 @@ class PartitioningSuite extends FunSuite with SharedSparkContext {
         .filter(_ >= 0.0)
 
     // Run the partitions, including the consecutive empty ones, through StatCounter
-    val stats: StatCounter = rdd.stats();
-    assert(abs(6.0 - stats.sum) < 0.01);
-    assert(abs(6.0/2 - rdd.mean) < 0.01);
-    assert(abs(1.0 - rdd.variance) < 0.01);
-    assert(abs(1.0 - rdd.stdev) < 0.01);
+    val stats: StatCounter = rdd.stats()
+    assert(abs(6.0 - stats.sum) < 0.01)
+    assert(abs(6.0/2 - rdd.mean) < 0.01)
+    assert(abs(1.0 - rdd.variance) < 0.01)
+    assert(abs(1.0 - rdd.stdev) < 0.01)
 
     // Add other tests here for classes that should be able to handle empty partitions correctly
   }
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..151af0d213c65e6144441f42be49163b2389faf4
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
@@ -0,0 +1,140 @@
+/*
+ * 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
+
+import org.scalatest.{FunSuite, PrivateMethodTester}
+
+import org.apache.spark.scheduler.TaskScheduler
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, SimrSchedulerBackend, SparkDeploySchedulerBackend}
+import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+import org.apache.spark.scheduler.local.LocalScheduler
+
+class SparkContextSchedulerCreationSuite
+  extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging {
+
+  def createTaskScheduler(master: String): TaskScheduler = {
+    // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the
+    // real schedulers, so we don't want to create a full SparkContext with the desired scheduler.
+    sc = new SparkContext("local", "test")
+    val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler)
+    SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test")
+  }
+
+  test("bad-master") {
+    val e = intercept[SparkException] {
+      createTaskScheduler("localhost:1234")
+    }
+    assert(e.getMessage.contains("Could not parse Master URL"))
+  }
+
+  test("local") {
+    createTaskScheduler("local") match {
+      case s: LocalScheduler =>
+        assert(s.threads === 1)
+        assert(s.maxFailures === 0)
+      case _ => fail()
+    }
+  }
+
+  test("local-n") {
+    createTaskScheduler("local[5]") match {
+      case s: LocalScheduler =>
+        assert(s.threads === 5)
+        assert(s.maxFailures === 0)
+      case _ => fail()
+    }
+  }
+
+  test("local-n-failures") {
+    createTaskScheduler("local[4, 2]") match {
+      case s: LocalScheduler =>
+        assert(s.threads === 4)
+        assert(s.maxFailures === 2)
+      case _ => fail()
+    }
+  }
+
+  test("simr") {
+    createTaskScheduler("simr://uri") match {
+      case s: ClusterScheduler =>
+        assert(s.backend.isInstanceOf[SimrSchedulerBackend])
+      case _ => fail()
+    }
+  }
+
+  test("local-cluster") {
+    createTaskScheduler("local-cluster[3, 14, 512]") match {
+      case s: ClusterScheduler =>
+        assert(s.backend.isInstanceOf[SparkDeploySchedulerBackend])
+      case _ => fail()
+    }
+  }
+
+  def testYarn(master: String, expectedClassName: String) {
+    try {
+      createTaskScheduler(master) match {
+        case s: ClusterScheduler =>
+          assert(s.getClass === Class.forName(expectedClassName))
+        case _ => fail()
+      }
+    } catch {
+      case e: SparkException =>
+        assert(e.getMessage.contains("YARN mode not available"))
+        logWarning("YARN not available, could not test actual YARN scheduler creation")
+      case e: Throwable => fail(e)
+    }
+  }
+
+  test("yarn-standalone") {
+    testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler")
+  }
+
+  test("yarn-client") {
+    testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
+  }
+
+  def testMesos(master: String, expectedClass: Class[_]) {
+    try {
+      createTaskScheduler(master) match {
+        case s: ClusterScheduler =>
+          assert(s.backend.getClass === expectedClass)
+        case _ => fail()
+      }
+    } catch {
+      case e: UnsatisfiedLinkError =>
+        assert(e.getMessage.contains("no mesos in"))
+        logWarning("Mesos not available, could not test actual Mesos scheduler creation")
+      case e: Throwable => fail(e)
+    }
+  }
+
+  test("mesos fine-grained") {
+    System.setProperty("spark.mesos.coarse", "false")
+    testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend])
+  }
+
+  test("mesos coarse-grained") {
+    System.setProperty("spark.mesos.coarse", "true")
+    testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend])
+  }
+
+  test("mesos with zookeeper") {
+    System.setProperty("spark.mesos.coarse", "false")
+    testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend])
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 05f8545c7b82837695769383c5bcd1ceeb49b4e0..0b38e239f9b024f787c4346d4b9adb076633b190 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -25,7 +25,7 @@ import net.liftweb.json.JsonAST.JValue
 import org.scalatest.FunSuite
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
-import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState, WorkerInfo}
 import org.apache.spark.deploy.worker.ExecutorRunner
 
 class JsonProtocolSuite extends FunSuite {
@@ -53,7 +53,8 @@ class JsonProtocolSuite extends FunSuite {
     val workers = Array[WorkerInfo](createWorkerInfo(), createWorkerInfo())
     val activeApps = Array[ApplicationInfo](createAppInfo())
     val completedApps = Array[ApplicationInfo]()
-    val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps)
+    val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps,
+      RecoveryState.ALIVE)
     val output = JsonProtocol.writeMasterState(stateResponse)
     assertValidJson(output)
   }
@@ -79,7 +80,7 @@ class JsonProtocolSuite extends FunSuite {
   }
   def createExecutorRunner() : ExecutorRunner = {
     new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
-      new File("sparkHome"), new File("workDir"))
+      new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
   }
 
   def assertValidJson(json: JValue) {
diff --git a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
similarity index 50%
rename from core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala
rename to core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 21f16ef2c6ececa8db7126f8897152ff5b523b28..4cb4ddc9cd66ae9385aa981618ebdd4baf23cbf3 100644
--- a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
@@ -15,31 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.spark
+package org.apache.spark.deploy.worker
 
+import java.io.File
 import org.scalatest.FunSuite
-import org.apache.spark.SparkContext._
-import org.apache.spark.rdd.{RDD, PartitionPruningRDD}
+import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription}
 
+class ExecutorRunnerTest extends FunSuite {
+  test("command includes appId") {
+    def f(s:String) = new File(s)
+    val sparkHome = sys.env("SPARK_HOME")
+    val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()),
+      sparkHome, "appUiUrl")
+    val appId = "12345-worker321-9876"
+    val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome),
+      f("ooga"), ExecutorState.RUNNING)
 
-class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext {
-
-  test("Pruned Partitions inherit locality prefs correctly") {
-    class TestPartition(i: Int) extends Partition {
-      def index = i
-    }
-    val rdd = new RDD[Int](sc, Nil) {
-      override protected def getPartitions = {
-        Array[Partition](
-            new TestPartition(1),
-            new TestPartition(2), 
-            new TestPartition(3))
-      }
-      def compute(split: Partition, context: TaskContext) = {Iterator()}
-    }
-    val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false})
-    val p = prunedRDD.partitions(0)
-    assert(p.index == 2)
-    assert(prunedRDD.partitions.length == 1)
+    assert(er.buildCommandSeq().last === appId)
   }
 }
diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..0d4c10db8ef33368cb662f4c896f045a80e41b4b
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
@@ -0,0 +1,202 @@
+/*
+ * 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.rdd
+
+import java.util.concurrent.Semaphore
+
+import scala.concurrent.{Await, TimeoutException}
+import scala.concurrent.duration.Duration
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+import org.scalatest.concurrent.Timeouts
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.{SparkContext, SparkException, LocalSparkContext}
+
+
+class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts {
+
+  @transient private var sc: SparkContext = _
+
+  override def beforeAll() {
+    sc = new SparkContext("local[2]", "test")
+  }
+
+  override def afterAll() {
+    LocalSparkContext.stop(sc)
+    sc = null
+  }
+
+  lazy val zeroPartRdd = new EmptyRDD[Int](sc)
+
+  test("countAsync") {
+    assert(zeroPartRdd.countAsync().get() === 0)
+    assert(sc.parallelize(1 to 10000, 5).countAsync().get() === 10000)
+  }
+
+  test("collectAsync") {
+    assert(zeroPartRdd.collectAsync().get() === Seq.empty)
+
+    val collected = sc.parallelize(1 to 1000, 3).collectAsync().get()
+    assert(collected === (1 to 1000))
+  }
+
+  test("foreachAsync") {
+    zeroPartRdd.foreachAsync(i => Unit).get()
+
+    val accum = sc.accumulator(0)
+    sc.parallelize(1 to 1000, 3).foreachAsync { i =>
+      accum += 1
+    }.get()
+    assert(accum.value === 1000)
+  }
+
+  test("foreachPartitionAsync") {
+    zeroPartRdd.foreachPartitionAsync(iter => Unit).get()
+
+    val accum = sc.accumulator(0)
+    sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter =>
+      accum += 1
+    }.get()
+    assert(accum.value === 9)
+  }
+
+  test("takeAsync") {
+    def testTake(rdd: RDD[Int], input: Seq[Int], num: Int) {
+      val expected = input.take(num)
+      val saw = rdd.takeAsync(num).get()
+      assert(saw == expected, "incorrect result for rdd with %d partitions (expected %s, saw %s)"
+        .format(rdd.partitions.size, expected, saw))
+    }
+    val input = Range(1, 1000)
+
+    var rdd = sc.parallelize(input, 1)
+    for (num <- Seq(0, 1, 999, 1000)) {
+      testTake(rdd, input, num)
+    }
+
+    rdd = sc.parallelize(input, 2)
+    for (num <- Seq(0, 1, 3, 500, 501, 999, 1000)) {
+      testTake(rdd, input, num)
+    }
+
+    rdd = sc.parallelize(input, 100)
+    for (num <- Seq(0, 1, 500, 501, 999, 1000)) {
+      testTake(rdd, input, num)
+    }
+
+    rdd = sc.parallelize(input, 1000)
+    for (num <- Seq(0, 1, 3, 999, 1000)) {
+      testTake(rdd, input, num)
+    }
+  }
+
+  /**
+   * Make sure onComplete, onSuccess, and onFailure are invoked correctly in the case
+   * of a successful job execution.
+   */
+  test("async success handling") {
+    val f = sc.parallelize(1 to 10, 2).countAsync()
+
+    // Use a semaphore to make sure onSuccess and onComplete's success path will be called.
+    // If not, the test will hang.
+    val sem = new Semaphore(0)
+
+    f.onComplete {
+      case scala.util.Success(res) =>
+        sem.release()
+      case scala.util.Failure(e) =>
+        info("Should not have reached this code path (onComplete matching Failure)")
+        throw new Exception("Task should succeed")
+    }
+    f.onSuccess { case a: Any =>
+      sem.release()
+    }
+    f.onFailure { case t =>
+      info("Should not have reached this code path (onFailure)")
+      throw new Exception("Task should succeed")
+    }
+    assert(f.get() === 10)
+
+    failAfter(10 seconds) {
+      sem.acquire(2)
+    }
+  }
+
+  /**
+   * Make sure onComplete, onSuccess, and onFailure are invoked correctly in the case
+   * of a failed job execution.
+   */
+  test("async failure handling") {
+    val f = sc.parallelize(1 to 10, 2).map { i =>
+      throw new Exception("intentional"); i
+    }.countAsync()
+
+    // Use a semaphore to make sure onFailure and onComplete's failure path will be called.
+    // If not, the test will hang.
+    val sem = new Semaphore(0)
+
+    f.onComplete {
+      case scala.util.Success(res) =>
+        info("Should not have reached this code path (onComplete matching Success)")
+        throw new Exception("Task should fail")
+      case scala.util.Failure(e) =>
+        sem.release()
+    }
+    f.onSuccess { case a: Any =>
+      info("Should not have reached this code path (onSuccess)")
+      throw new Exception("Task should fail")
+    }
+    f.onFailure { case t =>
+      sem.release()
+    }
+    intercept[SparkException] {
+      f.get()
+    }
+
+    failAfter(10 seconds) {
+      sem.acquire(2)
+    }
+  }
+
+  /**
+   * Awaiting FutureAction results
+   */
+  test("FutureAction result, infinite wait") {
+    val f = sc.parallelize(1 to 100, 4)
+              .countAsync()
+    assert(Await.result(f, Duration.Inf) === 100)
+  }
+
+  test("FutureAction result, finite wait") {
+    val f = sc.parallelize(1 to 100, 4)
+              .countAsync()
+    assert(Await.result(f, Duration(30, "seconds")) === 100)
+  }
+
+  test("FutureAction result, timeout") {
+    val f = sc.parallelize(1 to 100, 4)
+              .mapPartitions(itr => { Thread.sleep(20); itr })
+              .countAsync()
+    intercept[TimeoutException] {
+      Await.result(f, Duration(20, "milliseconds"))
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7f50a5a47c2ff3ef5f909469dca837bc399171f8
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
@@ -0,0 +1,271 @@
+/*
+ * 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.rdd
+
+import scala.math.abs
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd._
+import org.apache.spark._
+
+class DoubleRDDSuite extends FunSuite with SharedSparkContext {
+  // Verify tests on the histogram functionality. We test with both evenly
+  // and non-evenly spaced buckets as the bucket lookup function changes.
+  test("WorksOnEmpty") {
+    // Make sure that it works on an empty input
+    val rdd: RDD[Double] = sc.parallelize(Seq())
+    val buckets = Array(0.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksWithOutOfRangeWithOneBucket") {
+    // Verify that if all of the elements are out of range the counts are zero
+    val rdd = sc.parallelize(Seq(10.01, -0.01))
+    val buckets = Array(0.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksInRangeWithOneBucket") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+    val buckets = Array(0.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(4)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksInRangeWithOneBucketExactMatch") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+    val buckets = Array(1.0, 4.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(4)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksWithOutOfRangeWithTwoBuckets") {
+    // Verify that out of range works with two buckets
+    val rdd = sc.parallelize(Seq(10.01, -0.01))
+    val buckets = Array(0.0, 5.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(0, 0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksWithOutOfRangeWithTwoUnEvenBuckets") {
+    // Verify that out of range works with two un even buckets
+    val rdd = sc.parallelize(Seq(10.01, -0.01))
+    val buckets = Array(0.0, 4.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(0, 0)
+    assert(histogramResults === expectedHistogramResults)
+  }
+
+  test("WorksInRangeWithTwoBuckets") {
+    // Make sure that it works with two equally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6))
+    val buckets = Array(0.0, 5.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(3, 2)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksInRangeWithTwoBucketsAndNaN") {
+    // Make sure that it works with two equally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN))
+    val buckets = Array(0.0, 5.0, 10.0)
+    val histogramResults = rdd.histogram(buckets)
+    val histogramResults2 = rdd.histogram(buckets, true)
+    val expectedHistogramResults = Array(3, 2)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramResults2 === expectedHistogramResults)
+  }
+
+  test("WorksInRangeWithTwoUnevenBuckets") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6))
+    val buckets = Array(0.0, 5.0, 11.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(3, 2)
+    assert(histogramResults === expectedHistogramResults)
+  }
+
+  test("WorksMixedRangeWithTwoUnevenBuckets") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01))
+    val buckets = Array(0.0, 5.0, 11.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(4, 3)
+    assert(histogramResults === expectedHistogramResults)
+  }
+
+  test("WorksMixedRangeWithFourUnevenBuckets") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+      200.0, 200.1))
+    val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(4, 2, 1, 3)
+    assert(histogramResults === expectedHistogramResults)
+  }
+
+  test("WorksMixedRangeWithUnevenBucketsAndNaN") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+      200.0, 200.1, Double.NaN))
+    val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(4, 2, 1, 3)
+    assert(histogramResults === expectedHistogramResults)
+  }
+  // Make sure this works with a NaN end bucket
+  test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRange") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+      200.0, 200.1, Double.NaN))
+    val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(4, 2, 1, 2, 3)
+    assert(histogramResults === expectedHistogramResults)
+  }
+  // Make sure this works with a NaN end bucket and an inifity
+  test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") {
+    // Make sure that it works with two unequally spaced buckets and elements in each
+    val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+      200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN))
+    val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(4, 2, 1, 2, 4)
+    assert(histogramResults === expectedHistogramResults)
+  }
+
+  test("WorksWithOutOfRangeWithInfiniteBuckets") {
+    // Verify that out of range works with two buckets
+    val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN))
+    val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0)
+    val histogramResults = rdd.histogram(buckets)
+    val expectedHistogramResults = Array(1, 1)
+    assert(histogramResults === expectedHistogramResults)
+  }
+  // Test the failure mode with an invalid bucket array
+  test("ThrowsExceptionOnInvalidBucketArray") {
+    val rdd = sc.parallelize(Seq(1.0))
+    // Empty array
+    intercept[IllegalArgumentException] {
+      val buckets = Array.empty[Double]
+      val result = rdd.histogram(buckets)
+    }
+    // Single element array
+    intercept[IllegalArgumentException] {
+      val buckets = Array(1.0)
+      val result = rdd.histogram(buckets)
+    }
+  }
+
+  // Test automatic histogram function
+  test("WorksWithoutBucketsBasic") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+    val (histogramBuckets, histogramResults) = rdd.histogram(1)
+    val expectedHistogramResults = Array(4)
+    val expectedHistogramBuckets = Array(1.0, 4.0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramBuckets === expectedHistogramBuckets)
+  }
+  // Test automatic histogram function with a single element
+  test("WorksWithoutBucketsBasicSingleElement") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1))
+    val (histogramBuckets, histogramResults) = rdd.histogram(1)
+    val expectedHistogramResults = Array(1)
+    val expectedHistogramBuckets = Array(1.0, 1.0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramBuckets === expectedHistogramBuckets)
+  }
+  // Test automatic histogram function with a single element
+  test("WorksWithoutBucketsBasicNoRange") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 1, 1, 1))
+    val (histogramBuckets, histogramResults) = rdd.histogram(1)
+    val expectedHistogramResults = Array(4)
+    val expectedHistogramBuckets = Array(1.0, 1.0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramBuckets === expectedHistogramBuckets)
+  }
+
+  test("WorksWithoutBucketsBasicTwo") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+    val (histogramBuckets, histogramResults) = rdd.histogram(2)
+    val expectedHistogramResults = Array(2, 2)
+    val expectedHistogramBuckets = Array(1.0, 2.5, 4.0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramBuckets === expectedHistogramBuckets)
+  }
+
+  test("WorksWithoutBucketsWithMoreRequestedThanElements") {
+    // Verify the basic case of one bucket and all elements in that bucket works
+    val rdd = sc.parallelize(Seq(1, 2))
+    val (histogramBuckets, histogramResults) = rdd.histogram(10)
+    val expectedHistogramResults =
+      Array(1, 0, 0, 0, 0, 0, 0, 0, 0, 1)
+    val expectedHistogramBuckets =
+      Array(1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 2.0)
+    assert(histogramResults === expectedHistogramResults)
+    assert(histogramBuckets === expectedHistogramBuckets)
+  }
+
+  // Test the failure mode with an invalid RDD
+  test("ThrowsExceptionOnInvalidRDDs") {
+    // infinity
+    intercept[UnsupportedOperationException] {
+      val rdd = sc.parallelize(Seq(1, 1.0/0.0))
+      val result = rdd.histogram(1)
+    }
+    // NaN
+    intercept[UnsupportedOperationException] {
+      val rdd = sc.parallelize(Seq(1, Double.NaN))
+      val result = rdd.histogram(1)
+    }
+    // Empty
+    intercept[UnsupportedOperationException] {
+      val rdd: RDD[Double] = sc.parallelize(Seq())
+      val result = rdd.histogram(1)
+    }
+  }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
index 31f97fc1391e862a04a71aee36e0438aba4245bb..57d3382ed0b3fc30c47335743ecd5cae367ea6f3 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -106,7 +106,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
       }
     }
     visit(sums)
-    assert(deps.size === 2) // ShuffledRDD, ParallelCollection
+    assert(deps.size === 2) // ShuffledRDD, ParallelCollection.
   }
 
   test("join") {
diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..53a7b7c44df1c8345d0656c79f734cce6a1b47cd
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala
@@ -0,0 +1,86 @@
+/*
+ * 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.rdd
+
+import org.scalatest.FunSuite
+import org.apache.spark.{TaskContext, Partition, SharedSparkContext}
+
+
+class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext {
+
+
+  test("Pruned Partitions inherit locality prefs correctly") {
+
+    val rdd = new RDD[Int](sc, Nil) {
+      override protected def getPartitions = {
+        Array[Partition](
+          new TestPartition(0, 1),
+          new TestPartition(1, 1),
+          new TestPartition(2, 1))
+      }
+
+      def compute(split: Partition, context: TaskContext) = {
+        Iterator()
+      }
+    }
+    val prunedRDD = PartitionPruningRDD.create(rdd, {
+      x => if (x == 2) true else false
+    })
+    assert(prunedRDD.partitions.length == 1)
+    val p = prunedRDD.partitions(0)
+    assert(p.index == 0)
+    assert(p.asInstanceOf[PartitionPruningRDDPartition].parentSplit.index == 2)
+  }
+
+
+  test("Pruned Partitions can be unioned ") {
+
+    val rdd = new RDD[Int](sc, Nil) {
+      override protected def getPartitions = {
+        Array[Partition](
+          new TestPartition(0, 4),
+          new TestPartition(1, 5),
+          new TestPartition(2, 6))
+      }
+
+      def compute(split: Partition, context: TaskContext) = {
+        List(split.asInstanceOf[TestPartition].testValue).iterator
+      }
+    }
+    val prunedRDD1 = PartitionPruningRDD.create(rdd, {
+      x => if (x == 0) true else false
+    })
+
+    val prunedRDD2 = PartitionPruningRDD.create(rdd, {
+      x => if (x == 2) true else false
+    })
+
+    val merged = prunedRDD1 ++ prunedRDD2
+    assert(merged.count() == 2)
+    val take = merged.take(2)
+    assert(take.apply(0) == 4)
+    assert(take.apply(1) == 6)
+  }
+}
+
+class TestPartition(i: Int, value: Int) extends Partition with Serializable {
+  def index = i
+
+  def testValue = this.value
+
+}
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 8fd1115207ae5a73c1f83380c01ffed1f93a0a84..d8dcd6d14c23856e0d79cc519b45e3d14dc84072 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 2f933246b0ad219430ab45895e6ad90885f435ba..706d84a58b5630a80ea2bf23ea01e197ffa3f43a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -23,16 +23,15 @@ import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.LocalSparkContext
-import org.apache.spark.MapOutputTracker
-import org.apache.spark.rdd.RDD
+import org.apache.spark.MapOutputTrackerMaster
 import org.apache.spark.SparkContext
 import org.apache.spark.Partition
 import org.apache.spark.TaskContext
 import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency}
 import org.apache.spark.{FetchFailed, Success, TaskEndReason}
-import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster}
-
+import org.apache.spark.rdd.RDD
 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
 
 /**
  * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
@@ -60,11 +59,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
       taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch)
       taskSets += taskSet
     }
-    override def setListener(listener: TaskSchedulerListener) = {}
+    override def cancelTasks(stageId: Int) {}
+    override def setDAGScheduler(dagScheduler: DAGScheduler) = {}
     override def defaultParallelism() = 2
   }
 
-  var mapOutputTracker: MapOutputTracker = null
+  var mapOutputTracker: MapOutputTrackerMaster = null
   var scheduler: DAGScheduler = null
 
   /**
@@ -75,15 +75,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
   val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]]
   // stub out BlockManagerMaster.getLocations to use our cacheLocations
   val blockManagerMaster = new BlockManagerMaster(null) {
-      override def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
-        blockIds.map { name =>
-          val pieces = name.split("_")
-          if (pieces(0) == "rdd") {
-            val key = pieces(1).toInt -> pieces(2).toInt
-            cacheLocations.getOrElse(key, Seq())
-          } else {
-            Seq()
-          }
+      override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
+        blockIds.map {
+          _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)).
+            getOrElse(Seq())
         }.toSeq
       }
       override def removeExecutor(execId: String) {
@@ -104,8 +99,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     taskSets.clear()
     cacheLocations.clear()
     results.clear()
-    mapOutputTracker = new MapOutputTracker()
-    scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) {
+    mapOutputTracker = new MapOutputTrackerMaster()
+    scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) {
       override def runLocally(job: ActiveJob) {
         // don't bother with the thread while unit testing
         runLocallyWithinThread(job)
@@ -186,7 +181,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
       func: (TaskContext, Iterator[_]) => _ = jobComputeFunc,
       allowLocal: Boolean = false,
       listener: JobListener = listener) {
-    runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener))
+    val jobId = scheduler.nextJobId.getAndIncrement()
+    runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener))
   }
 
   /** Sends TaskSetFailed to the scheduler. */
@@ -210,6 +206,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     submit(rdd, Array(0))
     complete(taskSets(0), List((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("local job") {
@@ -220,8 +217,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
       override def getPreferredLocations(split: Partition) = Nil
       override def toString = "DAGSchedulerSuite Local RDD"
     }
-    runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener))
+    val jobId = scheduler.nextJobId.getAndIncrement()
+    runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("run trivial job w/ dependency") {
@@ -230,6 +229,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     submit(finalRdd, Array(0))
     complete(taskSets(0), Seq((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("cache location preferences w/ dependency") {
@@ -242,12 +242,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     assertLocations(taskSet, Seq(Seq("hostA", "hostB")))
     complete(taskSet, Seq((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("trivial job failure") {
     submit(makeRdd(1, Nil), Array(0))
     failed(taskSets(0), "some failure")
-    assert(failure.getMessage === "Job failed: some failure")
+    assert(failure.getMessage === "Job aborted: some failure")
+    assertDataStructuresEmpty
   }
 
   test("run trivial shuffle") {
@@ -263,6 +265,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
            Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
     complete(taskSets(1), Seq((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("run trivial shuffle with fetch failure") {
@@ -288,6 +291,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB"))
     complete(taskSets(3), Seq((Success, 43)))
     assert(results === Map(0 -> 42, 1 -> 43))
+    assertDataStructuresEmpty
   }
 
   test("ignore late map task completions") {
@@ -316,6 +320,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
            Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA")))
     complete(taskSets(1), Seq((Success, 42), (Success, 43)))
     assert(results === Map(0 -> 42, 1 -> 43))
+    assertDataStructuresEmpty
   }
 
   test("run trivial shuffle with out-of-band failure and retry") {
@@ -332,15 +337,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     complete(taskSets(0), Seq(
         (Success, makeMapStatus("hostA", 1)),
        (Success, makeMapStatus("hostB", 1))))
-   // have hostC complete the resubmitted task
-   complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1))))
-   assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
-          Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
-   complete(taskSets(2), Seq((Success, 42)))
-   assert(results === Map(0 -> 42))
- }
-
- test("recursive shuffle failures") {
+    // have hostC complete the resubmitted task
+    complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1))))
+    assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
+           Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
+    complete(taskSets(2), Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
+  }
+
+  test("recursive shuffle failures") {
     val shuffleOneRdd = makeRdd(2, Nil)
     val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
     val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne))
@@ -366,6 +372,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1))))
     complete(taskSets(5), Seq((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   test("cached post-shuffle") {
@@ -397,6 +404,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1))))
     complete(taskSets(4), Seq((Success, 42)))
     assert(results === Map(0 -> 42))
+    assertDataStructuresEmpty
   }
 
   /**
@@ -416,4 +424,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
   private def makeBlockManagerId(host: String): BlockManagerId =
     BlockManagerId("exec-" + host, host, 12345, 0)
 
+  private def assertDataStructuresEmpty = {
+    assert(scheduler.pendingTasks.isEmpty)
+    assert(scheduler.activeJobs.isEmpty)
+    assert(scheduler.failed.isEmpty)
+    assert(scheduler.idToActiveJob.isEmpty)
+    assert(scheduler.jobIdToStageIds.isEmpty)
+    assert(scheduler.stageIdToJobIds.isEmpty)
+    assert(scheduler.stageIdToStage.isEmpty)
+    assert(scheduler.stageToInfos.isEmpty)
+    assert(scheduler.resultStageToJob.isEmpty)
+    assert(scheduler.running.isEmpty)
+    assert(scheduler.shuffleToMapStage.isEmpty)
+    assert(scheduler.waiting.isEmpty)
+  }
 }
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..002368ff554f7226cc4f7f6b1643f8c1f61df0c8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
@@ -31,6 +31,7 @@ import org.apache.spark.rdd.RDD
 
 
 class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
+  val WAIT_TIMEOUT_MILLIS = 10000
 
   test("inner method") {
     sc = new SparkContext("local", "joblogger")
@@ -58,11 +59,14 @@ 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))
-    joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName)
+    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.getSimpleName)
     parentRdd.setName("MyRDD")
     joblogger.getRddNameTest(parentRdd) should be ("MyRDD")
     joblogger.createLogWriterTest(jobID)
@@ -88,8 +92,12 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
     sc.addSparkListener(joblogger)
     val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
     rdd.reduceByKey(_+_).collect()
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
+    val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER)
     
-    joblogger.getLogDir should be ("/tmp/spark")
+    joblogger.getLogDir should be ("/tmp/spark-%s".format(user))
     joblogger.getJobIDtoPrintWriter.size should be (1)
     joblogger.getStageIDToJobID.size should be (2)
     joblogger.getStageIDToJobID.get(0) should be (Some(0))
@@ -115,7 +123,9 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
     sc.addSparkListener(joblogger)
     val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
     rdd.reduceByKey(_+_).collect()
-    
+
+    assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
     joblogger.onJobStartCount should be (1)
     joblogger.onJobEndCount should be (1)
     joblogger.onTaskEndCount should be (8)
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 0d8742cb81b0bd3e2f9f4cc2f430a3e320bfd7a5..2e41438a527aac204662ec5f6c7b687a408b4da4 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)
@@ -37,9 +83,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
       i
     }
 
-    val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)}
-    d.count
-    val WAIT_TIMEOUT_MILLIS = 10000
+    val d = sc.parallelize(0 to 1e4.toInt, 64).map{i => w(i)}
+    d.count()
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
     listener.stageInfos.size should be (1)
 
@@ -50,7 +95,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
     val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)}
     d4.setName("A Cogroup")
 
-    d4.collectAsMap
+    d4.collectAsMap()
 
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
     listener.stageInfos.size should be (4)
@@ -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.tcp.maximum-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/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
index 2f12aaed18c6c99f485363b3d60e330feb795114..0f01515179f0b8683716ffcc0e2ba1b31b3dff87 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
@@ -17,10 +17,11 @@
 
 package org.apache.spark.scheduler.cluster
 
+import org.apache.spark.TaskContext
 import org.apache.spark.scheduler.{TaskLocation, Task}
 
-class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) {
-  override def run(attemptId: Long): Int = 0
+class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) {
+  override def runTask(context: TaskContext): Int = 0
 
   override def preferredLocations: Seq[TaskLocation] = prefLocs
 }
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
index a00198db8c3cdc6945e0070807f71ed8dcde905b..27c2d5336142f26f90f9903170ecfd0bc79b58fd 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
@@ -23,6 +23,7 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
 
 import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv}
 import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult}
+import org.apache.spark.storage.TaskResultBlockId
 
 /**
  * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter.
@@ -85,7 +86,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
     val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
     assert(result === 1.to(akkaFrameSize).toArray)
 
-    val RESULT_BLOCK_ID = "taskresult_0"
+    val RESULT_BLOCK_ID = TaskResultBlockId(0)
     assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0,
       "Expect result to be removed from the block manager.")
   }
diff --git a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
index af76c843e86b1704bf668d18d8ec333f18caeed0..1e676c1719337179e02940472632baeb7966b63a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
@@ -17,17 +17,15 @@
 
 package org.apache.spark.scheduler.local
 
-import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
-
-import org.apache.spark._
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster._
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.{ConcurrentMap, HashMap}
 import java.util.concurrent.Semaphore
 import java.util.concurrent.CountDownLatch
-import java.util.Properties
+
+import scala.collection.mutable.HashMap
+
+import org.scalatest.{BeforeAndAfterEach, FunSuite}
+
+import org.apache.spark._
+
 
 class Lock() {
   var finished = false
@@ -63,7 +61,12 @@ object TaskThreadInfo {
  * 5. each task(pending) must use "sleep" to  make sure it has been added to taskSetManager queue,
  *    thus it will be scheduled later when cluster has free cpu cores.
  */
-class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
+class LocalSchedulerSuite extends FunSuite with LocalSparkContext with BeforeAndAfterEach {
+
+  override def afterEach() {
+    super.afterEach()
+    System.clearProperty("spark.scheduler.mode")
+  }
 
   def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) {
 
@@ -148,12 +151,13 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
   }
 
   test("Local fair scheduler end-to-end test") {
-    sc = new SparkContext("local[8]", "LocalSchedulerSuite")
-    val sem = new Semaphore(0)
     System.setProperty("spark.scheduler.mode", "FAIR")
     val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
     System.setProperty("spark.scheduler.allocation.file", xmlPath)
 
+    sc = new SparkContext("local[8]", "LocalSchedulerSuite")
+    val sem = new Semaphore(0)
+
     createThread(10,"1",sc,sem)
     TaskThreadInfo.threadToStarted(10).await()
     createThread(20,"2",sc,sem)
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b647e8a6728eca95db66d428311b7eb3097e2d2d
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
@@ -0,0 +1,114 @@
+/*
+ * 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 org.scalatest.FunSuite
+
+class BlockIdSuite extends FunSuite {
+  def assertSame(id1: BlockId, id2: BlockId) {
+    assert(id1.name === id2.name)
+    assert(id1.hashCode === id2.hashCode)
+    assert(id1 === id2)
+  }
+
+  def assertDifferent(id1: BlockId, id2: BlockId) {
+    assert(id1.name != id2.name)
+    assert(id1.hashCode != id2.hashCode)
+    assert(id1 != id2)
+  }
+
+  test("test-bad-deserialization") {
+    try {
+      // Try to deserialize an invalid block id.
+      BlockId("myblock")
+      fail()
+    } catch {
+      case e: IllegalStateException => // OK
+      case _: Throwable => fail()
+    }
+  }
+
+  test("rdd") {
+    val id = RDDBlockId(1, 2)
+    assertSame(id, RDDBlockId(1, 2))
+    assertDifferent(id, RDDBlockId(1, 1))
+    assert(id.name === "rdd_1_2")
+    assert(id.asRDDId.get.rddId === 1)
+    assert(id.asRDDId.get.splitIndex === 2)
+    assert(id.isRDD)
+    assertSame(id, BlockId(id.toString))
+  }
+
+  test("shuffle") {
+    val id = ShuffleBlockId(1, 2, 3)
+    assertSame(id, ShuffleBlockId(1, 2, 3))
+    assertDifferent(id, ShuffleBlockId(3, 2, 3))
+    assert(id.name === "shuffle_1_2_3")
+    assert(id.asRDDId === None)
+    assert(id.shuffleId === 1)
+    assert(id.mapId === 2)
+    assert(id.reduceId === 3)
+    assert(id.isShuffle)
+    assertSame(id, BlockId(id.toString))
+  }
+
+  test("broadcast") {
+    val id = BroadcastBlockId(42)
+    assertSame(id, BroadcastBlockId(42))
+    assertDifferent(id, BroadcastBlockId(123))
+    assert(id.name === "broadcast_42")
+    assert(id.asRDDId === None)
+    assert(id.broadcastId === 42)
+    assert(id.isBroadcast)
+    assertSame(id, BlockId(id.toString))
+  }
+
+  test("taskresult") {
+    val id = TaskResultBlockId(60)
+    assertSame(id, TaskResultBlockId(60))
+    assertDifferent(id, TaskResultBlockId(61))
+    assert(id.name === "taskresult_60")
+    assert(id.asRDDId === None)
+    assert(id.taskId === 60)
+    assert(!id.isRDD)
+    assertSame(id, BlockId(id.toString))
+  }
+
+  test("stream") {
+    val id = StreamBlockId(1, 100)
+    assertSame(id, StreamBlockId(1, 100))
+    assertDifferent(id, StreamBlockId(2, 101))
+    assert(id.name === "input-1-100")
+    assert(id.asRDDId === None)
+    assert(id.streamId === 1)
+    assert(id.uniqueId === 100)
+    assert(!id.isBroadcast)
+    assertSame(id, BlockId(id.toString))
+  }
+
+  test("test") {
+    val id = TestBlockId("abc")
+    assertSame(id, TestBlockId("abc"))
+    assertDifferent(id, TestBlockId("ab"))
+    assert(id.name === "test_abc")
+    assert(id.asRDDId === None)
+    assert(id.id === "abc")
+    assert(!id.isShuffle)
+    assertSame(id, BlockId(id.toString))
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 038a9acb8593c07f45c0e660bf0b719e97a8b888..5b4d63b954cc850ff9b0bd8c2f08e6612edd2607 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -32,7 +32,6 @@ import org.scalatest.time.SpanSugar._
 import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream}
 import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
 
-
 class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
   var store: BlockManager = null
   var store2: BlockManager = null
@@ -46,6 +45,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   System.setProperty("spark.kryoserializer.buffer.mb", "1")
   val serializer = new KryoSerializer
 
+  // Implicitly convert strings to BlockIds for test clarity.
+  implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value)
+  def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId)
+
   before {
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0)
     this.actorSystem = actorSystem
@@ -53,7 +56,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     System.setProperty("spark.hostPort", "localhost:" + boundPort)
 
     master = new BlockManagerMaster(
-      actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+      Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
 
     // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
     oldArch = System.setProperty("os.arch", "amd64")
@@ -229,31 +232,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
     // Putting a1, a2 and a3 in memory.
-    store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY)
     store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY)
     master.removeRdd(0, blocking = false)
 
     eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
-      store.getSingle("rdd_0_0") should be (None)
-      master.getLocations("rdd_0_0") should have size 0
+      store.getSingle(rdd(0, 0)) should be (None)
+      master.getLocations(rdd(0, 0)) should have size 0
     }
     eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
-      store.getSingle("rdd_0_1") should be (None)
-      master.getLocations("rdd_0_1") should have size 0
+      store.getSingle(rdd(0, 1)) should be (None)
+      master.getLocations(rdd(0, 1)) should have size 0
     }
     eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
       store.getSingle("nonrddblock") should not be (None)
       master.getLocations("nonrddblock") should have size (1)
     }
 
-    store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY)
     master.removeRdd(0, blocking = true)
-    store.getSingle("rdd_0_0") should be (None)
-    master.getLocations("rdd_0_0") should have size 0
-    store.getSingle("rdd_0_1") should be (None)
-    master.getLocations("rdd_0_1") should have size 0
+    store.getSingle(rdd(0, 0)) should be (None)
+    master.getLocations(rdd(0, 0)) should have size 0
+    store.getSingle(rdd(0, 1)) should be (None)
+    master.getLocations(rdd(0, 1)) should have size 0
   }
 
   test("reregistration on heart beat") {
@@ -372,41 +375,41 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
-    store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY)
     // Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2
     // from the same RDD
-    assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
-    assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
-    assert(store.getSingle("rdd_0_1") != None, "rdd_0_1 was not in store")
+    assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+    assert(store.getSingle(rdd(0, 2)) != None, "rdd_0_2 was not in store")
+    assert(store.getSingle(rdd(0, 1)) != None, "rdd_0_1 was not in store")
     // Check that rdd_0_3 doesn't replace them even after further accesses
-    assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
-    assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
-    assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+    assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+    assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+    assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
   }
 
   test("in-memory LRU for partitions of multiple RDDs") {
     store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
-    store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     // At this point rdd_1_1 should've replaced rdd_0_1
-    assert(store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was not in store")
-    assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
-    assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
+    assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store")
+    assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store")
+    assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store")
     // Do a get() on rdd_0_2 so that it is the most recently used item
-    assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
+    assert(store.getSingle(rdd(0, 2)) != None, "rdd_0_2 was not in store")
     // Put in more partitions from RDD 0; they should replace rdd_1_1
-    store.putSingle("rdd_0_3", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
-    store.putSingle("rdd_0_4", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+    store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped
     // when we try to add rdd_0_4.
-    assert(!store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was in store")
-    assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
-    assert(!store.memoryStore.contains("rdd_0_4"), "rdd_0_4 was in store")
-    assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
-    assert(store.memoryStore.contains("rdd_0_3"), "rdd_0_3 was not in store")
+    assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store")
+    assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store")
+    assert(!store.memoryStore.contains(rdd(0, 4)), "rdd_0_4 was in store")
+    assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store")
+    assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store")
   }
 
   test("on-disk storage") {
@@ -590,43 +593,46 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     try {
       System.setProperty("spark.shuffle.compress", "true")
       store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
-      store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed")
+      store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100,
+        "shuffle_0_0_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.shuffle.compress", "false")
       store = new BlockManager("exec2", actorSystem, master, serializer, 2000)
-      store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed")
+      store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000,
+        "shuffle_0_0_0 was compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.broadcast.compress", "true")
       store = new BlockManager("exec3", actorSystem, master, serializer, 2000)
-      store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed")
+      store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100,
+        "broadcast_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.broadcast.compress", "false")
       store = new BlockManager("exec4", actorSystem, master, serializer, 2000)
-      store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed")
+      store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.rdd.compress", "true")
       store = new BlockManager("exec5", actorSystem, master, serializer, 2000)
-      store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed")
+      store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.rdd.compress", "false")
       store = new BlockManager("exec6", actorSystem, master, serializer, 2000)
-      store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
-      assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed")
+      store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+      assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed")
       store.stop()
       store = null
 
diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..070982e798f343b2bdbdeaf6c663b0e0b2575873
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -0,0 +1,111 @@
+/*
+ * 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.{FileWriter, File}
+
+import scala.collection.mutable
+
+import com.google.common.io.Files
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
+
+class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll {
+
+  val rootDir0 = Files.createTempDir()
+  rootDir0.deleteOnExit()
+  val rootDir1 = Files.createTempDir()
+  rootDir1.deleteOnExit()
+  val rootDirs = rootDir0.getName + "," + rootDir1.getName
+  println("Created root dirs: " + rootDirs)
+
+  // This suite focuses primarily on consolidation features,
+  // so we coerce consolidation if not already enabled.
+  val consolidateProp = "spark.shuffle.consolidateFiles"
+  val oldConsolidate = Option(System.getProperty(consolidateProp))
+  System.setProperty(consolidateProp, "true")
+
+  val shuffleBlockManager = new ShuffleBlockManager(null) {
+    var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]()
+    override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id)
+  }
+
+  var diskBlockManager: DiskBlockManager = _
+
+  override def afterAll() {
+    oldConsolidate.map(c => System.setProperty(consolidateProp, c))
+  }
+
+  override def beforeEach() {
+    diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs)
+    shuffleBlockManager.idToSegmentMap.clear()
+  }
+
+  test("basic block creation") {
+    val blockId = new TestBlockId("test")
+    assertSegmentEquals(blockId, blockId.name, 0, 0)
+
+    val newFile = diskBlockManager.getFile(blockId)
+    writeToFile(newFile, 10)
+    assertSegmentEquals(blockId, blockId.name, 0, 10)
+
+    newFile.delete()
+  }
+
+  test("block appending") {
+    val blockId = new TestBlockId("test")
+    val newFile = diskBlockManager.getFile(blockId)
+    writeToFile(newFile, 15)
+    assertSegmentEquals(blockId, blockId.name, 0, 15)
+    val newFile2 = diskBlockManager.getFile(blockId)
+    assert(newFile === newFile2)
+    writeToFile(newFile2, 12)
+    assertSegmentEquals(blockId, blockId.name, 0, 27)
+    newFile.delete()
+  }
+
+  test("block remapping") {
+    val filename = "test"
+    val blockId0 = new ShuffleBlockId(1, 2, 3)
+    val newFile = diskBlockManager.getFile(filename)
+    writeToFile(newFile, 15)
+    shuffleBlockManager.idToSegmentMap(blockId0) = new FileSegment(newFile, 0, 15)
+    assertSegmentEquals(blockId0, filename, 0, 15)
+
+    val blockId1 = new ShuffleBlockId(1, 2, 4)
+    val newFile2 = diskBlockManager.getFile(filename)
+    writeToFile(newFile2, 12)
+    shuffleBlockManager.idToSegmentMap(blockId1) = new FileSegment(newFile, 15, 12)
+    assertSegmentEquals(blockId1, filename, 15, 12)
+
+    assert(newFile === newFile2)
+    newFile.delete()
+  }
+
+  def assertSegmentEquals(blockId: BlockId, filename: String, offset: Int, length: Int) {
+    val segment = diskBlockManager.getBlockLocation(blockId)
+    assert(segment.file.getName === filename)
+    assert(segment.offset === offset)
+    assert(segment.length === length)
+  }
+
+  def writeToFile(file: File, numBytes: Int) {
+    val writer = new FileWriter(file, true)
+    for (i <- 0 until numBytes) writer.write(i)
+    writer.close()
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7177919a58157c89400ae2fe88eaa256b08b945e
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
@@ -0,0 +1,154 @@
+/*
+ * 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.util
+
+import scala.collection.mutable.HashSet
+
+import org.scalatest.FunSuite
+
+class AppendOnlyMapSuite extends FunSuite {
+  test("initialization") {
+    val goodMap1 = new AppendOnlyMap[Int, Int](1)
+    assert(goodMap1.size === 0)
+    val goodMap2 = new AppendOnlyMap[Int, Int](255)
+    assert(goodMap2.size === 0)
+    val goodMap3 = new AppendOnlyMap[Int, Int](256)
+    assert(goodMap3.size === 0)
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29
+    }
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](-1)
+    }
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](0)
+    }
+  }
+
+  test("object keys and values") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map("" + i) === "" + i)
+    }
+    assert(map("0") === null)
+    assert(map("101") === null)
+    assert(map(null) === null)
+    val set = new HashSet[(String, String)]
+    for ((k, v) <- map) {   // Test the foreach method
+      set += ((k, v))
+    }
+    assert(set === (1 to 100).map(_.toString).map(x => (x, x)).toSet)
+  }
+
+  test("primitive keys and values") {
+    val map = new AppendOnlyMap[Int, Int]()
+    for (i <- 1 to 100) {
+      map(i) = i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map(i) === i)
+    }
+    assert(map(0) === null)
+    assert(map(101) === null)
+    val set = new HashSet[(Int, Int)]
+    for ((k, v) <- map) {   // Test the foreach method
+      set += ((k, v))
+    }
+    assert(set === (1 to 100).map(x => (x, x)).toSet)
+  }
+
+  test("null keys") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    assert(map(null) === null)
+    map(null) = "hello"
+    assert(map.size === 101)
+    assert(map(null) === "hello")
+  }
+
+  test("null values") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = null
+    }
+    assert(map.size === 100)
+    assert(map("1") === null)
+    assert(map(null) === null)
+    assert(map.size === 100)
+    map(null) = null
+    assert(map.size === 101)
+    assert(map(null) === null)
+  }
+
+  test("changeValue") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      val res = map.changeValue("" + i, (hadValue, oldValue) => {
+        assert(hadValue === true)
+        assert(oldValue === "" + i)
+        oldValue + "!"
+      })
+      assert(res === i + "!")
+    }
+    // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a
+    // bug where changeValue would return the wrong result when the map grew on that insert
+    for (i <- 101 to 400) {
+      val res = map.changeValue("" + i, (hadValue, oldValue) => {
+        assert(hadValue === false)
+        i + "!"
+      })
+      assert(res === i + "!")
+    }
+    assert(map.size === 400)
+    assert(map(null) === null)
+    map.changeValue(null, (hadValue, oldValue) => {
+      assert(hadValue === false)
+      "null!"
+    })
+    assert(map.size === 401)
+    map.changeValue(null, (hadValue, oldValue) => {
+      assert(hadValue === true)
+      assert(oldValue === "null!")
+      "null!!"
+    })
+    assert(map.size === 401)
+  }
+
+  test("inserting in capacity-1 map") {
+    val map = new AppendOnlyMap[String, String](1)
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map("" + i) === "" + i)
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b78367b6cac028b217abea0fcbb79c08ab270045
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.util
+
+import java.util.Random
+import org.scalatest.FlatSpec
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+import org.apache.spark.util.Utils.times
+
+class XORShiftRandomSuite extends FunSuite with ShouldMatchers {
+
+  def fixture = new {
+    val seed = 1L
+    val xorRand = new XORShiftRandom(seed)
+    val hundMil = 1e8.toInt
+  }
+   
+  /*
+   * This test is based on a chi-squared test for randomness. The values are hard-coded 
+   * so as not to create Spark's dependency on apache.commons.math3 just to call one
+   * method for calculating the exact p-value for a given number of random numbers
+   * and bins. In case one would want to move to a full-fledged test based on 
+   * apache.commons.math3, the relevant class is here:
+   * org.apache.commons.math3.stat.inference.ChiSquareTest
+   */
+  test ("XORShift generates valid random numbers") {
+
+    val f = fixture
+
+    val numBins = 10
+    // create 10 bins
+    val bins = Array.fill(numBins)(0)
+
+    // populate bins based on modulus of the random number
+    times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1}
+
+    /* since the seed is deterministic, until the algorithm is changed, we know the result will be 
+     * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 
+     * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) 
+     * significance level. However, should the RNG implementation change, the test should still 
+     * pass at the same significance level. The chi-squared test done in R gave the following 
+     * results:
+     *   > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272,
+     *     10000790, 10002286, 9998699))
+     *     Chi-squared test for given probabilities
+     *     data:  c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, 
+     *            10002286, 9998699)
+     *     X-squared = 11.975, df = 9, p-value = 0.2147
+     * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million 
+     * random numbers
+     * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared
+     * is greater than or equal to that number.
+     */
+    val binSize = f.hundMil/numBins
+    val xSquared = bins.map(x => math.pow((binSize - x), 2)/binSize).sum
+    xSquared should be <  (16.9196)
+
+  }
+
+}
\ No newline at end of file
diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..0f1ab3d20eea4456385f26df2c724595450e6234
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.util.collection
+
+import org.scalatest.FunSuite
+
+
+class BitSetSuite extends FunSuite {
+
+  test("basic set and get") {
+    val setBits = Seq(0, 9, 1, 10, 90, 96)
+    val bitset = new BitSet(100)
+
+    for (i <- 0 until 100) {
+      assert(!bitset.get(i))
+    }
+
+    setBits.foreach(i => bitset.set(i))
+
+    for (i <- 0 until 100) {
+      if (setBits.contains(i)) {
+        assert(bitset.get(i))
+      } else {
+        assert(!bitset.get(i))
+      }
+    }
+    assert(bitset.cardinality() === setBits.size)
+  }
+
+  test("100% full bit set") {
+    val bitset = new BitSet(10000)
+    for (i <- 0 until 10000) {
+      assert(!bitset.get(i))
+      bitset.set(i)
+    }
+    for (i <- 0 until 10000) {
+      assert(bitset.get(i))
+    }
+    assert(bitset.cardinality() === 10000)
+  }
+
+  test("nextSetBit") {
+    val setBits = Seq(0, 9, 1, 10, 90, 96)
+    val bitset = new BitSet(100)
+    setBits.foreach(i => bitset.set(i))
+
+    assert(bitset.nextSetBit(0) === 0)
+    assert(bitset.nextSetBit(1) === 1)
+    assert(bitset.nextSetBit(2) === 9)
+    assert(bitset.nextSetBit(9) === 9)
+    assert(bitset.nextSetBit(10) === 10)
+    assert(bitset.nextSetBit(11) === 90)
+    assert(bitset.nextSetBit(80) === 90)
+    assert(bitset.nextSetBit(91) === 96)
+    assert(bitset.nextSetBit(96) === 96)
+    assert(bitset.nextSetBit(97) === -1)
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e9b62ea70db2243c562ff02d657d1bf7ad11db58
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
@@ -0,0 +1,177 @@
+/*
+ * 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.util.collection
+
+import scala.collection.mutable.HashSet
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+import org.apache.spark.util.SizeEstimator
+
+class OpenHashMapSuite extends FunSuite with ShouldMatchers {
+
+  test("size for specialized, primitive value (int)") {
+    val capacity = 1024
+    val map = new OpenHashMap[String, Int](capacity)
+    val actualSize = SizeEstimator.estimate(map)
+    // 64 bit for pointers, 32 bit for ints, and 1 bit for the bitset.
+    val expectedSize = capacity * (64 + 32 + 1) / 8
+    // Make sure we are not allocating a significant amount of memory beyond our expected.
+    actualSize should be <= (expectedSize * 1.1).toLong
+  }
+
+  test("initialization") {
+    val goodMap1 = new OpenHashMap[String, Int](1)
+    assert(goodMap1.size === 0)
+    val goodMap2 = new OpenHashMap[String, Int](255)
+    assert(goodMap2.size === 0)
+    val goodMap3 = new OpenHashMap[String, String](256)
+    assert(goodMap3.size === 0)
+    intercept[IllegalArgumentException] {
+      new OpenHashMap[String, Int](1 << 30) // Invalid map size: bigger than 2^29
+    }
+    intercept[IllegalArgumentException] {
+      new OpenHashMap[String, Int](-1)
+    }
+    intercept[IllegalArgumentException] {
+      new OpenHashMap[String, String](0)
+    }
+  }
+
+  test("primitive value") {
+    val map = new OpenHashMap[String, Int]
+
+    for (i <- 1 to 1000) {
+      map(i.toString) = i
+      assert(map(i.toString) === i)
+    }
+
+    assert(map.size === 1000)
+    assert(map(null) === 0)
+
+    map(null) = -1
+    assert(map.size === 1001)
+    assert(map(null) === -1)
+
+    for (i <- 1 to 1000) {
+      assert(map(i.toString) === i)
+    }
+
+    // Test iterator
+    val set = new HashSet[(String, Int)]
+    for ((k, v) <- map) {
+      set.add((k, v))
+    }
+    val expected = (1 to 1000).map(x => (x.toString, x)) :+ (null.asInstanceOf[String], -1)
+    assert(set === expected.toSet)
+  }
+
+  test("non-primitive value") {
+    val map = new OpenHashMap[String, String]
+
+    for (i <- 1 to 1000) {
+      map(i.toString) = i.toString
+      assert(map(i.toString) === i.toString)
+    }
+
+    assert(map.size === 1000)
+    assert(map(null) === null)
+
+    map(null) = "-1"
+    assert(map.size === 1001)
+    assert(map(null) === "-1")
+
+    for (i <- 1 to 1000) {
+      assert(map(i.toString) === i.toString)
+    }
+
+    // Test iterator
+    val set = new HashSet[(String, String)]
+    for ((k, v) <- map) {
+      set.add((k, v))
+    }
+    val expected = (1 to 1000).map(_.toString).map(x => (x, x)) :+ (null.asInstanceOf[String], "-1")
+    assert(set === expected.toSet)
+  }
+
+  test("null keys") {
+    val map = new OpenHashMap[String, String]()
+    for (i <- 1 to 100) {
+      map(i.toString) = i.toString
+    }
+    assert(map.size === 100)
+    assert(map(null) === null)
+    map(null) = "hello"
+    assert(map.size === 101)
+    assert(map(null) === "hello")
+  }
+
+  test("null values") {
+    val map = new OpenHashMap[String, String]()
+    for (i <- 1 to 100) {
+      map(i.toString) = null
+    }
+    assert(map.size === 100)
+    assert(map("1") === null)
+    assert(map(null) === null)
+    assert(map.size === 100)
+    map(null) = null
+    assert(map.size === 101)
+    assert(map(null) === null)
+  }
+
+  test("changeValue") {
+    val map = new OpenHashMap[String, String]()
+    for (i <- 1 to 100) {
+      map(i.toString) = i.toString
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      val res = map.changeValue(i.toString, { assert(false); "" }, v => {
+        assert(v === i.toString)
+        v + "!"
+      })
+      assert(res === i + "!")
+    }
+    // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a
+    // bug where changeValue would return the wrong result when the map grew on that insert
+    for (i <- 101 to 400) {
+      val res = map.changeValue(i.toString, { i + "!" }, v => { assert(false); v })
+      assert(res === i + "!")
+    }
+    assert(map.size === 400)
+    assert(map(null) === null)
+    map.changeValue(null, { "null!" }, v => { assert(false); v })
+    assert(map.size === 401)
+    map.changeValue(null, { assert(false); "" }, v => {
+      assert(v === "null!")
+      "null!!"
+    })
+    assert(map.size === 401)
+  }
+
+  test("inserting in capacity-1 map") {
+    val map = new OpenHashMap[String, String](1)
+    for (i <- 1 to 100) {
+      map(i.toString) = i.toString
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map(i.toString) === i.toString)
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1b24f8f287cfd3d00d9cc271cb6a186990acbdb2
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
@@ -0,0 +1,180 @@
+/*
+ * 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.util.collection
+
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.util.SizeEstimator
+
+
+class OpenHashSetSuite extends FunSuite with ShouldMatchers {
+
+  test("size for specialized, primitive int") {
+    val loadFactor = 0.7
+    val set = new OpenHashSet[Int](64, loadFactor)
+    for (i <- 0 until 1024) {
+      set.add(i)
+    }
+    assert(set.size === 1024)
+    assert(set.capacity > 1024)
+    val actualSize = SizeEstimator.estimate(set)
+    // 32 bits for the ints + 1 bit for the bitset
+    val expectedSize = set.capacity * (32 + 1) / 8
+    // Make sure we are not allocating a significant amount of memory beyond our expected.
+    actualSize should be <= (expectedSize * 1.1).toLong
+  }
+
+  test("primitive int") {
+    val set = new OpenHashSet[Int]
+    assert(set.size === 0)
+    assert(!set.contains(10))
+    assert(!set.contains(50))
+    assert(!set.contains(999))
+    assert(!set.contains(10000))
+
+    set.add(10)
+    assert(set.contains(10))
+    assert(!set.contains(50))
+    assert(!set.contains(999))
+    assert(!set.contains(10000))
+
+    set.add(50)
+    assert(set.size === 2)
+    assert(set.contains(10))
+    assert(set.contains(50))
+    assert(!set.contains(999))
+    assert(!set.contains(10000))
+
+    set.add(999)
+    assert(set.size === 3)
+    assert(set.contains(10))
+    assert(set.contains(50))
+    assert(set.contains(999))
+    assert(!set.contains(10000))
+
+    set.add(50)
+    assert(set.size === 3)
+    assert(set.contains(10))
+    assert(set.contains(50))
+    assert(set.contains(999))
+    assert(!set.contains(10000))
+  }
+
+  test("primitive long") {
+    val set = new OpenHashSet[Long]
+    assert(set.size === 0)
+    assert(!set.contains(10L))
+    assert(!set.contains(50L))
+    assert(!set.contains(999L))
+    assert(!set.contains(10000L))
+
+    set.add(10L)
+    assert(set.size === 1)
+    assert(set.contains(10L))
+    assert(!set.contains(50L))
+    assert(!set.contains(999L))
+    assert(!set.contains(10000L))
+
+    set.add(50L)
+    assert(set.size === 2)
+    assert(set.contains(10L))
+    assert(set.contains(50L))
+    assert(!set.contains(999L))
+    assert(!set.contains(10000L))
+
+    set.add(999L)
+    assert(set.size === 3)
+    assert(set.contains(10L))
+    assert(set.contains(50L))
+    assert(set.contains(999L))
+    assert(!set.contains(10000L))
+
+    set.add(50L)
+    assert(set.size === 3)
+    assert(set.contains(10L))
+    assert(set.contains(50L))
+    assert(set.contains(999L))
+    assert(!set.contains(10000L))
+  }
+
+  test("non-primitive") {
+    val set = new OpenHashSet[String]
+    assert(set.size === 0)
+    assert(!set.contains(10.toString))
+    assert(!set.contains(50.toString))
+    assert(!set.contains(999.toString))
+    assert(!set.contains(10000.toString))
+
+    set.add(10.toString)
+    assert(set.size === 1)
+    assert(set.contains(10.toString))
+    assert(!set.contains(50.toString))
+    assert(!set.contains(999.toString))
+    assert(!set.contains(10000.toString))
+
+    set.add(50.toString)
+    assert(set.size === 2)
+    assert(set.contains(10.toString))
+    assert(set.contains(50.toString))
+    assert(!set.contains(999.toString))
+    assert(!set.contains(10000.toString))
+
+    set.add(999.toString)
+    assert(set.size === 3)
+    assert(set.contains(10.toString))
+    assert(set.contains(50.toString))
+    assert(set.contains(999.toString))
+    assert(!set.contains(10000.toString))
+
+    set.add(50.toString)
+    assert(set.size === 3)
+    assert(set.contains(10.toString))
+    assert(set.contains(50.toString))
+    assert(set.contains(999.toString))
+    assert(!set.contains(10000.toString))
+  }
+
+  test("non-primitive set growth") {
+    val set = new OpenHashSet[String]
+    for (i <- 1 to 1000) {
+      set.add(i.toString)
+    }
+    assert(set.size === 1000)
+    assert(set.capacity > 1000)
+    for (i <- 1 to 100) {
+      set.add(i.toString)
+    }
+    assert(set.size === 1000)
+    assert(set.capacity > 1000)
+  }
+
+  test("primitive set growth") {
+    val set = new OpenHashSet[Long]
+    for (i <- 1 to 1000) {
+      set.add(i.toLong)
+    }
+    assert(set.size === 1000)
+    assert(set.capacity > 1000)
+    for (i <- 1 to 100) {
+      set.add(i.toLong)
+    }
+    assert(set.size === 1000)
+    assert(set.capacity > 1000)
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3b60decee9acb71c18ab19186d1b7df6aadaa4ca
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala
@@ -0,0 +1,119 @@
+/*
+ * 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.util.collection
+
+import scala.collection.mutable.HashSet
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+import org.apache.spark.util.SizeEstimator
+
+class PrimitiveKeyOpenHashMapSuite extends FunSuite with ShouldMatchers {
+
+  test("size for specialized, primitive key, value (int, int)") {
+    val capacity = 1024
+    val map = new PrimitiveKeyOpenHashMap[Int, Int](capacity)
+    val actualSize = SizeEstimator.estimate(map)
+    // 32 bit for keys, 32 bit for values, and 1 bit for the bitset.
+    val expectedSize = capacity * (32 + 32 + 1) / 8
+    // Make sure we are not allocating a significant amount of memory beyond our expected.
+    actualSize should be <= (expectedSize * 1.1).toLong
+  }
+
+  test("initialization") {
+    val goodMap1 = new PrimitiveKeyOpenHashMap[Int, Int](1)
+    assert(goodMap1.size === 0)
+    val goodMap2 = new PrimitiveKeyOpenHashMap[Int, Int](255)
+    assert(goodMap2.size === 0)
+    val goodMap3 = new PrimitiveKeyOpenHashMap[Int, Int](256)
+    assert(goodMap3.size === 0)
+    intercept[IllegalArgumentException] {
+      new PrimitiveKeyOpenHashMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29
+    }
+    intercept[IllegalArgumentException] {
+      new PrimitiveKeyOpenHashMap[Int, Int](-1)
+    }
+    intercept[IllegalArgumentException] {
+      new PrimitiveKeyOpenHashMap[Int, Int](0)
+    }
+  }
+
+  test("basic operations") {
+    val longBase = 1000000L
+    val map = new PrimitiveKeyOpenHashMap[Long, Int]
+
+    for (i <- 1 to 1000) {
+      map(i + longBase) = i
+      assert(map(i + longBase) === i)
+    }
+
+    assert(map.size === 1000)
+
+    for (i <- 1 to 1000) {
+      assert(map(i + longBase) === i)
+    }
+
+    // Test iterator
+    val set = new HashSet[(Long, Int)]
+    for ((k, v) <- map) {
+      set.add((k, v))
+    }
+    assert(set === (1 to 1000).map(x => (x + longBase, x)).toSet)
+  }
+
+  test("null values") {
+    val map = new PrimitiveKeyOpenHashMap[Long, String]()
+    for (i <- 1 to 100) {
+      map(i.toLong) = null
+    }
+    assert(map.size === 100)
+    assert(map(1.toLong) === null)
+  }
+
+  test("changeValue") {
+    val map = new PrimitiveKeyOpenHashMap[Long, String]()
+    for (i <- 1 to 100) {
+      map(i.toLong) = i.toString
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      val res = map.changeValue(i.toLong, { assert(false); "" }, v => {
+        assert(v === i.toString)
+        v + "!"
+      })
+      assert(res === i + "!")
+    }
+    // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a
+    // bug where changeValue would return the wrong result when the map grew on that insert
+    for (i <- 101 to 400) {
+      val res = map.changeValue(i.toLong, { i + "!" }, v => { assert(false); v })
+      assert(res === i + "!")
+    }
+    assert(map.size === 400)
+  }
+
+  test("inserting in capacity-1 map") {
+    val map = new PrimitiveKeyOpenHashMap[Long, String](1)
+    for (i <- 1 to 100) {
+      map(i.toLong) = i.toString
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map(i.toLong) === i.toString)
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..970dade628fe467f3c6c9a3344983fd77ebeeb2e
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala
@@ -0,0 +1,117 @@
+/*
+ * 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.util.collection
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.util.SizeEstimator
+
+class PrimitiveVectorSuite extends FunSuite {
+
+  test("primitive value") {
+    val vector = new PrimitiveVector[Int]
+
+    for (i <- 0 until 1000) {
+      vector += i
+      assert(vector(i) === i)
+    }
+
+    assert(vector.size === 1000)
+    assert(vector.size == vector.length)
+    intercept[IllegalArgumentException] {
+      vector(1000)
+    }
+
+    for (i <- 0 until 1000) {
+      assert(vector(i) == i)
+    }
+  }
+
+  test("non-primitive value") {
+    val vector = new PrimitiveVector[String]
+
+    for (i <- 0 until 1000) {
+      vector += i.toString
+      assert(vector(i) === i.toString)
+    }
+
+    assert(vector.size === 1000)
+    assert(vector.size == vector.length)
+    intercept[IllegalArgumentException] {
+      vector(1000)
+    }
+
+    for (i <- 0 until 1000) {
+      assert(vector(i) == i.toString)
+    }
+  }
+
+  test("ideal growth") {
+    val vector = new PrimitiveVector[Long](initialSize = 1)
+    vector += 1
+    for (i <- 1 until 1024) {
+      vector += i
+      assert(vector.size === i + 1)
+      assert(vector.capacity === Integer.highestOneBit(i) * 2)
+    }
+    assert(vector.capacity === 1024)
+    vector += 1024
+    assert(vector.capacity === 2048)
+  }
+
+  test("ideal size") {
+    val vector = new PrimitiveVector[Long](8192)
+    for (i <- 0 until 8192) {
+      vector += i
+    }
+    assert(vector.size === 8192)
+    assert(vector.capacity === 8192)
+    val actualSize = SizeEstimator.estimate(vector)
+    val expectedSize = 8192 * 8
+    // Make sure we are not allocating a significant amount of memory beyond our expected.
+    // Due to specialization wonkiness, we need to ensure we don't have 2 copies of the array.
+    assert(actualSize < expectedSize * 1.1)
+  }
+
+  test("resizing") {
+    val vector = new PrimitiveVector[Long]
+    for (i <- 0 until 4097) {
+      vector += i
+    }
+    assert(vector.size === 4097)
+    assert(vector.capacity === 8192)
+    vector.trim()
+    assert(vector.size === 4097)
+    assert(vector.capacity === 4097)
+    vector.resize(5000)
+    assert(vector.size === 4097)
+    assert(vector.capacity === 5000)
+    vector.resize(4000)
+    assert(vector.size === 4000)
+    assert(vector.capacity === 4000)
+    vector.resize(5000)
+    assert(vector.size === 4000)
+    assert(vector.capacity === 5000)
+    for (i <- 0 until 4000) {
+      assert(vector(i) == i)
+    }
+    intercept[IllegalArgumentException] {
+      vector(4000)
+    }
+  }
+}
diff --git a/docker/README.md b/docker/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..bf59e77d111f9c121ea64b949dba3a557a38622d
--- /dev/null
+++ b/docker/README.md
@@ -0,0 +1,5 @@
+Spark docker files
+===========
+
+Drawn from Matt Massie's docker files (https://github.com/massie/dockerfiles),
+as well as some updates from Andre Schumacher (https://github.com/AndreSchumacher/docker).
\ No newline at end of file
diff --git a/docker/build b/docker/build
new file mode 100755
index 0000000000000000000000000000000000000000..253a2fc8dd8e7a0a9dce47930874f000cfa18293
--- /dev/null
+++ b/docker/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+docker images > /dev/null || { echo Please install docker in non-sudo mode. ; exit; }
+
+./spark-test/build
\ No newline at end of file
diff --git a/docker/spark-test/README.md b/docker/spark-test/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..ec0baf6e6d4196c511f044f4cbffe478338376e0
--- /dev/null
+++ b/docker/spark-test/README.md
@@ -0,0 +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 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
+your Spark source and have them immediately usable in the docker images
+(without rebuilding them).
diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..60962776dda57d6b6432eaba24b9a82ae5daca3f
--- /dev/null
+++ b/docker/spark-test/base/Dockerfile
@@ -0,0 +1,38 @@
+#
+# 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.
+#
+
+FROM ubuntu:precise
+
+RUN echo "deb http://archive.ubuntu.com/ubuntu precise main universe" > /etc/apt/sources.list
+
+# Upgrade package index
+RUN apt-get update
+
+# install a few other useful packages plus Open Jdk 7
+RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server
+
+ENV SCALA_VERSION 2.9.3
+ENV SPARK_VERSION 0.8.1
+ENV CDH_VERSION cdh4
+ENV SCALA_HOME /opt/scala-$SCALA_VERSION
+ENV SPARK_HOME /opt/spark
+ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH
+
+# Install Scala
+ADD http://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz /
+RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -)
+RUN rm /scala-$SCALA_VERSION.tgz
diff --git a/docker/spark-test/build b/docker/spark-test/build
new file mode 100755
index 0000000000000000000000000000000000000000..6f9e19743370b85c4dbeadf89ded369a1a3b75f4
--- /dev/null
+++ b/docker/spark-test/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+docker build -t spark-test-base spark-test/base/
+docker build -t spark-test-master spark-test/master/
+docker build -t spark-test-worker spark-test/worker/
diff --git a/docker/spark-test/master/Dockerfile b/docker/spark-test/master/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..f729534ab63ed429fd6aba90739645974d56170a
--- /dev/null
+++ b/docker/spark-test/master/Dockerfile
@@ -0,0 +1,21 @@
+# Spark Master
+#
+# 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.
+#
+
+FROM spark-test-base
+ADD default_cmd /root/
+CMD ["/root/default_cmd"]
diff --git a/docker/spark-test/master/default_cmd b/docker/spark-test/master/default_cmd
new file mode 100755
index 0000000000000000000000000000000000000000..a5b1303c2ebdb957d3f2249fd13a13f537582599
--- /dev/null
+++ b/docker/spark-test/master/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.master.Master -i $IP
diff --git a/docker/spark-test/worker/Dockerfile b/docker/spark-test/worker/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..890febe7b68e29a55639ae5a8d21d283cd5d33a7
--- /dev/null
+++ b/docker/spark-test/worker/Dockerfile
@@ -0,0 +1,22 @@
+# Spark Worker
+#
+# 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.
+#
+
+FROM spark-test-base
+ENV SPARK_WORKER_PORT 8888
+ADD default_cmd /root/
+ENTRYPOINT ["/root/default_cmd"]
diff --git a/docker/spark-test/worker/default_cmd b/docker/spark-test/worker/default_cmd
new file mode 100755
index 0000000000000000000000000000000000000000..ab6336f70c1c6a3913a8f217d0619514ac287752
--- /dev/null
+++ b/docker/spark-test/worker/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.worker.Worker $1
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 0c1d657cde470abbb0ca00c0cd266dbd2fe587bf..ad7969d012283a009a9b3bf71c6da3e80fbbdbcd 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -74,12 +74,12 @@
                         <li class="dropdown">
                             <a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a>
                             <ul class="dropdown-menu">
-                                <li><a href="api/core/index.html">Spark Core for Java/Scala</a></li>
+                                <li><a href="api/core/index.html#org.apache.spark.package">Spark Core for Java/Scala</a></li>
                                 <li><a href="api/pyspark/index.html">Spark Core for Python</a></li>
                                 <li class="divider"></li>
-                                <li><a href="api/streaming/index.html">Spark Streaming</a></li>
-                                <li><a href="api/mllib/index.html">MLlib (Machine Learning)</a></li>
-                                <li><a href="api/bagel/index.html">Bagel (Pregel on Spark)</a></li>
+                                <li><a href="api/streaming/index.html#org.apache.spark.streaming.package">Spark Streaming</a></li>
+                                <li><a href="api/mllib/index.html#org.apache.spark.mllib.package">MLlib (Machine Learning)</a></li>
+                                <li><a href="api/bagel/index.html#org.apache.spark.bagel.package">Bagel (Pregel on Spark)</a></li>
                             </ul>
                         </li>
 
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index 140190a38cfca8242683d7c4a18bdb8b86345872..de001e6c52f85e39c6ca62afd7e699039221ec35 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -106,7 +106,7 @@ _Example_
 
 ## Operations
 
-Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details.
+Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details.
 
 ### Actions
 
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index 19c01e179f35f2c00c9519adcbebb13636d3f466..c709001632261c1d0443fdcb0ac2d94211d0add3 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -45,6 +45,12 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with
     # Cloudera CDH 4.2.0 with MapReduce v2
     $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
 
+Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows:
+ 
+    # Apache Hadoop 2.2.X and newer
+    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
+
+The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. 
 
 ## Spark Tests in Maven ##
 
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index f679cad713769d45abfc9f10936fa39c3d724d8b..e16703292cc226f49dec84bfc603cc64e747b4f9 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.
 
@@ -45,7 +45,7 @@ The system currently supports three cluster managers:
   easy to set up a cluster.
 * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
   and service applications.
-* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.0.
+* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
 
 In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone
 cluster on Amazon EC2.
@@ -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..677d182e50b3d8d4a32bf8672784ce1689236bd0 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>
@@ -275,11 +275,32 @@ Apart from these, the following properties are also available, and may be useful
 </tr>
 <tr>
   <td>spark.akka.timeout</td>
-  <td>20</td>
+  <td>100</td>
   <td>
     Communication timeout between Spark nodes, in seconds.
   </td>
 </tr>
+<tr>
+  <td>spark.akka.heartbeat.pauses</td>
+  <td>600</td>
+  <td>
+     This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if you need to.
+  </td>
+</tr>
+<tr>
+  <td>spark.akka.failure-detector.threshold</td>
+  <td>300.0</td>
+  <td>
+     This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). This maps to akka's `akka.remote.transport-failure-detector.threshold`. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
+  </td>
+</tr>
+<tr>
+  <td>spark.akka.heartbeat.interval</td>
+  <td>1000</td>
+  <td>
+    This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. 
+  </td>
+</tr>
 <tr>
   <td>spark.driver.host</td>
   <td>(local hostname)</td>
@@ -319,7 +340,49 @@ 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>
+<tr>
+  <td>spark.shuffle.consolidateFiles</td>
+  <td>false</td>
+  <td>
+    If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
+  </td>
+</tr>
+<tr>
+  <td>spark.speculation</td>
+  <td>false</td>
+  <td>
+    If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched.
+  </td>
+</tr>
+<tr>
+  <td>spark.speculation.interval</td>
+  <td>100</td>
+  <td>
+    How often Spark will check for tasks to speculate, in milliseconds.
+  </td>
+</tr>
+<tr>
+  <td>spark.speculation.quantile</td>
+  <td>0.75</td>
+  <td>
+    Percentage of tasks which must be complete before speculation is enabled for a particular stage.
+  </td>
+</tr>
+<tr>
+  <td>spark.speculation.multiplier</td>
+  <td>1.5</td>
+  <td>
+    How many times slower a task is than the median to be considered for speculation.
+  </td>
+</tr>
 </table>
 
 # Environment Variables
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/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md
index f706625fe9d5bb1b754a95bab01b3370c4338995..de6a2b0a43bd5ee4de7dcd1c22a75991c50f2581 100644
--- a/docs/hadoop-third-party-distributions.md
+++ b/docs/hadoop-third-party-distributions.md
@@ -10,7 +10,7 @@ with these distributions:
 # Compile-time Hadoop Version
 
 When compiling Spark, you'll need to 
-[set the SPARK_HADOOP_VERSION flag](http://localhost:4000/index.html#a-note-about-hadoop-versions):
+[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions):
 
     SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly
 
@@ -25,8 +25,8 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
       <h3>CDH Releases</h3>
       <table class="table" style="width:350px; margin-right: 20px;">
         <tr><th>Release</th><th>Version code</th></tr>
-        <tr><td>CDH 4.X.X (YARN mode)</td><td>2.0.0-chd4.X.X</td></tr>
-        <tr><td>CDH 4.X.X</td><td>2.0.0-mr1-chd4.X.X</td></tr>
+        <tr><td>CDH 4.X.X (YARN mode)</td><td>2.0.0-cdh4.X.X</td></tr>
+        <tr><td>CDH 4.X.X</td><td>2.0.0-mr1-cdh4.X.X</td></tr>
         <tr><td>CDH 3u6</td><td>0.20.2-cdh3u6</td></tr>
         <tr><td>CDH 3u5</td><td>0.20.2-cdh3u5</td></tr>
         <tr><td>CDH 3u4</td><td>0.20.2-cdh3u4</td></tr>
@@ -40,6 +40,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
         <tr><td>HDP 1.2</td><td>1.1.2</td></tr>
         <tr><td>HDP 1.1</td><td>1.0.3</td></tr>
         <tr><td>HDP 1.0</td><td>1.0.3</td></tr>
+        <tr><td>HDP 2.0</td><td>2.2.0</td></tr>
       </table>
     </td>
   </tr>
diff --git a/docs/index.md b/docs/index.md
index bd386a8a8fdb6579fc7c7641c7a4a753f394e279..d3ac696d1e818a0dee8587756a6f8974276ac289 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -56,14 +56,16 @@ Hadoop, you must build Spark against the same version that your cluster uses.
 By default, Spark links to Hadoop 1.0.4. You can change this by setting the
 `SPARK_HADOOP_VERSION` variable when compiling:
 
-    SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly
+    SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly
 
-In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
+In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set
 `SPARK_YARN` to `true`:
 
     SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
 
-(Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.)
+Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.
+
+For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes.
 
 # Where to Go from Here
 
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index d304c5497bdb35ef0a48294f79fa1c3c000d2f79..dbcb9ae343357f648637eb3487819725beb304bd 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -91,7 +91,7 @@ The fair scheduler also supports grouping jobs into _pools_, and setting differe
 (e.g. weight) for each pool. This can be useful to create a "high-priority" pool for more important jobs,
 for example, or to group the jobs of each user together and give _users_ equal shares regardless of how
 many concurrent jobs they have instead of giving _jobs_ equal shares. This approach is modeled after the
-[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/stable/fair_scheduler.html).
+[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html).
 
 Without any intervention, newly submitted jobs go into a _default pool_, but jobs' pools can be set by
 adding the `spark.scheduler.pool` "local property" to the SparkContext in the thread that's submitting them.
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 5f456b999b0e187434bfcee09b572a2cde508657..5ed0474477302b2768bdd59e6a66afea867a126c 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -50,6 +50,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the
 * `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
 * `JmxSink`: Registers metrics for viewing in a JXM console.
 * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.
+* `GraphiteSink`: Sends metrics to a Graphite node.
 
 The syntax of the metrics configuration file is defined in an example configuration file, 
 `$SPARK_HOME/conf/metrics.conf.template`.
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/running-on-yarn.md b/docs/running-on-yarn.md
index 30128ec45db04d39e3537303f8bfc39d0e50c5cd..9e4c4e123f0aff51de0b29190e557cb92fd57b1f 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -17,10 +17,12 @@ This can be built by setting the Hadoop version and `SPARK_YARN` environment var
 The assembled JAR will be something like this:
 `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`.
 
+The build process now also supports new YARN versions (2.2.x). See below.
 
 # Preparations
 
 - Building a YARN-enabled assembly (see above).
+- The assembled jar can be installed into HDFS or used locally.
 - Your application code must be packaged into a separate JAR file.
 
 If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different.
@@ -34,12 +36,20 @@ Environment variables:
 
 System Properties:
 * 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
+* 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
+* 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
+* 'spark.yarn.scheduler.heartbeat.interval-ms', the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. 
+* 'spark.yarn.max.worker.failures', the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
 
 # Launching Spark on YARN
 
 Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster.
 This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager.
 
+There are two scheduler mode that can be used to launch spark application on YARN.
+
+## Launch spark application by YARN Client with yarn-standalone mode.
+
 The command to launch the YARN Client is as follows:
 
     SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
@@ -47,11 +57,15 @@ The command to launch the YARN Client is as follows:
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
       --num-workers <NUMBER_OF_WORKER_MACHINES> \
+      --master-class <ApplicationMaster_CLASS>
       --master-memory <MEMORY_FOR_MASTER> \
       --worker-memory <MEMORY_PER_WORKER> \
       --worker-cores <CORES_PER_WORKER> \
       --name <application_name> \
-      --queue <queue_name>
+      --queue <queue_name> \
+      --addJars <any_local_files_used_in_SparkContext.addJar> \
+      --files <files_for_distributed_cache> \
+      --archives <archives_for_distributed_cache>
 
 For example:
 
@@ -77,10 +91,37 @@ For example:
     $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout
     Pi is roughly 3.13794
 
-The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
+The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will  periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
+
+With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell.
+
+## Launch spark application with yarn-client mode.
+
+With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR
+
+In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh
+
+For example:
+
+    SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+    SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+    ./run-example org.apache.spark.examples.SparkPi yarn-client
+
+
+    SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+    SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+    MASTER=yarn-client ./spark-shell
+
+# Building Spark for Hadoop/YARN 2.2.x
+
+Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. 
+
+See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process.
 
 # Important Notes
 
-- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above.
 - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
+- The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
+- The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
+- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally.  
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/spark-standalone.md b/docs/spark-standalone.md
index 81cdbefd0c7ae83a3de3369da54b47bb3d9591cd..b822265b5a0cc4479e3540752b745aadeae8cfbc 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -3,6 +3,9 @@ layout: global
 title: Spark Standalone Mode
 ---
 
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
 In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing.
 
 # Installing Spark Standalone to a Cluster
@@ -48,11 +51,11 @@ Finally, the following configuration options can be passed to the master and wor
   </tr>
   <tr>
     <td><code>-c CORES</code>, <code>--cores CORES</code></td>
-    <td>Total CPU cores to allow Spark applicatons to use on the machine (default: all available); only on worker</td>
+    <td>Total CPU cores to allow Spark applications to use on the machine (default: all available); only on worker</td>
   </tr>
   <tr>
     <td><code>-m MEM</code>, <code>--memory MEM</code></td>
-    <td>Total amount of memory to allow Spark applicatons to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
+    <td>Total amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
   </tr>
   <tr>
     <td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
@@ -169,3 +172,75 @@ In addition, detailed log output for each job is also written to the work direct
 
 You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).
 
+
+# High Availability
+
+By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below.
+
+## Standby Masters with ZooKeeper
+
+**Overview**
+
+Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected.
+
+Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html).
+
+**Configuration**
+
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+  <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+  <tr>
+    <td><code>spark.deploy.recoveryMode</code></td>
+    <td>Set to ZOOKEEPER to enable standby Master recovery mode (default: NONE).</td>
+  </tr>
+  <tr>
+    <td><code>spark.deploy.zookeeper.url</code></td>
+    <td>The ZooKeeper cluster url (e.g., 192.168.1.100:2181,192.168.1.101:2181).</td>
+  </tr>
+  <tr>
+    <td><code>spark.deploy.zookeeper.dir</code></td>
+    <td>The directory in ZooKeeper to store recovery state (default: /spark).</td>
+  </tr>
+</table>
+
+Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently).
+
+**Details**
+
+After you have a ZooKeeper cluster set up, enabling high availability is straightforward. Simply start multiple Master processes on different nodes with the same ZooKeeper configuration (ZooKeeper URL and directory). Masters can be added and removed at any time.
+
+In order to schedule new applications or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if ``host1`` goes down, this configuration would still be correct as we'd find the new leader, ``host2``.
+
+There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, an application or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered applications and Workers to inform them of the change in leadership, so they need not even have known of the existence of the new Master at startup.
+
+Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of.
+
+## Single-Node Recovery with Local File System
+
+**Overview**
+
+ZooKeeper is the best way to go for production-level high availability, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When applications and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process.
+
+**Configuration**
+
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+  <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+  <tr>
+    <td><code>spark.deploy.recoveryMode</code></td>
+    <td>Set to FILESYSTEM to enable single-node recovery mode (default: NONE).</td>
+  </tr>
+  <tr>
+    <td><code>spark.deploy.recoveryDirectory</code></td>
+    <td>The directory in which Spark will store recovery state, accessible from the Master's perspective.</td>
+  </tr>
+</table>
+
+**Details**
+
+* This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart.
+* While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout.
+* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index c7df1720249481eed6a4b4268ac9c8796d0587b6..82f42e0b8d16a74e2d3e9b4881a6e6bfdd75c93a 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>
@@ -122,12 +126,12 @@ Spark Streaming features windowed computations, which allow you to apply transfo
 <table class="table">
 <tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
 <tr>
-  <td> <b>window</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> <b>window</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
   <td> Return a new DStream which is computed based on windowed batches of the source DStream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
   </td>
 </tr>
 <tr>
-  <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> <b>countByWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
   <td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
   </td>
 </tr>
@@ -161,7 +165,6 @@ Spark Streaming features windowed computations, which allow you to apply transfo
  <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
 </td>
 </tr>
-
 </table>
 
 A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#org.apache.spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.PairDStreamFunctions).
@@ -211,7 +214,7 @@ ssc.stop()
 {% endhighlight %}
 
 # Example
-A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` .
+A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala` .
 
 {% highlight scala %}
 import org.apache.spark.streaming.{Seconds, StreamingContext}
@@ -280,7 +283,7 @@ Time: 1357008430000 ms
 </td>
 </table>
 
-You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
 
 # DStream Persistence
 Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
@@ -480,7 +483,7 @@ Similar to [Spark's Java API](java-programming-guide.html), we also provide a Ja
 1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#org.apache.spark.api.java.function.Function) and [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2)
 1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types.
 
-Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java`
+Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java`
 
 The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart.
 
@@ -524,5 +527,5 @@ JavaPairDStream<String, Integer> wordCounts = words.map(
 # Where to Go from Here
 
 * API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package)
-* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/spark/streaming/examples)
+* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/streaming/examples)
 * [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
diff --git a/docs/tuning.md b/docs/tuning.md
index 28d88a26592725ff750d09b2c924fe3dd72a8ce2..a4be18816928e7fe08c24b76186af63131ed03cd 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -39,7 +39,8 @@ in your operations) and performance. It provides two serialization libraries:
   for best performance.
 
 You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`
-*before* creating your SparkContext. The only reason it is not the default is because of the custom
+*before* creating your SparkContext. This setting configures the serializer used for not only shuffling data between worker
+nodes but also when serializing RDDs to disk.  The only reason Kryo is not the default is because of the custom
 registration requirement, but we recommend trying it in any network-intensive application.
 
 Finally, to register your classes with Kryo, create a public class that extends
@@ -67,7 +68,7 @@ The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced
 registration options, such as adding custom serialization code.
 
 If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
-system property. The default is 32, but this value needs to be large enough to hold the *largest*
+system property. The default is 2, but this value needs to be large enough to hold the *largest*
 object you will serialize.
 
 Finally, if you don't register your classes, Kryo will still work, but it will have to store the
@@ -175,7 +176,7 @@ To further tune garbage collection, we first need to understand some basic infor
 * Java Heap space is divided in to two regions Young and Old. The Young generation is meant to hold short-lived objects
   while the Old generation is intended for objects with longer lifetimes.
 
-* The Young generation is further divided into three regions [Eden, Survivor1, Survivor2].
+* The Young generation is further divided into three regions \[Eden, Survivor1, Survivor2\].
 
 * A simplified description of the garbage collection procedure: When Eden is full, a minor GC is run on Eden and objects
   that are alive from Eden and Survivor1 are copied to Survivor2. The Survivor regions are swapped. If an object is old
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 65868b76b91fcfdd895ba7a14bbf2895407471e4..11892324286a5dbc070dad4bad7be8322d3c17db 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -72,12 +72,12 @@ def parse_args():
   parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use")
   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", 
+  parser.add_option("--spark-git-repo",
+      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)")
-  parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", 
+  parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port",
       help="Use SSH dynamic port forwarding to create a SOCKS proxy at " +
             "the given local address (for use with login)")
   parser.add_option("--resume", action="store_true", default=False,
@@ -101,6 +101,8 @@ def parse_args():
       help="The SSH user you want to connect as (default: root)")
   parser.add_option("--delete-groups", action="store_true", default=False,
       help="When destroying a cluster, delete the security groups that were created")
+  parser.add_option("--use-existing-master", action="store_true", default=False,
+      help="Launch fresh slaves, but use an existing stopped master if possible")
 
   (opts, args) = parser.parse_args()
   if len(args) != 2:
@@ -191,7 +193,7 @@ def get_spark_ami(opts):
     instance_type = "pvm"
     print >> stderr,\
         "Don't recognize %s, assuming type is pvm" % opts.instance_type
-  
+
   ami_path = "%s/%s/%s" % (AMI_PREFIX, opts.region, instance_type)
   try:
     ami = urllib2.urlopen(ami_path).read().strip()
@@ -215,6 +217,7 @@ def launch_cluster(conn, opts, cluster_name):
     master_group.authorize(src_group=slave_group)
     master_group.authorize('tcp', 22, 22, '0.0.0.0/0')
     master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
+    master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0')
     master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
     master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')
     master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0')
@@ -232,9 +235,9 @@ def launch_cluster(conn, opts, cluster_name):
     slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0')
 
   # Check if instances are already running in our groups
-  active_nodes = get_existing_cluster(conn, opts, cluster_name,
-                                      die_on_error=False)
-  if any(active_nodes):
+  existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name,
+                                                           die_on_error=False)
+  if existing_slaves or (existing_masters and not opts.use_existing_master):
     print >> stderr, ("ERROR: There are already instances running in " +
         "group %s or %s" % (master_group.name, slave_group.name))
     sys.exit(1)
@@ -335,21 +338,28 @@ def launch_cluster(conn, opts, cluster_name):
                                                         zone, slave_res.id)
       i += 1
 
-  # Launch masters
-  master_type = opts.master_instance_type
-  if master_type == "":
-    master_type = opts.instance_type
-  if opts.zone == 'all':
-    opts.zone = random.choice(conn.get_all_zones()).name
-  master_res = image.run(key_name = opts.key_pair,
-                         security_groups = [master_group],
-                         instance_type = master_type,
-                         placement = opts.zone,
-                         min_count = 1,
-                         max_count = 1,
-                         block_device_map = block_map)
-  master_nodes = master_res.instances
-  print "Launched master in %s, regid = %s" % (zone, master_res.id)
+  # Launch or resume masters
+  if existing_masters:
+    print "Starting master..."
+    for inst in existing_masters:
+      if inst.state not in ["shutting-down", "terminated"]:
+        inst.start()
+    master_nodes = existing_masters
+  else:
+    master_type = opts.master_instance_type
+    if master_type == "":
+      master_type = opts.instance_type
+    if opts.zone == 'all':
+      opts.zone = random.choice(conn.get_all_zones()).name
+    master_res = image.run(key_name = opts.key_pair,
+                           security_groups = [master_group],
+                           instance_type = master_type,
+                           placement = opts.zone,
+                           min_count = 1,
+                           max_count = 1,
+                           block_device_map = block_map)
+    master_nodes = master_res.instances
+    print "Launched master in %s, regid = %s" % (zone, master_res.id)
 
   # Return all the instances
   return (master_nodes, slave_nodes)
@@ -403,8 +413,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
       print slave.public_dns_name
       ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar)
 
-  modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', 
-             'mapreduce', 'spark-standalone']
+  modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs',
+             'mapreduce', 'spark-standalone', 'tachyon']
 
   if opts.hadoop_major_version == "1":
     modules = filter(lambda x: x != "mapreduce", modules)
@@ -668,12 +678,12 @@ def real_main():
       print "Terminating slaves..."
       for inst in slave_nodes:
         inst.terminate()
-      
+
       # Delete security groups as well
       if opts.delete_groups:
         print "Deleting security groups (this will take some time)..."
         group_names = [cluster_name + "-master", cluster_name + "-slaves"]
-        
+
         attempt = 1;
         while attempt <= 3:
           print "Attempt %d" % attempt
@@ -731,6 +741,7 @@ def real_main():
         cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " +
         "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" +
         "AMAZON EBS IF IT IS EBS-BACKED!!\n" +
+        "All data on spot-instance slaves will be lost.\n" +
         "Stop cluster " + cluster_name + " (y/N): ")
     if response == "y":
       (master_nodes, slave_nodes) = get_existing_cluster(
@@ -742,7 +753,10 @@ def real_main():
       print "Stopping slaves..."
       for inst in slave_nodes:
         if inst.state not in ["shutting-down", "terminated"]:
-          inst.stop()
+          if inst.spot_instance_request_id:
+            inst.terminate()
+          else:
+            inst.stop()
 
   elif action == "start":
     (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name)
diff --git a/examples/pom.xml b/examples/pom.xml
index c6c9def5be75569486dfb9a2b8e618b3521c58e1..97f6dfea6643dcb8f308d6e5852a3c97e53e7132 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -26,41 +26,48 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-examples_${scala-short.version}</artifactId>
+  <artifactId>spark-examples_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Examples</name>
   <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>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala-short.version}</artifactId>
+      <artifactId>spark-streaming_2.10</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-mllib_${scala-short.version}</artifactId>
+      <artifactId>spark-mllib_2.10</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-bagel_${scala-short.version}</artifactId>
+      <artifactId>spark-bagel_2.10</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
@@ -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.10</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>
@@ -91,17 +107,17 @@
     </dependency>
     <dependency>
       <groupId>com.twitter</groupId>
-      <artifactId>algebird-core_${scala-short.version}</artifactId>
+      <artifactId>algebird-core_2.10</artifactId>
       <version>0.1.11</version>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -137,13 +153,21 @@
           <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>
 
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
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..a1199809923bce78865d96dff4f23e53198aa4ab 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -22,27 +22,36 @@ 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 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",
       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
 
-    var arr1 = new Array[Int](num)
+    val arr1 = new Array[Int](num)
     for (i <- 0 until arr1.length) {
       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)
-      }
+      val observedSizes = sc.parallelize(1 to 10, slices).map(_ => barr1.value.size)
+      // Collect the small RDD so we can print the observed sizes locally.
+      observedSizes.collect().foreach(i => println(i))
+      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/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 4af45b2b4a0670df6d8c5072c818cdf0be72fe1b..83db8b9e26411cb23f08892be5f59a5664a23813 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -120,7 +120,7 @@ object LocalALS {
         System.exit(1)
       }
     }
-    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
+    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
     
     val R = generateR()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index f79f0142b8679134210db7616284176a994f3c36..e1afc29f9aca9cb170af59a07f7f3ea47da69bb6 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -18,35 +18,38 @@
 package org.apache.spark.examples
 
 import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
 
 object MultiBroadcastTest {
   def main(args: Array[String]) {
     if (args.length == 0) {
-      System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
+      System.err.println("Usage: MultiBroadcastTest <master> [<slices>] [numElem]")
       System.exit(1)
     }
 
-    val sc = new SparkContext(args(0), "Broadcast Test",
+    val sc = new SparkContext(args(0), "Multi-Broadcast Test",
       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
 
-    var arr1 = new Array[Int](num)
+    val arr1 = new Array[Int](num)
     for (i <- 0 until arr1.length) {
       arr1(i) = i
     }
 
-    var arr2 = new Array[Int](num)
+    val arr2 = new Array[Int](num)
     for (i <- 0 until arr2.length) {
       arr2(i) = i
     }
 
     val barr1 = sc.broadcast(arr1)
     val barr2 = sc.broadcast(arr2)
-    sc.parallelize(1 to 10, slices).foreach {
-      i => println(barr1.value.size + barr2.value.size)
+    val observedSizes: RDD[(Int, Int)] = sc.parallelize(1 to 10, slices).map { _ =>
+      (barr1.value.size, barr2.value.size)
     }
+    // Collect the small RDD so we can print the observed sizes locally.
+    observedSizes.collect().foreach(i => println(i))
 
     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/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index f7bf75b4e5038c10e374ae6e26d4e561b2cda568..bc2db39c1231ba71af9a5fe1b346d692554ffc8e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -21,8 +21,6 @@ import java.util.Random
 import org.apache.spark.SparkContext
 import org.apache.spark.util.Vector
 import org.apache.spark.SparkContext._
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
 
 /**
  * K-means clustering.
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/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 5a7a9d1bd8f744bcc92c5e8f0a30c50051fd64bc..8543ce0e3285e3fe8f45121045012bba51712b54 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -65,7 +65,7 @@ object SparkTC {
       oldCount = nextCount
       // Perform the join, obtaining an RDD of (y, (z, x)) pairs,
       // then project the result to obtain the new (x, z) paths.
-      tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache();
+      tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache()
       nextCount = tc.count()
     } while (nextCount != oldCount)
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index 08e399f9ee16803d10b90f5d70ead4a27b9af99b..50e3f9639cd5b81d5fa20d94ab9f30ca2367102d 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -86,7 +86,7 @@ class FeederActor extends Actor {
 class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
 extends Actor with Receiver {
 
-  lazy private val remotePublisher = context.actorFor(urlOfPublisher)
+  lazy private val remotePublisher = context.actorSelection(urlOfPublisher)
 
   override def preStart = remotePublisher ! SubscribeReceiver(context.self)
 
@@ -121,7 +121,7 @@ object FeederActor {
 
     println("Feeder started as:" + feeder)
 
-    actorSystem.awaitTermination();
+    actorSystem.awaitTermination()
   }
 }
 
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..ff332a02821290f98faebcbf6a06ec6c091bc31b
--- /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/mllib/pom.xml b/mllib/pom.xml
index a57bddeff3118d4c6c85a4c23088cb1b002da60f..228f8c029ba1bb69761da2d6cc3f439cb98d68d7 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-mllib_${scala-short.version}</artifactId>
+  <artifactId>spark-mllib_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project ML Library</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -48,12 +48,12 @@
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -63,8 +63,8 @@
     </dependency>
   </dependencies>
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.scalatest</groupId>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
index edbf77dbcc3a44b9d7b38a6080904a8dbcf2079b..0dee9399a86ea2bd5a9d9292f5f3d550fdce76c1 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
@@ -18,15 +18,16 @@
 package org.apache.spark.mllib.clustering
 
 import scala.collection.mutable.ArrayBuffer
-import scala.util.Random
+
+import org.jblas.DoubleMatrix
 
 import org.apache.spark.SparkContext
 import org.apache.spark.SparkContext._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.Logging
 import org.apache.spark.mllib.util.MLUtils
+import org.apache.spark.util.XORShiftRandom
 
-import org.jblas.DoubleMatrix
 
 
 /**
@@ -195,7 +196,7 @@ class KMeans private (
    */
   private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = {
     // Sample all the cluster centers in one pass to avoid repeated scans
-    val sample = data.takeSample(true, runs * k, new Random().nextInt()).toSeq
+    val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq
     Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).toArray)
   }
 
@@ -210,7 +211,7 @@ class KMeans private (
    */
   private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = {
     // Initialize each run's center to a random point
-    val seed = new Random().nextInt()
+    val seed = new XORShiftRandom().nextInt()
     val sample = data.takeSample(true, runs, seed).toSeq
     val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r)))
 
@@ -222,7 +223,7 @@ class KMeans private (
         for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point))
       }.reduceByKey(_ + _).collectAsMap()
       val chosen = data.mapPartitionsWithIndex { (index, points) =>
-        val rand = new Random(seed ^ (step << 16) ^ index)
+        val rand = new XORShiftRandom(seed ^ (step << 16) ^ index)
         for {
           p <- points
           r <- 0 until runs
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..8a065c6d7d1d7f458789f3ab50f0054c8a947dfb
--- /dev/null
+++ b/new-yarn/pom.xml
@@ -0,0 +1,161 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-yarn_2.9.3</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Support</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_2.9.3</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${yarn.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-ipc</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_2.9.3</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <configuration>
+          <shadedArtifactAttached>false</shadedArtifactAttached>
+          <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile>
+          <artifactSet>
+            <includes>
+              <include>*:*</include>
+            </includes>
+          </artifactSet>
+          <filters>
+            <filter>
+              <artifact>*:*</artifact>
+              <excludes>
+                <exclude>META-INF/*.SF</exclude>
+                <exclude>META-INF/*.DSA</exclude>
+                <exclude>META-INF/*.RSA</exclude>
+              </excludes>
+            </filter>
+          </filters>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                  <resource>reference.conf</resource>
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>test</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <exportAntProperties>true</exportAntProperties>
+              <tasks>
+                <property name="spark.classpath" refid="maven.test.classpath" />
+                <property environment="env" />
+                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">  
+                  <condition>
+                    <not>
+                      <or>
+                        <isset property="env.SCALA_HOME" />
+                        <isset property="env.SCALA_LIBRARY_PATH" />
+                      </or>
+                    </not>
+                  </condition>
+                </fail>
+              </tasks>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+        <configuration>
+          <environmentVariables>
+            <SPARK_HOME>${basedir}/..</SPARK_HOME>
+            <SPARK_TESTING>1</SPARK_TESTING>
+            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
+          </environmentVariables>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000000000000000000000000000000000000..eeeca3ea8a33e4f77562383dfef50a7507628894
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,446 @@
+/*
+ * 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.deploy.yarn
+
+import java.io.IOException
+import java.net.Socket
+import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+
+import scala.collection.JavaConversions._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.util.ShutdownHookManager
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.Utils
+
+
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  
+  private var rpc: YarnRPC = YarnRPC.create(conf)
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+  private var appAttemptId: ApplicationAttemptId = _
+  private var userThread: Thread = _
+  private val fs = FileSystem.get(yarnConf)
+
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var isFinished: Boolean = false
+  private var uiAddress: String = _
+  private val maxAppAttempts: Int = conf.getInt(
+    YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
+  private var isLastAMRetry: Boolean = true
+  private var amClient: AMRMClient[ContainerRequest] = _
+
+  // Default to numWorkers * 2, with minimum of 3
+  private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
+    math.max(args.numWorkers * 2, 3).toString()).toInt
+
+  def run() {
+    // Setup the directories so things go to YARN approved directories rather
+    // than user specified and /tmp.
+    System.setProperty("spark.local.dir", getLocalDirs())
+
+    // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using.
+    ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
+
+    appAttemptId = getApplicationAttemptId()
+    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
+    amClient = AMRMClient.createAMRMClient()
+    amClient.init(yarnConf)
+    amClient.start()
+
+    // Workaround until hadoop moves to something which has
+    // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
+    // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
+    
+    ApplicationMaster.register(this)
+
+    // Start the user's JAR
+    userThread = startUserClass()
+    
+    // This a bit hacky, but we need to wait until the spark.driver.port property has
+    // been set by the Thread executing the user class.
+    waitForSparkMaster()
+
+    waitForSparkContextInitialized()
+
+    // Do this after Spark master is up and SparkContext is created so that we can register UI Url.
+    val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
+
+    // Allocate all containers
+    allocateWorkers()
+
+    // Wait for the user class to Finish     
+    userThread.join()
+
+    System.exit(0)
+  }
+
+  /** Get the Yarn approved local directories. */
+  private def getLocalDirs(): String = {
+    // Hadoop 0.23 and 2.x have different Environment variable names for the
+    // local dirs, so lets check both. We assume one of the 2 is set.
+    // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
+    val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
+      .getOrElse(Option(System.getenv("LOCAL_DIRS"))
+        .getOrElse(""))
+
+    if (localDirs.isEmpty()) {
+      throw new Exception("Yarn Local dirs can't be empty")
+    }
+    localDirs
+  }
+  
+  private def getApplicationAttemptId(): ApplicationAttemptId = {
+    val envs = System.getenv()
+    val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    appAttemptId
+  }
+  
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
+  }
+  
+  private def waitForSparkMaster() {
+    logInfo("Waiting for Spark driver to be reachable.")
+    var driverUp = false
+    var tries = 0
+    val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt
+    while (!driverUp && tries < numTries) {
+      val driverHost = System.getProperty("spark.driver.host")
+      val driverPort = System.getProperty("spark.driver.port")
+      try {
+        val socket = new Socket(driverHost, driverPort.toInt)
+        socket.close()
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+        driverUp = true
+      } catch {
+        case e: Exception => {
+          logWarning("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+          Thread.sleep(100)
+          tries = tries + 1
+        }
+      }
+    }
+  }
+
+  private def startUserClass(): Thread  = {
+    logInfo("Starting the user JAR in a separate Thread")
+    val mainMethod = Class.forName(
+      args.userClass,
+      false /* initialize */,
+      Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
+    val t = new Thread {
+      override def run() {
+        var successed = false
+        try {
+          // Copy
+          var mainArgs: Array[String] = new Array[String](args.userArgs.size)
+          args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size)
+          mainMethod.invoke(null, mainArgs)
+          // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR
+          // userThread will stop here unless it has uncaught exception thrown out
+          // It need shutdown hook to set SUCCEEDED
+          successed = true
+        } finally {
+          logDebug("finishing main")
+          isLastAMRetry = true
+          if (successed) {
+            ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+          } else {
+            ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
+          }
+        }
+      }
+    }
+    t.start()
+    t
+  }
+
+  // This need to happen before allocateWorkers()
+  private def waitForSparkContextInitialized() {
+    logInfo("Waiting for Spark context initialization")
+    try {
+      var sparkContext: SparkContext = null
+      ApplicationMaster.sparkContextRef.synchronized {
+        var numTries = 0
+        val waitTime = 10000L
+        val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+        while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) {
+          logInfo("Waiting for Spark context initialization ... " + numTries)
+          numTries = numTries + 1
+          ApplicationMaster.sparkContextRef.wait(waitTime)
+        }
+        sparkContext = ApplicationMaster.sparkContextRef.get()
+        assert(sparkContext != null || numTries >= maxNumTries)
+
+        if (sparkContext != null) {
+          uiAddress = sparkContext.ui.appUIAddress
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            amClient,
+            appAttemptId,
+            args, 
+            sparkContext.preferredNodeLocationData)
+        } else {
+          logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d".
+            format(numTries * waitTime, maxNumTries))
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            amClient,
+            appAttemptId,
+            args)
+        }
+      }
+    } finally {
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT :
+      // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks.
+      ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+    }
+  }
+
+  private def allocateWorkers() {
+    try {
+      logInfo("Allocating " + args.numWorkers + " workers.")
+      // Wait until all containers have finished
+      // TODO: This is a bit ugly. Can we make it nicer?
+      // TODO: Handle container failure
+      yarnAllocator.addResourceRequests(args.numWorkers)
+      // Exits the loop if the user thread exits.
+      while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+        if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+          finishApplicationMaster(FinalApplicationStatus.FAILED,
+            "max number of worker failures reached")
+        }
+        yarnAllocator.allocateResources()
+        ApplicationMaster.incrementAllocatorLoop(1)
+        Thread.sleep(100)
+      }
+    } finally {
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+      // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
+      ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+    }
+    logInfo("All workers have launched.")
+
+    // Launch a progress reporter thread, else the app will get killed after expiration
+    // (def: 10mins) timeout.
+    if (userThread.isAlive) {
+      // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
+      val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+
+      // we want to be reasonably responsive without causing too many requests to RM.
+      val schedulerInterval =
+        System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+
+      // must be <= timeoutInterval / 2.
+      val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
+      launchReporterThread(interval)
+    }
+  }
+
+  private def launchReporterThread(_sleepTime: Long): Thread = {
+    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+
+    val t = new Thread {
+      override def run() {
+        while (userThread.isAlive) {
+          if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+            finishApplicationMaster(FinalApplicationStatus.FAILED,
+              "max number of worker failures reached")
+          }
+          val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+            yarnAllocator.getNumPendingAllocate
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
+            yarnAllocator.addResourceRequests(missingWorkerCount)
+          }
+          sendProgress()
+          Thread.sleep(sleepTime)
+        }
+      }
+    }
+    // Setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.start()
+    logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+    t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // Simulated with an allocate request with no nodes requested.
+    yarnAllocator.allocateResources()
+  }
+
+  /*
+  def printContainers(containers: List[Container]) = {
+    for (container <- containers) {
+      logInfo("Launching shell command on a new container."
+        + ", containerId=" + container.getId()
+        + ", containerNode=" + container.getNodeId().getHost() 
+        + ":" + container.getNodeId().getPort()
+        + ", containerNodeURI=" + container.getNodeHttpAddress()
+        + ", containerState" + container.getState()
+        + ", containerResourceMemory"  
+        + container.getResource().getMemory())
+    }
+  }
+  */
+
+  def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
+    synchronized {
+      if (isFinished) {
+        return
+      }
+      isFinished = true
+    }
+
+    logInfo("finishApplicationMaster with " + status)
+    // Set tracking URL to empty since we don't have a history server.
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+  }
+
+  /**
+   * Clean up the staging directory. 
+   */
+  private def cleanupStagingDir() { 
+    var stagingDirPath: Path = null
+    try {
+      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      if (!preserveFiles) {
+        stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
+        if (stagingDirPath == null) {
+          logError("Staging directory is null")
+          return
+        }
+        logInfo("Deleting staging directory " + stagingDirPath)
+        fs.delete(stagingDirPath, true)
+      }
+    } catch {
+      case ioe: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
+    }
+  }
+
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
+
+    def run() {
+      logInfo("AppMaster received a signal.")
+      // we need to clean up staging dir before HDFS is shut down
+      // make sure we don't delete it until this is the last AM
+      if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
+    }
+  }
+}
+
+object ApplicationMaster {
+  // Number of times to wait for the allocator loop to complete.
+  // Each loop iteration waits for 100ms, so maximum of 3 seconds.
+  // This is to ensure that we have reasonable number of containers before we start
+  // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+  // optimal as more containers are available. Might need to handle this better.
+  private val ALLOCATOR_LOOP_WAIT_COUNT = 30
+
+  private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]()
+
+  val sparkContextRef: AtomicReference[SparkContext] =
+    new AtomicReference[SparkContext](null /* initialValue */)
+
+  val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
+
+  def incrementAllocatorLoop(by: Int) {
+    val count = yarnAllocatorLoop.getAndAdd(by)
+    if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
+      yarnAllocatorLoop.synchronized {
+        // to wake threads off wait ...
+        yarnAllocatorLoop.notifyAll()
+      }
+    }
+  }
+
+  def register(master: ApplicationMaster) {
+    applicationMasters.add(master)
+  }
+
+  // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm...
+  def sparkContextInitialized(sc: SparkContext): Boolean = {
+    var modified = false
+    sparkContextRef.synchronized {
+      modified = sparkContextRef.compareAndSet(null, sc)
+      sparkContextRef.notifyAll()
+    }
+
+    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do
+    // System.exit.
+    // Should not really have to do this, but it helps YARN to evict resources earlier.
+    // Not to mention, prevent the Client from declaring failure even though we exited properly.
+    // Note that this will unfortunately not properly clean up the staging files because it gets
+    // called too late, after the filesystem is already shutdown.
+    if (modified) {
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+        // This is not only logs, but also ensures that log system is initialized for this instance
+        // when we are actually 'run'-ing.
+        logInfo("Adding shutdown hook for context " + sc)
+        override def run() { 
+          logInfo("Invoking sc stop from shutdown hook") 
+          sc.stop() 
+          // Best case ...
+          for (master <- applicationMasters) {
+            master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+          }
+        } 
+      } )
+    }
+
+    // Wait for initialization to complete and atleast 'some' nodes can get allocated.
+    yarnAllocatorLoop.synchronized {
+      while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
+        yarnAllocatorLoop.wait(1000L)
+      }
+    }
+    modified
+  }
+
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new ApplicationMaster(args).run()
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f76a5ddd39e90d4998d7712113dd0949823d5e18
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -0,0 +1,94 @@
+/*
+ * 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.deploy.yarn
+
+import org.apache.spark.util.IntParam
+import collection.mutable.ArrayBuffer
+
+class ApplicationMasterArguments(val args: Array[String]) {
+  var userJar: String = null
+  var userClass: String = null
+  var userArgs: Seq[String] = Seq[String]()
+  var workerMemory = 1024
+  var workerCores = 1
+  var numWorkers = 2
+
+  parseArgs(args.toList)
+  
+  private def parseArgs(inputArgs: List[String]): Unit = {
+    val userArgsBuffer = new ArrayBuffer[String]()
+
+    var args = inputArgs
+
+    while (! args.isEmpty) {
+
+      args match {
+        case ("--jar") :: value :: tail =>
+          userJar = value
+          args = tail
+
+        case ("--class") :: value :: tail =>
+          userClass = value
+          args = tail
+
+        case ("--args") :: value :: tail =>
+          userArgsBuffer += value
+          args = tail
+
+        case ("--num-workers") :: IntParam(value) :: tail =>
+          numWorkers = value
+          args = tail
+
+        case ("--worker-memory") :: IntParam(value) :: tail =>
+          workerMemory = value
+          args = tail
+
+        case ("--worker-cores") :: IntParam(value) :: tail =>
+          workerCores = value
+          args = tail
+
+        case Nil =>
+          if (userJar == null || userClass == null) {
+            printUsageAndExit(1)
+          }
+
+        case _ =>
+          printUsageAndExit(1, args)
+      }
+    }
+
+    userArgs = userArgsBuffer.readOnly
+  }
+  
+  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+    if (unknownParam != null) {
+      System.err.println("Unknown/unsupported param " + unknownParam)
+    }
+    System.err.println(
+      "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" +
+      "Options:\n" +
+      "  --jar JAR_PATH       Path to your application's JAR file (required)\n" +
+      "  --class CLASS_NAME   Name of your application's main class (required)\n" +
+      "  --args ARGS          Arguments to be passed to your application's main class.\n" +
+      "                       Mutliple invocations are possible, each will be passed in order.\n" +
+      "  --num-workers NUM    Number of workers to start (Default: 2)\n" +
+      "  --worker-cores NUM   Number of cores for the workers (Default: 1)\n" +
+      "  --worker-memory MEM  Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n")
+    System.exit(exitCode)
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000000000000000000000000000000000000..94678815e806ad66680880e59ac4f7c01d1fd67c
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -0,0 +1,519 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Apps, Records}
+
+import org.apache.spark.Logging 
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
+
+
+/**
+ * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The
+ * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
+ * which will launch a Spark master process and negotiate resources throughout its duration.
+ */
+class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+
+  var rpc: YarnRPC = YarnRPC.create(conf)
+  val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+  val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+  private val SPARK_STAGING: String = ".sparkStaging"
+  private val distCacheMgr = new ClientDistributedCacheManager()
+
+  // Staging directory is private! -> rwx--------
+  val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short)
+  // App files are world-wide readable and owner writable -> rw-r--r--
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
+
+  def this(args: ClientArguments) = this(new Configuration(), args)
+
+  def runApp(): ApplicationId = {
+    validateArgs()
+    // Initialize and start the client service.
+    init(yarnConf)
+    start()
+
+    // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers).
+    logClusterResourceDetails()
+
+    // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM)
+    // interface).
+
+    // Get a new client application.
+    val newApp = super.createApplication()
+    val newAppResponse = newApp.getNewApplicationResponse()
+    val appId = newAppResponse.getApplicationId()
+
+    verifyClusterResources(newAppResponse)
+
+    // Set up resource and environment variables.
+    val appStagingDir = getAppStagingDir(appId)
+    val localResources = prepareLocalResources(appStagingDir)
+    val launchEnv = setupLaunchEnv(localResources, appStagingDir)
+    val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv)
+
+    // Set up an application submission context.
+    val appContext = newApp.getApplicationSubmissionContext()
+    appContext.setApplicationName(args.appName)
+    appContext.setQueue(args.amQueue)
+    appContext.setAMContainerSpec(amContainer)
+
+    // Memory for the ApplicationMaster.
+    val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
+    memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+    appContext.setResource(memoryResource)
+
+    // Finally, submit and monitor the application.
+    submitApp(appContext)
+    appId
+  }
+
+  def run() {
+    val appId = runApp()
+    monitorApplication(appId)
+    System.exit(0)
+  }
+
+  // TODO(harvey): This could just go in ClientArguments.
+  def validateArgs() = {
+    Map(
+      (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+      (args.userJar == null) -> "Error: You must specify a user jar!",
+      (args.userClass == null) -> "Error: You must specify a user class!",
+      (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!",
+      (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" +
+        "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+      (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" +
+        "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString)
+    ).foreach { case(cond, errStr) =>
+      if (cond) {
+        logError(errStr)
+        args.printUsageAndExit(1)
+      }
+    }
+  }
+
+  def getAppStagingDir(appId: ApplicationId): String = {
+    SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
+  }
+
+  def logClusterResourceDetails() {
+    val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
+    logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " +
+      clusterMetrics.getNumNodeManagers)
+
+    val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
+    logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
+      queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+        queueInfo.getQueueName,
+        queueInfo.getCurrentCapacity,
+        queueInfo.getMaximumCapacity,
+        queueInfo.getApplications.size,
+        queueInfo.getChildQueues.size))
+  }
+
+  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+    val maxMem = app.getMaximumResourceCapability().getMemory()
+    logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
+
+    // If we have requested more then the clusters max for a single resource then exit.
+    if (args.workerMemory > maxMem) {
+      logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.".
+        format(args.workerMemory, maxMem))
+      System.exit(1)
+    }
+    val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+    if (amMem > maxMem) {
+      logError("Required AM memory (%d) is above the max threshold (%d) of this cluster".
+        format(args.amMemory, maxMem))
+      System.exit(1)
+    }
+
+    // We could add checks to make sure the entire cluster has enough resources but that involves
+    // getting all the node reports and computing ourselves.
+  }
+
+  /** See if two file systems are the same or not. */
+  private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+    val srcUri = srcFs.getUri()
+    val dstUri = destFs.getUri()
+    if (srcUri.getScheme() == null) {
+      return false
+    }
+    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+      return false
+    }
+    var srcHost = srcUri.getHost()
+    var dstHost = dstUri.getHost()
+    if ((srcHost != null) && (dstHost != null)) {
+      try {
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+      } catch {
+        case e: UnknownHostException =>
+          return false
+      }
+      if (!srcHost.equals(dstHost)) {
+        return false
+      }
+    } else if (srcHost == null && dstHost != null) {
+      return false
+    } else if (srcHost != null && dstHost == null) {
+      return false
+    }
+    //check for ports
+    if (srcUri.getPort() != dstUri.getPort()) {
+      return false
+    }
+    return true
+  }
+
+  /** Copy the file into HDFS if needed. */
+  private def copyRemoteFile(
+      dstDir: Path,
+      originalPath: Path,
+      replication: Short,
+      setPerms: Boolean = false): Path = {
+    val fs = FileSystem.get(conf)
+    val remoteFs = originalPath.getFileSystem(conf)
+    var newPath = originalPath
+    if (! compareFs(remoteFs, fs)) {
+      newPath = new Path(dstDir, originalPath.getName())
+      logInfo("Uploading " + originalPath + " to " + newPath)
+      FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
+      fs.setReplication(newPath, replication)
+      if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
+    } 
+    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+    // version shows the specific version in the distributed cache configuration
+    val qualPath = fs.makeQualified(newPath)
+    val fc = FileContext.getFileContext(qualPath.toUri(), conf)
+    val destPath = fc.resolvePath(qualPath)
+    destPath
+  }
+
+  def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
+    logInfo("Preparing Local resources")
+    // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+    // local resources to the application master.
+    val fs = FileSystem.get(conf)
+
+    val delegTokenRenewer = Master.getMasterPrincipal(conf)
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+        logError("Can't get Master Kerberos principal for use as renewer")
+        System.exit(1)
+      }
+    }
+    val dst = new Path(fs.getHomeDirectory(), appStagingDir)
+    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val dstFs = dst.getFileSystem(conf)
+      dstFs.addDelegationTokens(delegTokenRenewer, credentials)
+    }
+
+    val localResources = HashMap[String, LocalResource]()
+    FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
+
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+    Map(
+      Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
+      Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")
+    ).foreach { case(destName, _localPath) =>
+      val localPath: String = if (_localPath != null) _localPath.trim() else ""
+      if (! localPath.isEmpty()) {
+        var localURI = new URI(localPath)
+        // If not specified assume these are in the local filesystem to keep behavior like Hadoop
+        if (localURI.getScheme() == null) {
+          localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
+        }
+        val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
+        val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          destName, statCache)
+      }
+    }
+
+    // Handle jars local to the ApplicationMaster.
+    if ((args.addJars != null) && (!args.addJars.isEmpty())){
+      args.addJars.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        // Only add the resource to the Spark ApplicationMaster.
+        val appMasterOnly = true
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache, appMasterOnly)
+      }
+    }
+
+    // Handle any distributed cache files
+    if ((args.files != null) && (!args.files.isEmpty())){
+      args.files.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache)
+      }
+    }
+
+    // Handle any distributed cache archives
+    if ((args.archives != null) && (!args.archives.isEmpty())) {
+      args.archives.split(',').foreach { case file:String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+          linkname, statCache)
+      }
+    }
+
+    UserGroupInformation.getCurrentUser().addCredentials(credentials)
+    localResources
+  }
+
+  def setupLaunchEnv(
+      localResources: HashMap[String, LocalResource], 
+      stagingDir: String): HashMap[String, String] = {
+    logInfo("Setting up the launch environment")
+    val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
+
+    val env = new HashMap[String, String]()
+
+    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    env("SPARK_YARN_MODE") = "true"
+    env("SPARK_YARN_STAGING_DIR") = stagingDir
+
+    // Set the environment variables to be passed on to the Workers.
+    distCacheMgr.setDistFilesEnv(env)
+    distCacheMgr.setDistArchivesEnv(env)
+
+    // Allow users to specify some environment variables.
+    Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+    // Add each SPARK_* key to the environment.
+    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+
+    env
+  }
+
+  def userArgsToString(clientArgs: ClientArguments): String = {
+    val prefix = " --args "
+    val args = clientArgs.userArgs
+    val retval = new StringBuilder()
+    for (arg <- args){
+      retval.append(prefix).append(" '").append(arg).append("' ")
+    }
+    retval.toString
+  }
+
+  def createContainerLaunchContext(
+      newApp: GetNewApplicationResponse,
+      localResources: HashMap[String, LocalResource],
+      env: HashMap[String, String]): ContainerLaunchContext = {
+    logInfo("Setting up container launch context")
+    val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+    amContainer.setLocalResources(localResources)
+    amContainer.setEnvironment(env)
+
+    // TODO: Need a replacement for the following code to fix -Xmx?
+    // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
+    // var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
+    //  ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+    //    YarnAllocationHandler.MEMORY_OVERHEAD)
+
+    // Extra options for the JVM
+    var JAVA_OPTS = ""
+
+    // Add Xmx for AM memory
+    JAVA_OPTS += "-Xmx" + args.amMemory + "m"
+
+    val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+    JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
+
+    // TODO: Remove once cpuset version is pushed out.
+    // The context is, default gc for server class machines ends up using all cores to do gc -
+    // hence if there are multiple containers in same node, Spark GC affects all other containers'
+    // performance (which can be that of other Spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
+    // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
+    // of cores on a node.
+    val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+      java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+    if (useConcurrentAndIncrementalGC) {
+      // In our expts, using (default) throughput collector has severe perf ramifications in
+      // multi-tenant machines
+      JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+      JAVA_OPTS += " -XX:+CMSIncrementalMode "
+      JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+    }
+
+    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+      JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
+    }
+
+    // Command for the ApplicationMaster
+    var javaCommand = "java"
+    val javaHome = System.getenv("JAVA_HOME")
+    if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+      javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+    }
+
+    val commands = List[String](
+      javaCommand + 
+      " -server " +
+      JAVA_OPTS +
+      " " + args.amClass +
+      " --class " + args.userClass + 
+      " --jar " + args.userJar +
+      userArgsToString(args) +
+      " --worker-memory " + args.workerMemory +
+      " --worker-cores " + args.workerCores +
+      " --num-workers " + args.numWorkers +
+      " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+      " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+    logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
+    amContainer.setCommands(commands)
+
+    // Setup security tokens.
+    val dob = new DataOutputBuffer()
+    credentials.writeTokenStorageToStream(dob)
+    amContainer.setTokens(ByteBuffer.wrap(dob.getData()))
+
+    amContainer
+  }
+
+  def submitApp(appContext: ApplicationSubmissionContext) = {
+    // Submit the application to the applications manager.
+    logInfo("Submitting application to ASM")
+    super.submitApplication(appContext)
+  }
+
+  def monitorApplication(appId: ApplicationId): Boolean = {  
+    while (true) {
+      Thread.sleep(1000)
+      val report = super.getApplicationReport(appId)
+
+      logInfo("Application report from ASM: \n" +
+        "\t application identifier: " + appId.toString() + "\n" +
+        "\t appId: " + appId.getId() + "\n" +
+        "\t clientToAMToken: " + report.getClientToAMToken() + "\n" +
+        "\t appDiagnostics: " + report.getDiagnostics() + "\n" +
+        "\t appMasterHost: " + report.getHost() + "\n" +
+        "\t appQueue: " + report.getQueue() + "\n" +
+        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+        "\t appStartTime: " + report.getStartTime() + "\n" +
+        "\t yarnAppState: " + report.getYarnApplicationState() + "\n" +
+        "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" +
+        "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
+        "\t appUser: " + report.getUser()
+      )
+
+      val state = report.getYarnApplicationState()
+      val dsStatus = report.getFinalApplicationStatus()
+      if (state == YarnApplicationState.FINISHED || 
+        state == YarnApplicationState.FAILED ||
+        state == YarnApplicationState.KILLED) {
+        return true
+      }
+    }
+    true
+  }
+}
+
+object Client {
+  val SPARK_JAR: String = "spark.jar"
+  val APP_JAR: String = "app.jar"
+  val LOG4J_PROP: String = "log4j.properties"
+
+  def main(argStrings: Array[String]) {
+    // Set an env variable indicating we are running in YARN mode.
+    // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
+    // see Client#setupLaunchEnv().
+    System.setProperty("SPARK_YARN_MODE", "true")
+
+    val args = new ClientArguments(argStrings)
+
+    (new Client(args)).run()
+  }
+
+  // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
+  def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
+    for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+      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_PROP)
+    }
+    // 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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9efb28a9426726c4e41e4287291b8ab939ba942e
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.deploy.yarn
+
+import scala.collection.mutable.{ArrayBuffer, HashMap}
+
+import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
+import org.apache.spark.util.IntParam
+import org.apache.spark.util.MemoryParam
+
+
+// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
+class ClientArguments(val args: Array[String]) {
+  var addJars: String = null
+  var files: String = null
+  var archives: String = null
+  var userJar: String = null
+  var userClass: String = null
+  var userArgs: Seq[String] = Seq[String]()
+  var workerMemory = 1024 // MB
+  var workerCores = 1
+  var numWorkers = 2
+  var amQueue = System.getProperty("QUEUE", "default")
+  var amMemory: Int = 512 // MB
+  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
+  var appName: String = "Spark"
+  // TODO
+  var inputFormatInfo: List[InputFormatInfo] = null
+  // TODO(harvey)
+  var priority = 0
+
+  parseArgs(args.toList)
+
+  private def parseArgs(inputArgs: List[String]): Unit = {
+    val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]()
+    val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]()
+
+    var args = inputArgs
+
+    while (!args.isEmpty) {
+      args match {
+        case ("--jar") :: value :: tail =>
+          userJar = value
+          args = tail
+
+        case ("--class") :: value :: tail =>
+          userClass = value
+          args = tail
+
+        case ("--args") :: value :: tail =>
+          userArgsBuffer += value
+          args = tail
+
+        case ("--master-class") :: value :: tail =>
+          amClass = value
+          args = tail
+
+        case ("--master-memory") :: MemoryParam(value) :: tail =>
+          amMemory = value
+          args = tail
+
+        case ("--num-workers") :: IntParam(value) :: tail =>
+          numWorkers = value
+          args = tail
+
+        case ("--worker-memory") :: MemoryParam(value) :: tail =>
+          workerMemory = value
+          args = tail
+
+        case ("--worker-cores") :: IntParam(value) :: tail =>
+          workerCores = value
+          args = tail
+
+        case ("--queue") :: value :: tail =>
+          amQueue = value
+          args = tail
+
+        case ("--name") :: value :: tail =>
+          appName = value
+
+        case ("--addJars") :: value :: tail =>
+          addJars = value
+          args = tail
+
+        case ("--files") :: value :: tail =>
+          files = value
+          args = tail
+
+        case ("--archives") :: value :: tail =>
+          archives = value
+          args = tail
+
+        case Nil =>
+          if (userJar == null || userClass == null) {
+            printUsageAndExit(1)
+          }
+
+        case _ =>
+          printUsageAndExit(1, args)
+      }
+    }
+
+    userArgs = userArgsBuffer.readOnly
+    inputFormatInfo = inputFormatMap.values.toList
+  }
+
+
+  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+    if (unknownParam != null) {
+      System.err.println("Unknown/unsupported param " + unknownParam)
+    }
+    System.err.println(
+      "Usage: org.apache.spark.deploy.yarn.Client [options] \n" +
+      "Options:\n" +
+      "  --jar JAR_PATH             Path to your application's JAR file (required)\n" +
+      "  --class CLASS_NAME         Name of your application's main class (required)\n" +
+      "  --args ARGS                Arguments to be passed to your application's main class.\n" +
+      "                             Mutliple invocations are possible, each will be passed in order.\n" +
+      "  --num-workers NUM          Number of workers to start (Default: 2)\n" +
+      "  --worker-cores NUM         Number of cores for the workers (Default: 1). This is unsused right now.\n" +
+      "  --master-class CLASS_NAME  Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" +
+      "  --master-memory MEM        Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
+      "  --worker-memory MEM        Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
+      "  --name NAME                The name of your application (Default: Spark)\n" +
+      "  --queue QUEUE              The hadoop queue to use for allocation requests (Default: 'default')\n" +
+      "  --addJars jars             Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
+      "  --files files              Comma separated list of files to be distributed with the job.\n" +
+      "  --archives archives        Comma separated list of archives to be distributed with the job."
+      )
+    System.exit(exitCode)
+  }
+
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5f159b073f5372dc9e1f73347f88ae8aa48e0cbb
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -0,0 +1,228 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.URI
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
+
+import org.apache.spark.Logging 
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.LinkedHashMap
+import scala.collection.mutable.Map
+
+
+/** Client side methods to setup the Hadoop distributed cache */
+class ClientDistributedCacheManager() extends Logging {
+  private val distCacheFiles: Map[String, Tuple3[String, String, String]] = 
+    LinkedHashMap[String, Tuple3[String, String, String]]()
+  private val distCacheArchives: Map[String, Tuple3[String, String, String]] = 
+    LinkedHashMap[String, Tuple3[String, String, String]]()
+
+
+  /**
+   * Add a resource to the list of distributed cache resources. This list can
+   * be sent to the ApplicationMaster and possibly the workers so that it can 
+   * be downloaded into the Hadoop distributed cache for use by this application.
+   * Adds the LocalResource to the localResources HashMap passed in and saves 
+   * the stats of the resources to they can be sent to the workers and verified.
+   *
+   * @param fs FileSystem
+   * @param conf Configuration
+   * @param destPath path to the resource
+   * @param localResources localResource hashMap to insert the resource into
+   * @param resourceType LocalResourceType 
+   * @param link link presented in the distributed cache to the destination
+   * @param statCache cache to store the file/directory stats 
+   * @param appMasterOnly Whether to only add the resource to the app master
+   */
+  def addResource(
+      fs: FileSystem,
+      conf: Configuration,
+      destPath: Path, 
+      localResources: HashMap[String, LocalResource],
+      resourceType: LocalResourceType,
+      link: String,
+      statCache: Map[URI, FileStatus],
+      appMasterOnly: Boolean = false) = {
+    val destStatus = fs.getFileStatus(destPath)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(resourceType)
+    val visibility = getVisibility(conf, destPath.toUri(), statCache)
+    amJarRsrc.setVisibility(visibility)
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
+    amJarRsrc.setTimestamp(destStatus.getModificationTime())
+    amJarRsrc.setSize(destStatus.getLen())
+    if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name")
+    localResources(link) = amJarRsrc
+    
+    if (appMasterOnly == false) {
+      val uri = destPath.toUri()
+      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
+      if (resourceType == LocalResourceType.FILE) {
+        distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), 
+          destStatus.getModificationTime().toString(), visibility.name())
+      } else {
+        distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), 
+          destStatus.getModificationTime().toString(), visibility.name())
+      }
+    }
+  }
+
+  /**
+   * Adds the necessary cache file env variables to the env passed in
+   * @param env
+   */
+  def setDistFilesEnv(env: Map[String, String]) = {
+    val (keys, tupleValues) = distCacheFiles.unzip
+    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
+
+    if (keys.size > 0) {
+      env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = 
+        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = 
+        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = 
+        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
+    }
+  }
+
+  /**
+   * Adds the necessary cache archive env variables to the env passed in
+   * @param env
+   */
+  def setDistArchivesEnv(env: Map[String, String]) = {
+    val (keys, tupleValues) = distCacheArchives.unzip
+    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
+
+    if (keys.size > 0) {
+      env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = 
+        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") =
+        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = 
+        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
+    }
+  }
+
+  /**
+   * Returns the local resource visibility depending on the cache file permissions
+   * @param conf
+   * @param uri
+   * @param statCache
+   * @return LocalResourceVisibility
+   */
+  def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
+      LocalResourceVisibility = {
+    if (isPublic(conf, uri, statCache)) {
+      return LocalResourceVisibility.PUBLIC 
+    } 
+    return LocalResourceVisibility.PRIVATE
+  }
+
+  /**
+   * Returns a boolean to denote whether a cache file is visible to all(public)
+   * or not
+   * @param conf
+   * @param uri
+   * @param statCache
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
+    val fs = FileSystem.get(uri, conf)
+    val current = new Path(uri.getPath())
+    //the leaf level file should be readable by others
+    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
+      return false
+    }
+    return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
+  }
+
+  /**
+   * Returns true if all ancestors of the specified path have the 'execute'
+   * permission set for all users (i.e. that other users can traverse
+   * the directory heirarchy to the given path)
+   * @param fs
+   * @param path
+   * @param statCache
+   * @return true if all ancestors have the 'execute' permission set for all users
+   */
+  def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, 
+      statCache: Map[URI, FileStatus]): Boolean =  {
+    var current = path
+    while (current != null) {
+      //the subdirs in the path should have execute permissions for others
+      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
+        return false
+      }
+      current = current.getParent()
+    }
+    return true
+  }
+
+  /**
+   * Checks for a given path whether the Other permissions on it 
+   * imply the permission in the passed FsAction
+   * @param fs
+   * @param path
+   * @param action
+   * @param statCache
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  def checkPermissionOfOther(fs: FileSystem, path: Path,
+      action: FsAction, statCache: Map[URI, FileStatus]): Boolean = {
+    val status = getFileStatus(fs, path.toUri(), statCache)
+    val perms = status.getPermission()
+    val otherAction = perms.getOtherAction()
+    if (otherAction.implies(action)) {
+      return true
+    }
+    return false
+  }
+
+  /**
+   * Checks to see if the given uri exists in the cache, if it does it 
+   * returns the existing FileStatus, otherwise it stats the uri, stores
+   * it in the cache, and returns the FileStatus.
+   * @param fs
+   * @param uri
+   * @param statCache
+   * @return FileStatus
+   */
+  def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = {
+    val stat = statCache.get(uri) match {
+      case Some(existstat) => existstat
+      case None => 
+        val newStat = fs.getFileStatus(new Path(uri))
+        statCache.put(uri, newStat)
+        newStat
+    }
+    return stat
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c38f33e212fbf2e2bf60fa1dd4793012ea288f81
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -0,0 +1,223 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.Socket
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+import akka.actor._
+import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
+import akka.actor.Terminated
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+
+  private var appAttemptId: ApplicationAttemptId = _
+  private var reporterThread: Thread = _
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var driverClosed:Boolean = false
+
+  private var amClient: AMRMClient[ContainerRequest] = _
+
+  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
+  var actor: ActorRef = _
+
+  // This actor just working as a monitor to watch on Driver Actor.
+  class MonitorActor(driverUrl: String) extends Actor {
+
+    var driver: ActorRef = null
+
+    override def preStart() {
+      logInfo("Listen to driver: " + driverUrl)
+      driver = context.actorFor(driverUrl)
+      context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+      context.watch(driver) // Doesn't work with remote actors, but useful for testing
+    }
+
+    override def receive = {
+      case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+        logInfo("Driver terminated or disconnected! Shutting down.")
+        driverClosed = true
+    }
+  }
+
+  def run() {
+
+    amClient = AMRMClient.createAMRMClient()
+    amClient.init(yarnConf)
+    amClient.start()
+
+    appAttemptId = getApplicationAttemptId()
+    registerApplicationMaster()
+
+    waitForSparkMaster()
+
+    // Allocate all containers
+    allocateWorkers()
+
+    // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+    // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
+
+    val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+    // must be <= timeoutInterval/ 2.
+    // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
+    // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
+    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+    reporterThread = launchReporterThread(interval)
+
+    // Wait for the reporter thread to Finish.
+    reporterThread.join()
+
+    finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+    actorSystem.shutdown()
+
+    logInfo("Exited")
+    System.exit(0)
+  }
+
+  private def getApplicationAttemptId(): ApplicationAttemptId = {
+    val envs = System.getenv()
+    val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    appAttemptId
+  }
+
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    // TODO:(Raymond) Find out Spark UI address and fill in here?
+    amClient.registerApplicationMaster(Utils.localHostName(), 0, "")
+  }
+
+  private def waitForSparkMaster() {
+    logInfo("Waiting for Spark driver to be reachable.")
+    var driverUp = false
+    val hostport = args.userArgs(0)
+    val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+    while(!driverUp) {
+      try {
+        val socket = new Socket(driverHost, driverPort)
+        socket.close()
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+        Thread.sleep(100)
+      }
+    }
+    System.setProperty("spark.driver.host", driverHost)
+    System.setProperty("spark.driver.port", driverPort.toString)
+
+    val driverUrl = "akka://spark@%s:%s/user/%s".format(
+      driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+    actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
+  }
+
+
+  private def allocateWorkers() {
+
+    // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
+    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+      scala.collection.immutable.Map()
+
+    yarnAllocator = YarnAllocationHandler.newAllocator(
+      yarnConf,
+      amClient,
+      appAttemptId,
+      args,
+      preferredNodeLocationData)
+
+    logInfo("Allocating " + args.numWorkers + " workers.")
+    // Wait until all containers have finished
+    // TODO: This is a bit ugly. Can we make it nicer?
+    // TODO: Handle container failure
+
+    yarnAllocator.addResourceRequests(args.numWorkers)
+    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+      yarnAllocator.allocateResources()
+      Thread.sleep(100)
+    }
+
+    logInfo("All workers have launched.")
+
+  }
+
+  // TODO: We might want to extend this to allocate more containers in case they die !
+  private def launchReporterThread(_sleepTime: Long): Thread = {
+    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+
+    val t = new Thread {
+      override def run() {
+        while (!driverClosed) {
+          val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+            yarnAllocator.getNumPendingAllocate
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
+            yarnAllocator.addResourceRequests(missingWorkerCount)
+          }
+          sendProgress()
+          Thread.sleep(sleepTime)
+        }
+      }
+    }
+    // setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.start()
+    logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+    t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // simulated with an allocate request with no nodes requested ...
+    yarnAllocator.allocateResources()
+  }
+
+  def finishApplicationMaster(status: FinalApplicationStatus) {
+    logInfo("finish ApplicationMaster with " + status)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+  }
+
+}
+
+
+object WorkerLauncher {
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new WorkerLauncher(args).run()
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9f5523c4b97a8811c14a19f9e34c6cd1e32028e7
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -0,0 +1,209 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.URI
+import java.nio.ByteBuffer
+import java.security.PrivilegedExceptionAction
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.client.api.NMClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
+
+import org.apache.spark.Logging
+
+
+class WorkerRunnable(
+    container: Container,
+    conf: Configuration,
+    masterAddress: String,
+    slaveId: String,
+    hostname: String,
+    workerMemory: Int,
+    workerCores: Int) 
+  extends Runnable with Logging {
+
+  var rpc: YarnRPC = YarnRPC.create(conf)
+  var nmClient: NMClient = _
+  val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  def run = {
+    logInfo("Starting Worker Container")
+    nmClient = NMClient.createNMClient()
+    nmClient.init(yarnConf)
+    nmClient.start()
+    startContainer
+  }
+
+  def startContainer = {
+    logInfo("Setting up ContainerLaunchContext")
+
+    val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+      .asInstanceOf[ContainerLaunchContext]
+
+    val localResources = prepareLocalResources
+    ctx.setLocalResources(localResources)
+
+    val env = prepareEnvironment
+    ctx.setEnvironment(env)
+
+    // Extra options for the JVM
+    var JAVA_OPTS = ""
+    // Set the JVM memory
+    val workerMemoryString = workerMemory + "m"
+    JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
+    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+      JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+    }
+
+    JAVA_OPTS += " -Djava.io.tmpdir=" + 
+      new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
+
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out.
+    // The context is, default gc for server class machines end up using all cores to do gc - hence
+    // if there are multiple containers in same node, spark gc effects all other containers
+    // performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
+/*
+    else {
+      // If no java_opts specified, default to using -XX:+CMSIncrementalMode
+      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+      // want to mess with it.
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tennent machines
+      // The options are based on
+      // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
+      JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+      JAVA_OPTS += " -XX:+CMSIncrementalMode "
+      JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+    }
+*/
+
+    val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+    val dob = new DataOutputBuffer()
+    credentials.writeTokenStorageToStream(dob)
+    ctx.setTokens(ByteBuffer.wrap(dob.getData()))
+
+    var javaCommand = "java"
+    val javaHome = System.getenv("JAVA_HOME")
+    if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+      javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+    }
+
+    val commands = List[String](javaCommand +
+      " -server " +
+      // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+      // an inconsistent state.
+      // 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.CoarseGrainedExecutorBackend " +
+      masterAddress + " " +
+      slaveId + " " +
+      hostname + " " +
+      workerCores +
+      " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+      " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+    logInfo("Setting up worker with commands: " + commands)
+    ctx.setCommands(commands)
+
+    // Send the start request to the ContainerManager
+    nmClient.startContainer(container, ctx)
+  }
+
+  private def setupDistributedCache(
+      file: String,
+      rtype: LocalResourceType,
+      localResources: HashMap[String, LocalResource],
+      timestamp: String,
+      size: String, 
+      vis: String) = {
+    val uri = new URI(file)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(rtype)
+    amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+    amJarRsrc.setTimestamp(timestamp.toLong)
+    amJarRsrc.setSize(size.toLong)
+    localResources(uri.getFragment()) = amJarRsrc
+  }
+
+  def prepareLocalResources: HashMap[String, LocalResource] = {
+    logInfo("Preparing Local resources")
+    val localResources = HashMap[String, LocalResource]()
+
+    if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+      val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
+      for( i <- 0 to distFiles.length - 1) {
+        setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+          fileSizes(i), visibilities(i))
+      }
+    }
+
+    if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+      val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
+      for( i <- 0 to distArchives.length - 1) {
+        setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, 
+          timeStamps(i), fileSizes(i), visibilities(i))
+      }
+    }
+
+    logInfo("Prepared Local resources " + localResources)
+    localResources
+  }
+
+  def prepareEnvironment: HashMap[String, String] = {
+    val env = new HashMap[String, String]()
+
+    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"))
+
+    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+    env
+  }
+
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dba0f7640e67cc88bbe432ec12e218c6103244ff
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -0,0 +1,687 @@
+/*
+ * 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.deploy.yarn
+
+import java.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import org.apache.spark.Logging
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
+import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.util.{RackResolver, Records}
+
+
+object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+  type AllocationType = Value
+  val HOST, RACK, ANY = Value
+}
+
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
+// Note that right now, we assume all node asks as uniform in terms of capabilities and priority
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+    val conf: Configuration,
+    val amClient: AMRMClient[ContainerRequest],
+    val appAttemptId: ApplicationAttemptId,
+    val maxWorkers: Int,
+    val workerMemory: Int,
+    val workerCores: Int,
+    val preferredHostToCount: Map[String, Int], 
+    val preferredRackToCount: Map[String, Int])
+  extends Logging {
+  // These three are locked on allocatedHostToContainersMap. Complementary data structures
+  // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
+  // allocatedContainerToHostMap: container to host mapping.
+  private val allocatedHostToContainersMap =
+    new HashMap[String, collection.mutable.Set[ContainerId]]()
+
+  private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
+
+  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+  // allocated node)
+  // As with the two data structures above, tightly coupled with them, and to be locked on
+  // allocatedHostToContainersMap
+  private val allocatedRackCount = new HashMap[String, Int]()
+
+  // Containers which have been released.
+  private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
+  // Containers to be released in next request to RM
+  private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
+
+  // Number of container requests that have been sent to, but not yet allocated by the
+  // ApplicationMaster.
+  private val numPendingAllocate = new AtomicInteger()
+  private val numWorkersRunning = new AtomicInteger()
+  // Used to generate a unique id per worker
+  private val workerIdCounter = new AtomicInteger()
+  private val lastResponseId = new AtomicInteger()
+  private val numWorkersFailed = new AtomicInteger()
+
+  def getNumPendingAllocate: Int = numPendingAllocate.intValue
+
+  def getNumWorkersRunning: Int = numWorkersRunning.intValue
+
+  def getNumWorkersFailed: Int = numWorkersFailed.intValue
+
+  def isResourceConstraintSatisfied(container: Container): Boolean = {
+    container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+  }
+
+  def releaseContainer(container: Container) {
+    val containerId = container.getId
+    pendingReleaseContainers.put(containerId, true)
+    amClient.releaseAssignedContainer(containerId)
+  }
+
+  def allocateResources() {
+    // We have already set the container request. Poll the ResourceManager for a response.
+    // This doubles as a heartbeat if there are no pending container requests.
+    val progressIndicator = 0.1f
+    val allocateResponse = amClient.allocate(progressIndicator)
+
+    val allocatedContainers = allocateResponse.getAllocatedContainers()
+    if (allocatedContainers.size > 0) {
+      var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size)
+
+      if (numPendingAllocateNow < 0) {
+        numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow)
+      }
+
+      logDebug("""
+        Allocated containers: %d
+        Current worker count: %d
+        Containers released: %s
+        Containers to-be-released: %s
+        Cluster resources: %s
+        """.format(
+          allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers,
+          allocateResponse.getAvailableResources))
+
+      val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+      for (container <- allocatedContainers) {
+        if (isResourceConstraintSatisfied(container)) {
+          // Add the accepted `container` to the host's list of already accepted,
+          // allocated containers
+          val host = container.getNodeId.getHost
+          val containersForHost = hostToContainers.getOrElseUpdate(host,
+            new ArrayBuffer[Container]())
+          containersForHost += container
+        } else {
+          // Release container, since it doesn't satisfy resource constraints.
+          releaseContainer(container)
+        }
+      }
+
+       // Find the appropriate containers to use.
+      // TODO: Cleanup this group-by...
+      val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+      val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+      val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+      for (candidateHost <- hostToContainers.keySet) {
+        val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0)
+        val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost)
+
+        val remainingContainersOpt = hostToContainers.get(candidateHost)
+        assert(remainingContainersOpt.isDefined)
+        var remainingContainers = remainingContainersOpt.get
+
+        if (requiredHostCount >= remainingContainers.size) {
+          // Since we have <= required containers, add all remaining containers to
+          // `dataLocalContainers`.
+          dataLocalContainers.put(candidateHost, remainingContainers)
+          // There are no more free containers remaining.
+          remainingContainers = null
+        } else if (requiredHostCount > 0) {
+          // Container list has more containers than we need for data locality.
+          // Split the list into two: one based on the data local container count,
+          // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+          // containers.
+          val (dataLocal, remaining) = remainingContainers.splitAt(
+            remainingContainers.size - requiredHostCount)
+          dataLocalContainers.put(candidateHost, dataLocal)
+
+          // Invariant: remainingContainers == remaining
+
+          // YARN has a nasty habit of allocating a ton of containers on a host - discourage this.
+          // Add each container in `remaining` to list of containers to release. If we have an
+          // insufficient number of containers, then the next allocation cycle will reallocate
+          // (but won't treat it as data local).
+          // TODO(harvey): Rephrase this comment some more.
+          for (container <- remaining) releaseContainer(container)
+          remainingContainers = null
+        }
+
+        // For rack local containers
+        if (remainingContainers != null) {
+          val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+          if (rack != null) {
+            val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0)
+            val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) -
+              rackLocalContainers.getOrElse(rack, List()).size
+
+            if (requiredRackCount >= remainingContainers.size) {
+              // Add all remaining containers to to `dataLocalContainers`.
+              dataLocalContainers.put(rack, remainingContainers)
+              remainingContainers = null
+            } else if (requiredRackCount > 0) {
+              // Container list has more containers that we need for data locality.
+              // Split the list into two: one based on the data local container count,
+              // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+              // containers.
+              val (rackLocal, remaining) = remainingContainers.splitAt(
+                remainingContainers.size - requiredRackCount)
+              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+                new ArrayBuffer[Container]())
+
+              existingRackLocal ++= rackLocal
+
+              remainingContainers = remaining
+            }
+          }
+        }
+
+        if (remainingContainers != null) {
+          // Not all containers have been consumed - add them to the list of off-rack containers.
+          offRackContainers.put(candidateHost, remainingContainers)
+        }
+      }
+
+      // Now that we have split the containers into various groups, go through them in order:
+      // first host-local, then rack-local, and finally off-rack.
+      // Note that the list we create below tries to ensure that not all containers end up within
+      // a host if there is a sufficiently large number of hosts/containers.
+      val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+
+      // Run each of the allocated containers.
+      for (container <- allocatedContainersToProcess) {
+        val numWorkersRunningNow = numWorkersRunning.incrementAndGet()
+        val workerHostname = container.getNodeId.getHost
+        val containerId = container.getId
+
+        val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+        assert(container.getResource.getMemory >= workerMemoryOverhead)
+
+        if (numWorkersRunningNow > maxWorkers) {
+          logInfo("""Ignoring container %s at host %s, since we already have the required number of
+            containers for it.""".format(containerId, workerHostname))
+          releaseContainer(container)
+          numWorkersRunning.decrementAndGet()
+        } else {
+          val workerId = workerIdCounter.incrementAndGet().toString
+          val driverUrl = "akka://spark@%s:%s/user/%s".format(
+            System.getProperty("spark.driver.host"),
+            System.getProperty("spark.driver.port"),
+            CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+          logInfo("Launching container %s for on host %s".format(containerId, workerHostname))
+
+          // To be safe, remove the container from `pendingReleaseContainers`.
+          pendingReleaseContainers.remove(containerId)
+
+          val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
+          allocatedHostToContainersMap.synchronized {
+            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+              new HashSet[ContainerId]())
+
+            containerSet += containerId
+            allocatedContainerToHostMap.put(containerId, workerHostname)
+
+            if (rack != null) {
+              allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            }
+          }
+          logInfo("Launching WorkerRunnable. driverUrl: %s,  workerHostname: %s".format(driverUrl, workerHostname))
+          val workerRunnable = new WorkerRunnable(
+            container,
+            conf,
+            driverUrl,
+            workerId,
+            workerHostname,
+            workerMemory,
+            workerCores)
+          new Thread(workerRunnable).start()
+        }
+      }
+      logDebug("""
+        Finished allocating %s containers (from %s originally).
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          allocatedContainersToProcess,
+          allocatedContainers,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
+    }
+
+    val completedContainers = allocateResponse.getCompletedContainersStatuses()
+    if (completedContainers.size > 0) {
+      logDebug("Completed %d containers".format(completedContainers.size))
+
+      for (completedContainer <- completedContainers) {
+        val containerId = completedContainer.getContainerId
+
+        if (pendingReleaseContainers.containsKey(containerId)) {
+          // YarnAllocationHandler already marked the container for release, so remove it from
+          // `pendingReleaseContainers`.
+          pendingReleaseContainers.remove(containerId)
+        } else {
+          // Decrement the number of workers running. The next iteration of the ApplicationMaster's
+          // reporting thread will take care of allocating.
+          numWorkersRunning.decrementAndGet()
+          logInfo("Completed container %s (state: %s, exit status: %s)".format(
+            containerId,
+            completedContainer.getState,
+            completedContainer.getExitStatus()))
+          // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
+          // there are some exit status' we shouldn't necessarily count against us, but for
+          // now I think its ok as none of the containers are expected to exit
+          if (completedContainer.getExitStatus() != 0) {
+            logInfo("Container marked as failed: " + containerId)
+            numWorkersFailed.incrementAndGet()
+          }
+        }
+
+        allocatedHostToContainersMap.synchronized {
+          if (allocatedContainerToHostMap.containsKey(containerId)) {
+            val hostOpt = allocatedContainerToHostMap.get(containerId)
+            assert(hostOpt.isDefined)
+            val host = hostOpt.get
+
+            val containerSetOpt = allocatedHostToContainersMap.get(host)
+            assert(containerSetOpt.isDefined)
+            val containerSet = containerSetOpt.get
+
+            containerSet.remove(containerId)
+            if (containerSet.isEmpty) {
+              allocatedHostToContainersMap.remove(host)
+            } else {
+              allocatedHostToContainersMap.update(host, containerSet)
+            }
+
+            allocatedContainerToHostMap.remove(containerId)
+
+            // TODO: Move this part outside the synchronized block?
+            val rack = YarnAllocationHandler.lookupRack(conf, host)
+            if (rack != null) {
+              val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
+              if (rackCount > 0) {
+                allocatedRackCount.put(rack, rackCount)
+              } else {
+                allocatedRackCount.remove(rack)
+              }
+            }
+          }
+        }
+      }
+      logDebug("""
+        Finished processing %d completed containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          completedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
+    }
+  }
+
+  def createRackResourceRequests(
+      hostContainers: ArrayBuffer[ContainerRequest]
+    ): ArrayBuffer[ContainerRequest] = {
+    // Generate modified racks and new set of hosts under it before issuing requests.
+    val rackToCounts = new HashMap[String, Int]()
+
+    for (container <- hostContainers) {
+      val candidateHost = container.getNodes.last
+      assert(YarnAllocationHandler.ANY_HOST != candidateHost)
+
+      val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+      if (rack != null) {
+        var count = rackToCounts.getOrElse(rack, 0)
+        count += 1
+        rackToCounts.put(rack, count)
+      }
+    }
+
+    val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size)
+    for ((rack, count) <- rackToCounts) {
+      requestedContainers ++= createResourceRequests(
+        AllocationType.RACK,
+        rack,
+        count,
+        YarnAllocationHandler.PRIORITY)
+    }
+
+    requestedContainers
+  }
+
+  def allocatedContainersOnHost(host: String): Int = {
+    var retval = 0
+    allocatedHostToContainersMap.synchronized {
+      retval = allocatedHostToContainersMap.getOrElse(host, Set()).size
+    }
+    retval
+  }
+
+  def allocatedContainersOnRack(rack: String): Int = {
+    var retval = 0
+    allocatedHostToContainersMap.synchronized {
+      retval = allocatedRackCount.getOrElse(rack, 0)
+    }
+    retval
+  }
+
+  def addResourceRequests(numWorkers: Int) {
+    val containerRequests: List[ContainerRequest] =
+      if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
+        logDebug("numWorkers: " + numWorkers + ", host preferences: " +
+          preferredHostToCount.isEmpty)
+        createResourceRequests(
+          AllocationType.ANY,
+          resource = null,
+          numWorkers,
+          YarnAllocationHandler.PRIORITY).toList
+      } else {
+        // Request for all hosts in preferred nodes and for numWorkers - 
+        // candidates.size, request by default allocation policy.
+        val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size)
+        for ((candidateHost, candidateCount) <- preferredHostToCount) {
+          val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
+
+          if (requiredCount > 0) {
+            hostContainerRequests ++= createResourceRequests(
+              AllocationType.HOST,
+              candidateHost,
+              requiredCount,
+              YarnAllocationHandler.PRIORITY)
+          }
+        }
+        val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests(
+          hostContainerRequests).toList
+
+        val anyContainerRequests = createResourceRequests(
+          AllocationType.ANY,
+          resource = null,
+          numWorkers,
+          YarnAllocationHandler.PRIORITY)
+
+        val containerRequestBuffer = new ArrayBuffer[ContainerRequest](
+          hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size)
+
+        containerRequestBuffer ++= hostContainerRequests
+        containerRequestBuffer ++= rackContainerRequests
+        containerRequestBuffer ++= anyContainerRequests
+        containerRequestBuffer.toList
+      }
+
+    for (request <- containerRequests) {
+      amClient.addContainerRequest(request)
+    }
+
+    if (numWorkers > 0) {
+      numPendingAllocate.addAndGet(numWorkers)
+      logInfo("Will Allocate %d worker containers, each with %d memory".format(
+        numWorkers,
+        (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)))
+    } else {
+      logDebug("Empty allocation request ...")
+    }
+
+    for (request <- containerRequests) {
+      val nodes = request.getNodes
+      var hostStr = if (nodes == null || nodes.isEmpty) {
+        "Any"
+      } else {
+        nodes.last
+      }
+      logInfo("Container request (host: %s, priority: %s, capability: %s".format(
+        hostStr,
+        request.getPriority().getPriority,
+        request.getCapability))
+    }
+  }
+
+  private def createResourceRequests(
+      requestType: AllocationType.AllocationType,
+      resource: String,
+      numWorkers: Int,
+      priority: Int
+    ): ArrayBuffer[ContainerRequest] = {
+
+    // If hostname is specified, then we need at least two requests - node local and rack local.
+    // There must be a third request, which is ANY. That will be specially handled.
+    requestType match {
+      case AllocationType.HOST => {
+        assert(YarnAllocationHandler.ANY_HOST != resource)
+        val hostname = resource
+        val nodeLocal = constructContainerRequests(
+          Array(hostname),
+          racks = null,
+          numWorkers,
+          priority)
+
+        // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler.
+        YarnAllocationHandler.populateRackInfo(conf, hostname)
+        nodeLocal
+      }
+      case AllocationType.RACK => {
+        val rack = resource
+        constructContainerRequests(hosts = null, Array(rack), numWorkers, priority)
+      }
+      case AllocationType.ANY => constructContainerRequests(
+        hosts = null, racks = null, numWorkers, priority)
+      case _ => throw new IllegalArgumentException(
+        "Unexpected/unsupported request type: " + requestType)
+    }
+  }
+
+  private def constructContainerRequests(
+      hosts: Array[String],
+      racks: Array[String],
+      numWorkers: Int,
+      priority: Int
+    ): ArrayBuffer[ContainerRequest] = {
+
+    val memoryResource = Records.newRecord(classOf[Resource])
+    memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+
+    val prioritySetting = Records.newRecord(classOf[Priority])
+    prioritySetting.setPriority(priority)
+
+    val requests = new ArrayBuffer[ContainerRequest]()
+    for (i <- 0 until numWorkers) {
+      requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting)
+    }
+    requests
+  }
+}
+
+object YarnAllocationHandler {
+
+  val ANY_HOST = "*"
+  // All requests are issued with same priority : we do not (yet) have any distinction between 
+  // request types (like map/reduce in hadoop for example)
+  val PRIORITY = 1
+
+  // Additional memory overhead - in mb.
+  val MEMORY_OVERHEAD = 384
+
+  // Host to rack map - saved from allocation requests. We are expecting this not to change.
+  // Note that it is possible for this to change : and ResurceManager will indicate that to us via
+  // update response to allocate. But we are punting on handling that for now.
+  private val hostToRack = new ConcurrentHashMap[String, String]()
+  private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
+
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      args: ApplicationMasterArguments
+    ): YarnAllocationHandler = {
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      Map[String, Int](),
+      Map[String, Int]())
+  }
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      args: ApplicationMasterArguments,
+      map: collection.Map[String,
+      collection.Set[SplitInfo]]
+    ): YarnAllocationHandler = {
+    val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map)
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      hostToSplitCount,
+      rackToSplitCount)
+  }
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      maxWorkers: Int,
+      workerMemory: Int,
+      workerCores: Int,
+      map: collection.Map[String, collection.Set[SplitInfo]]
+    ): YarnAllocationHandler = {
+    val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      maxWorkers,
+      workerMemory,
+      workerCores,
+      hostToCount,
+      rackToCount)
+  }
+
+  // A simple method to copy the split info map.
+  private def generateNodeToWeight(
+      conf: Configuration,
+      input: collection.Map[String, collection.Set[SplitInfo]]
+    ): (Map[String, Int], Map[String, Int]) = {
+
+    if (input == null) {
+      return (Map[String, Int](), Map[String, Int]())
+    }
+
+    val hostToCount = new HashMap[String, Int]
+    val rackToCount = new HashMap[String, Int]
+
+    for ((host, splits) <- input) {
+      val hostCount = hostToCount.getOrElse(host, 0)
+      hostToCount.put(host, hostCount + splits.size)
+
+      val rack = lookupRack(conf, host)
+      if (rack != null){
+        val rackCount = rackToCount.getOrElse(host, 0)
+        rackToCount.put(host, rackCount + splits.size)
+      }
+    }
+
+    (hostToCount.toMap, rackToCount.toMap)
+  }
+
+  def lookupRack(conf: Configuration, host: String): String = {
+    if (!hostToRack.contains(host)) {
+      populateRackInfo(conf, host)
+    }
+    hostToRack.get(host)
+  }
+
+  def fetchCachedHostsForRack(rack: String): Option[Set[String]] = {
+    Option(rackToHostSet.get(rack)).map { set =>
+      val convertedSet: collection.mutable.Set[String] = set
+      // TODO: Better way to get a Set[String] from JSet.
+      convertedSet.toSet
+    }
+  }
+
+  def populateRackInfo(conf: Configuration, hostname: String) {
+    Utils.checkHost(hostname)
+
+    if (!hostToRack.containsKey(hostname)) {
+      // If there are repeated failures to resolve, all to an ignore list.
+      val rackInfo = RackResolver.resolve(conf, hostname)
+      if (rackInfo != null && rackInfo.getNetworkLocation != null) {
+        val rack = rackInfo.getNetworkLocation
+        hostToRack.put(hostname, rack)
+        if (! rackToHostSet.containsKey(rack)) {
+          rackToHostSet.putIfAbsent(rack,
+            Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+        }
+        rackToHostSet.get(rack).add(hostname)
+
+        // TODO(harvey): Figure out what this comment means...
+        // Since RackResolver caches, we are disabling this for now ...
+      } /* else {
+        // right ? Else we will keep calling rack resolver in case we cant resolve rack info ...
+        hostToRack.put(hostname, null)
+      } */
+    }
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2ba2366ead17113c764663ed934a5221f0eeb0ee
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.deploy.yarn
+
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.conf.Configuration
+
+/**
+ * Contains util methods to interact with Hadoop from spark.
+ */
+class YarnSparkHadoopUtil extends SparkHadoopUtil {
+
+  // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true.
+  override def isYarnMode(): Boolean = { true }
+
+  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
+  // Always create a new config, dont reuse yarnConf.
+  override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration())
+
+  // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
+  override def addCredentials(conf: JobConf) {
+    val jobCreds = conf.getCredentials()
+    jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..63a0449e5a0730085554d2b8ae86067135fa8dba
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.spark._
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.deploy.yarn.YarnAllocationHandler
+import org.apache.spark.util.Utils
+
+/**
+ *
+ * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
+ */
+private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+
+  def this(sc: SparkContext) = this(sc, new Configuration())
+
+  // By default, rack is unknown
+  override def getRackForHost(hostPort: String): Option[String] = {
+    val host = Utils.parseHostPort(hostPort)._1
+    val retval = YarnAllocationHandler.lookupRack(conf, host)
+    if (retval != null) Some(retval) else None
+  }
+
+  override def postStartHook() {
+
+    // The yarn application is running, but the worker might not yet ready
+    // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
+    Thread.sleep(2000L)
+    logInfo("YarnClientClusterScheduler.postStartHook done")
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b206780c7806e15c84944db05876f89c8f848040
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.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.scheduler.cluster
+
+import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
+import org.apache.spark.{SparkException, Logging, SparkContext}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments}
+
+private[spark] class YarnClientSchedulerBackend(
+    scheduler: ClusterScheduler,
+    sc: SparkContext)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+  with Logging {
+
+  var client: Client = null
+  var appId: ApplicationId = null
+
+  override def start() {
+    super.start()
+
+    val defalutWorkerCores = "2"
+    val defalutWorkerMemory = "512m"
+    val defaultWorkerNumber = "1"
+
+    val userJar = System.getenv("SPARK_YARN_APP_JAR")
+    var workerCores = System.getenv("SPARK_WORKER_CORES")
+    var workerMemory = System.getenv("SPARK_WORKER_MEMORY")
+    var workerNumber = System.getenv("SPARK_WORKER_INSTANCES")
+
+    if (userJar == null)
+      throw new SparkException("env SPARK_YARN_APP_JAR is not set")
+
+    if (workerCores == null)
+      workerCores = defalutWorkerCores
+    if (workerMemory == null)
+      workerMemory = defalutWorkerMemory
+    if (workerNumber == null)
+      workerNumber = defaultWorkerNumber
+
+    val driverHost = System.getProperty("spark.driver.host")
+    val driverPort = System.getProperty("spark.driver.port")
+    val hostport = driverHost + ":" + driverPort
+
+    val argsArray = Array[String](
+      "--class", "notused",
+      "--jar", userJar,
+      "--args", hostport,
+      "--worker-memory", workerMemory,
+      "--worker-cores", workerCores,
+      "--num-workers", workerNumber,
+      "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
+    )
+
+    val args = new ClientArguments(argsArray)
+    client = new Client(args)
+    appId = client.runApp()
+    waitForApp()
+  }
+
+  def waitForApp() {
+
+    // TODO : need a better way to find out whether the workers are ready or not
+    // maybe by resource usage report?
+    while(true) {
+      val report = client.getApplicationReport(appId)
+
+      logInfo("Application report from ASM: \n" +
+        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+        "\t appStartTime: " + report.getStartTime() + "\n" +
+        "\t yarnAppState: " + report.getYarnApplicationState() + "\n"
+      )
+
+      // Ready to go, or already gone.
+      val state = report.getYarnApplicationState()
+      if (state == YarnApplicationState.RUNNING) {
+        return
+      } else if (state == YarnApplicationState.FINISHED ||
+        state == YarnApplicationState.FAILED ||
+        state == YarnApplicationState.KILLED) {
+        throw new SparkException("Yarn application already ended," +
+          "might be killed or not able to launch application master.")
+      }
+
+      Thread.sleep(1000)
+    }
+  }
+
+  override def stop() {
+    super.stop()
+    client.stop()
+    logInfo("Stoped")
+  }
+
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..29b3f22e13697b38bc501e2f914d8fc0a202d722
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.spark._
+import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+import org.apache.spark.util.Utils
+import org.apache.hadoop.conf.Configuration
+
+/**
+ *
+ * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done
+ */
+private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+
+  logInfo("Created YarnClusterScheduler")
+
+  def this(sc: SparkContext) = this(sc, new Configuration())
+
+  // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate
+  // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?)
+  // Subsequent creations are ignored - since nodes are already allocated by then.
+
+
+  // By default, rack is unknown
+  override def getRackForHost(hostPort: String): Option[String] = {
+    val host = Utils.parseHostPort(hostPort)._1
+    val retval = YarnAllocationHandler.lookupRack(conf, host)
+    if (retval != null) Some(retval) else None
+  }
+
+  override def postStartHook() {
+    val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc)
+    if (sparkContextInitialized){
+      // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
+      Thread.sleep(3000L)
+    }
+    logInfo("YarnClusterScheduler.postStartHook done")
+  }
+}
diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5
--- /dev/null
+++ b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -0,0 +1,220 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.URI
+
+import org.scalatest.FunSuite
+import org.scalatest.mock.MockitoSugar
+import org.mockito.Mockito.when
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+
+class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar {
+
+  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
+    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): 
+        LocalResourceVisibility = {
+      return LocalResourceVisibility.PRIVATE
+    }
+  }
+  
+  test("test getFileStatus empty") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath() === null)
+  }
+
+  test("test getFileStatus cached") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath().toString() === "/tmp/testing")
+  }
+
+  test("test addResource") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", 
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 0)
+    assert(resource.getSize() === 0)
+    assert(resource.getType() === LocalResourceType.FILE)
+
+    val env = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env)
+    assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0")
+    assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0")
+    assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
+
+    //add another one and verify both there and order correct
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing2"))
+    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
+    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
+    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", 
+      statCache, false)
+    val resource2 = localResources("link2")
+    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
+    assert(resource2.getTimestamp() === 10)
+    assert(resource2.getSize() === 20)
+    assert(resource2.getType() === LocalResourceType.FILE)
+
+    val env2 = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env2)
+    val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+    val files = env2("SPARK_YARN_CACHE_FILES").split(',') 
+    val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+    val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',')
+    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(timestamps(0)  === "0")
+    assert(sizes(0)  === "0")
+    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
+
+    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
+    assert(timestamps(1)  === "10")
+    assert(sizes(1)  === "20")
+    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
+  }
+
+  test("test addResource link null") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    intercept[Exception] {
+      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, 
+        statCache, false)
+    }
+    assert(localResources.get("link") === None)
+    assert(localResources.size === 0)
+  }
+
+  test("test addResource appmaster only") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
+      statCache, true)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val env = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
+  }
+
+  test("test addResource archive") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val env = new HashMap[String, String]()
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
+
+    distMgr.setDistFilesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
+  }
+
+
+}
diff --git a/pom.xml b/pom.xml
index 9278856e5a920ef89bd68207ef24836ee788183f..6906ad2d3246696b3b47efbca19b51d24b2ca66b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache</groupId>
     <artifactId>apache</artifactId>
-    <version>11</version>
+    <version>13</version>
   </parent>
   <groupId>org.apache.spark</groupId>
   <artifactId>spark-parent</artifactId>
@@ -61,6 +61,29 @@
     <maven>3.0.0</maven>
   </prerequisites>
 
+  <mailingLists>
+    <mailingList>
+      <name>Dev Mailing List</name>
+      <post>dev@spark.incubator.apache.org</post>
+      <subscribe>dev-subscribe@spark.incubator.apache.org</subscribe>
+      <unsubscribe>dev-unsubscribe@spark.incubator.apache.org</unsubscribe>
+    </mailingList>
+
+    <mailingList>
+      <name>User Mailing List</name>
+      <post>user@spark.incubator.apache.org</post>
+      <subscribe>user-subscribe@spark.incubator.apache.org</subscribe>
+      <unsubscribe>user-unsubscribe@spark.incubator.apache.org</unsubscribe>
+    </mailingList>
+
+    <mailingList>
+      <name>Commits Mailing List</name>
+      <post>commits@spark.incubator.apache.org</post>
+      <subscribe>commits-subscribe@spark.incubator.apache.org</subscribe>
+      <unsubscribe>commits-unsubscribe@spark.incubator.apache.org</unsubscribe>
+    </mailingList>
+  </mailingLists>
+
   <modules>
     <module>core</module>
     <module>bagel</module>
@@ -78,10 +101,11 @@
 
     <java.version>1.6</java.version>
 
-    <scala-short.version>2.10</scala-short.version>
     <scala.version>2.10.3</scala.version>
     <mesos.version>0.13.0</mesos.version>
-    <akka.version>2.2.1</akka.version>
+    <akka.version>2.2.3</akka.version>
+    <akka.group>com.typesafe.akka</akka.group>
+    <protobuf.version>2.4.1</protobuf.version>
     <slf4j.version>1.7.2</slf4j.version>
     <log4j.version>1.2.17</log4j.version>
     <hadoop.version>1.0.4</hadoop.version>
@@ -93,10 +117,10 @@
   </properties>
 
   <repositories>
-    <repository>
-      <id>typesafe-repo</id>
-      <name>Typesafe Repository</name>
-      <url>http://repo.typesafe.com/typesafe/releases/</url>
+    <repository> 
+      <id>maven-repo</id> <!-- This should be at top, it makes maven try the central repo first and then others and hence faster dep resolution -->
+      <name>Maven Repository</name>
+      <url>http://repo.maven.apache.org/maven2/</url>
       <releases>
         <enabled>true</enabled>
       </releases>
@@ -116,9 +140,9 @@
       </snapshots>
     </repository>
     <repository>
-      <id>akka-repo</id>
-      <name>Akka Repository</name>
-      <url>http://repo.akka.io/releases/</url>
+      <id>mqtt-repo</id>
+      <name>MQTT Repository</name>
+      <url>https://repo.eclipse.org/content/repositories/paho-releases/</url>
       <releases>
         <enabled>true</enabled>
       </releases>
@@ -127,41 +151,6 @@
       </snapshots>
     </repository>
   </repositories>
-  <pluginRepositories>
-    <pluginRepository>
-      <id>oss-sonatype-releases</id>
-      <name>OSS Sonatype</name>
-      <url>https://oss.sonatype.org/content/repositories/releases</url>
-      <releases>
-        <enabled>true</enabled>
-      </releases>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </pluginRepository>
-    <pluginRepository>
-      <id>oss-sonatype-snapshots</id>
-      <name>OSS Sonatype</name>
-      <url>https://oss.sonatype.org/content/repositories/snapshots</url>
-      <releases>
-        <enabled>false</enabled>
-      </releases>
-      <snapshots>
-        <enabled>true</enabled>
-      </snapshots>
-    </pluginRepository>
-    <pluginRepository>
-      <id>oss-sonatype</id>
-      <name>OSS Sonatype</name>
-      <url>https://oss.sonatype.org/content/groups/public</url>
-      <releases>
-        <enabled>true</enabled>
-      </releases>
-      <snapshots>
-        <enabled>true</enabled>
-      </snapshots>
-    </pluginRepository>
-  </pluginRepositories>
 
   <dependencyManagement>
     <dependencies>
@@ -213,11 +202,11 @@
       <dependency>
         <groupId>com.google.protobuf</groupId>
         <artifactId>protobuf-java</artifactId>
-        <version>2.4.1</version>
+        <version>${protobuf.version}</version>
       </dependency>
       <dependency>
         <groupId>com.twitter</groupId>
-        <artifactId>chill_${scala-short.version}</artifactId>
+        <artifactId>chill_2.10</artifactId>
         <version>0.3.1</version>
       </dependency>
       <dependency>
@@ -226,14 +215,48 @@
         <version>0.3.1</version>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
-        <artifactId>akka-remote_${scala-short.version}</artifactId>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-actor_2.10</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_${scala-short.version}</artifactId>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-remote_2.10</artifactId>
         <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-slf4j_2.10</artifactId>
+        <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-zeromq_2.10</artifactId>
+        <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>it.unimi.dsi</groupId>
@@ -245,11 +268,6 @@
         <artifactId>colt</artifactId>
         <version>1.2.0</version>
       </dependency>
-      <dependency>
-        <groupId>com.github.scala-incubator.io</groupId>
-        <artifactId>scala-io-file_${scala-short.version}</artifactId>
-        <version>0.4.1</version>
-      </dependency>
       <dependency>
         <groupId>org.apache.mesos</groupId>
         <artifactId>mesos</artifactId>
@@ -268,7 +286,7 @@
       </dependency>
       <dependency>
         <groupId>net.liftweb</groupId>
-        <artifactId>lift-json_${scala-short.version}</artifactId>
+        <artifactId>lift-json_2.10</artifactId>
         <version>2.5.1</version>
         <exclusions>
           <exclusion>
@@ -297,6 +315,11 @@
         <artifactId>metrics-ganglia</artifactId>
         <version>3.0.0</version>
       </dependency>
+      <dependency>
+        <groupId>com.codahale.metrics</groupId>
+        <artifactId>metrics-graphite</artifactId>
+        <version>3.0.0</version>
+      </dependency>
       <dependency>
         <groupId>org.scala-lang</groupId>
         <artifactId>scala-compiler</artifactId>
@@ -319,7 +342,7 @@
       </dependency>
       <dependency>
         <groupId>org.scalatest</groupId>
-        <artifactId>scalatest_${scala-short.version}</artifactId>
+        <artifactId>scalatest_2.10</artifactId>
         <version>1.9.1</version>
         <scope>test</scope>
       </dependency>
@@ -334,9 +357,15 @@
         <version>3.1</version>
         <scope>test</scope>
       </dependency>
+     <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-all</artifactId>
+        <version>1.8.5</version>
+        <scope>test</scope>
+      </dependency>
       <dependency>
         <groupId>org.scalacheck</groupId>
-        <artifactId>scalacheck_${scala-short.version}</artifactId>
+        <artifactId>scalacheck_2.10</artifactId>
         <version>1.10.0</version>
         <scope>test</scope>
       </dependency>
@@ -346,6 +375,17 @@
         <version>0.9</version>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.apache.zookeeper</groupId>
+        <artifactId>zookeeper</artifactId>
+        <version>3.4.5</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-client</artifactId>
@@ -361,19 +401,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>
@@ -397,19 +429,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>
@@ -428,19 +452,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>
@@ -459,19 +475,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>
-          </exclusion>
-          <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
+            <artifactId>*</artifactId>
           </exclusion>
           <exclusion>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
+            <groupId>org.sonatype.sisu.inject</groupId>
+            <artifactId>*</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -729,6 +737,41 @@
         </dependencies>
       </dependencyManagement>
     </profile>
+
+    <profile>
+      <id>new-yarn</id>
+      <properties>
+        <akka.group>org.spark-project</akka.group>
+        <akka.version>2.0.5-protobuf-2.5-java-1.5</akka.version>
+        <hadoop.major.version>2</hadoop.major.version>
+        <hadoop.version>2.2.0</hadoop.version>
+        <protobuf.version>2.5.0</protobuf.version>
+      </properties>
+
+      <modules>
+        <module>new-yarn</module>
+      </modules>
+
+      <repositories>
+        <repository>
+          <id>maven-root</id>
+          <name>Maven root repository</name>
+          <url>http://repo1.maven.org/maven2/</url>
+          <releases>
+            <enabled>true</enabled>
+          </releases>
+          <snapshots>
+            <enabled>false</enabled>
+          </snapshots>
+        </repository>
+      </repositories>
+
+      <dependencyManagement>
+        <dependencies>
+        </dependencies>
+      </dependencyManagement>
+    </profile>
+
     <profile>
       <id>repl-bin</id>
       <activation>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 43db97f6800b950e80d9258ca7e363a9cdea365a..05d3f18ea219e93b6216050e84c4471a71a6abac 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -28,6 +28,11 @@ object SparkBuild extends Build {
   // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set
   // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN.
   val DEFAULT_HADOOP_VERSION = "1.0.4"
+
+  // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set
+  // through the SPARK_IS_NEW_HADOOP environment variable.
+  val DEFAULT_IS_NEW_HADOOP = false
+
   val DEFAULT_YARN = false
 
   // HBase version; set as appropriate.
@@ -55,27 +60,43 @@ object SparkBuild extends Build {
 
   lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
 
-  lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
-
   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")
 
   // Allows build configuration to be set through environment variables
   lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
+  lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
+    case None => {
+      val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
+      (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP)
+    }
+    case Some(v) => v.toBoolean
+  }
   lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
     case None => DEFAULT_YARN
     case Some(v) => v.toBoolean
   }
 
+  // Build against a protobuf-2.5 compatible Akka if Hadoop 2 is used.
+  lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1"
+  lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5"
+  lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka"
+
   // 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
+  lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
+  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
+  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+
+  // 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",
@@ -100,7 +121,7 @@ object SparkBuild extends Build {
     // also check the local Maven repository ~/.m2
     resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))),
 
-    // For Sonatype publishing
+   // 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/"),
 
@@ -161,6 +182,7 @@ object SparkBuild extends Build {
         "org.scalacheck"   %% "scalacheck"      % "1.10.0" % "test",
         "com.novocode"      % "junit-interface" % "0.9"    % "test",
         "org.easymock"      % "easymock"        % "3.1"    % "test",
+        "org.mockito"       % "mockito-all"     % "1.8.5"  % "test",
         "commons-io"        % "commons-io"      % "2.4"    % "test"
     ),
 
@@ -199,26 +221,28 @@ object SparkBuild extends Build {
         "log4j"                    % "log4j"            % "1.2.17",
         "org.slf4j"                % "slf4j-api"        % slf4jVersion,
         "org.slf4j"                % "slf4j-log4j12"    % slf4jVersion,
+        "commons-daemon"           % "commons-daemon"   % "1.0.10", // workaround for bug HADOOP-9407
         "com.ning"                 % "compress-lzf"     % "0.8.4",
         "org.xerial.snappy"        % "snappy-java"      % "1.0.5",
-        "commons-daemon"           % "commons-daemon"   % "1.0.10", // workaround for bug HADOOP-9407
         "org.ow2.asm"              % "asm"              % "4.0",
         "com.google.protobuf"      % "protobuf-java"    % "2.4.1",
-        "com.typesafe.akka"       %% "akka-remote"      % "2.2.3"  excludeAll(excludeNetty), 
-        "com.typesafe.akka"       %% "akka-slf4j"       % "2.2.3"  excludeAll(excludeNetty),
+        akkaGroup                 %% "akka-remote"      % "2.2.3"  excludeAll(excludeNetty),
+        akkaGroup                 %% "akka-slf4j"       % "2.2.3"  excludeAll(excludeNetty),
         "net.liftweb"             %% "lift-json"        % "2.5.1"  excludeAll(excludeNetty),
         "it.unimi.dsi"             % "fastutil"         % "6.4.4",
         "colt"                     % "colt"             % "1.2.0",
         "org.apache.mesos"         % "mesos"            % "0.13.0",
         "net.java.dev.jets3t"      % "jets3t"           % "0.7.1",
         "org.apache.derby"         % "derby"            % "10.4.2.0"                     % "test",
-        "org.apache.hadoop"        % "hadoop-client"    % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
+        "org.apache.hadoop"        % "hadoop-client"    % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
         "org.apache.avro"          % "avro"             % "1.7.4",
         "org.apache.avro"          % "avro-ipc"         % "1.7.4" excludeAll(excludeNetty),
+        "org.apache.zookeeper"     % "zookeeper"        % "3.4.5" excludeAll(excludeNetty),
         "com.codahale.metrics"     % "metrics-core"     % "3.0.0",
         "com.codahale.metrics"     % "metrics-jvm"      % "3.0.0",
         "com.codahale.metrics"     % "metrics-json"     % "3.0.0",
         "com.codahale.metrics"     % "metrics-ganglia"  % "3.0.0",
+        "com.codahale.metrics"     % "metrics-graphite" % "3.0.0",
         "com.twitter"             %% "chill"            % "0.3.1",
         "com.twitter"              % "chill-java"       % "0.3.1"
       )
@@ -257,7 +281,7 @@ object SparkBuild extends Build {
 
   def toolsSettings = sharedSettings ++ Seq(
     name := "spark-tools"
-  )
+  ) ++ assemblySettings ++ extraAssemblySettings
 
   def bagelSettings = sharedSettings ++ Seq(
     name := "spark-bagel"
@@ -272,11 +296,22 @@ object SparkBuild extends Build {
 
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
+    resolvers ++= Seq(
+      "Eclipse Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/",
+      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
+    ),
+
     libraryDependencies ++= Seq(
       "org.apache.flume"      % "flume-ng-sdk"     % "1.2.0" % "compile"  excludeAll(excludeNetty, excludeSnappy),
+      "com.sksamuel.kafka"   %% "kafka"            % "0.8.0-beta1"
+        exclude("com.sun.jdmk", "jmxtools")
+        exclude("com.sun.jmx", "jmxri")
+        exclude("net.sf.jopt-simple", "jopt-simple")
+        excludeAll(excludeNetty),
+      "org.eclipse.paho"      % "mqtt-client"      % "0.4.0",
       "com.github.sgroschupf" % "zkclient"         % "0.1"                excludeAll(excludeNetty),
       "org.twitter4j"         % "twitter4j-stream" % "3.0.3"              excludeAll(excludeNetty),
-      "com.typesafe.akka"    %%  "akka-zeromq"     % "2.2.3"              excludeAll(excludeNetty)
+      akkaGroup              %% "akka-zeromq"      % "2.2.3"              excludeAll(excludeNetty)
     )
   )
 
@@ -300,7 +335,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(
@@ -308,7 +345,8 @@ 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 "META-INF/services/org.apache.hadoop.fs.FileSystem" => MergeStrategy.concat
+      case "log4j.properties" => MergeStrategy.discard
+      case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines
       case "reference.conf" => MergeStrategy.concat
       case _ => MergeStrategy.first
     }
diff --git a/project/plugins.sbt b/project/plugins.sbt
index cfcd85082a8cc5c60b9b850710028f154b3682e5..4ba0e4280a9deb4ac9259f1af12daf33cca2c7dd 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -4,7 +4,7 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release
 
 resolvers += "Spray Repository" at "http://repo.spray.cc/"
 
-addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.1")
+addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2")
 
 addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0")
 
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 1d0d002d36623409f341688811d4d846ec6ee9fe..0b42e729f8dcc756c711584de2b2a4f071b480c5 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -32,6 +32,6 @@ target: docs/
 
 private: no
 
-exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers
+exclude: pyspark.cloudpickle pyspark.worker pyspark.join
          pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test
          pyspark.rddsampler pyspark.daemon
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index d367f91967ff5379946da1a5226c88b12bce3878..2204e9c9ca7011f10ca60c900896db13868c35ff 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):
@@ -83,8 +90,10 @@ import struct
 import SocketServer
 import threading
 from pyspark.cloudpickle import CloudPickler
-from pyspark.serializers import read_int, read_with_length, load_pickle
+from pyspark.serializers import read_int, PickleSerializer
+
 
+pickleSer = PickleSerializer()
 
 # Holds accumulators registered on the current machine, keyed by ID. This is then used to send
 # the local accumulator updates back to the driver program at the end of a task.
@@ -139,9 +148,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):
@@ -200,7 +213,7 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler):
         from pyspark.accumulators import _accumulatorRegistry
         num_updates = read_int(self.rfile)
         for _ in range(num_updates):
-            (aid, update) = load_pickle(read_with_length(self.rfile))
+            (aid, update) = pickleSer._read_with_length(self.rfile)
             _accumulatorRegistry[aid] += update
         # Write a byte in acknowledgement
         self.wfile.write(struct.pack("!b", 1))
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 597110321a86370c29063052eb892f9213a4bfb3..cbd41e58c4a780392b2a6b8c58320535e416cd36 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -26,7 +26,7 @@ from pyspark.accumulators import Accumulator
 from pyspark.broadcast import Broadcast
 from pyspark.files import SparkFiles
 from pyspark.java_gateway import launch_gateway
-from pyspark.serializers import dump_pickle, write_with_length, batched
+from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer
 from pyspark.storagelevel import StorageLevel
 from pyspark.rdd import RDD
 
@@ -42,15 +42,16 @@ class SparkContext(object):
 
     _gateway = None
     _jvm = None
-    _writeIteratorToPickleFile = None
+    _writeToFile = None
     _takePartition = None
     _next_accum_id = 0
     _active_spark_context = None
     _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):
+        environment=None, batchSize=1024, serializer=PickleSerializer()):
         """
         Create a new SparkContext.
 
@@ -66,24 +67,30 @@ 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.
+        @param serializer: The serializer for RDDs.
+
+
+        >>> 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
         self.environment = environment or {}
-        self.batchSize = batchSize  # -1 represents a unlimited batch size
+        self._batchSize = batchSize  # -1 represents an unlimited batch size
+        self._unbatched_serializer = serializer
+        if batchSize == 1:
+            self.serializer = self._unbatched_serializer
+        else:
+            self.serializer = BatchedSerializer(self._unbatched_serializer,
+                                                batchSize)
 
         # Create the Java SparkContext through Py4J
         empty_string_array = self._gateway.new_array(self._jvm.String, 0)
@@ -119,6 +126,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._writeToFile = \
+                    SparkContext._jvm.PythonRDD.writeToFile
+                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):
         """
@@ -158,15 +191,17 @@ class SparkContext(object):
         # Make sure we distribute data evenly if it's smaller than self.batchSize
         if "__len__" not in dir(c):
             c = list(c)    # Make it a list so we can compute its length
-        batchSize = min(len(c) // numSlices, self.batchSize)
+        batchSize = min(len(c) // numSlices, self._batchSize)
         if batchSize > 1:
-            c = batched(c, batchSize)
-        for x in c:
-            write_with_length(dump_pickle(x), tempFile)
+            serializer = BatchedSerializer(self._unbatched_serializer,
+                                           batchSize)
+        else:
+            serializer = self._unbatched_serializer
+        serializer.dump_stream(c, tempFile)
         tempFile.close()
-        readRDDFromPickleFile = self._jvm.PythonRDD.readRDDFromPickleFile
-        jrdd = readRDDFromPickleFile(self._jsc, tempFile.name, numSlices)
-        return RDD(jrdd, self)
+        readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile
+        jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices)
+        return RDD(jrdd, self, serializer)
 
     def textFile(self, name, minSplits=None):
         """
@@ -175,21 +210,39 @@ class SparkContext(object):
         RDD of Strings.
         """
         minSplits = minSplits or min(self.defaultParallelism, 2)
-        jrdd = self._jsc.textFile(name, minSplits)
-        return RDD(jrdd, self)
+        return RDD(self._jsc.textFile(name, minSplits), self,
+                   MUTF8Deserializer())
 
-    def _checkpointFile(self, name):
+    def _checkpointFile(self, name, input_deserializer):
         jrdd = self._jsc.checkpointFile(name)
-        return RDD(jrdd, self)
+        return RDD(jrdd, self, input_deserializer)
 
     def union(self, rdds):
         """
         Build the union of a list of RDDs.
+
+        This supports unions() of RDDs with different serialized formats,
+        although this forces them to be reserialized using the default
+        serializer:
+
+        >>> path = os.path.join(tempdir, "union-text.txt")
+        >>> with open(path, "w") as testFile:
+        ...    testFile.write("Hello")
+        >>> textFile = sc.textFile(path)
+        >>> textFile.collect()
+        [u'Hello']
+        >>> parallelized = sc.parallelize(["World!"])
+        >>> sorted(sc.union([textFile, parallelized]).collect())
+        [u'Hello', 'World!']
         """
+        first_jrdd_deserializer = rdds[0]._jrdd_deserializer
+        if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds):
+            rdds = [x._reserialize() for x in rdds]
         first = rdds[0]._jrdd
         rest = [x._jrdd for x in rdds[1:]]
-        rest = ListConverter().convert(rest, self.gateway._gateway_client)
-        return RDD(self._jsc.union(first, rest), self)
+        rest = ListConverter().convert(rest, self._gateway._gateway_client)
+        return RDD(self._jsc.union(first, rest), self,
+                   rdds[0]._jrdd_deserializer)
 
     def broadcast(self, value):
         """
@@ -197,7 +250,9 @@ class SparkContext(object):
         object for reading it in distributed functions. The variable will be
         sent to each cluster only once.
         """
-        jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value)))
+        pickleSer = PickleSerializer()
+        pickled = pickleSer.dumps(value)
+        jbroadcast = self._jsc.broadcast(bytearray(pickled))
         return Broadcast(jbroadcast.id(), value, jbroadcast,
                          self._pickled_broadcast_vars)
 
@@ -209,7 +264,7 @@ class SparkContext(object):
         and floating-point numbers if you do not provide one. For other types,
         a custom AccumulatorParam can be used.
         """
-        if accum_param == None:
+        if accum_param is None:
             if isinstance(value, int):
                 accum_param = accumulators.INT_ACCUMULATOR_PARAM
             elif isinstance(value, float):
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 245a132dfdcecfc88b3d7a9e85b6cc09409d1671..61720dcf1af9d4614f2bc7902efb2b2c93af1b40 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -18,7 +18,7 @@
 from base64 import standard_b64encode as b64enc
 import copy
 from collections import defaultdict
-from itertools import chain, ifilter, imap, product
+from itertools import chain, ifilter, imap
 import operator
 import os
 import sys
@@ -27,9 +27,8 @@ from subprocess import Popen, PIPE
 from tempfile import NamedTemporaryFile
 from threading import Thread
 
-from pyspark import cloudpickle
-from pyspark.serializers import batched, Batch, dump_pickle, load_pickle, \
-    read_from_pickle_file, pack_long
+from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
+    BatchedSerializer, CloudPickleSerializer, pack_long
 from pyspark.join import python_join, python_left_outer_join, \
     python_right_outer_join, python_cogroup
 from pyspark.statcounter import StatCounter
@@ -48,12 +47,12 @@ class RDD(object):
     operated on in parallel.
     """
 
-    def __init__(self, jrdd, ctx):
+    def __init__(self, jrdd, ctx, jrdd_deserializer):
         self._jrdd = jrdd
         self.is_cached = False
         self.is_checkpointed = False
         self.ctx = ctx
-        self._partitionFunc = None
+        self._jrdd_deserializer = jrdd_deserializer
 
     @property
     def context(self):
@@ -247,7 +246,23 @@ class RDD(object):
         >>> rdd.union(rdd).collect()
         [1, 1, 2, 3, 1, 1, 2, 3]
         """
-        return RDD(self._jrdd.union(other._jrdd), self.ctx)
+        if self._jrdd_deserializer == other._jrdd_deserializer:
+            rdd = RDD(self._jrdd.union(other._jrdd), self.ctx,
+                      self._jrdd_deserializer)
+            return rdd
+        else:
+            # These RDDs contain data in different serialized formats, so we
+            # must normalize them to the default serializer.
+            self_copy = self._reserialize()
+            other_copy = other._reserialize()
+            return RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx,
+                       self.ctx.serializer)
+
+    def _reserialize(self):
+        if self._jrdd_deserializer == self.ctx.serializer:
+            return self
+        else:
+            return self.map(lambda x: x, preservesPartitioning=True)
 
     def __add__(self, other):
         """
@@ -334,17 +349,9 @@ class RDD(object):
         [(1, 1), (1, 2), (2, 1), (2, 2)]
         """
         # Due to batching, we can't use the Java cartesian method.
-        java_cartesian = RDD(self._jrdd.cartesian(other._jrdd), self.ctx)
-        def unpack_batches(pair):
-            (x, y) = pair
-            if type(x) == Batch or type(y) == Batch:
-                xs = x.items if type(x) == Batch else [x]
-                ys = y.items if type(y) == Batch else [y]
-                for pair in product(xs, ys):
-                    yield pair
-            else:
-                yield pair
-        return java_cartesian.flatMap(unpack_batches)
+        deserializer = CartesianDeserializer(self._jrdd_deserializer,
+                                             other._jrdd_deserializer)
+        return RDD(self._jrdd.cartesian(other._jrdd), self.ctx, deserializer)
 
     def groupBy(self, f, numPartitions=None):
         """
@@ -391,8 +398,8 @@ class RDD(object):
         """
         Return a list that contains all of the elements in this RDD.
         """
-        picklesInJava = self._jrdd.collect().iterator()
-        return list(self._collect_iterator_through_file(picklesInJava))
+        bytesInJava = self._jrdd.collect().iterator()
+        return list(self._collect_iterator_through_file(bytesInJava))
 
     def _collect_iterator_through_file(self, iterator):
         # Transferring lots of data through Py4J can be slow because
@@ -400,10 +407,10 @@ class RDD(object):
         # file and read it back.
         tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir)
         tempFile.close()
-        self.ctx._writeIteratorToPickleFile(iterator, tempFile.name)
+        self.ctx._writeToFile(iterator, tempFile.name)
         # Read the data into Python and deserialize it:
         with open(tempFile.name, 'rb') as tempFile:
-            for item in read_from_pickle_file(tempFile):
+            for item in self._jrdd_deserializer.load_stream(tempFile):
                 yield item
         os.unlink(tempFile.name)
 
@@ -571,7 +578,7 @@ class RDD(object):
         items = []
         for partition in range(mapped._jrdd.splits().size()):
             iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition)
-            items.extend(self._collect_iterator_through_file(iterator))
+            items.extend(mapped._collect_iterator_through_file(iterator))
             if len(items) >= num:
                 break
         return items[:num]
@@ -598,7 +605,10 @@ class RDD(object):
         '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n'
         """
         def func(split, iterator):
-            return (str(x).encode("utf-8") for x in iterator)
+            for x in iterator:
+                if not isinstance(x, basestring):
+                    x = unicode(x)
+                yield x.encode("utf-8")
         keyed = PipelinedRDD(self, func)
         keyed._bypass_serializer = True
         keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path)
@@ -735,6 +745,7 @@ class RDD(object):
         # Transferring O(n) objects to Java is too expensive.  Instead, we'll
         # form the hash buckets in Python, transferring O(numPartitions) objects
         # to Java.  Each object is a (splitNumber, [objects]) pair.
+        outputSerializer = self.ctx._unbatched_serializer
         def add_shuffle_key(split, iterator):
 
             buckets = defaultdict(list)
@@ -743,14 +754,14 @@ class RDD(object):
                 buckets[partitionFunc(k) % numPartitions].append((k, v))
             for (split, items) in buckets.iteritems():
                 yield pack_long(split)
-                yield dump_pickle(Batch(items))
+                yield outputSerializer.dumps(items)
         keyed = PipelinedRDD(self, add_shuffle_key)
         keyed._bypass_serializer = True
         pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
         partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
                                                      id(partitionFunc))
         jrdd = pairRDD.partitionBy(partitioner).values()
-        rdd = RDD(jrdd, self.ctx)
+        rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer))
         # This is required so that id(partitionFunc) remains unique, even if
         # partitionFunc is a lambda:
         rdd._partitionFunc = partitionFunc
@@ -787,7 +798,8 @@ class RDD(object):
             numPartitions = self.ctx.defaultParallelism
         def combineLocally(iterator):
             combiners = {}
-            for (k, v) in iterator:
+            for x in iterator:
+                (k, v) = x
                 if k not in combiners:
                     combiners[k] = createCombiner(v)
                 else:
@@ -929,38 +941,39 @@ class PipelinedRDD(RDD):
     20
     """
     def __init__(self, prev, func, preservesPartitioning=False):
-        if isinstance(prev, PipelinedRDD) and prev._is_pipelinable():
+        if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable():
+            # This transformation is the first in its stage:
+            self.func = func
+            self.preservesPartitioning = preservesPartitioning
+            self._prev_jrdd = prev._jrdd
+            self._prev_jrdd_deserializer = prev._jrdd_deserializer
+        else:
             prev_func = prev.func
             def pipeline_func(split, iterator):
                 return func(split, prev_func(split, iterator))
             self.func = pipeline_func
             self.preservesPartitioning = \
                 prev.preservesPartitioning and preservesPartitioning
-            self._prev_jrdd = prev._prev_jrdd
-        else:
-            self.func = func
-            self.preservesPartitioning = preservesPartitioning
-            self._prev_jrdd = prev._jrdd
+            self._prev_jrdd = prev._prev_jrdd  # maintain the pipeline
+            self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer
         self.is_cached = False
         self.is_checkpointed = False
         self.ctx = prev.ctx
         self.prev = prev
         self._jrdd_val = None
+        self._jrdd_deserializer = self.ctx.serializer
         self._bypass_serializer = False
 
     @property
     def _jrdd(self):
         if self._jrdd_val:
             return self._jrdd_val
-        func = self.func
-        if not self._bypass_serializer and self.ctx.batchSize != 1:
-            oldfunc = self.func
-            batchSize = self.ctx.batchSize
-            def batched_func(split, iterator):
-                return batched(oldfunc(split, iterator), batchSize)
-            func = batched_func
-        cmds = [func, self._bypass_serializer]
-        pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds)
+        if self._bypass_serializer:
+            serializer = NoOpSerializer()
+        else:
+            serializer = self.ctx.serializer
+        command = (self.func, self._prev_jrdd_deserializer, serializer)
+        pickled_command = CloudPickleSerializer().dumps(command)
         broadcast_vars = ListConverter().convert(
             [x._jbroadcast for x in self.ctx._pickled_broadcast_vars],
             self.ctx._gateway._gateway_client)
@@ -971,8 +984,9 @@ class PipelinedRDD(RDD):
         includes = ListConverter().convert(self.ctx._python_includes,
                                      self.ctx._gateway._gateway_client)
         python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(),
-            pipe_command, env, includes, self.preservesPartitioning, self.ctx.pythonExec,
-            broadcast_vars, self.ctx._javaAccumulator, class_tag)
+            bytearray(pickled_command), env, includes, self.preservesPartitioning,
+            self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator,
+            class_tag)
         self._jrdd_val = python_rdd.asJavaRDD()
         return self._jrdd_val
 
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 54fed1c9c70f66e503abb5c523d6327bb9bae8b4..811fa6f018b23f3c9883bd2a770f03c044786850 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -15,45 +15,269 @@
 # limitations under the License.
 #
 
-import struct
+"""
+PySpark supports custom serializers for transferring data; this can improve
+performance.
+
+By default, PySpark uses L{PickleSerializer} to serialize objects using Python's
+C{cPickle} serializer, which can serialize nearly any Python object.
+Other serializers, like L{MarshalSerializer}, support fewer datatypes but can be
+faster.
+
+The serializer is chosen when creating L{SparkContext}:
+
+>>> from pyspark.context import SparkContext
+>>> from pyspark.serializers import MarshalSerializer
+>>> sc = SparkContext('local', 'test', serializer=MarshalSerializer())
+>>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10)
+[0, 2, 4, 6, 8, 10, 12, 14, 16, 18]
+>>> sc.stop()
+
+By default, PySpark serialize objects in batches; the batch size can be
+controlled through SparkContext's C{batchSize} parameter
+(the default size is 1024 objects):
+
+>>> sc = SparkContext('local', 'test', batchSize=2)
+>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x)
+
+Behind the scenes, this creates a JavaRDD with four partitions, each of
+which contains two batches of two objects:
+
+>>> rdd.glom().collect()
+[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]]
+>>> rdd._jrdd.count()
+8L
+>>> sc.stop()
+
+A batch size of -1 uses an unlimited batch size, and a size of 1 disables
+batching:
+
+>>> sc = SparkContext('local', 'test', batchSize=1)
+>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x)
+>>> rdd.glom().collect()
+[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]]
+>>> rdd._jrdd.count()
+16L
+"""
+
 import cPickle
+from itertools import chain, izip, product
+import marshal
+import struct
+from pyspark import cloudpickle
+
+
+__all__ = ["PickleSerializer", "MarshalSerializer"]
+
+
+class SpecialLengths(object):
+    END_OF_DATA_SECTION = -1
+    PYTHON_EXCEPTION_THROWN = -2
+    TIMING_DATA = -3
+
+
+class Serializer(object):
+
+    def dump_stream(self, iterator, stream):
+        """
+        Serialize an iterator of objects to the output stream.
+        """
+        raise NotImplementedError
+
+    def load_stream(self, stream):
+        """
+        Return an iterator of deserialized objects from the input stream.
+        """
+        raise NotImplementedError
+
+
+    def _load_stream_without_unbatching(self, stream):
+        return self.load_stream(stream)
+
+    # Note: our notion of "equality" is that output generated by
+    # equal serializers can be deserialized using the same serializer.
+
+    # This default implementation handles the simple cases;
+    # subclasses should override __eq__ as appropriate.
+
+    def __eq__(self, other):
+        return isinstance(other, self.__class__)
+
+    def __ne__(self, other):
+        return not self.__eq__(other)
+
+
+class FramedSerializer(Serializer):
+    """
+    Serializer that writes objects as a stream of (length, data) pairs,
+    where C{length} is a 32-bit integer and data is C{length} bytes.
+    """
+
+    def dump_stream(self, iterator, stream):
+        for obj in iterator:
+            self._write_with_length(obj, stream)
+
+    def load_stream(self, stream):
+        while True:
+            try:
+                yield self._read_with_length(stream)
+            except EOFError:
+                return
+
+    def _write_with_length(self, obj, stream):
+        serialized = self.dumps(obj)
+        write_int(len(serialized), stream)
+        stream.write(serialized)
+
+    def _read_with_length(self, stream):
+        length = read_int(stream)
+        obj = stream.read(length)
+        if obj == "":
+            raise EOFError
+        return self.loads(obj)
+
+    def dumps(self, obj):
+        """
+        Serialize an object into a byte array.
+        When batching is used, this will be called with an array of objects.
+        """
+        raise NotImplementedError
+
+    def loads(self, obj):
+        """
+        Deserialize an object from a byte array.
+        """
+        raise NotImplementedError
+
+
+class BatchedSerializer(Serializer):
+    """
+    Serializes a stream of objects in batches by calling its wrapped
+    Serializer with streams of objects.
+    """
+
+    UNLIMITED_BATCH_SIZE = -1
+
+    def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE):
+        self.serializer = serializer
+        self.batchSize = batchSize
+
+    def _batched(self, iterator):
+        if self.batchSize == self.UNLIMITED_BATCH_SIZE:
+            yield list(iterator)
+        else:
+            items = []
+            count = 0
+            for item in iterator:
+                items.append(item)
+                count += 1
+                if count == self.batchSize:
+                    yield items
+                    items = []
+                    count = 0
+            if items:
+                yield items
+
+    def dump_stream(self, iterator, stream):
+        self.serializer.dump_stream(self._batched(iterator), stream)
+
+    def load_stream(self, stream):
+        return chain.from_iterable(self._load_stream_without_unbatching(stream))
+
+    def _load_stream_without_unbatching(self, stream):
+            return self.serializer.load_stream(stream)
+
+    def __eq__(self, other):
+        return isinstance(other, BatchedSerializer) and \
+               other.serializer == self.serializer
+
+    def __str__(self):
+        return "BatchedSerializer<%s>" % str(self.serializer)
 
 
-class Batch(object):
+class CartesianDeserializer(FramedSerializer):
     """
-    Used to store multiple RDD entries as a single Java object.
+    Deserializes the JavaRDD cartesian() of two PythonRDDs.
+    """
+
+    def __init__(self, key_ser, val_ser):
+        self.key_ser = key_ser
+        self.val_ser = val_ser
+
+    def load_stream(self, stream):
+        key_stream = self.key_ser._load_stream_without_unbatching(stream)
+        val_stream = self.val_ser._load_stream_without_unbatching(stream)
+        key_is_batched = isinstance(self.key_ser, BatchedSerializer)
+        val_is_batched = isinstance(self.val_ser, BatchedSerializer)
+        for (keys, vals) in izip(key_stream, val_stream):
+            keys = keys if key_is_batched else [keys]
+            vals = vals if val_is_batched else [vals]
+            for pair in product(keys, vals):
+                yield pair
+
+    def __eq__(self, other):
+        return isinstance(other, CartesianDeserializer) and \
+               self.key_ser == other.key_ser and self.val_ser == other.val_ser
+
+    def __str__(self):
+        return "CartesianDeserializer<%s, %s>" % \
+               (str(self.key_ser), str(self.val_ser))
 
-    This relieves us from having to explicitly track whether an RDD
-    is stored as batches of objects and avoids problems when processing
-    the union() of batched and unbatched RDDs (e.g. the union() of textFile()
-    with another RDD).
+
+class NoOpSerializer(FramedSerializer):
+
+    def loads(self, obj): return obj
+    def dumps(self, obj): return obj
+
+
+class PickleSerializer(FramedSerializer):
     """
-    def __init__(self, items):
-        self.items = items
+    Serializes objects using Python's cPickle serializer:
 
+        http://docs.python.org/2/library/pickle.html
 
-def batched(iterator, batchSize):
-    if batchSize == -1: # unlimited batch size
-        yield Batch(list(iterator))
-    else:
-        items = []
-        count = 0
-        for item in iterator:
-            items.append(item)
-            count += 1
-            if count == batchSize:
-                yield Batch(items)
-                items = []
-                count = 0
-        if items:
-            yield Batch(items)
+    This serializer supports nearly any Python object, but may
+    not be as fast as more specialized serializers.
+    """
 
+    def dumps(self, obj): return cPickle.dumps(obj, 2)
+    loads = cPickle.loads
 
-def dump_pickle(obj):
-    return cPickle.dumps(obj, 2)
+class CloudPickleSerializer(PickleSerializer):
 
+    def dumps(self, obj): return cloudpickle.dumps(obj, 2)
 
-load_pickle = cPickle.loads
+
+class MarshalSerializer(FramedSerializer):
+    """
+    Serializes objects using Python's Marshal serializer:
+
+        http://docs.python.org/2/library/marshal.html
+
+    This serializer is faster than PickleSerializer but supports fewer datatypes.
+    """
+
+    dumps = marshal.dumps
+    loads = marshal.loads
+
+
+class MUTF8Deserializer(Serializer):
+    """
+    Deserializes streams written by Java's DataOutputStream.writeUTF().
+    """
+
+    def loads(self, stream):
+        length = struct.unpack('>H', stream.read(2))[0]
+        return stream.read(length).decode('utf8')
+
+    def load_stream(self, stream):
+        while True:
+            try:
+                yield self.loads(stream)
+            except struct.error:
+                return
+            except EOFError:
+                return
 
 
 def read_long(stream):
@@ -84,25 +308,4 @@ def write_int(value, stream):
 
 def write_with_length(obj, stream):
     write_int(len(obj), stream)
-    stream.write(obj)
-
-
-def read_with_length(stream):
-    length = read_int(stream)
-    obj = stream.read(length)
-    if obj == "":
-        raise EOFError
-    return obj
-
-
-def read_from_pickle_file(stream):
-    try:
-        while True:
-            obj = load_pickle(read_with_length(stream))
-            if type(obj) == Batch:  # We don't care about inheritance
-                for item in obj.items:
-                    yield item
-            else:
-                yield obj
-    except EOFError:
-        return
+    stream.write(obj)
\ No newline at end of file
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 29d6a128f6a9b83dc742c676ef010a90f54ab73e..3987642bf4d5c75fb74a3e1071ce7f1882ac905d 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -19,6 +19,8 @@
 Unit tests for PySpark; additional tests are implemented as doctests in
 individual modules.
 """
+from fileinput import input
+from glob import glob
 import os
 import shutil
 import sys
@@ -86,7 +88,8 @@ class TestCheckpoint(PySparkTestCase):
         time.sleep(1)  # 1 second
 
         self.assertTrue(flatMappedRDD.getCheckpointFile() is not None)
-        recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile())
+        recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(),
+                                            flatMappedRDD._jrdd_deserializer)
         self.assertEquals([1, 2, 3, 4], recovered.collect())
 
 
@@ -137,6 +140,19 @@ class TestAddFile(PySparkTestCase):
         self.assertEqual("Hello World from inside a package!", UserClass().hello())
 
 
+class TestRDDFunctions(PySparkTestCase):
+
+    def test_save_as_textfile_with_unicode(self):
+        # Regression test for SPARK-970
+        x = u"\u00A1Hola, mundo!"
+        data = self.sc.parallelize([x])
+        tempFile = NamedTemporaryFile(delete=True)
+        tempFile.close()
+        data.saveAsTextFile(tempFile.name)
+        raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*")))
+        self.assertEqual(x, unicode(raw_contents.strip(), "utf-8"))
+
+
 class TestIO(PySparkTestCase):
 
     def test_stdout_redirection(self):
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index d63c2aaef772de62eef3bf913ad4a4859cf30512..f2b3f3c1421d12d48637ca47b3ac39ed61bbcfac 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -23,23 +23,22 @@ import sys
 import time
 import socket
 import traceback
-from base64 import standard_b64decode
 # CloudPickler needs to be imported so that depicklers are registered using the
 # copy_reg module.
 from pyspark.accumulators import _accumulatorRegistry
 from pyspark.broadcast import Broadcast, _broadcastRegistry
 from pyspark.cloudpickle import CloudPickler
 from pyspark.files import SparkFiles
-from pyspark.serializers import write_with_length, read_with_length, write_int, \
-    read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file
+from pyspark.serializers import write_with_length, write_int, read_long, \
+    write_long, read_int, SpecialLengths, MUTF8Deserializer, PickleSerializer
 
 
-def load_obj(infile):
-    return load_pickle(standard_b64decode(infile.readline().strip()))
+pickleSer = PickleSerializer()
+mutf8_deserializer = MUTF8Deserializer()
 
 
 def report_times(outfile, boot, init, finish):
-    write_int(-3, outfile)
+    write_int(SpecialLengths.TIMING_DATA, outfile)
     write_long(1000 * boot, outfile)
     write_long(1000 * init, outfile)
     write_long(1000 * finish, outfile)
@@ -52,7 +51,7 @@ def main(infile, outfile):
         return
 
     # fetch name of workdir
-    spark_files_dir = load_pickle(read_with_length(infile))
+    spark_files_dir = mutf8_deserializer.loads(infile)
     SparkFiles._root_directory = spark_files_dir
     SparkFiles._is_running_on_worker = True
 
@@ -60,38 +59,33 @@ def main(infile, outfile):
     num_broadcast_variables = read_int(infile)
     for _ in range(num_broadcast_variables):
         bid = read_long(infile)
-        value = read_with_length(infile)
-        _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value))
+        value = pickleSer._read_with_length(infile)
+        _broadcastRegistry[bid] = Broadcast(bid, value)
 
     # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH
     sys.path.append(spark_files_dir) # *.py files that were added will be copied here
     num_python_includes =  read_int(infile)
     for _ in range(num_python_includes):
-        sys.path.append(os.path.join(spark_files_dir, load_pickle(read_with_length(infile))))
+        filename = mutf8_deserializer.loads(infile)
+        sys.path.append(os.path.join(spark_files_dir, filename))
 
-    # now load function
-    func = load_obj(infile)
-    bypassSerializer = load_obj(infile)
-    if bypassSerializer:
-        dumps = lambda x: x
-    else:
-        dumps = dump_pickle
+    command = pickleSer._read_with_length(infile)
+    (func, deserializer, serializer) = command
     init_time = time.time()
-    iterator = read_from_pickle_file(infile)
     try:
-        for obj in func(split_index, iterator):
-            write_with_length(dumps(obj), outfile)
+        iterator = deserializer.load_stream(infile)
+        serializer.dump_stream(func(split_index, iterator), outfile)
     except Exception as e:
-        write_int(-2, outfile)
+        write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile)
         write_with_length(traceback.format_exc(), outfile)
         sys.exit(-1)
     finish_time = time.time()
     report_times(outfile, boot_time, init_time, finish_time)
     # Mark the beginning of the accumulators section of the output
-    write_int(-1, outfile)
-    for aid, accum in _accumulatorRegistry.items():
-        write_with_length(dump_pickle((aid, accum._value)), outfile)
-    write_int(-1, outfile)
+    write_int(SpecialLengths.END_OF_DATA_SECTION, outfile)
+    write_int(len(_accumulatorRegistry), outfile)
+    for (aid, accum) in _accumulatorRegistry.items():
+        pickleSer._write_with_length((aid, accum._value), outfile)
 
 
 if __name__ == '__main__':
diff --git a/python/run-tests b/python/run-tests
index cbc554ea9db0d2cdd18323a408879d98f80810cb..d4dad672d284299a0a1fc116b24f7833eefe1b7b 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -37,6 +37,7 @@ run_test "pyspark/rdd.py"
 run_test "pyspark/context.py"
 run_test "-m doctest pyspark/broadcast.py"
 run_test "-m doctest pyspark/accumulators.py"
+run_test "-m doctest pyspark/serializers.py"
 run_test "pyspark/tests.py"
 
 if [[ $FAILED != 0 ]]; then
diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py
index 5bb6f5009ffbb1bb0ca54035903ba33b2967e922..8e4a6292bc17c53391fd50acc5d838b9173af8f2 100755
--- a/python/test_support/userlibrary.py
+++ b/python/test_support/userlibrary.py
@@ -1,3 +1,20 @@
+#
+# 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.
+#
+
 """
 Used to test shipping of code depenencies with SparkContext.addPyFile().
 """
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index c983ea5dfb97b42f3aad337375444e1fc4fdc899..c2a4efa36c485342f597a008dbd5dffaa0c7ddb7 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-repl-bin_${scala-short.version}</artifactId>
+  <artifactId>spark-repl-bin_2.10</artifactId>
   <packaging>pom</packaging>
   <name>Spark Project REPL binary packaging</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -40,18 +40,18 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-bagel_${scala-short.version}</artifactId>
+      <artifactId>spark-bagel_2.10</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-repl_${scala-short.version}</artifactId>
+      <artifactId>spark-repl_2.10</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor
index bcfae22677fdd8372af55c77a688c2634e2a5cb8..052d76fb8d81cad0b27af8f0ac86b51b6acc8b7c 100755
--- a/repl-bin/src/deb/bin/spark-executor
+++ b/repl-bin/src/deb/bin/spark-executor
@@ -19,4 +19,4 @@
 
 FWDIR="$(cd `dirname $0`; pwd)"
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/run spark.executor.MesosExecutorBackend
+exec $FWDIR/run org.apache.spark.executor.MesosExecutorBackend
diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell
index ec7e33e1e3deccefe314578f41d7ed8b27cb7379..118349d7c30f26ec8770724fca797a3aed69cbd4 100755
--- a/repl-bin/src/deb/bin/spark-shell
+++ b/repl-bin/src/deb/bin/spark-shell
@@ -18,4 +18,4 @@
 #
 
 FWDIR="$(cd `dirname $0`; pwd)"
-exec $FWDIR/run spark.repl.Main "$@"
+exec $FWDIR/run org.apache.spark.repl.Main "$@"
diff --git a/repl/pom.xml b/repl/pom.xml
index ff6649322911b0edef7b94bd4fce3abd9a256f05..bf06d730d4ac61f4c4e71378c5472f14d5a45a23 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -26,7 +26,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-repl_${scala-short.version}</artifactId>
+  <artifactId>spark-repl_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project REPL</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -39,18 +39,18 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-bagel_${scala-short.version}</artifactId>
+      <artifactId>spark-bagel_2.10</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-mllib_${scala-short.version}</artifactId>
+      <artifactId>spark-mllib_2.10</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
@@ -78,18 +78,18 @@
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
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 988b624febc0e27ab53e28878c7fca6f471e4694..523fd1222dd71dff1770f1488c21ceb73617b9f6 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -675,6 +675,20 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     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) {
@@ -915,10 +929,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
   }
 
   def createSparkContext(): SparkContext = {
-   val uri = System.getenv("SPARK_EXECUTOR_URI")
-   if (uri != null) {
-         System.setProperty("spark.executor.uri", uri)
-   }
+    val uri = System.getenv("SPARK_EXECUTOR_URI")
+    if (uri != null) {
+      System.setProperty("spark.executor.uri", uri)
+    }
     val master = this.master match {
       case Some(m) => m
       case None => {
diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index fccb6e652c47e52badf88b05f87b042705105ca1..daaa2a0305113527d66d1d69cb54c791933c91ae 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -5,10 +5,13 @@ import java.net.URLClassLoader
 
 import scala.collection.mutable.ArrayBuffer
 
-import org.scalatest.FunSuite
 import com.google.common.io.Files
+import org.scalatest.FunSuite
+import org.apache.spark.SparkContext
+
 
 class ReplSuite extends FunSuite {
+
   def runInterpreter(master: String, input: String): String = {
     val in = new BufferedReader(new StringReader(input + "\n"))
     val out = new StringWriter()
@@ -46,24 +49,55 @@ class ReplSuite extends FunSuite {
       "Interpreter output contained '" + message + "':\n" + output)
   }
 
+  test("propagation of local properties") {
+    // A mock ILoop that doesn't install the SIGINT handler.
+    class ILoop(out: PrintWriter) extends SparkILoop(None, out, None) {
+      settings = new scala.tools.nsc.Settings
+      settings.usejavacp.value = true
+      org.apache.spark.repl.Main.interp = this
+      override def createInterpreter() {
+        intp = new SparkILoopInterpreter
+        intp.setContextClassLoader()
+      }
+    }
+
+    val out = new StringWriter()
+    val interp = new ILoop(new PrintWriter(out))
+    interp.sparkContext = new SparkContext("local", "repl-test")
+    interp.createInterpreter()
+    interp.intp.initialize()
+    interp.sparkContext.setLocalProperty("someKey", "someValue")
+
+    // Make sure the value we set in the caller to interpret is propagated in the thread that
+    // interprets the command.
+    interp.interpret("org.apache.spark.repl.Main.interp.sparkContext.getLocalProperty(\"someKey\")")
+    assert(out.toString.contains("someValue"))
+
+    interp.sparkContext.stop()
+    System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
+  }
+
   test("simple foreach with accumulator") {
-    val output = runInterpreter("local", """
-                                           |val accum = sc.accumulator(0)
-                                           |sc.parallelize(1 to 10).foreach(x => accum += x)
-                                           |accum.value
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |val accum = sc.accumulator(0)
+        |sc.parallelize(1 to 10).foreach(x => accum += x)
+        |accum.value
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res1: Int = 55", output)
   }
 
   test("external vars") {
-    val output = runInterpreter("local", """
-                                           |var v = 7
-                                           |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
-                                           |v = 10
-                                           |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |var v = 7
+        |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+        |v = 10
+        |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Int = 70", output)
@@ -71,35 +105,38 @@ class ReplSuite extends FunSuite {
   }
 
   test("external classes") {
-    val output = runInterpreter("local", """
-                                           |class C {
-                                           |def foo = 5
-                                           |}
-                                           |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |class C {
+        |def foo = 5
+        |}
+        |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Int = 50", output)
   }
 
   test("external functions") {
-    val output = runInterpreter("local", """
-                                           |def double(x: Int) = x + x
-                                           |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |def double(x: Int) = x + x
+        |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Int = 110", output)
   }
 
   test("external functions that access vars") {
-    val output = runInterpreter("local", """
-                                           |var v = 7
-                                           |def getV() = v
-                                           |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                           |v = 10
-                                           |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |var v = 7
+        |def getV() = v
+        |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+        |v = 10
+        |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Int = 70", output)
@@ -110,13 +147,14 @@ class ReplSuite extends FunSuite {
     // Test that the value that a broadcast var had when it was created is used,
     // even if that variable is then modified in the driver program
     // TODO: This doesn't actually work for arrays when we run in local mode!
-    val output = runInterpreter("local", """
-                                           |var array = new Array[Int](5)
-                                           |val broadcastArray = sc.broadcast(array)
-                                           |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                           |array(0) = 5
-                                           |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                         """.stripMargin)
+    val output = runInterpreter("local",
+      """
+        |var array = new Array[Int](5)
+        |val broadcastArray = sc.broadcast(array)
+        |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+        |array(0) = 5
+        |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output)
@@ -130,12 +168,13 @@ class ReplSuite extends FunSuite {
     out.write("What's up?\n")
     out.write("Goodbye\n")
     out.close()
-    val output = runInterpreter("local", """
-                                           |var file = sc.textFile("%s/input").cache()
-                                           |file.count()
-                                           |file.count()
-                                           |file.count()
-                                         """.stripMargin.format(tempDir.getAbsolutePath))
+    val output = runInterpreter("local",
+      """
+        |var file = sc.textFile("%s/input").cache()
+        |file.count()
+        |file.count()
+        |file.count()
+      """.stripMargin.format(tempDir.getAbsolutePath))
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Long = 3", output)
@@ -144,18 +183,19 @@ class ReplSuite extends FunSuite {
   }
 
   test("local-cluster mode") {
-    val output = runInterpreter("local-cluster[1,1,512]", """
-                                                            |var v = 7
-                                                            |def getV() = v
-                                                            |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                                            |v = 10
-                                                            |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                                            |var array = new Array[Int](5)
-                                                            |val broadcastArray = sc.broadcast(array)
-                                                            |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                                            |array(0) = 5
-                                                            |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                                          """.stripMargin)
+    val output = runInterpreter("local-cluster[1,1,512]",
+      """
+        |var v = 7
+        |def getV() = v
+        |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+        |v = 10
+        |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+        |var array = new Array[Int](5)
+        |val broadcastArray = sc.broadcast(array)
+        |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+        |array(0) = 5
+        |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+      """.stripMargin)
     assertDoesNotContain("error:", output)
     assertDoesNotContain("Exception", output)
     assertContains("res0: Int = 70", output)
@@ -166,18 +206,19 @@ class ReplSuite extends FunSuite {
 
   if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
     test("running on Mesos") {
-      val output = runInterpreter("localquiet", """
-                                                  |var v = 7
-                                                  |def getV() = v
-                                                  |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                                  |v = 10
-                                                  |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
-                                                  |var array = new Array[Int](5)
-                                                  |val broadcastArray = sc.broadcast(array)
-                                                  |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                                  |array(0) = 5
-                                                  |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
-                                                """.stripMargin)
+      val output = runInterpreter("localquiet",
+        """
+          |var v = 7
+          |def getV() = v
+          |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+          |v = 10
+          |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+          |var array = new Array[Int](5)
+          |val broadcastArray = sc.broadcast(array)
+          |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+          |array(0) = 5
+          |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+        """.stripMargin)
       assertDoesNotContain("error:", output)
       assertDoesNotContain("Exception", output)
       assertContains("res0: Int = 70", output)
diff --git a/spark-class b/spark-class
index 5305b3d02598f229d36e9b0f89be58f06f593de0..713404d07755758ea327f16106e1a07cba599cb3 100755
--- a/spark-class
+++ b/spark-class
@@ -95,16 +95,36 @@ 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
+
+TOOLS_DIR="$FWDIR"/tools
+SPARK_TOOLS_JAR=""
+if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
+  # Use the JAR from the SBT build
+  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
+fi
+if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then
+  # Use the JAR from the Maven build
+  # TODO: this also needs to become an assembly!
+  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`
 fi
 
 # Compute classpath using external script
 CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+CLASSPATH="$SPARK_TOOLS_JAR:$CLASSPATH"
 export CLASSPATH
 
 if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
@@ -116,3 +136,4 @@ fi
 
 exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
 
+
diff --git a/spark-class2.cmd b/spark-class2.cmd
index d4d853e8ad930931e7ddf2295c6c1448846a29e0..3869d0761bfaa8e7ba0e3688b1ec23f8e8a56d87 100644
--- a/spark-class2.cmd
+++ b/spark-class2.cmd
@@ -65,10 +65,17 @@ if "%FOUND_JAR%"=="0" (
 )
 :skip_build_test
 
+set TOOLS_DIR=%FWDIR%tools
+set SPARK_TOOLS_JAR=
+for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do (
+  set SPARK_TOOLS_JAR=%%d
+)
+
 rem Compute classpath using external script
 set DONT_PRINT_CLASSPATH=1
 call "%FWDIR%bin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
+set CLASSPATH=%SPARK_TOOLS_JAR%;%CLASSPATH%
 
 rem Figure out where java is.
 set RUNNER=java
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 3f2033f34a4d03af6e496b2445d8a2ed7751bf10..e27b437aae37dee116e8564000a826f3e750f9da 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -26,23 +26,29 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-streaming_${scala-short.version}</artifactId>
+  <artifactId>spark-streaming_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Streaming</name>
   <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>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -55,10 +61,23 @@
       <version>1.9.11</version>
     </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>
+      <groupId>com.sksamuel.kafka</groupId>
+      <artifactId>kafka_2.10</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,35 +88,39 @@
           <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>
       <artifactId>scala-library</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
-      <artifactId>akka-zeromq_${scala-short.version}</artifactId>
-      <version>${akka.version}</version>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-zeromq_2.10</artifactId>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala-short.version}</artifactId>
+      <artifactId>scalacheck_2.10</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -114,10 +137,15 @@
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.eclipse.paho</groupId>
+      <artifactId>mqtt-client</artifactId>
+       <version>0.4.0</version>
+    </dependency>
   </dependencies>
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.scalatest</groupId>
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..9271914eb536a6565210913758d12db91e3a1306 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")
@@ -92,7 +94,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
           fs.delete(file, false)
           fs.rename(writeFile, file)
 
-          val finishTime = System.currentTimeMillis();
+          val finishTime = System.currentTimeMillis()
           logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
             "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
           return
@@ -122,7 +124,9 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
 
   def stop() {
     synchronized {
-      if (stopped) return ;
+      if (stopped) {
+        return
+      }
       stopped = true
     }
     executor.shutdown()
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 cd404fd408b645a254b429874ded94416d32d39f..329d2b58357ae36aa9f20f36e9e3e49776679655 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration
 
 /**
  * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
  * for more details on RDDs). DStreams can either be created from live data (such as, data from
  * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
  * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -439,6 +439,13 @@ abstract class DStream[T: ClassTag] (
    */
   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
@@ -480,7 +487,7 @@ abstract class DStream[T: ClassTag] (
 
   /**
    * 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))
@@ -488,7 +495,7 @@ abstract class DStream[T: ClassTag] (
 
   /**
    * 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) {
     ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc)))
@@ -496,18 +503,52 @@ abstract class DStream[T: ClassTag] (
 
   /**
    * 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: ClassTag](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: ClassTag](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: ClassTag, V: ClassTag](
+      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: ClassTag, V: ClassTag](
+      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/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
index b761646dff4d261c53351b62093b3ca39a1bcb3e..6e9a781978d2054f5836b53e321a3ec27521f314 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
@@ -25,14 +25,16 @@ import org.apache.spark.SparkContext._
 
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.Queue
+import scala.concurrent.duration._
 
 import akka.actor._
 import akka.pattern.ask
-import scala.concurrent.duration._
+import akka.dispatch._
+import org.apache.spark.storage.BlockId
 
 private[streaming] sealed trait NetworkInputTrackerMessage
 private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
-private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[String], metadata: Any) extends NetworkInputTrackerMessage
+private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) extends NetworkInputTrackerMessage
 private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage
 
 /**
@@ -47,7 +49,7 @@ class NetworkInputTracker(
   val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*)
   val receiverExecutor = new ReceiverExecutor()
   val receiverInfo = new HashMap[Int, ActorRef]
-  val receivedBlockIds = new HashMap[Int, Queue[String]]
+  val receivedBlockIds = new HashMap[Int, Queue[BlockId]]
   val timeout = 5000.milliseconds
 
   var currentTime: Time = null
@@ -66,9 +68,9 @@ class NetworkInputTracker(
   }
 
   /** Return all the blocks received from a receiver. */
-  def getBlockIds(receiverId: Int, time: Time): Array[String] = synchronized {
+  def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized {
     val queue =  receivedBlockIds.synchronized {
-      receivedBlockIds.getOrElse(receiverId, new Queue[String]())
+      receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]())
     }
     val result = queue.synchronized {
       queue.dequeueAll(x => true)
@@ -91,7 +93,7 @@ class NetworkInputTracker(
       case AddBlocks(streamId, blockIds, metadata) => {
         val tmp = receivedBlockIds.synchronized {
           if (!receivedBlockIds.contains(streamId)) {
-            receivedBlockIds += ((streamId, new Queue[String]))
+            receivedBlockIds += ((streamId, new Queue[BlockId]))
           }
           receivedBlockIds(streamId)
         }
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 f021e29619ad41915f052a47d19a990057a004eb..80af96c060a14b994f8b8ae9873512df77dd94c6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -18,9 +18,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.{MapValuedDStream, FlatMapValuedDStream}
+import org.apache.spark.streaming.dstream._
 
 import org.apache.spark.{Partitioner, HashPartitioner}
 import org.apache.spark.SparkContext._
@@ -360,7 +358,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
@@ -399,11 +397,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: ClassTag](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: ClassTag](
       flatMapValuesFunc: V => TraversableOnce[U]
     ): DStream[(K, U)] = {
@@ -411,9 +416,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: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
@@ -421,56 +425,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: ClassTag](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: ClassTag](
       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)(
-      classTag[K],
-      ClassTags.seqSeqClassTag
+    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: ClassTag](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: ClassTag](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: ClassTag](
       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: ClassTag](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: ClassTag](
+      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: ClassTag](
+      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: ClassTag](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: ClassTag](
+      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: ClassTag](
+      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,
@@ -480,8 +560,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 c722aa15ab35febb332aface1b537ffff0a8cd68..d2c4fdee657f756dbc8f57b9b8d999b150e9f05e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -102,6 +102,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)")
@@ -254,10 +258,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)
   }
 
   /**
@@ -268,12 +276,16 @@ class StreamingContext private (
    *               in its own thread.
    * @param storageLevel  Storage level to use for storing the received objects
    */
-  def kafkaStream[T: ClassTag, D <: kafka.serializer.Decoder[_]: Manifest](
+  def kafkaStream[
+    K: ClassTag,
+    V: ClassTag,
+    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
   }
@@ -452,13 +464,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: ClassTag](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: ClassTag](
+      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 0d54d78ed33c8bc86f97dcf394e24a15f99aa14b..d29033df3223f31f6c1ee807ad996ae5773c77c2 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
@@ -27,7 +27,7 @@ import scala.reflect.ClassTag
 
 /**
  * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
  * for more details on RDDs). DStreams can either be created from live data (such as, data from
  * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
  * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -96,6 +96,12 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T]
    */
   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 4508e485908ebae60e4dc597de43bdcc3827a476..64f38ce1c0283278c358c44572d79881652b3b37 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
@@ -25,7 +25,8 @@ import scala.reflect.ClassTag
 
 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._
@@ -121,10 +122,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. */
@@ -239,7 +242,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)))
@@ -247,7 +250,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))
@@ -255,7 +258,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: ClassTag[U] =
@@ -267,7 +270,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: ClassTag[U] =
@@ -279,7 +282,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] = {
@@ -294,7 +297,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] = {
@@ -307,6 +310,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: ClassTag[U] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+    implicit val cmv: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[U] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+    implicit val cmk2: ClassTag[K2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+    implicit val cmv2: ClassTag[V2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[K2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+    implicit val cmv2: ClassTag[V2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+    implicit val cmw: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[K2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+    implicit val cmv2: ClassTag[V2] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+    implicit val cmk3: ClassTag[K3] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K3]]
+    implicit val cmv3: ClassTag[V3] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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 c80545b5304db2f56f255254cbf8016fbedece88..dfd6e27c3e9101dbe125c26b0e5a4b99469c4a27 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
@@ -25,7 +25,7 @@ import scala.reflect.ClassTag
 
 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}
@@ -37,8 +37,8 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.PairRDDFunctions
 
 class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
-    implicit val kTag: ClassTag[K],
-    implicit val vTag: ClassTag[V])
+    implicit val kManifest: ClassTag[K],
+    implicit val vManifest: ClassTag[V])
     extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
 
   override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
@@ -60,6 +60,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 {
@@ -149,7 +155,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],
@@ -414,7 +420,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
@@ -429,7 +435,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
@@ -437,15 +443,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: ClassTag](
+  def updateStateByKey[S](
       updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
       numPartitions: Int)
   : JavaPairDStream[K, S] = {
+    implicit val cm: ClassTag[S] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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
@@ -453,19 +461,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: ClassTag](
+  def updateStateByKey[S](
       updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
       partitioner: Partitioner
   ): JavaPairDStream[K, S] = {
+    implicit val cm: ClassTag[S] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[U] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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
@@ -475,9 +494,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])] = {
@@ -487,21 +505,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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
@@ -510,17 +543,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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[W] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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".
@@ -591,14 +718,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 classTag: ClassTag[(K, V)] =
     implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]]
 }
 
 object JavaPairDStream {
-  implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)])
-  :JavaPairDStream[K, V] =
+  implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = {
     new JavaPairDStream[K, V](dstream)
+  }
 
   def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
     implicit val cmk: ClassTag[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 8242af6d5f8d13d8b13f22dcf02ffc03dd9105a0..80dcf87491ea3450f062084483a6fed66126d428 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
@@ -17,9 +17,9 @@
 
 package org.apache.spark.streaming.api.java
 
-import java.lang.{Long => JLong, Integer => JInt}
+import java.lang.{Integer => JInt}
 import java.io.InputStream
-import java.util.{Map => JMap}
+import java.util.{Map => JMap, List => JList}
 
 import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
@@ -36,10 +36,9 @@ 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, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.dstream._
-import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy}
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -144,7 +143,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     zkQuorum: String,
     groupId: String,
     topics: JMap[String, JInt])
-  : JavaDStream[String] = {
+  : JavaPairDStream[String, String] = {
     implicit val cmt: ClassTag[String] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
     ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
@@ -166,7 +165,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     groupId: String,
     topics: JMap[String, JInt],
     storageLevel: StorageLevel)
-  : JavaDStream[String] = {
+  : JavaPairDStream[String, String] = {
     implicit val cmt: ClassTag[String] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
     ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
@@ -175,25 +174,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: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]]
-    ssc.kafkaStream[T, D](
+  : JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassTag[K] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val valueCmt: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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)
@@ -306,7 +314,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
     implicit val cmf: ClassTag[F] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[F]]
-    ssc.fileStream[K, V, F](directory);
+    ssc.fileStream[K, V, F](directory)
   }
 
   /**
@@ -588,6 +596,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: ClassTag[T] = first.classTag
+    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: ClassTag[(K, V)] = first.classTag
+    implicit val kcm: ClassTag[K] = first.kManifest
+    implicit val vcm: ClassTag[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: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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: ClassTag[K] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val cmv: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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 16c1567355850e53857d37643043f4c3d711ce8d..0000000000000000000000000000000000000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
+++ /dev/null
@@ -1,59 +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}
-import scala.reflect.ClassTag
-
-private[streaming]
-class CoGroupedDStream[K : ClassTag](
-    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/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
index b72ab90e605c0e2d3c8366ec6f7d5e757122513f..60d79175f1e8a32ca64806342761e38018fbd57c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
@@ -138,8 +138,8 @@ class FlumeReceiver(
 
   protected override def onStart() {
     val responder = new SpecificResponder(
-      classOf[AvroSourceProtocol], new FlumeEventServer(this));
-    val server = new NettyServer(responder, new InetSocketAddress(host, port));
+      classOf[AvroSourceProtocol], new FlumeEventServer(this))
+    val server = new NettyServer(responder, new InetSocketAddress(host, port))
     blockGenerator.start()
     server.start()
     logInfo("Flume receiver started")
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 96134868ccfa59a50ddd4af9bf5299e949af49f3..526f5564c733d48161bf49563a6f920e4e6a2442 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._
 import scala.reflect.ClassTag
 
 /**
@@ -46,25 +42,32 @@ import scala.reflect.ClassTag
  * @param storageLevel RDD storage level.
  */
 private[streaming]
-class KafkaInputDStream[T: ClassTag, D <: Decoder[_]: Manifest](
+class KafkaInputDStream[
+  K: ClassTag,
+  V: ClassTag,
+  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: ClassTag, D <: Decoder[_]: Manifest](
-  kafkaParams: Map[String, String],
-  topics: Map[String, Int],
-  storageLevel: StorageLevel
+class KafkaReceiver[
+  K: ClassTag,
+  V: ClassTag,
+  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,35 @@ class KafkaReceiver[T: ClassTag, 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"))
     }
 
+    val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[K]]
+    val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[V]]
+
     // Create Threads for each Topic/Message Stream we are listening
-    val decoder = manifest[D].runtimeClass.newInstance.asInstanceOf[Decoder[T]]
-    val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder)
+    val topicMessageStreams = consumerConnector.createMessageStreams(
+      topics, keyDecoder, valueDecoder)
+
 
     // Start the messages handler for each partition
     topicMessageStreams.values.foreach { streams =>
@@ -112,11 +123,12 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest](
   }
 
   // Handles Kafka Messages
-  private class MessageHandler[T: ClassTag](stream: KafkaStream[T]) extends Runnable {
+  private class MessageHandler[K: ClassTag, V: ClassTag](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..ef4a7375685d36ea3205dd73f07f46730bb79842
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
@@ -0,0 +1,110 @@
+/*
+ * 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._
+import scala.reflect.ClassTag
+
+/**
+ * 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: ClassTag](
+  @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/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 394a39fbb0af427274ba39c0c9765f79e8773806..d5ae8aef9278a79630474c2710718e11af26bdc7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -32,7 +32,7 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
 import org.apache.spark.streaming._
 import org.apache.spark.{Logging, SparkEnv}
 import org.apache.spark.rdd.{RDD, BlockRDD}
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
 
 /**
  * Abstract class for defining any InputDStream that has to start a receiver on worker
@@ -70,7 +70,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte
       val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
       Some(new BlockRDD[T](ssc.sc, blockIds))
     } else {
-      Some(new BlockRDD[T](ssc.sc, Array[String]()))
+      Some(new BlockRDD[T](ssc.sc, Array[BlockId]()))
     }
   }
 }
@@ -78,7 +78,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte
 
 private[streaming] sealed trait NetworkReceiverMessage
 private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage
-private[streaming] case class ReportBlock(blockId: String, metadata: Any) extends NetworkReceiverMessage
+private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) extends NetworkReceiverMessage
 private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage
 
 /**
@@ -159,7 +159,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
   /**
    * Pushes a block (as an ArrayBuffer filled with data) into the block manager.
    */
-  def pushBlock(blockId: String, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) {
+  def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) {
     env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level)
     actor ! ReportBlock(blockId, metadata)
   }
@@ -167,7 +167,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
   /**
    * Pushes a block (as bytes) into the block manager.
    */
-  def pushBlock(blockId: String, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
+  def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
     env.blockManager.putBytes(blockId, bytes, level)
     actor ! ReportBlock(blockId, metadata)
   }
@@ -178,7 +178,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
     val ip = System.getProperty("spark.driver.host", "localhost")
     val port = System.getProperty("spark.driver.port", "7077").toInt
     val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
-    val tracker = env.actorSystem.actorFor(url)
+    val tracker = env.actorSystem.actorSelection(url)
     val timeout = 5.seconds
 
     override def preStart() {
@@ -210,7 +210,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
   class BlockGenerator(storageLevel: StorageLevel)
     extends Serializable with Logging {
 
-    case class Block(id: String, buffer: ArrayBuffer[T], metadata: Any = null)
+    case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
 
     val clock = new SystemClock()
     val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong
@@ -233,16 +233,16 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
       logInfo("Data handler stopped")
     }
 
-    def += (obj: T) {
+    def += (obj: T): Unit = synchronized {
       currentBuffer += obj
     }
 
-    private def updateCurrentBuffer(time: Long) {
+    private def updateCurrentBuffer(time: Long): Unit = synchronized {
       try {
         val newBlockBuffer = currentBuffer
         currentBuffer = new ArrayBuffer[T]
         if (newBlockBuffer.size > 0) {
-          val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval)
+          val blockId = StreamBlockId(NetworkReceiver.this.streamId, time - blockInterval)
           val newBlock = new Block(blockId, newBlockBuffer)
           blocksForPushing.add(newBlock)
         }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
index a4746f06ada33b43752d9c017ef1e4b695cea102..dea0f26f908fbb8327168afb4dd53f2212b81475 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.streaming.dstream
 
 import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{StorageLevel, StreamBlockId}
 import org.apache.spark.streaming.StreamingContext
 
 import scala.reflect.ClassTag
@@ -73,7 +73,7 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel)
         var nextBlockNumber = 0
         while (true) {
           val buffer = queue.take()
-          val blockId = "input-" + streamId + "-" + nextBlockNumber
+          val blockId = StreamBlockId(streamId, nextBlockNumber)
           nextBlockNumber += 1
           pushBlock(blockId, buffer, null, storageLevel)
         }
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 73e1ddf7a473205c5c25bed58b3edd0957069a64..aeea060df7161fe33206a40e488a780bb1dedd9d 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
@@ -22,16 +22,22 @@ import org.apache.spark.streaming.{Duration, DStream, Time}
 import scala.reflect.ClassTag
 
 private[streaming]
-class TransformedDStream[T: ClassTag, U: ClassTag] (
-    parent: DStream[T],
-    transformFunc: (RDD[T], Time) => RDD[U]
-  ) extends DStream[U](parent.ssc) {
+class TransformedDStream[U: ClassTag] (
+    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/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
index ee087a1cf0b990f184a3cd5f0dcaa1c89327ab14..fdf5371a8958708c73253c479406a71b7068c986 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
@@ -25,7 +25,7 @@ import akka.actor.SupervisorStrategy._
 import scala.concurrent.duration._
 import scala.reflect.ClassTag
 
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{StorageLevel, StreamBlockId}
 import org.apache.spark.streaming.dstream.NetworkReceiver
 
 import java.util.concurrent.atomic.AtomicInteger
@@ -160,7 +160,7 @@ private[streaming] class ActorReceiver[T: ClassTag](
   protected def pushBlock(iter: Iterator[T]) {
     val buffer = new ArrayBuffer[T]
     buffer ++= iter
-    pushBlock("block-" + streamId + "-" + System.nanoTime(), buffer, null, storageLevel)
+    pushBlock(StreamBlockId(streamId, System.nanoTime()), buffer, null, storageLevel)
   }
 
   protected def onStart() = {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
index ce8c56fa8a4e3b0f41101c351aa70d1240da7d2a..f164d516b0b3f8da846ac243f17cf4efdf02a30d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
@@ -17,14 +17,14 @@
 
 package org.apache.spark.streaming.receivers
 
+import scala.reflect.ClassTag
+
 import akka.actor.Actor
 import akka.util.ByteString
 import akka.zeromq._
 
 import org.apache.spark.Logging
 
-import scala.reflect.ClassTag
-
 /**
  * A receiver to subscribe to ZeroMQ stream.
  */
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 076fb53fa1b546d110af5bbbd5e3da71864f0d10..daeb99f5b7bf32f24f0db3079768a15876791e23 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -25,6 +25,7 @@ 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;
@@ -185,6 +186,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(
@@ -225,7 +259,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);
   }
@@ -294,8 +328,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();
@@ -322,23 +356,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(
@@ -1101,7 +1447,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);
   }
@@ -1144,7 +1490,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);
   }
@@ -1222,14 +1599,20 @@ public class JavaAPISuite implements Serializable {
   @Test
   public void testKafkaStream() {
     HashMap<String, Integer> topics = Maps.newHashMap();
-    JavaDStream<String> test1 = ssc.kafkaStream("localhost:12345", "group", topics);
-    JavaDStream<String> 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<String> 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 d5cdad4998c792a9766d580f98e00bc465ccc0b8..42ab9590d6df5f797e55e195ef9df12158ca02f7 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -35,9 +35,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: ClassTag[T] =
@@ -52,12 +52,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: ClassTag[T] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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)
   }
 
@@ -65,9 +64,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: ClassTag[V] =
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
     val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
@@ -75,6 +76,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: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[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 07de51bebbc5d3c070df08422ebce78268463905..e81287b44e51f2d02b38dec9dfb2088bc48c773f 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -372,7 +372,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..7dc82decefd6e0ac3cb69acf51d95840b2aeef55 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -23,15 +23,15 @@ import akka.actor.IOManager
 import akka.actor.Props
 import akka.util.ByteString
 
-import dstream.SparkFlumeEvent
+import org.apache.spark.streaming.dstream.{NetworkReceiver, SparkFlumeEvent}
 import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
 import java.io.{File, BufferedWriter, OutputStreamWriter}
-import java.util.concurrent.{TimeUnit, ArrayBlockingQueue}
+import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue}
 import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
 import util.ManualClock
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.receivers.Receiver
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, Logging}
 import scala.util.Random
 import org.apache.commons.io.FileUtils
 import org.scalatest.BeforeAndAfter
@@ -44,6 +44,7 @@ import java.nio.ByteBuffer
 import collection.JavaConversions._
 import java.nio.charset.Charset
 import com.google.common.io.Files
+import java.util.concurrent.atomic.AtomicInteger
 
 class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
 
@@ -61,7 +62,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     System.clearProperty("spark.hostPort")
   }
 
-
   test("socket input stream") {
     // Start the server
     val testServer = new TestServer()
@@ -124,9 +124,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
     val input = Seq(1, 2, 3, 4, 5)
     Thread.sleep(1000)
-    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort));
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
     val client = SpecificRequestor.getClient(
-      classOf[AvroSourceProtocol], transceiver);
+      classOf[AvroSourceProtocol], transceiver)
 
     for (i <- 0 until input.size) {
       val event = new AvroFlumeEvent
@@ -268,13 +268,56 @@ 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)
+  }
+
+  test("multi-thread receiver") {
+    // set up the test receiver
+    val numThreads = 10
+    val numRecordsPerThread = 1000
+    val numTotalRecords = numThreads * numRecordsPerThread
+    val testReceiver = new MultiThreadTestReceiver(numThreads, numRecordsPerThread)
+    MultiThreadTestReceiver.haveAllThreadsFinished = false
+
+    // set up the network stream using the test receiver
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val networkStream = ssc.networkStream[Int](testReceiver)
+    val countStream = networkStream.count
+    val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]]
+    val outputStream = new TestOutputStream(countStream, outputBuffer)
+    def output = outputBuffer.flatMap(x => x)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    // Let the data from the receiver be received
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val startTime = System.currentTimeMillis()
+    while((!MultiThreadTestReceiver.haveAllThreadsFinished || output.sum < numTotalRecords) &&
+      System.currentTimeMillis() - startTime < 5000) {
+      Thread.sleep(100)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+    Thread.sleep(1000)
+    logInfo("Stopping context")
+    ssc.stop()
+
+    // Verify whether data received was as expected
+    logInfo("--------------------------------")
+    logInfo("output.size = " + outputBuffer.size)
+    logInfo("output")
+    outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("--------------------------------")
+    assert(output.sum === numTotalRecords)
   }
 }
 
 
-/** This is server to test the network input stream */
+/** This is a server to test the network input stream */
 class TestServer() extends Logging {
 
   val queue = new ArrayBlockingQueue[String](100)
@@ -336,6 +379,7 @@ object TestServer {
   }
 }
 
+/** This is an actor for testing actor input stream */
 class TestActor(port: Int) extends Actor with Receiver {
 
   def bytesToString(byteString: ByteString) = byteString.utf8String
@@ -347,3 +391,36 @@ class TestActor(port: Int) extends Actor with Receiver {
       pushBlock(bytesToString(bytes))
   }
 }
+
+/** This is a receiver to test multiple threads inserting data using block generator */
+class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int)
+  extends NetworkReceiver[Int] {
+  lazy val executorPool = Executors.newFixedThreadPool(numThreads)
+  lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY)
+  lazy val finishCount = new AtomicInteger(0)
+
+  protected def onStart() {
+    blockGenerator.start()
+    (1 to numThreads).map(threadId => {
+      val runnable = new Runnable {
+        def run() {
+          (1 to numRecordsPerThread).foreach(i =>
+            blockGenerator += (threadId * numRecordsPerThread + i) )
+          if (finishCount.incrementAndGet == numThreads) {
+            MultiThreadTestReceiver.haveAllThreadsFinished = true
+          }
+          logInfo("Finished thread " + threadId)
+        }
+      }
+      executorPool.submit(runnable)
+    })
+  }
+
+  protected def onStop() {
+    executorPool.shutdown()
+  }
+}
+
+object MultiThreadTestReceiver {
+  var haveAllThreadsFinished = false
+}
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 c91f9ba46d04da6daae6fcbaa2a7916debdfc8e0..2f34e812a1009df2ccc68f3be07c8426b090b080 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -61,8 +61,11 @@ class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]],
 /**
  * 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: ClassTag](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
+class TestOutputStream[T: ClassTag](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
@@ -76,6 +79,30 @@ class TestOutputStream[T: ClassTag](parent: DStream[T], val output: ArrayBuffer[
   }
 }
 
+/**
+ * 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: ClassTag](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.
@@ -109,7 +136,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
    */
   def setupStreams[U: ClassTag, V: ClassTag](
       input: Seq[Seq[U]],
-      operation: DStream[U] => DStream[V]
+      operation: DStream[U] => DStream[V],
+      numPartitions: Int = numInputPartitions
     ): StreamingContext = {
 
     // Create StreamingContext
@@ -119,9 +147,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
@@ -147,7 +176,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)
@@ -158,18 +188,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: ClassTag](
       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: ClassTag](
+      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 {
@@ -203,7 +252,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
 
       Thread.sleep(500) // Give some time for the forgetting old RDDs to complete
     } catch {
-      case e: Exception => e.printStackTrace(); throw e;
+      case e: Exception => {e.printStackTrace(); throw e}
     } finally {
       ssc.stop()
     }
diff --git a/tools/pom.xml b/tools/pom.xml
index db87b54deca5529c9521027576a91c7fac18af98..2bad494fc5bd8c1639edabfba7e68b6dc2ca5bfe 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -25,7 +25,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-tools_${scala-short.version}</artifactId>
+  <artifactId>spark-tools_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Tools</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -33,24 +33,24 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala-short.version}</artifactId>
+      <artifactId>spark-streaming_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala-short.version}</artifactId>
+      <artifactId>scalatest_2.10</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
 
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
index f824c472aee62bff4b985b1aecf1ca81d0ca109c..f670f65bf5b38ca6c56703cc955c2f2b9236a751 100644
--- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
@@ -199,7 +199,7 @@ object JavaAPICompletenessChecker {
 
   private def toJavaMethod(method: SparkMethod): SparkMethod = {
     val params = method.parameters
-      .filterNot(_.name == "scala.reflect.ClassManifest")
+      .filterNot(_.name == "scala.reflect.ClassTag")
       .map(toJavaType(_, isReturnType = false))
     SparkMethod(method.name, toJavaType(method.returnType, isReturnType = true), params)
   }
@@ -212,7 +212,7 @@ object JavaAPICompletenessChecker {
     // internal Spark components.
     val excludedNames = Seq(
       "org.apache.spark.rdd.RDD.origin",
-      "org.apache.spark.rdd.RDD.elementClassManifest",
+      "org.apache.spark.rdd.RDD.elementClassTag",
       "org.apache.spark.rdd.RDD.checkpointData",
       "org.apache.spark.rdd.RDD.partitioner",
       "org.apache.spark.rdd.RDD.partitions",
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 7770cbb0cc678d6d1e09be689562b51574fe34bb..d18ac3736d707a421d34bfbdaa572eeae6b1c15d 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -25,7 +25,7 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-yarn_${scala-short.version}</artifactId>
+  <artifactId>spark-yarn_2.10</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project YARN Support</name>
   <url>http://spark.incubator.apache.org/</url>
@@ -33,7 +33,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala-short.version}</artifactId>
+      <artifactId>spark-core_2.10</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -61,11 +61,21 @@
       <groupId>org.apache.avro</groupId>
       <artifactId>avro-ipc</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_2.10</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
-    <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
+    <outputDirectory>target/scala-2.10/classes</outputDirectory>
+    <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
@@ -106,6 +116,46 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>test</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <exportAntProperties>true</exportAntProperties>
+              <tasks>
+                <property name="spark.classpath" refid="maven.test.classpath" />
+                <property environment="env" />
+                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">  
+                  <condition>
+                    <not>
+                      <or>
+                        <isset property="env.SCALA_HOME" />
+                        <isset property="env.SCALA_LIBRARY_PATH" />
+                      </or>
+                    </not>
+                  </condition>
+                </fail>
+              </tasks>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+        <configuration>
+          <environmentVariables>
+            <SPARK_HOME>${basedir}/..</SPARK_HOME>
+            <SPARK_TESTING>1</SPARK_TESTING>
+            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
+          </environmentVariables>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 </project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 858b58d338d71491b21d6098f84be8d34d0491f5..240ed8b32ae93b7099b7d83adb2fcbf600a444b8 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -17,51 +17,67 @@
 
 package org.apache.spark.deploy.yarn
 
+import java.io.IOException
 import java.net.Socket
 import java.util.concurrent.CopyOnWriteArrayList
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+
+import scala.collection.JavaConversions._
+
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.util.ShutdownHookManager
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-import scala.collection.JavaConversions._
+
 import org.apache.spark.{SparkContext, Logging}
 import org.apache.spark.util.Utils
-import org.apache.hadoop.security.UserGroupInformation
-import java.security.PrivilegedExceptionAction
+
 
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
 
   def this(args: ApplicationMasterArguments) = this(args, new Configuration())
   
   private var rpc: YarnRPC = YarnRPC.create(conf)
-  private var resourceManager: AMRMProtocol = null
-  private var appAttemptId: ApplicationAttemptId = null
-  private var userThread: Thread = null
+  private var resourceManager: AMRMProtocol = _
+  private var appAttemptId: ApplicationAttemptId = _
+  private var userThread: Thread = _
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-
-  private var yarnAllocator: YarnAllocationHandler = null
-  private var isFinished:Boolean = false
-  private var uiAddress: String = ""
-
+  private val fs = FileSystem.get(yarnConf)
+
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var isFinished: Boolean = false
+  private var uiAddress: String = _
+  private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
+    YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
+  private var isLastAMRetry: Boolean = true
+  // default to numWorkers * 2, with minimum of 3
+  private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
+    math.max(args.numWorkers * 2, 3).toString()).toInt
 
   def run() {
-    // setup the directories so things go to yarn approved directories rather
-    // then user specified and /tmp
+    // Setup the directories so things go to yarn approved directories rather
+    // then user specified and /tmp.
     System.setProperty("spark.local.dir", getLocalDirs())
+
+    // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using.
+    ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
     
     appAttemptId = getApplicationAttemptId()
+    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
     resourceManager = registerWithResourceManager()
 
     // Workaround until hadoop moves to something which has
     // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
-    // ignore result
+    // ignore result.
     // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times
-    // Hence args.workerCores = numCore disabled above. Any better option ?
+    // Hence args.workerCores = numCore disabled above. Any better option?
 
     // Compute number of threads for akka
     //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
@@ -87,7 +103,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     waitForSparkContextInitialized()
 
-    // do this after spark master is up and SparkContext is created so that we can register UI Url
+    // Do this after spark master is up and SparkContext is created so that we can register UI Url
     val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
     
     // Allocate all containers
@@ -106,12 +122,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
     val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
-      .getOrElse(""))
+        .getOrElse(""))
 
     if (localDirs.isEmpty()) {
       throw new Exception("Yarn Local dirs can't be empty")
     }
-    return localDirs
+    localDirs
   }
   
   private def getApplicationAttemptId(): ApplicationAttemptId = {
@@ -120,7 +136,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     val containerId = ConverterUtils.toContainerId(containerIdString)
     val appAttemptId = containerId.getApplicationAttemptId()
     logInfo("ApplicationAttemptId: " + appAttemptId)
-    return appAttemptId
+    appAttemptId
   }
   
   private def registerWithResourceManager(): AMRMProtocol = {
@@ -128,7 +144,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       YarnConfiguration.RM_SCHEDULER_ADDRESS,
       YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
     logInfo("Connecting to ResourceManager at " + rmAddress)
-    return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+    rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
   }
   
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
@@ -136,12 +152,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
       .asInstanceOf[RegisterApplicationMasterRequest]
     appMasterRequest.setApplicationAttemptId(appAttemptId)
-    // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. 
+    // Setting this to master host,port - so that the ApplicationReport at client has some
+    // sensible info. 
     // Users can then monitor stderr/stdout on that node if required.
     appMasterRequest.setHost(Utils.localHostName())
     appMasterRequest.setRpcPort(0)
     appMasterRequest.setTrackingUrl(uiAddress)
-    return resourceManager.registerApplicationMaster(appMasterRequest)
+    resourceManager.registerApplicationMaster(appMasterRequest)
   }
   
   private def waitForSparkMaster() {
@@ -155,34 +172,40 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       try {
         val socket = new Socket(driverHost, driverPort.toInt)
         socket.close()
-        logInfo("Driver now available: " + driverHost + ":" + driverPort)
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
         driverUp = true
       } catch {
-        case e: Exception =>
-          logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying")
-        Thread.sleep(100)
-        tries = tries + 1
+        case e: Exception => {
+          logWarning("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+          Thread.sleep(100)
+          tries = tries + 1
+        }
       }
     }
   }
 
   private def startUserClass(): Thread  = {
     logInfo("Starting the user JAR in a separate Thread")
-    val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader)
-      .getMethod("main", classOf[Array[String]])
+    val mainMethod = Class.forName(
+      args.userClass,
+      false /* initialize */,
+      Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
         var successed = false
         try {
           // Copy
-          var mainArgs: Array[String] = new Array[String](args.userArgs.size())
-          args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
+          var mainArgs: Array[String] = new Array[String](args.userArgs.size)
+          args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size)
           mainMethod.invoke(null, mainArgs)
           // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR
           // userThread will stop here unless it has uncaught exception thrown out
           // It need shutdown hook to set SUCCEEDED
           successed = true
         } finally {
+          logDebug("finishing main")
+          isLastAMRetry = true
           if (successed) {
             ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           } else {
@@ -192,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       }
     }
     t.start()
-    return t
+    t
   }
 
   // this need to happen before allocateWorkers
@@ -214,12 +237,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
         if (null != sparkContext) {
           uiAddress = sparkContext.ui.appUIAddress
-          this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, 
-                                               sparkContext.preferredNodeLocationData) 
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            resourceManager,
+            appAttemptId,
+            args, 
+            sparkContext.preferredNodeLocationData) 
         } else {
-          logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime + 
-                  ", numTries = " + numTries)
-          this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args)
+          logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".
+            format(count * waitTime, numTries))
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            resourceManager,
+            appAttemptId,
+            args)
         }
       }
     } finally {
@@ -229,52 +260,63 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-
-
   private def allocateWorkers() {
     try {
       logInfo("Allocating " + args.numWorkers + " workers.")
       // Wait until all containers have finished
       // TODO: This is a bit ugly. Can we make it nicer?
       // TODO: Handle container failure
-      while(yarnAllocator.getNumWorkersRunning < args.numWorkers &&
-        // If user thread exists, then quit !
-        userThread.isAlive) {
 
-          this.yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
-          ApplicationMaster.incrementAllocatorLoop(1)
-          Thread.sleep(100)
+      // Exists the loop if the user thread exits.
+      while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+        if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+          finishApplicationMaster(FinalApplicationStatus.FAILED,
+            "max number of worker failures reached")
+        }
+        yarnAllocator.allocateContainers(
+          math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+        ApplicationMaster.incrementAllocatorLoop(1)
+        Thread.sleep(100)
       }
     } finally {
-      // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : 
-      // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+      // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
       ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
     }
     logInfo("All workers have launched.")
 
-    // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+    // Launch a progress reporter thread, else the app will get killed after expiration
+    // (def: 10mins) timeout.
+    // TODO(harvey): Verify the timeout
     if (userThread.isAlive) {
-      // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
-
+      // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
       val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
-      // must be <= timeoutInterval/ 2.
-      // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
-      // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
-      val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+
+      // we want to be reasonably responsive without causing too many requests to RM.
+      val schedulerInterval =
+        System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+
+      // must be <= timeoutInterval / 2.
+      val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
       launchReporterThread(interval)
     }
   }
 
-  // TODO: We might want to extend this to allocate more containers in case they die !
   private def launchReporterThread(_sleepTime: Long): Thread = {
     val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
         while (userThread.isAlive) {
+          if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+            finishApplicationMaster(FinalApplicationStatus.FAILED,
+              "max number of worker failures reached")
+          }
           val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
           if (missingWorkerCount > 0) {
-            logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
             yarnAllocator.allocateContainers(missingWorkerCount)
           }
           else sendProgress()
@@ -282,16 +324,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
         }
       }
     }
-    // setting to daemon status, though this is usually not a good idea.
+    // Setting to daemon status, though this is usually not a good idea.
     t.setDaemon(true)
     t.start()
     logInfo("Started progress reporter thread - sleep time : " + sleepTime)
-    return t
+    t
   }
 
   private def sendProgress() {
     logDebug("Sending progress")
-    // simulated with an allocate request with no nodes requested ...
+    // Simulated with an allocate request with no nodes requested ...
     yarnAllocator.allocateContainers(0)
   }
 
@@ -310,8 +352,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
   */
 
-  def finishApplicationMaster(status: FinalApplicationStatus) {
-
+  def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
     synchronized {
       if (isFinished) {
         return
@@ -324,20 +365,52 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       .asInstanceOf[FinishApplicationMasterRequest]
     finishReq.setAppAttemptId(appAttemptId)
     finishReq.setFinishApplicationStatus(status)
-    // set tracking url to empty since we don't have a history server
+    finishReq.setDiagnostics(diagnostics)
+    // Set tracking url to empty since we don't have a history server.
     finishReq.setTrackingUrl("")
     resourceManager.finishApplicationMaster(finishReq)
+  }
 
+  /**
+   * Clean up the staging directory. 
+   */
+  private def cleanupStagingDir() { 
+    var stagingDirPath: Path = null
+    try {
+      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      if (!preserveFiles) {
+        stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
+        if (stagingDirPath == null) {
+          logError("Staging directory is null")
+          return
+        }
+        logInfo("Deleting staging directory " + stagingDirPath)
+        fs.delete(stagingDirPath, true)
+      }
+    } catch {
+      case ioe: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
+    }
+  }
+
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
+
+    def run() {
+      logInfo("AppMaster received a signal.")
+      // we need to clean up staging dir before HDFS is shut down
+      // make sure we don't delete it until this is the last AM
+      if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
+    }
   }
- 
 }
 
 object ApplicationMaster {
-  // number of times to wait for the allocator loop to complete.
-  // each loop iteration waits for 100ms, so maximum of 3 seconds.
+  // Number of times to wait for the allocator loop to complete.
+  // Each loop iteration waits for 100ms, so maximum of 3 seconds.
   // This is to ensure that we have reasonable number of containers before we start
-  // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be optimal as more 
-  // containers are available. Might need to handle this better.
+  // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+  // optimal as more containers are available. Might need to handle this better.
   private val ALLOCATOR_LOOP_WAIT_COUNT = 30
   def incrementAllocatorLoop(by: Int) {
     val count = yarnAllocatorLoop.getAndAdd(by)
@@ -355,7 +428,8 @@ object ApplicationMaster {
     applicationMasters.add(master)
   }
 
-  val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null)
+  val sparkContextRef: AtomicReference[SparkContext] =
+    new AtomicReference[SparkContext](null /* initialValue */)
   val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
 
   def sparkContextInitialized(sc: SparkContext): Boolean = {
@@ -365,17 +439,21 @@ object ApplicationMaster {
       sparkContextRef.notifyAll()
     }
 
-    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit
-    // Should not really have to do this, but it helps yarn to evict resources earlier.
-    // not to mention, prevent Client declaring failure even though we exit'ed properly.
+    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do
+    // System.exit.
+    // Should not really have to do this, but it helps YARN to evict resources earlier.
+    // Not to mention, prevent the Client from declaring failure even though we exited properly.
+    // Note that this will unfortunately not properly clean up the staging files because it gets
+    // called too late, after the filesystem is already shutdown.
     if (modified) {
       Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
-        // This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run'
+        // This is not only logs, but also ensures that log system is initialized for this instance
+        // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
         override def run() { 
           logInfo("Invoking sc stop from shutdown hook") 
           sc.stop() 
-          // best case ...
+          // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
@@ -383,7 +461,7 @@ object ApplicationMaster {
       } )
     }
 
-    // Wait for initialization to complete and atleast 'some' nodes can get allocated
+    // Wait for initialization to complete and atleast 'some' nodes can get allocated.
     yarnAllocatorLoop.synchronized {
       while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
         yarnAllocatorLoop.wait(1000L)
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 076dd3c9b023695e3d22ff9daa890e3c843accd0..79dd03806523bcea066fcafd05160fe732032d22 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
@@ -17,37 +17,54 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.net.{InetSocketAddress, URI}
+import java.net.{InetAddress, UnknownHostException, URI}
 import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.mapred.Master
 import org.apache.hadoop.net.NetUtils
-import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.client.YarnClientImpl
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
+import org.apache.hadoop.yarn.util.{Apps, Records}
+
 import org.apache.spark.Logging 
 import org.apache.spark.util.Utils
-import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils}
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.spark.deploy.SparkHadoopUtil
 
+
 class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
-  
+
   def this(args: ClientArguments) = this(new Configuration(), args)
-  
+
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-  val credentials = UserGroupInformation.getCurrentUser().getCredentials();
-  
-  def run() {
+  val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+  private val SPARK_STAGING: String = ".sparkStaging"
+  private val distCacheMgr = new ClientDistributedCacheManager()
+
+  // Staging directory is private! -> rwx--------
+  val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
+
+  // App files are world-wide readable and owner writable -> rw-r--r--
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) 
+
+  // for client user who want to monitor app status by itself.
+  def runApp() = {
+    validateArgs()
+
     init(yarnConf)
     start()
     logClusterResourceDetails()
@@ -57,8 +74,9 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     verifyClusterResources(newApp)
     val appContext = createApplicationSubmissionContext(appId)
-    val localResources = prepareLocalResources(appId, "spark")
-    val env = setupLaunchEnv(localResources)
+    val appStagingDir = getAppStagingDir(appId)
+    val localResources = prepareLocalResources(appStagingDir)
+    val env = setupLaunchEnv(localResources, appStagingDir)
     val amContainer = createContainerLaunchContext(newApp, localResources, env)
 
     appContext.setQueue(args.amQueue)
@@ -66,30 +84,59 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
 
     submitApp(appContext)
-    
+    appId
+  }
+
+  def run() {
+    val appId = runApp()
     monitorApplication(appId)
     System.exit(0)
   }
-  
+
+  def validateArgs() = {
+    Map(
+      (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+      (args.userJar == null) -> "Error: You must specify a user jar!",
+      (args.userClass == null) -> "Error: You must specify a user class!",
+      (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!",
+      (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " +
+        "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+      (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
+        "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
+    ).foreach { case(cond, errStr) => 
+      if (cond) {
+        logError(errStr)
+        args.printUsageAndExit(1)
+      }
+    }
+  }
+
+  def getAppStagingDir(appId: ApplicationId): String = {
+    SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
+  }
 
   def logClusterResourceDetails() {
     val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
-    logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers)
+    logInfo("Got Cluster metric info from ASM, numNodeManagers = " +
+      clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity +
-      ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size +
-      ", queueChildQueueCount=" + queueInfo.getChildQueues.size)
+    logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
+      queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+        queueInfo.getQueueName,
+        queueInfo.getCurrentCapacity,
+        queueInfo.getMaximumCapacity,
+        queueInfo.getApplications.size,
+        queueInfo.getChildQueues.size))
   }
 
-  
   def verifyClusterResources(app: GetNewApplicationResponse) = { 
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
-    
-    // if we have requested more then the clusters max for a single resource then exit.
+
+    // If we have requested more then the clusters max for a single resource then exit.
     if (args.workerMemory > maxMem) {
-      logError("the worker size is to large to run on this cluster " + args.workerMemory);
+      logError("the worker size is to large to run on this cluster " + args.workerMemory)
       System.exit(1)
     }
     val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
@@ -98,10 +145,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       System.exit(1)
     }
 
-    // We could add checks to make sure the entire cluster has enough resources but that involves getting
-    // all the node reports and computing ourselves 
+    // We could add checks to make sure the entire cluster has enough resources but that involves
+    // getting all the node reports and computing ourselves 
   }
-  
+
   def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
     logInfo("Setting up application submission context for ASM")
     val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
@@ -109,89 +156,170 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     appContext.setApplicationName(args.appName)
     return appContext
   }
-  
-  def prepareLocalResources(appId: ApplicationId, appName: String): HashMap[String, LocalResource] = {
+
+  /** See if two file systems are the same or not. */
+  private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+    val srcUri = srcFs.getUri()
+    val dstUri = destFs.getUri()
+    if (srcUri.getScheme() == null) {
+      return false
+    }
+    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+      return false
+    }
+    var srcHost = srcUri.getHost()
+    var dstHost = dstUri.getHost()
+    if ((srcHost != null) && (dstHost != null)) {
+      try {
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+      } catch {
+        case e: UnknownHostException =>
+          return false
+      }
+      if (!srcHost.equals(dstHost)) {
+        return false
+      }
+    } else if (srcHost == null && dstHost != null) {
+      return false
+    } else if (srcHost != null && dstHost == null) {
+      return false
+    }
+    //check for ports
+    if (srcUri.getPort() != dstUri.getPort()) {
+      return false
+    }
+    return true
+  }
+
+  /** Copy the file into HDFS if needed. */
+  private def copyRemoteFile(
+      dstDir: Path,
+      originalPath: Path,
+      replication: Short,
+      setPerms: Boolean = false): Path = {
+    val fs = FileSystem.get(conf)
+    val remoteFs = originalPath.getFileSystem(conf)
+    var newPath = originalPath
+    if (! compareFs(remoteFs, fs)) {
+      newPath = new Path(dstDir, originalPath.getName())
+      logInfo("Uploading " + originalPath + " to " + newPath)
+      FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
+      fs.setReplication(newPath, replication)
+      if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
+    } 
+    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+    // version shows the specific version in the distributed cache configuration
+    val qualPath = fs.makeQualified(newPath)
+    val fc = FileContext.getFileContext(qualPath.toUri(), conf)
+    val destPath = fc.resolvePath(qualPath)
+    destPath
+  }
+
+  def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
-    val locaResources = HashMap[String, LocalResource]()
-    // Upload Spark and the application JAR to the remote file system
-    // Add them as local resources to the AM
+    // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+    // local resources to the AM.
     val fs = FileSystem.get(conf)
 
-    val delegTokenRenewer = Master.getMasterPrincipal(conf);
+    val delegTokenRenewer = Master.getMasterPrincipal(conf)
     if (UserGroupInformation.isSecurityEnabled()) {
       if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
         logError("Can't get Master Kerberos principal for use as renewer")
         System.exit(1)
       }
     }
+    val dst = new Path(fs.getHomeDirectory(), appStagingDir)
+    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
 
-    Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF"))
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val dstFs = dst.getFileSystem(conf)
+      dstFs.addDelegationTokens(delegTokenRenewer, credentials)
+    }
+    val localResources = HashMap[String, LocalResource]()
+    FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
+
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+    Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, 
+      Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF"))
     .foreach { case(destName, _localPath) =>
       val localPath: String = if (_localPath != null) _localPath.trim() else ""
       if (! localPath.isEmpty()) {
-        val src = new Path(localPath)
-        val pathSuffix = appName + "/" + appId.getId() + destName
-        val dst = new Path(fs.getHomeDirectory(), pathSuffix)
-        logInfo("Uploading " + src + " to " + dst)
-        fs.copyFromLocalFile(false, true, src, dst)
-        val destStatus = fs.getFileStatus(dst)
-
-        // get tokens for anything we upload to hdfs
-        if (UserGroupInformation.isSecurityEnabled()) {
-          fs.addDelegationTokens(delegTokenRenewer, credentials);
+        var localURI = new URI(localPath)
+        // if not specified assume these are in the local filesystem to keep behavior like Hadoop
+        if (localURI.getScheme() == null) {
+          localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
         }
+        val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
+        val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          destName, statCache)
+      }
+    }
+
+    // handle any add jars
+    if ((args.addJars != null) && (!args.addJars.isEmpty())){
+      args.addJars.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache, true)
+      }
+    }
+
+    // handle any distributed cache files
+    if ((args.files != null) && (!args.files.isEmpty())){
+      args.files.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache)
+      }
+    }
 
-        val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-        amJarRsrc.setType(LocalResourceType.FILE)
-        amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
-        amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(dst))
-        amJarRsrc.setTimestamp(destStatus.getModificationTime())
-        amJarRsrc.setSize(destStatus.getLen())
-        locaResources(destName) = amJarRsrc
+    // handle any distributed cache archives
+    if ((args.archives != null) && (!args.archives.isEmpty())) {
+      args.archives.split(',').foreach { case file:String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+          linkname, statCache)
       }
     }
-    UserGroupInformation.getCurrentUser().addCredentials(credentials);
-    return locaResources
+
+    UserGroupInformation.getCurrentUser().addCredentials(credentials)
+    return localResources
   }
-  
-  def setupLaunchEnv(localResources: HashMap[String, LocalResource]): HashMap[String, String] = {
+
+  def setupLaunchEnv(
+      localResources: HashMap[String, LocalResource], 
+      stagingDir: String): HashMap[String, String] = {
     logInfo("Setting up the launch environment")
-    val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null)
+    val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
 
     val env = new HashMap[String, String]()
 
-    // If log4j present, ensure ours overrides all others
-    if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
-
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
-    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() + "://" +
-      localResources("spark.jar").getResource().getFile().toString()
-    env("SPARK_YARN_JAR_TIMESTAMP") =  localResources("spark.jar").getTimestamp().toString()
-    env("SPARK_YARN_JAR_SIZE") =  localResources("spark.jar").getSize().toString()
-
-    env("SPARK_YARN_USERJAR_PATH") =
-      localResources("app.jar").getResource().getScheme.toString() + "://" +
-      localResources("app.jar").getResource().getFile().toString()
-    env("SPARK_YARN_USERJAR_TIMESTAMP") =  localResources("app.jar").getTimestamp().toString()
-    env("SPARK_YARN_USERJAR_SIZE") =  localResources("app.jar").getSize().toString()
-
-    if (log4jConfLocalRes != null) {
-      env("SPARK_YARN_LOG4J_PATH") =
-        log4jConfLocalRes.getResource().getScheme.toString() + "://" + log4jConfLocalRes.getResource().getFile().toString()
-      env("SPARK_YARN_LOG4J_TIMESTAMP") =  log4jConfLocalRes.getTimestamp().toString()
-      env("SPARK_YARN_LOG4J_SIZE") =  log4jConfLocalRes.getSize().toString()
-    }
+    env("SPARK_YARN_STAGING_DIR") = stagingDir
+
+    // Set the environment variables to be passed on to the Workers.
+    distCacheMgr.setDistFilesEnv(env)
+    distCacheMgr.setDistArchivesEnv(env)
 
-    // allow users to specify some environment variables
+    // Allow users to specify some environment variables.
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
 
-    // Add each SPARK-* key to the environment
+    // Add each SPARK-* key to the environment.
     System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
-    return env
+    env
   }
 
   def userArgsToString(clientArgs: ClientArguments): String = {
@@ -201,13 +329,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     for (arg <- args){
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
-
     retval.toString
   }
 
-  def createContainerLaunchContext(newApp: GetNewApplicationResponse,
-                                   localResources: HashMap[String, LocalResource],
-                                   env: HashMap[String, String]): ContainerLaunchContext = {
+  def createContainerLaunchContext(
+      newApp: GetNewApplicationResponse,
+      localResources: HashMap[String, LocalResource],
+      env: HashMap[String, String]): ContainerLaunchContext = {
     logInfo("Setting up container launch context")
     val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
     amContainer.setLocalResources(localResources)
@@ -215,8 +343,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
 
+    // TODO(harvey): This can probably be a val.
     var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
-        (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD
+      ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+        YarnAllocationHandler.MEMORY_OVERHEAD)
 
     // Extra options for the JVM
     var JAVA_OPTS = ""
@@ -227,14 +357,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     JAVA_OPTS += " -Djava.io.tmpdir=" + 
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
-
-    // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
-    // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same
-    // node, spark gc effects all other containers performance (which can also be other spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
-    // limited to subset of cores on a node.
-    if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) {
-      // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out. The context is, default gc for server class machines
+    // end up using all cores to do gc - hence if there are multiple containers in same node,
+    // spark gc effects all other containers performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
+    val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+      java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+    if (useConcurrentAndIncrementalGC) {
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tenant machines
       JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
       JAVA_OPTS += " -XX:+CMSIncrementalMode "
       JAVA_OPTS += " -XX:+CMSIncrementalPacing "
@@ -247,7 +381,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
 
     // Command for the ApplicationMaster
-    var javaCommand = "java";
+    var javaCommand = "java"
     val javaHome = System.getenv("JAVA_HOME")
     if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
@@ -256,7 +390,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val commands = List[String](javaCommand + 
       " -server " +
       JAVA_OPTS +
-      " org.apache.spark.deploy.yarn.ApplicationMaster" +
+      " " + args.amClass +
       " --class " + args.userClass + 
       " --jar " + args.userJar +
       userArgsToString(args) +
@@ -267,28 +401,28 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
     logInfo("Command for the ApplicationMaster: " + commands(0))
     amContainer.setCommands(commands)
-    
+
     val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
-    // Memory for the ApplicationMaster
+    // Memory for the ApplicationMaster.
     capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
     amContainer.setResource(capability)
 
-    // Setup security tokens
+    // Setup security tokens.
     val dob = new DataOutputBuffer()
     credentials.writeTokenStorageToStream(dob)
     amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData()))
 
-    return amContainer
+    amContainer
   }
-  
+
   def submitApp(appContext: ApplicationSubmissionContext) = {
-    // Submit the application to the applications manager
+    // Submit the application to the applications manager.
     logInfo("Submitting application to ASM")
     super.submitApplication(appContext)
   }
-  
+
   def monitorApplication(appId: ApplicationId): Boolean = {  
-    while(true) {
+    while (true) {
       Thread.sleep(1000)
       val report = super.getApplicationReport(appId)
 
@@ -306,26 +440,31 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
         "\t appUser: " + report.getUser()
       )
-      
+
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
       if (state == YarnApplicationState.FINISHED || 
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
-          return true
+        return true
       }
     }
-    return true
+    true
   }
 }
 
 object Client {
+  val SPARK_JAR: String = "spark.jar"
+  val APP_JAR: String = "app.jar"
+  val LOG4J_PROP: String = "log4j.properties"
+
   def main(argStrings: Array[String]) {
     // Set an env variable indicating we are running in YARN mode.
     // Note that anything with SPARK prefix gets propagated to all (remote) processes
     System.setProperty("SPARK_YARN_MODE", "true")
 
     val args = new ClientArguments(argStrings)
+
     new Client(args).run
   }
 
@@ -335,4 +474,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_PROP)
+    }
+    // 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/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index c56dbd99ba4d9767151843159d5d2ca3e273ac3b..b9dbc3fb87a1f4ba4e9a37650bee52ae88e0a16c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -24,6 +24,9 @@ import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
 
 // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
 class ClientArguments(val args: Array[String]) {
+  var addJars: String = null
+  var files: String = null
+  var archives: String = null
   var userJar: String = null
   var userClass: String = null
   var userArgs: Seq[String] = Seq[String]()
@@ -32,6 +35,7 @@ class ClientArguments(val args: Array[String]) {
   var numWorkers = 2
   var amQueue = System.getProperty("QUEUE", "default")
   var amMemory: Int = 512
+  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
   var appName: String = "Spark"
   // TODO
   var inputFormatInfo: List[InputFormatInfo] = null
@@ -59,18 +63,22 @@ class ClientArguments(val args: Array[String]) {
           userArgsBuffer += value
           args = tail
 
-        case ("--master-memory") :: MemoryParam(value) :: tail =>
-          amMemory = value
+        case ("--master-class") :: value :: tail =>
+          amClass = value
           args = tail
 
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
+        case ("--master-memory") :: MemoryParam(value) :: tail =>
+          amMemory = value
           args = tail
 
         case ("--worker-memory") :: MemoryParam(value) :: tail =>
           workerMemory = value
           args = tail
 
+        case ("--num-workers") :: IntParam(value) :: tail =>
+          numWorkers = value
+          args = tail
+
         case ("--worker-cores") :: IntParam(value) :: tail =>
           workerCores = value
           args = tail
@@ -81,6 +89,17 @@ class ClientArguments(val args: Array[String]) {
 
         case ("--name") :: value :: tail =>
           appName = value
+
+        case ("--addJars") :: value :: tail =>
+          addJars = value
+          args = tail
+
+        case ("--files") :: value :: tail =>
+          files = value
+          args = tail
+
+        case ("--archives") :: value :: tail =>
+          archives = value
           args = tail
 
         case Nil =>
@@ -97,7 +116,7 @@ class ClientArguments(val args: Array[String]) {
     inputFormatInfo = inputFormatMap.values.toList
   }
 
-  
+
   def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
     if (unknownParam != null) {
       System.err.println("Unknown/unsupported param " + unknownParam)
@@ -105,18 +124,22 @@ class ClientArguments(val args: Array[String]) {
     System.err.println(
       "Usage: org.apache.spark.deploy.yarn.Client [options] \n" +
       "Options:\n" +
-      "  --jar JAR_PATH       Path to your application's JAR file (required)\n" +
-      "  --class CLASS_NAME   Name of your application's main class (required)\n" +
-      "  --args ARGS          Arguments to be passed to your application's main class.\n" +
-      "                       Mutliple invocations are possible, each will be passed in order.\n" +
-      "  --num-workers NUM    Number of workers to start (Default: 2)\n" +
-      "  --worker-cores NUM   Number of cores for the workers (Default: 1). This is unsused right now.\n" +
-      "  --master-memory MEM  Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
-      "  --worker-memory MEM  Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
-      "  --name NAME          The name of your application (Default: Spark)\n" + 
-      "  --queue QUEUE        The hadoop queue to use for allocation requests (Default: 'default')"
+      "  --jar JAR_PATH             Path to your application's JAR file (required)\n" +
+      "  --class CLASS_NAME         Name of your application's main class (required)\n" +
+      "  --args ARGS                Arguments to be passed to your application's main class.\n" +
+      "                             Mutliple invocations are possible, each will be passed in order.\n" +
+      "  --num-workers NUM          Number of workers to start (Default: 2)\n" +
+      "  --worker-cores NUM         Number of cores for the workers (Default: 1). This is unsused right now.\n" +
+      "  --master-class CLASS_NAME  Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" +
+      "  --master-memory MEM        Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
+      "  --worker-memory MEM        Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
+      "  --name NAME                The name of your application (Default: Spark)\n" +
+      "  --queue QUEUE              The hadoop queue to use for allocation requests (Default: 'default')\n" +
+      "  --addJars jars             Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
+      "  --files files              Comma separated list of files to be distributed with the job.\n" +
+      "  --archives archives        Comma separated list of archives to be distributed with the job."
       )
     System.exit(exitCode)
   }
-  
+
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5f159b073f5372dc9e1f73347f88ae8aa48e0cbb
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -0,0 +1,228 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.URI
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
+
+import org.apache.spark.Logging 
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.LinkedHashMap
+import scala.collection.mutable.Map
+
+
+/** Client side methods to setup the Hadoop distributed cache */
+class ClientDistributedCacheManager() extends Logging {
+  private val distCacheFiles: Map[String, Tuple3[String, String, String]] = 
+    LinkedHashMap[String, Tuple3[String, String, String]]()
+  private val distCacheArchives: Map[String, Tuple3[String, String, String]] = 
+    LinkedHashMap[String, Tuple3[String, String, String]]()
+
+
+  /**
+   * Add a resource to the list of distributed cache resources. This list can
+   * be sent to the ApplicationMaster and possibly the workers so that it can 
+   * be downloaded into the Hadoop distributed cache for use by this application.
+   * Adds the LocalResource to the localResources HashMap passed in and saves 
+   * the stats of the resources to they can be sent to the workers and verified.
+   *
+   * @param fs FileSystem
+   * @param conf Configuration
+   * @param destPath path to the resource
+   * @param localResources localResource hashMap to insert the resource into
+   * @param resourceType LocalResourceType 
+   * @param link link presented in the distributed cache to the destination
+   * @param statCache cache to store the file/directory stats 
+   * @param appMasterOnly Whether to only add the resource to the app master
+   */
+  def addResource(
+      fs: FileSystem,
+      conf: Configuration,
+      destPath: Path, 
+      localResources: HashMap[String, LocalResource],
+      resourceType: LocalResourceType,
+      link: String,
+      statCache: Map[URI, FileStatus],
+      appMasterOnly: Boolean = false) = {
+    val destStatus = fs.getFileStatus(destPath)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(resourceType)
+    val visibility = getVisibility(conf, destPath.toUri(), statCache)
+    amJarRsrc.setVisibility(visibility)
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
+    amJarRsrc.setTimestamp(destStatus.getModificationTime())
+    amJarRsrc.setSize(destStatus.getLen())
+    if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name")
+    localResources(link) = amJarRsrc
+    
+    if (appMasterOnly == false) {
+      val uri = destPath.toUri()
+      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
+      if (resourceType == LocalResourceType.FILE) {
+        distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), 
+          destStatus.getModificationTime().toString(), visibility.name())
+      } else {
+        distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), 
+          destStatus.getModificationTime().toString(), visibility.name())
+      }
+    }
+  }
+
+  /**
+   * Adds the necessary cache file env variables to the env passed in
+   * @param env
+   */
+  def setDistFilesEnv(env: Map[String, String]) = {
+    val (keys, tupleValues) = distCacheFiles.unzip
+    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
+
+    if (keys.size > 0) {
+      env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = 
+        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = 
+        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = 
+        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
+    }
+  }
+
+  /**
+   * Adds the necessary cache archive env variables to the env passed in
+   * @param env
+   */
+  def setDistArchivesEnv(env: Map[String, String]) = {
+    val (keys, tupleValues) = distCacheArchives.unzip
+    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
+
+    if (keys.size > 0) {
+      env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = 
+        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") =
+        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
+      env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = 
+        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
+    }
+  }
+
+  /**
+   * Returns the local resource visibility depending on the cache file permissions
+   * @param conf
+   * @param uri
+   * @param statCache
+   * @return LocalResourceVisibility
+   */
+  def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
+      LocalResourceVisibility = {
+    if (isPublic(conf, uri, statCache)) {
+      return LocalResourceVisibility.PUBLIC 
+    } 
+    return LocalResourceVisibility.PRIVATE
+  }
+
+  /**
+   * Returns a boolean to denote whether a cache file is visible to all(public)
+   * or not
+   * @param conf
+   * @param uri
+   * @param statCache
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
+    val fs = FileSystem.get(uri, conf)
+    val current = new Path(uri.getPath())
+    //the leaf level file should be readable by others
+    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
+      return false
+    }
+    return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
+  }
+
+  /**
+   * Returns true if all ancestors of the specified path have the 'execute'
+   * permission set for all users (i.e. that other users can traverse
+   * the directory heirarchy to the given path)
+   * @param fs
+   * @param path
+   * @param statCache
+   * @return true if all ancestors have the 'execute' permission set for all users
+   */
+  def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, 
+      statCache: Map[URI, FileStatus]): Boolean =  {
+    var current = path
+    while (current != null) {
+      //the subdirs in the path should have execute permissions for others
+      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
+        return false
+      }
+      current = current.getParent()
+    }
+    return true
+  }
+
+  /**
+   * Checks for a given path whether the Other permissions on it 
+   * imply the permission in the passed FsAction
+   * @param fs
+   * @param path
+   * @param action
+   * @param statCache
+   * @return true if the path in the uri is visible to all, false otherwise
+   */
+  def checkPermissionOfOther(fs: FileSystem, path: Path,
+      action: FsAction, statCache: Map[URI, FileStatus]): Boolean = {
+    val status = getFileStatus(fs, path.toUri(), statCache)
+    val perms = status.getPermission()
+    val otherAction = perms.getOtherAction()
+    if (otherAction.implies(action)) {
+      return true
+    }
+    return false
+  }
+
+  /**
+   * Checks to see if the given uri exists in the cache, if it does it 
+   * returns the existing FileStatus, otherwise it stats the uri, stores
+   * it in the cache, and returns the FileStatus.
+   * @param fs
+   * @param uri
+   * @param statCache
+   * @return FileStatus
+   */
+  def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = {
+    val stat = statCache.get(uri) match {
+      case Some(existstat) => existstat
+      case None => 
+        val newStat = fs.getFileStatus(new Path(uri))
+        statCache.put(uri, newStat)
+        newStat
+    }
+    return stat
+  }
+}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000000000000000000000000000000000000..69038844bbe4324d3f28cb1a902c99a7c147b087
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -0,0 +1,243 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.Socket
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+import akka.actor._
+import akka.remote._
+import akka.actor.Terminated
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.scheduler.SplitInfo
+
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+
+  private val rpc: YarnRPC = YarnRPC.create(conf)
+  private var resourceManager: AMRMProtocol = null
+  private var appAttemptId: ApplicationAttemptId = null
+  private var reporterThread: Thread = null
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  private var yarnAllocator: YarnAllocationHandler = null
+  private var driverClosed:Boolean = false
+
+  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
+  var actor: ActorRef = null
+
+  // This actor just working as a monitor to watch on Driver Actor.
+  class MonitorActor(driverUrl: String) extends Actor {
+
+    var driver: ActorSelection = null
+
+    override def preStart() {
+      logInfo("Listen to driver: " + driverUrl)
+      driver = context.actorSelection(driverUrl)
+      driver ! "hello"
+      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+    }
+
+    override def receive = {
+      case x: DisassociatedEvent =>
+        logInfo(s"Driver terminated or disconnected! Shutting down. $x")
+        driverClosed = true
+    }
+  }
+
+  def run() {
+
+    appAttemptId = getApplicationAttemptId()
+    resourceManager = registerWithResourceManager()
+    val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
+
+    // Compute number of threads for akka
+    val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
+
+    if (minimumMemory > 0) {
+      val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+      val numCore = (mem  / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0)
+
+      if (numCore > 0) {
+        // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406
+        // TODO: Uncomment when hadoop is on a version which has this fixed.
+        // args.workerCores = numCore
+      }
+    }
+
+    waitForSparkMaster()
+
+    // Allocate all containers
+    allocateWorkers()
+
+    // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+    // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
+
+    val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+    // must be <= timeoutInterval/ 2.
+    // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
+    // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
+    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+    reporterThread = launchReporterThread(interval)
+
+    // Wait for the reporter thread to Finish.
+    reporterThread.join()
+
+    finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+    actorSystem.shutdown()
+
+    logInfo("Exited")
+    System.exit(0)
+  }
+
+  private def getApplicationAttemptId(): ApplicationAttemptId = {
+    val envs = System.getenv()
+    val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV)
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    return appAttemptId
+  }
+
+  private def registerWithResourceManager(): AMRMProtocol = {
+    val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
+      YarnConfiguration.RM_SCHEDULER_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
+    logInfo("Connecting to ResourceManager at " + rmAddress)
+    return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+  }
+
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
+      .asInstanceOf[RegisterApplicationMasterRequest]
+    appMasterRequest.setApplicationAttemptId(appAttemptId)
+    // Setting this to master host,port - so that the ApplicationReport at client has some sensible info.
+    // Users can then monitor stderr/stdout on that node if required.
+    appMasterRequest.setHost(Utils.localHostName())
+    appMasterRequest.setRpcPort(0)
+    // What do we provide here ? Might make sense to expose something sensible later ?
+    appMasterRequest.setTrackingUrl("")
+    return resourceManager.registerApplicationMaster(appMasterRequest)
+  }
+
+  private def waitForSparkMaster() {
+    logInfo("Waiting for spark driver to be reachable.")
+    var driverUp = false
+    val hostport = args.userArgs(0)
+    val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+    while(!driverUp) {
+      try {
+        val socket = new Socket(driverHost, driverPort)
+        socket.close()
+        logInfo("Master now available: " + driverHost + ":" + driverPort)
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at " + driverHost + ":" + driverPort)
+        Thread.sleep(100)
+      }
+    }
+    System.setProperty("spark.driver.host", driverHost)
+    System.setProperty("spark.driver.port", driverPort.toString)
+
+    val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+      driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+    actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
+  }
+
+
+  private def allocateWorkers() {
+
+    // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
+    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map()
+
+    yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData)
+
+    logInfo("Allocating " + args.numWorkers + " workers.")
+    // Wait until all containers have finished
+    // TODO: This is a bit ugly. Can we make it nicer?
+    // TODO: Handle container failure
+    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+      yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+      Thread.sleep(100)
+    }
+
+    logInfo("All workers have launched.")
+
+  }
+
+  // TODO: We might want to extend this to allocate more containers in case they die !
+  private def launchReporterThread(_sleepTime: Long): Thread = {
+    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+
+    val t = new Thread {
+      override def run() {
+        while (!driverClosed) {
+          val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+            yarnAllocator.allocateContainers(missingWorkerCount)
+          }
+          else sendProgress()
+          Thread.sleep(sleepTime)
+        }
+      }
+    }
+    // setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.start()
+    logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+    return t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // simulated with an allocate request with no nodes requested ...
+    yarnAllocator.allocateContainers(0)
+  }
+
+  def finishApplicationMaster(status: FinalApplicationStatus) {
+
+    logInfo("finish ApplicationMaster with " + status)
+    val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest])
+      .asInstanceOf[FinishApplicationMasterRequest]
+    finishReq.setAppAttemptId(appAttemptId)
+    finishReq.setFinishApplicationStatus(status)
+    resourceManager.finishApplicationMaster(finishReq)
+  }
+
+}
+
+
+object WorkerLauncher {
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new WorkerLauncher(args).run()
+  }
+}
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 a60e8a300786e31df2d7408d34a833b1deb54f28..6a90cc51cfbaf71ef5683703dd84fa6d5b872ccb 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
@@ -21,53 +21,59 @@ import java.net.URI
 import java.nio.ByteBuffer
 import java.security.PrivilegedExceptionAction
 
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
 
 import org.apache.spark.Logging
-import org.apache.spark.util.Utils
 
-class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
-    slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) 
-    extends Runnable with Logging {
-  
+
+class WorkerRunnable(
+    container: Container,
+    conf: Configuration,
+    masterAddress: String,
+    slaveId: String,
+    hostname: String,
+    workerMemory: Int,
+    workerCores: Int) 
+  extends Runnable with Logging {
+
   var rpc: YarnRPC = YarnRPC.create(conf)
   var cm: ContainerManager = null
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-  
+
   def run = {
     logInfo("Starting Worker Container")
     cm = connectToCM
     startContainer
   }
-  
+
   def startContainer = {
     logInfo("Setting up ContainerLaunchContext")
-    
+
     val ctx = Records.newRecord(classOf[ContainerLaunchContext])
       .asInstanceOf[ContainerLaunchContext]
-    
+
     ctx.setContainerId(container.getId())
     ctx.setResource(container.getResource())
     val localResources = prepareLocalResources
     ctx.setLocalResources(localResources)
-    
+
     val env = prepareEnvironment
     ctx.setEnvironment(env)
-    
+
     // Extra options for the JVM
     var JAVA_OPTS = ""
     // Set the JVM memory
@@ -80,17 +86,21 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     JAVA_OPTS += " -Djava.io.tmpdir=" + 
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
-
-    // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
-    // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same
-    // node, spark gc effects all other containers performance (which can also be other spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
-    // limited to subset of cores on a node.
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out.
+    // The context is, default gc for server class machines end up using all cores to do gc - hence
+    // if there are multiple containers in same node, spark gc effects all other containers
+    // performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
 /*
     else {
       // If no java_opts specified, default to using -XX:+CMSIncrementalMode
-      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it.
-      // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines
+      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+      // want to mess with it.
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tennent machines
       // The options are based on
       // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
       JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
@@ -108,7 +118,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     credentials.writeTokenStorageToStream(dob)
     ctx.setContainerTokens(ByteBuffer.wrap(dob.getData()))
 
-    var javaCommand = "java";
+    var javaCommand = "java"
     val javaHome = System.getenv("JAVA_HOME")
     if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
@@ -117,11 +127,13 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     val commands = List[String](javaCommand +
       " -server " +
       // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
-      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state.
-      // 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 ?
+      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+      // an inconsistent state.
+      // 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 + " " +
@@ -130,85 +142,82 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
     logInfo("Setting up worker with commands: " + commands)
     ctx.setCommands(commands)
-    
+
     // Send the start request to the ContainerManager
     val startReq = Records.newRecord(classOf[StartContainerRequest])
     .asInstanceOf[StartContainerRequest]
     startReq.setContainerLaunchContext(ctx)
     cm.startContainer(startReq)
   }
-  
-  
+
+  private def setupDistributedCache(
+      file: String,
+      rtype: LocalResourceType,
+      localResources: HashMap[String, LocalResource],
+      timestamp: String,
+      size: String, 
+      vis: String) = {
+    val uri = new URI(file)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(rtype)
+    amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+    amJarRsrc.setTimestamp(timestamp.toLong)
+    amJarRsrc.setSize(size.toLong)
+    localResources(uri.getFragment()) = amJarRsrc
+  }
+
   def prepareLocalResources: HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
-    val locaResources = HashMap[String, LocalResource]()
-    
-    // Spark JAR
-    val sparkJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-    sparkJarResource.setType(LocalResourceType.FILE)
-    sparkJarResource.setVisibility(LocalResourceVisibility.APPLICATION)
-    sparkJarResource.setResource(ConverterUtils.getYarnUrlFromURI(
-      new URI(System.getenv("SPARK_YARN_JAR_PATH"))))
-    sparkJarResource.setTimestamp(System.getenv("SPARK_YARN_JAR_TIMESTAMP").toLong)
-    sparkJarResource.setSize(System.getenv("SPARK_YARN_JAR_SIZE").toLong)
-    locaResources("spark.jar") = sparkJarResource
-    // User JAR
-    val userJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-    userJarResource.setType(LocalResourceType.FILE)
-    userJarResource.setVisibility(LocalResourceVisibility.APPLICATION)
-    userJarResource.setResource(ConverterUtils.getYarnUrlFromURI(
-      new URI(System.getenv("SPARK_YARN_USERJAR_PATH"))))
-    userJarResource.setTimestamp(System.getenv("SPARK_YARN_USERJAR_TIMESTAMP").toLong)
-    userJarResource.setSize(System.getenv("SPARK_YARN_USERJAR_SIZE").toLong)
-    locaResources("app.jar") = userJarResource
-
-    // Log4j conf - if available
-    if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
-      val log4jConfResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-      log4jConfResource.setType(LocalResourceType.FILE)
-      log4jConfResource.setVisibility(LocalResourceVisibility.APPLICATION)
-      log4jConfResource.setResource(ConverterUtils.getYarnUrlFromURI(
-        new URI(System.getenv("SPARK_YARN_LOG4J_PATH"))))
-      log4jConfResource.setTimestamp(System.getenv("SPARK_YARN_LOG4J_TIMESTAMP").toLong)
-      log4jConfResource.setSize(System.getenv("SPARK_YARN_LOG4J_SIZE").toLong)
-      locaResources("log4j.properties") = log4jConfResource
+    val localResources = HashMap[String, LocalResource]()
+
+    if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+      val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
+      for( i <- 0 to distFiles.length - 1) {
+        setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+          fileSizes(i), visibilities(i))
+      }
     }
 
-    
-    logInfo("Prepared Local resources " + locaResources)
-    return locaResources
+    if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+      val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
+      for( i <- 0 to distArchives.length - 1) {
+        setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, 
+          timeStamps(i), fileSizes(i), visibilities(i))
+      }
+    }
+
+    logInfo("Prepared Local resources " + localResources)
+    return localResources
   }
-  
+
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    // If log4j present, ensure ours overrides all others
-    if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
-      // Which is correct ?
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./log4j.properties")
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
-    }
+    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
-    Client.populateHadoopClasspath(yarnConf, env)
-
-    // allow users to specify some environment variables
+    // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
 
     System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
     return env
   }
-  
+
   def connectToCM: ContainerManager = {
     val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort()
     val cmAddress = NetUtils.createSocketAddr(cmHostPortStr)
     logInfo("Connecting to ContainerManager at " + cmHostPortStr)
 
-    // use doAs and remoteUser here so we can add the container token and not 
-    // pollute the current users credentials with all of the individual container tokens
-    val user = UserGroupInformation.createRemoteUser(container.getId().toString());
-    val containerToken = container.getContainerToken();
+    // Use doAs and remoteUser here so we can add the container token and not pollute the current
+    // users credentials with all of the individual container tokens
+    val user = UserGroupInformation.createRemoteUser(container.getId().toString())
+    val containerToken = container.getContainerToken()
     if (containerToken != null) {
       user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress))
     }
@@ -219,8 +228,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
             return rpc.getProxy(classOf[ContainerManager],
                 cmAddress, conf).asInstanceOf[ContainerManager]
           }
-        });
-    return proxy;
+        })
+    proxy
   }
-  
+
 }
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 d222f412a0cc8c1aae479418efd68f5da7fb59b9..9ab20735299c0e35a1e1c13fcd11485ee6ee87d0 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
@@ -17,87 +17,112 @@
 
 package org.apache.spark.deploy.yarn
 
+import java.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
 import org.apache.spark.Logging
-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.spark.util.Utils
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.AMRMProtocol
+import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId}
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
 import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
-import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
-import java.util.concurrent.atomic.AtomicInteger
-import org.apache.hadoop.yarn.api.AMRMProtocol
-import collection.JavaConversions._
-import collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import org.apache.hadoop.conf.Configuration
-import java.util.{Collections, Set => JSet}
-import java.lang.{Boolean => JBoolean}
 
-object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+
+object AllocationType extends Enumeration {
   type AllocationType = Value
   val HOST, RACK, ANY = Value
 }
 
-// too many params ? refactor it 'somehow' ?
-// needs to be mt-safe
-// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it 
-// more proactive and decoupled.
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
 // Note that right now, we assume all node asks as uniform in terms of capabilities and priority
-// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info
-// on how we are requesting for containers.
-private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol, 
-                                          val appAttemptId: ApplicationAttemptId,
-                                          val maxWorkers: Int, val workerMemory: Int, val workerCores: Int,
-                                          val preferredHostToCount: Map[String, Int], 
-                                          val preferredRackToCount: Map[String, Int])
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+    val conf: Configuration,
+    val resourceManager: AMRMProtocol, 
+    val appAttemptId: ApplicationAttemptId,
+    val maxWorkers: Int,
+    val workerMemory: Int,
+    val workerCores: Int,
+    val preferredHostToCount: Map[String, Int], 
+    val preferredRackToCount: Map[String, Int])
   extends Logging {
-
-
   // These three are locked on allocatedHostToContainersMap. Complementary data structures
   // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
-  // allocatedContainerToHostMap: container to host mapping
-  private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]()
+  // allocatedContainerToHostMap: container to host mapping.
+  private val allocatedHostToContainersMap =
+    new HashMap[String, collection.mutable.Set[ContainerId]]()
+
   private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
-  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node)
-  // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap
+
+  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+  // allocated node)
+  // As with the two data structures above, tightly coupled with them, and to be locked on
+  // allocatedHostToContainersMap
   private val allocatedRackCount = new HashMap[String, Int]()
 
-  // containers which have been released.
+  // Containers which have been released.
   private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
-  // containers to be released in next request to RM
+  // Containers to be released in next request to RM
   private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
 
   private val numWorkersRunning = new AtomicInteger()
   // Used to generate a unique id per worker
   private val workerIdCounter = new AtomicInteger()
   private val lastResponseId = new AtomicInteger()
+  private val numWorkersFailed = new AtomicInteger()
 
   def getNumWorkersRunning: Int = numWorkersRunning.intValue
 
+  def getNumWorkersFailed: Int = numWorkersFailed.intValue
 
   def isResourceConstraintSatisfied(container: Container): Boolean = {
     container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
   }
 
   def allocateContainers(workersToRequest: Int) {
-    // We need to send the request only once from what I understand ... but for now, not modifying this much.
+    // We need to send the request only once from what I understand ... but for now, not modifying
+    // this much.
 
     // Keep polling the Resource Manager for containers
     val amResp = allocateWorkerResources(workersToRequest).getAMResponse
 
     val _allocatedContainers = amResp.getAllocatedContainers()
-    if (_allocatedContainers.size > 0) {
 
-
-      logDebug("Allocated " + _allocatedContainers.size + " containers, current count " + 
-        numWorkersRunning.get() + ", to-be-released " + releasedContainerList + 
-        ", pendingReleaseContainers : " + pendingReleaseContainers)
-      logDebug("Cluster Resources: " + amResp.getAvailableResources)
+    if (_allocatedContainers.size > 0) {
+      logDebug("""
+        Allocated containers: %d
+        Current worker count: %d
+        Containers released: %s
+        Containers to be released: %s
+        Cluster resources: %s
+        """.format(
+          _allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers,
+          amResp.getAvailableResources))
 
       val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
 
-      // ignore if not satisfying constraints      {
+      // Ignore if not satisfying constraints      {
       for (container <- _allocatedContainers) {
         if (isResourceConstraintSatisfied(container)) {
           // allocatedContainers += container
@@ -111,8 +136,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         else releasedContainerList.add(container.getId())
       }
 
-      // Find the appropriate containers to use
-      // Slightly non trivial groupBy I guess ...
+      // Find the appropriate containers to use. Slightly non trivial groupBy ...
       val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
       val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
       val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
@@ -132,21 +156,22 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           remainingContainers = null
         }
         else if (requiredHostCount > 0) {
-          // container list has more containers than we need for data locality.
-          // Split into two : data local container count of (remainingContainers.size - requiredHostCount) 
-          // and rest as remainingContainer
-          val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount)
+          // Container list has more containers than we need for data locality.
+          // Split into two : data local container count of (remainingContainers.size -
+          // requiredHostCount) and rest as remainingContainer
+          val (dataLocal, remaining) = remainingContainers.splitAt(
+            remainingContainers.size - requiredHostCount)
           dataLocalContainers.put(candidateHost, dataLocal)
           // remainingContainers = remaining
 
           // yarn has nasty habit of allocating a tonne of containers on a host - discourage this :
-          // add remaining to release list. If we have insufficient containers, next allocation cycle 
-          // will reallocate (but wont treat it as data local)
+          // add remaining to release list. If we have insufficient containers, next allocation 
+          // cycle will reallocate (but wont treat it as data local)
           for (container <- remaining) releasedContainerList.add(container.getId())
           remainingContainers = null
         }
 
-        // now rack local
+        // Now rack local
         if (remainingContainers != null){
           val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
 
@@ -159,15 +184,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
             if (requiredRackCount >= remainingContainers.size){
               // Add all to dataLocalContainers
               dataLocalContainers.put(rack, remainingContainers)
-              // all consumed
+              // All consumed
               remainingContainers = null
             }
             else if (requiredRackCount > 0) {
               // container list has more containers than we need for data locality.
-              // Split into two : data local container count of (remainingContainers.size - requiredRackCount) 
-              // and rest as remainingContainer
-              val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount)
-              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]())
+              // Split into two : data local container count of (remainingContainers.size -
+              // requiredRackCount) and rest as remainingContainer
+              val (rackLocal, remaining) = remainingContainers.splitAt(
+                remainingContainers.size - requiredRackCount)
+              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+                new ArrayBuffer[Container]())
 
               existingRackLocal ++= rackLocal
               remainingContainers = remaining
@@ -183,8 +210,8 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
       // Now that we have split the containers into various groups, go through them in order : 
       // first host local, then rack local and then off rack (everything else).
-      // Note that the list we create below tries to ensure that not all containers end up within a host 
-      // if there are sufficiently large number of hosts/containers.
+      // Note that the list we create below tries to ensure that not all containers end up within a
+      // host if there are sufficiently large number of hosts/containers.
 
       val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size)
       allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
@@ -197,33 +224,39 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         val workerHostname = container.getNodeId.getHost
         val containerId = container.getId
 
-        assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
+        assert(
+          container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
 
         if (numWorkersRunningNow > maxWorkers) {
-          logInfo("Ignoring container " + containerId + " at host " + workerHostname + 
-            " .. we already have required number of containers")
+          logInfo("""Ignoring container %s at host %s, since we already have the required number of
+            containers for it.""".format(containerId, workerHostname))
           releasedContainerList.add(containerId)
           // reset counter back to old value.
           numWorkersRunning.decrementAndGet()
         }
         else {
-          // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter)
+          // Deallocate + allocate can result in reusing id's wrongly - so use a different counter
+          // (workerIdCounter)
           val workerId = workerIdCounter.incrementAndGet().toString
           val driverUrl = "akka.tcp://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 ..
+          // Just to be safe, simply remove it from pendingReleaseContainers.
+          // Should not be there, but ..
           pendingReleaseContainers.remove(containerId)
 
           val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
           allocatedHostToContainersMap.synchronized {
-            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]())
+            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+              new HashSet[ContainerId]())
 
             containerSet += containerId
             allocatedContainerToHostMap.put(containerId, workerHostname)
-            if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            if (rack != null) {
+              allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            }
           }
 
           new Thread(
@@ -232,17 +265,23 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           ).start()
         }
       }
-      logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + 
-        _allocatedContainers.size + "), current count " + numWorkersRunning.get() +
-        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
+      logDebug("""
+        Finished processing %d containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
     }
 
 
     val completedContainers = amResp.getCompletedContainersStatuses()
     if (completedContainers.size > 0){
-      logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() +
-        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
-
+      logDebug("Completed %d containers, to-be-released: %s".format(
+        completedContainers.size, releasedContainerList))
       for (completedContainer <- completedContainers){
         val containerId = completedContainer.getContainerId
 
@@ -251,10 +290,19 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           pendingReleaseContainers.remove(containerId)
         }
         else {
-          // simply decrement count - next iteration of ReporterThread will take care of allocating !
+          // Simply decrement count - next iteration of ReporterThread will take care of allocating.
           numWorkersRunning.decrementAndGet()
-          logInfo("Container completed ? nodeId: " + containerId + ", state " + completedContainer.getState +
-            " httpaddress: " + completedContainer.getDiagnostics)
+          logInfo("Completed container %s (state: %s, exit status: %s)".format(
+            containerId,
+            completedContainer.getState,
+            completedContainer.getExitStatus()))
+          // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
+          // there are some exit status' we shouldn't necessarily count against us, but for
+          // now I think its ok as none of the containers are expected to exit
+          if (completedContainer.getExitStatus() != 0) {
+            logInfo("Container marked as failed: " + containerId)
+            numWorkersFailed.incrementAndGet()
+          }
         }
 
         allocatedHostToContainersMap.synchronized {
@@ -271,7 +319,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
             allocatedContainerToHostMap -= containerId
 
-            // doing this within locked context, sigh ... move to outside ?
+            // Doing this within locked context, sigh ... move to outside ?
             val rack = YarnAllocationHandler.lookupRack(conf, host)
             if (rack != null) {
               val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
@@ -281,9 +329,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           }
         }
       }
-      logDebug("After completed " + completedContainers.size + " containers, current count " + 
-        numWorkersRunning.get() + ", to-be-released " + releasedContainerList + 
-        ", pendingReleaseContainers : " + pendingReleaseContainers)
+      logDebug("""
+        Finished processing %d completed containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          completedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
     }
   }
 
@@ -337,7 +392,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
       // default.
     if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
-      logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty)
+      logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty)
       resourceRequests = List(
         createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY))
     }
@@ -350,17 +405,24 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
 
         if (requiredCount > 0) {
-          hostContainerRequests += 
-            createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY)
+          hostContainerRequests += createResourceRequest(
+            AllocationType.HOST,
+            candidateHost,
+            requiredCount,
+            YarnAllocationHandler.PRIORITY)
         }
       }
-      val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList)
+      val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(
+        hostContainerRequests.toList)
 
-      val anyContainerRequests: ResourceRequest = 
-        createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)
+      val anyContainerRequests: ResourceRequest = createResourceRequest(
+        AllocationType.ANY,
+        resource = null,
+        numWorkers,
+        YarnAllocationHandler.PRIORITY)
 
-      val containerRequests: ArrayBuffer[ResourceRequest] =
-        new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1)
+      val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](
+        hostContainerRequests.size + rackContainerRequests.size + 1)
 
       containerRequests ++= hostContainerRequests
       containerRequests ++= rackContainerRequests
@@ -378,55 +440,60 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
     val releasedContainerList = createReleasedContainerList()
     req.addAllReleases(releasedContainerList)
 
-
-
     if (numWorkers > 0) {
-      logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.")
+      logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers,
+        workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
     }
     else {
       logDebug("Empty allocation req ..  release : " + releasedContainerList)
     }
 
-    for (req <- resourceRequests) {
-      logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers +
-        ", p = " + req.getPriority().getPriority + ", capability: "  + req.getCapability)
+    for (request <- resourceRequests) {
+      logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)".
+        format(
+          request.getHostName,
+          request.getNumContainers,
+          request.getPriority,
+          request.getCapability))
     }
     resourceManager.allocate(req)
   }
 
 
-  private def createResourceRequest(requestType: AllocationType.AllocationType, 
-                                    resource:String, numWorkers: Int, priority: Int): ResourceRequest = {
+  private def createResourceRequest(
+    requestType: AllocationType.AllocationType, 
+    resource:String,
+    numWorkers: Int,
+    priority: Int): ResourceRequest = {
 
     // If hostname specified, we need atleast two requests - node local and rack local.
     // There must be a third request - which is ANY : that will be specially handled.
     requestType match {
       case AllocationType.HOST => {
-        assert (YarnAllocationHandler.ANY_HOST != resource)
-
+        assert(YarnAllocationHandler.ANY_HOST != resource)
         val hostname = resource
         val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority)
 
-        // add to host->rack mapping
+        // Add to host->rack mapping
         YarnAllocationHandler.populateRackInfo(conf, hostname)
 
         nodeLocal
       }
-
       case AllocationType.RACK => {
         val rack = resource
         createResourceRequestImpl(rack, numWorkers, priority)
       }
-
-      case AllocationType.ANY => {
-        createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority)
-      }
-
-      case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType)
+      case AllocationType.ANY => createResourceRequestImpl(
+        YarnAllocationHandler.ANY_HOST, numWorkers, priority)
+      case _ => throw new IllegalArgumentException(
+        "Unexpected/unsupported request type: " + requestType)
     }
   }
 
-  private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = {
+  private def createResourceRequestImpl(
+    hostname:String,
+    numWorkers: Int,
+    priority: Int): ResourceRequest = {
 
     val rsrcRequest = Records.newRecord(classOf[ResourceRequest])
     val memCapability = Records.newRecord(classOf[Resource])
@@ -447,11 +514,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
   def createReleasedContainerList(): ArrayBuffer[ContainerId] = {
 
     val retval = new ArrayBuffer[ContainerId](1)
-    // iterator on COW list ...
+    // Iterator on COW list ...
     for (container <- releasedContainerList.iterator()){
       retval += container
     }
-    // remove from the original list.
+    // Remove from the original list.
     if (! retval.isEmpty) {
       releasedContainerList.removeAll(retval)
       for (v <- retval) pendingReleaseContainers.put(v, true)
@@ -466,14 +533,14 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 object YarnAllocationHandler {
 
   val ANY_HOST = "*"
-  // all requests are issued with same priority : we do not (yet) have any distinction between 
+  // All requests are issued with same priority : we do not (yet) have any distinction between 
   // request types (like map/reduce in hadoop for example)
   val PRIORITY = 1
 
   // Additional memory overhead - in mb
   val MEMORY_OVERHEAD = 384
 
-  // host to rack map - saved from allocation requests
+  // Host to rack map - saved from allocation requests
   // We are expecting this not to change.
   // Note that it is possible for this to change : and RM will indicate that to us via update 
   // response to allocate. But we are punting on handling that for now.
@@ -481,38 +548,69 @@ object YarnAllocationHandler {
   private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
 
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   args: ApplicationMasterArguments): YarnAllocationHandler = {
-
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, 
-      args.workerMemory, args.workerCores, Map[String, Int](), Map[String, Int]())
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    args: ApplicationMasterArguments): YarnAllocationHandler = {
+
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      Map[String, Int](),
+      Map[String, Int]())
   }
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   args: ApplicationMasterArguments,
-                   map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    args: ApplicationMasterArguments,
+    map: collection.Map[String,
+    collection.Set[SplitInfo]]): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
-
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, 
-      args.workerMemory, args.workerCores, hostToCount, rackToCount)
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      hostToCount,
+      rackToCount)
   }
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   maxWorkers: Int, workerMemory: Int, workerCores: Int,
-                   map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    maxWorkers: Int,
+    workerMemory: Int,
+    workerCores: Int,
+    map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
 
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers,
-      workerMemory, workerCores, hostToCount, rackToCount)
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      maxWorkers,
+      workerMemory,
+      workerCores,
+      hostToCount,
+      rackToCount)
   }
 
   // A simple method to copy the split info map.
-  private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) :
+  private def generateNodeToWeight(
+    conf: Configuration,
+    input: collection.Map[String, collection.Set[SplitInfo]]) :
   // host to count, rack to count
   (Map[String, Int], Map[String, Int]) = {
 
@@ -536,7 +634,7 @@ object YarnAllocationHandler {
   }
 
   def lookupRack(conf: Configuration, host: String): String = {
-    if (! hostToRack.contains(host)) populateRackInfo(conf, host)
+    if (!hostToRack.contains(host)) populateRackInfo(conf, host)
     hostToRack.get(host)
   }
 
@@ -559,10 +657,12 @@ object YarnAllocationHandler {
         val rack = rackInfo.getNetworkLocation
         hostToRack.put(hostname, rack)
         if (! rackToHostSet.containsKey(rack)) {
-          rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+          rackToHostSet.putIfAbsent(rack,
+            Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
         }
         rackToHostSet.get(rack).add(hostname)
 
+        // TODO(harvey): Figure out this comment...
         // Since RackResolver caches, we are disabling this for now ...
       } /* else {
         // right ? Else we will keep calling rack resolver in case we cant resolve rack info ...
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index ca2f1e2565b9a8a7956ffe00e2b453779956853a..2ba2366ead17113c764663ed934a5221f0eeb0ee 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -18,13 +18,10 @@
 package org.apache.spark.deploy.yarn
 
 import org.apache.spark.deploy.SparkHadoopUtil
-import collection.mutable.HashMap
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import java.security.PrivilegedExceptionAction
 
 /**
  * Contains util methods to interact with Hadoop from spark.
@@ -40,7 +37,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
 
   // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
   override def addCredentials(conf: JobConf) {
-    val jobCreds = conf.getCredentials();
+    val jobCreds = conf.getCredentials()
     jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
   }
 }
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..63a0449e5a0730085554d2b8ae86067135fa8dba
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.spark._
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.deploy.yarn.YarnAllocationHandler
+import org.apache.spark.util.Utils
+
+/**
+ *
+ * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
+ */
+private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+
+  def this(sc: SparkContext) = this(sc, new Configuration())
+
+  // By default, rack is unknown
+  override def getRackForHost(hostPort: String): Option[String] = {
+    val host = Utils.parseHostPort(hostPort)._1
+    val retval = YarnAllocationHandler.lookupRack(conf, host)
+    if (retval != null) Some(retval) else None
+  }
+
+  override def postStartHook() {
+
+    // The yarn application is running, but the worker might not yet ready
+    // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
+    Thread.sleep(2000L)
+    logInfo("YarnClientClusterScheduler.postStartHook done")
+  }
+}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b206780c7806e15c84944db05876f89c8f848040
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.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.scheduler.cluster
+
+import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
+import org.apache.spark.{SparkException, Logging, SparkContext}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments}
+
+private[spark] class YarnClientSchedulerBackend(
+    scheduler: ClusterScheduler,
+    sc: SparkContext)
+  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+  with Logging {
+
+  var client: Client = null
+  var appId: ApplicationId = null
+
+  override def start() {
+    super.start()
+
+    val defalutWorkerCores = "2"
+    val defalutWorkerMemory = "512m"
+    val defaultWorkerNumber = "1"
+
+    val userJar = System.getenv("SPARK_YARN_APP_JAR")
+    var workerCores = System.getenv("SPARK_WORKER_CORES")
+    var workerMemory = System.getenv("SPARK_WORKER_MEMORY")
+    var workerNumber = System.getenv("SPARK_WORKER_INSTANCES")
+
+    if (userJar == null)
+      throw new SparkException("env SPARK_YARN_APP_JAR is not set")
+
+    if (workerCores == null)
+      workerCores = defalutWorkerCores
+    if (workerMemory == null)
+      workerMemory = defalutWorkerMemory
+    if (workerNumber == null)
+      workerNumber = defaultWorkerNumber
+
+    val driverHost = System.getProperty("spark.driver.host")
+    val driverPort = System.getProperty("spark.driver.port")
+    val hostport = driverHost + ":" + driverPort
+
+    val argsArray = Array[String](
+      "--class", "notused",
+      "--jar", userJar,
+      "--args", hostport,
+      "--worker-memory", workerMemory,
+      "--worker-cores", workerCores,
+      "--num-workers", workerNumber,
+      "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
+    )
+
+    val args = new ClientArguments(argsArray)
+    client = new Client(args)
+    appId = client.runApp()
+    waitForApp()
+  }
+
+  def waitForApp() {
+
+    // TODO : need a better way to find out whether the workers are ready or not
+    // maybe by resource usage report?
+    while(true) {
+      val report = client.getApplicationReport(appId)
+
+      logInfo("Application report from ASM: \n" +
+        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+        "\t appStartTime: " + report.getStartTime() + "\n" +
+        "\t yarnAppState: " + report.getYarnApplicationState() + "\n"
+      )
+
+      // Ready to go, or already gone.
+      val state = report.getYarnApplicationState()
+      if (state == YarnApplicationState.RUNNING) {
+        return
+      } else if (state == YarnApplicationState.FINISHED ||
+        state == YarnApplicationState.FAILED ||
+        state == YarnApplicationState.KILLED) {
+        throw new SparkException("Yarn application already ended," +
+          "might be killed or not able to launch application master.")
+      }
+
+      Thread.sleep(1000)
+    }
+  }
+
+  override def stop() {
+    super.stop()
+    client.stop()
+    logInfo("Stoped")
+  }
+
+}
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5
--- /dev/null
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -0,0 +1,220 @@
+/*
+ * 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.deploy.yarn
+
+import java.net.URI
+
+import org.scalatest.FunSuite
+import org.scalatest.mock.MockitoSugar
+import org.mockito.Mockito.when
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+
+class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar {
+
+  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
+    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): 
+        LocalResourceVisibility = {
+      return LocalResourceVisibility.PRIVATE
+    }
+  }
+  
+  test("test getFileStatus empty") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath() === null)
+  }
+
+  test("test getFileStatus cached") {
+    val distMgr = new ClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val uri = new URI("/tmp/testing")
+    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
+    val stat = distMgr.getFileStatus(fs, uri, statCache)
+    assert(stat.getPath().toString() === "/tmp/testing")
+  }
+
+  test("test addResource") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", 
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 0)
+    assert(resource.getSize() === 0)
+    assert(resource.getType() === LocalResourceType.FILE)
+
+    val env = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env)
+    assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0")
+    assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0")
+    assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
+
+    //add another one and verify both there and order correct
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing2"))
+    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
+    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
+    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", 
+      statCache, false)
+    val resource2 = localResources("link2")
+    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
+    assert(resource2.getTimestamp() === 10)
+    assert(resource2.getSize() === 20)
+    assert(resource2.getType() === LocalResourceType.FILE)
+
+    val env2 = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env2)
+    val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+    val files = env2("SPARK_YARN_CACHE_FILES").split(',') 
+    val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+    val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',')
+    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(timestamps(0)  === "0")
+    assert(sizes(0)  === "0")
+    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
+
+    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
+    assert(timestamps(1)  === "10")
+    assert(sizes(1)  === "20")
+    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
+  }
+
+  test("test addResource link null") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
+
+    intercept[Exception] {
+      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, 
+        statCache, false)
+    }
+    assert(localResources.get("link") === None)
+    assert(localResources.size === 0)
+  }
+
+  test("test addResource appmaster only") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
+      statCache, true)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val env = new HashMap[String, String]()
+    distMgr.setDistFilesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
+  }
+
+  test("test addResource archive") {
+    val distMgr = new MockClientDistributedCacheManager()
+    val fs = mock[FileSystem]
+    val conf = new Configuration()
+    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
+    val localResources = HashMap[String, LocalResource]()
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
+      null, new Path("/tmp/testing"))
+    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
+
+    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
+      statCache, false)
+    val resource = localResources("link")
+    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
+    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
+    assert(resource.getTimestamp() === 10)
+    assert(resource.getSize() === 20)
+    assert(resource.getType() === LocalResourceType.ARCHIVE)
+
+    val env = new HashMap[String, String]()
+
+    distMgr.setDistArchivesEnv(env)
+    assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20")
+    assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
+
+    distMgr.setDistFilesEnv(env)
+    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
+    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
+  }
+
+
+}