diff --git a/.gitignore b/.gitignore
index 155e785b01beb809a13c45c40d96f04f4dd6343b..b87fc1ee7944c332a976201f705728b0c57506ac 100644
--- a/.gitignore
+++ b/.gitignore
@@ -36,3 +36,4 @@ streaming-tests.log
 dependency-reduced-pom.xml
 .ensime
 .ensime_lucene
+derby.log
diff --git a/README.md b/README.md
index b0fc3524fa6b769c5c6f8ba95fae8cbadd0970da..ba24ab43b1a10c75ecd697d27fc4b178af59ab69 100644
--- a/README.md
+++ b/README.md
@@ -12,11 +12,16 @@ This README file only contains basic setup instructions.
 
 ## Building
 
-Spark requires Scala 2.9.2. The project is built using Simple Build Tool (SBT),
-which is packaged with it. To build Spark and its example programs, run:
+Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is
+built using Simple Build Tool (SBT), which is packaged with it. To build
+Spark and its example programs, run:
 
     sbt/sbt package
 
+Spark also supports building using Maven. If you would like to build using Maven,
+see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html)
+in the spark documentation..
+
 To run Spark, you will need to have Scala's bin directory in your `PATH`, or
 you will need to set the `SCALA_HOME` environment variable to point to where
 you've installed Scala. Scala must be accessible through one of these
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 667d28c1a2ba2b1fa730c01dc0bc025acde846f5..b83a0ef6c0f375e9faab544b44e4731786ff9194 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -102,5 +102,42 @@
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2-yarn</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>
diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala
index e10c03f6bad1c8640143afc8808c90afb16ca7d6..5ecdd7d0045fae68116792eae13da43dbdf1bb39 100644
--- a/bagel/src/main/scala/spark/bagel/Bagel.scala
+++ b/bagel/src/main/scala/spark/bagel/Bagel.scala
@@ -7,8 +7,7 @@ import scala.collection.mutable.ArrayBuffer
 import storage.StorageLevel
 
 object Bagel extends Logging {
-
-  val DEFAULT_STORAGE_LEVEL  = StorageLevel.MEMORY_ONLY
+  val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK
 
   /**
    * Runs a Bagel program.
@@ -63,8 +62,9 @@ object Bagel extends Logging {
       val combinedMsgs = msgs.combineByKey(
         combiner.createCombiner _, combiner.mergeMsg _, combiner.mergeCombiners _, partitioner)
       val grouped = combinedMsgs.groupWith(verts)
+      val superstep_ = superstep  // Create a read-only copy of superstep for capture in closure
       val (processed, numMsgs, numActiveVerts) =
-        comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep), storageLevel)
+        comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel)
 
       val timeTaken = System.currentTimeMillis - startTime
       logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000))
diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala
index 25db395c22128013d259aae8722d567e3c0ff76f..a09c97806869ef707a6b374a0c253d9f81410519 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/bagel/BagelSuite.scala
@@ -23,6 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
     }
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   test("halting by voting") {
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
index 0c584055c794685255450b041ab6a8c7b9d9b893..8ee3ec481fe0b90005d6a65b89117ecb84171bd9 100755
--- a/bin/spark-daemon.sh
+++ b/bin/spark-daemon.sh
@@ -30,7 +30,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> <args...>"
+usage="Usage: spark-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <spark-command> <spark-instance-number> <args...>"
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
@@ -48,6 +48,8 @@ startStop=$1
 shift
 command=$1
 shift
+instance=$1
+shift
 
 spark_rotate_log ()
 {
@@ -92,10 +94,10 @@ if [ "$SPARK_PID_DIR" = "" ]; then
 fi
 
 # some variables
-export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.log
+export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log
 export SPARK_ROOT_LOGGER="INFO,DRFA"
-log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.out
-pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command.pid
+log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out
+pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid
 
 # Set default scheduling priority
 if [ "$SPARK_NICENESS" = "" ]; then
diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh
index 4f9719ee809e800bef0055681a433b350a6ca2b4..0619097e4dc3a512920e8f70b0cbffe2ab75c3d3 100755
--- a/bin/spark-daemons.sh
+++ b/bin/spark-daemons.sh
@@ -2,7 +2,7 @@
 
 # Run a Spark command on all slave hosts.
 
-usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command args..."
+usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
diff --git a/bin/start-master.sh b/bin/start-master.sh
index 87feb261fe86bb498eedcf40c1d98b3773cf3576..83a3e1f3dc1a3caa04b83d60d9dd78f506db583c 100755
--- a/bin/start-master.sh
+++ b/bin/start-master.sh
@@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start spark.deploy.master.Master --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
+"$bin"/spark-daemon.sh start spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
diff --git a/bin/start-slave.sh b/bin/start-slave.sh
index 45a0cf7a6b7ac526fb0651e6a3f12f4cbfca8b51..616c76e4ee6e01eecbcfaf241ec87bd7e9dc9554 100755
--- a/bin/start-slave.sh
+++ b/bin/start-slave.sh
@@ -11,4 +11,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start spark.deploy.worker.Worker $1
+"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@"
diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh
index 390247ca4aa4916e0f53b001c6cb3927df02da65..4e05224190e3b5edd17d173578691fca03dd51fa 100755
--- a/bin/start-slaves.sh
+++ b/bin/start-slaves.sh
@@ -21,4 +21,13 @@ fi
 echo "Master IP: $SPARK_MASTER_IP"
 
 # Launch the slaves
-exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
+  exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+else
+  if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
+    SPARK_WORKER_WEBUI_PORT=8081
+  fi
+  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
+    "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+  done
+fi
diff --git a/bin/stop-master.sh b/bin/stop-master.sh
index f75167dd2c72d9352140b47d6ae074850364a0c2..172ee5891d17f6d6fb76c9ff12c258ca14edcbff 100755
--- a/bin/stop-master.sh
+++ b/bin/stop-master.sh
@@ -7,4 +7,4 @@ bin=`cd "$bin"; pwd`
 
 . "$bin/spark-config.sh"
 
-"$bin"/spark-daemon.sh stop spark.deploy.master.Master
\ No newline at end of file
+"$bin"/spark-daemon.sh stop spark.deploy.master.Master 1
diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
index 21c9ebf324fdc69f6ab1680a75f8b5ed198d28ab..fbfc594472fe7522c30a6989e171aa4fa3396c7f 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -7,4 +7,14 @@ bin=`cd "$bin"; pwd`
 
 . "$bin/spark-config.sh"
 
-"$bin"/spark-daemons.sh stop spark.deploy.worker.Worker
\ No newline at end of file
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
+  "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker 1
+else
+  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
+    "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker $(( $i + 1 ))
+  done
+fi
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 6d71ec56914c3e1fa418ff069e1dbaa372bd6db2..37565ca827980d3d2b48312dfa30bb9d3fd10cea 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -12,6 +12,7 @@
 # - SPARK_WORKER_CORES, to set the number of cores to use on this machine
 # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g)
 # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
+# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine
 #
 # Finally, Spark also relies on the following variables, but these can be set
 # on just the *master* (i.e. in your driver program), and will automatically
diff --git a/core/pom.xml b/core/pom.xml
index 9d46d94c1cce328baf27cf5ad51c5821a8be7d94..d8687bf9911b3d45a26b93310041bf359e66f28c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -73,7 +73,7 @@
     </dependency>
     <dependency>
       <groupId>cc.spray</groupId>
-      <artifactId>spray-json_${scala.version}</artifactId>
+      <artifactId>spray-json_2.9.2</artifactId>
     </dependency>
     <dependency>
       <groupId>org.tomdz.twirl</groupId>
@@ -81,13 +81,26 @@
     </dependency>
     <dependency>
       <groupId>com.github.scala-incubator.io</groupId>
-      <artifactId>scala-io-file_${scala.version}</artifactId>
+      <artifactId>scala-io-file_2.9.2</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.mesos</groupId>
       <artifactId>mesos</artifactId>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
 
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.version}</artifactId>
@@ -275,5 +288,72 @@
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>add-source</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>add-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/main/scala</source>
+                    <source>src/hadoop2-yarn/scala</source>
+                  </sources>
+                </configuration>
+              </execution>
+              <execution>
+                <id>add-scala-test-sources</id>
+                <phase>generate-test-sources</phase>
+                <goals>
+                  <goal>add-test-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/test/scala</source>
+                  </sources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2-yarn</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
index ca9f7219de7399218634d1833377e3e8719b7886..f286f2cf9c5122935fe39e77861f44304566ac79 100644
--- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -4,4 +4,7 @@ trait HadoopMapRedUtil {
   def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContext(conf, jobId)
 
   def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
+    jobId, isMap, taskId, attemptId)
 }
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
index de7b0f81e38016cc03eafad530afa9d38c35135e..264d421d14a18964117ec8970b05fb8ffc501f03 100644
--- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -6,4 +6,7 @@ trait HadoopMapReduceUtil {
   def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContext(conf, jobId)
 
   def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
+    jobId, isMap, taskId, attemptId)
 }
diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a0fb4fe25d188b02f1ac62245eaaa0de86b5b46c
--- /dev/null
+++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
@@ -0,0 +1,23 @@
+package spark.deploy
+import org.apache.hadoop.conf.Configuration
+
+
+/**
+ * Contains util methods to interact with Hadoop from spark.
+ */
+object SparkHadoopUtil {
+
+  def getUserNameFromEnvironment(): String = {
+    // defaulting to -D ...
+    System.getProperty("user.name")
+  }
+
+  def runAsUser(func: (Product) => Unit, args: Product) {
+
+    // Add support, if exists - for now, simply run func !
+    func(args)
+  }
+
+  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
+  def newConfiguration(): Configuration = new Configuration()
+}
diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..875c0a220bd36a911ffb5e12a3c99658276df41f
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -0,0 +1,13 @@
+
+package org.apache.hadoop.mapred
+
+import org.apache.hadoop.mapreduce.TaskType
+
+trait HadoopMapRedUtil {
+  def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
+
+  def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) =
+    new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId)
+}
diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..8bc6fb6dea18687a2df0dfbb032c9ba0f600d349
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -0,0 +1,13 @@
+package org.apache.hadoop.mapreduce
+
+import org.apache.hadoop.conf.Configuration
+import task.{TaskAttemptContextImpl, JobContextImpl}
+
+trait HadoopMapReduceUtil {
+  def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
+
+  def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) =
+    new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId)
+}
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ab1ab9d8a7a8e8530938f6a1ddc4c860cffb1af0
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
@@ -0,0 +1,63 @@
+package spark.deploy
+
+import collection.mutable.HashMap
+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.
+ */
+object SparkHadoopUtil {
+
+  val yarnConf = newConfiguration()
+
+  def getUserNameFromEnvironment(): String = {
+    // defaulting to env if -D is not present ...
+    val retval = System.getProperty(Environment.USER.name, System.getenv(Environment.USER.name))
+
+    // If nothing found, default to user we are running as
+    if (retval == null) System.getProperty("user.name") else retval
+  }
+
+  def runAsUser(func: (Product) => Unit, args: Product) {
+    runAsUser(func, args, getUserNameFromEnvironment())
+  }
+
+  def runAsUser(func: (Product) => Unit, args: Product, user: String) {
+
+    // println("running as user " + jobUserName)
+
+    UserGroupInformation.setConfiguration(yarnConf)
+    val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(user)
+    appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] {
+      def run: AnyRef = {
+        func(args)
+        // no return value ...
+        null
+      }
+    })
+  }
+
+  // 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.
+  def isYarnMode(): Boolean = {
+    val yarnMode = System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))
+    java.lang.Boolean.valueOf(yarnMode)
+  }
+
+  // Set an env variable indicating we are running in YARN mode.
+  // Note that anything with SPARK prefix gets propagated to all (remote) processes
+  def setYarnMode() {
+    System.setProperty("SPARK_YARN_MODE", "true")
+  }
+
+  def setYarnMode(env: HashMap[String, String]) {
+    env("SPARK_YARN_MODE") = "true"
+  }
+
+  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
+  // Always create a new config, dont reuse yarnConf.
+  def newConfiguration(): Configuration = new YarnConfiguration(new Configuration())
+}
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000000000000000000000000000000000000..aa72c1e5fef1c71484ccd82331206e7748d5ddb8
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,329 @@
+package spark.deploy.yarn
+
+import java.net.Socket
+import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+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 scala.collection.JavaConversions._
+import spark.{SparkContext, Logging, 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 val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  private var yarnAllocator: YarnAllocationHandler = null
+
+  def run() {
+    
+    // Initialization
+    val jobUserName = Utils.getUserNameFromEnvironment()
+    logInfo("running as user " + jobUserName)
+
+    // run as user ...
+    UserGroupInformation.setConfiguration(yarnConf)
+    val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(jobUserName)
+    appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] {
+      def run: AnyRef = {
+        runImpl()
+        return null
+      }
+    })
+  }
+
+  private def runImpl() {
+
+    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
+      }
+    }
+
+    // Workaround until hadoop moves to something which has
+    // https://issues.apache.org/jira/browse/HADOOP-8406
+    // 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 ?
+    // 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()
+    
+    // Allocate all containers
+    allocateWorkers()
+    
+    // Wait for the user class to Finish     
+    userThread.join()
+     
+    // Finish the ApplicationMaster
+    finishApplicationMaster()
+    // TODO: Exit based on success/failure
+    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
+    while(!driverUp) {
+      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("Master now available: " + driverHost + ":" + driverPort)
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at " + driverHost + ":" + driverPort)
+        Thread.sleep(100)
+      }
+    }
+  }
+  
+  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 t = new Thread {
+      override def run() {
+        // Copy
+        var mainArgs: Array[String] = new Array[String](args.userArgs.size())
+        args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
+        mainMethod.invoke(null, mainArgs)
+      }
+    }
+    t.start()
+    return t
+  }
+
+  private def allocateWorkers() {
+    logInfo("Waiting for spark context initialization")
+
+    try {
+      var sparkContext: SparkContext = null
+      ApplicationMaster.sparkContextRef.synchronized {
+        var count = 0
+        while (ApplicationMaster.sparkContextRef.get() == null) {
+          logInfo("Waiting for spark context initialization ... " + count)
+          count = count + 1
+          ApplicationMaster.sparkContextRef.wait(10000L)
+        }
+        sparkContext = ApplicationMaster.sparkContextRef.get()
+        assert(sparkContext != null)
+        this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, sparkContext.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 &&
+        // If user thread exists, then quit !
+        userThread.isAlive) {
+
+          this.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
+      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
+    if (userThread.isAlive){
+      // 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))
+      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){
+          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 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() { 
+    val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest])
+      .asInstanceOf[FinishApplicationMasterRequest]
+    finishReq.setAppAttemptId(appAttemptId)
+    // TODO: Check if the application has failed or succeeded
+    finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED)
+    resourceManager.finishApplicationMaster(finishReq)
+  }
+ 
+}
+
+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
+  def incrementAllocatorLoop(by: Int) {
+    val count = yarnAllocatorLoop.getAndAdd(by)
+    if (count >= ALLOCATOR_LOOP_WAIT_COUNT){
+      yarnAllocatorLoop.synchronized {
+        // to wake threads off wait ...
+        yarnAllocatorLoop.notifyAll()
+      }
+    }
+  }
+
+  private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]()
+
+  def register(master: ApplicationMaster) {
+    applicationMasters.add(master)
+  }
+
+  val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null)
+  val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
+
+  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 Client declaring failure even though we exit'ed properly.
+    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'
+        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
+        } 
+      } )
+    }
+
+    // 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/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1b00208511b1680190bba7b111a3dd808faac7eb
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -0,0 +1,77 @@
+package spark.deploy.yarn
+
+import 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: 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/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7a881e26dfffc423f7ab60d719926a9581db4dd9
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
@@ -0,0 +1,272 @@
+package spark.deploy.yarn
+
+import java.net.{InetSocketAddress, URI}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+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.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 spark.{Logging, Utils}
+import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils}
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import 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)
+  
+  def run() {
+    init(yarnConf)
+    start()
+    logClusterResourceDetails()
+
+    val newApp = super.getNewApplication()
+    val appId = newApp.getApplicationId()
+
+    verifyClusterResources(newApp)
+    val appContext = createApplicationSubmissionContext(appId)
+    val localResources = prepareLocalResources(appId, "spark")
+    val env = setupLaunchEnv(localResources)
+    val amContainer = createContainerLaunchContext(newApp, localResources, env)
+
+    appContext.setQueue(args.amQueue)
+    appContext.setAMContainerSpec(amContainer)
+    appContext.setUser(args.amUser)
+    
+    submitApp(appContext)
+    
+    monitorApplication(appId)
+    System.exit(0)
+  }
+  
+
+  def logClusterResourceDetails() {
+    val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
+    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)
+  }
+
+  
+  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+    val maxMem = app.getMaximumResourceCapability().getMemory()
+    logInfo("Max mem capabililty of resources in this cluster " + maxMem)
+    
+    // If the cluster does not have enough memory resources, exit.
+    val requestedMem = (args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + args.numWorkers * args.workerMemory
+    if (requestedMem > maxMem) {
+      logError("Cluster cannot satisfy memory resource request of " + requestedMem)
+      System.exit(1)
+    }
+  }
+  
+  def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
+    logInfo("Setting up application submission context for ASM")
+    val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
+    appContext.setApplicationId(appId)
+    appContext.setApplicationName("Spark")
+    return appContext
+  }
+  
+  def prepareLocalResources(appId: ApplicationId, appName: 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
+    val fs = FileSystem.get(conf)
+    Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> 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)
+
+        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
+      }
+    }
+    return locaResources
+  }
+  
+  def setupLaunchEnv(localResources: HashMap[String, LocalResource]): HashMap[String, String] = {
+    logInfo("Setting up the launch environment")
+    val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null)
+
+    val env = new HashMap[String, String]()
+    Apps.addToEnvironment(env, Environment.USER.name, args.amUser)
+
+    // 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)
+    SparkHadoopUtil.setYarnMode(env)
+    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()
+    }
+
+    // Add each SPARK-* key to the environment
+    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+    return 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)
+
+    val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
+
+    var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
+        (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD
+
+    // Extra options for the JVM
+    var JAVA_OPTS = ""
+
+    // Add Xmx for am memory
+    JAVA_OPTS += "-Xmx" + amMemory + "m "
+
+    // 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
+      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
+    val commands = List[String]("java " +
+      " -server " +
+      JAVA_OPTS +
+      " spark.deploy.yarn.ApplicationMaster" +
+      " --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 the ApplicationMaster: " + commands(0))
+    amContainer.setCommands(commands)
+    
+    val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
+    // Memory for the ApplicationMaster
+    capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+    amContainer.setResource(capability)
+    
+    return 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 clientToken: " + report.getClientToken() + "\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
+      }
+    }
+    return true
+  }
+}
+
+object Client {
+  def main(argStrings: Array[String]) {
+    val args = new ClientArguments(argStrings)
+    SparkHadoopUtil.setYarnMode()
+    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)
+    }
+  }
+}
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..24110558e7da8333efae3fb25a0f3d375e610ba4
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
@@ -0,0 +1,105 @@
+package spark.deploy.yarn
+
+import spark.util.MemoryParam
+import spark.util.IntParam
+import collection.mutable.{ArrayBuffer, HashMap}
+import 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 userJar: String = null
+  var userClass: String = null
+  var userArgs: Seq[String] = Seq[String]()
+  var workerMemory = 1024
+  var workerCores = 1
+  var numWorkers = 2
+  var amUser = System.getProperty("user.name")
+  var amQueue = System.getProperty("QUEUE", "default")
+  var amMemory: Int = 512
+  // TODO
+  var inputFormatInfo: List[InputFormatInfo] = null
+
+  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-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 ("--user") :: value :: tail =>
+          amUser = value
+          args = tail
+
+        case ("--queue") :: value :: tail =>
+          amQueue = 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: 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" +
+      "  --queue QUEUE        The hadoop queue to use for allocation requests (Default: 'default')\n" +
+      "  --user USERNAME      Run the ApplicationMaster (and slaves) as a different user\n"
+      )
+    System.exit(exitCode)
+  }
+  
+}
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a2bf0af762417e2d9d3085abef1983a6caa39448
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
@@ -0,0 +1,171 @@
+package spark.deploy.yarn
+
+import java.net.URI
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.net.NetUtils
+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.protocolrecords._
+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.hadoop.yarn.api.ApplicationConstants.Environment
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
+import spark.{Logging, Utils}
+
+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
+    val workerMemoryString = workerMemory + "m"
+    JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
+    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+      JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+    }
+    // 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 "
+    }
+*/
+
+    ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
+    val commands = List[String]("java " +
+      " -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 +
+      " spark.executor.StandaloneExecutorBackend " +
+      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
+    val startReq = Records.newRecord(classOf[StartContainerRequest])
+    .asInstanceOf[StartContainerRequest]
+    startReq.setContainerLaunchContext(ctx)
+    cm.startContainer(startReq)
+  }
+  
+  
+  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
+    }
+
+    
+    logInfo("Prepared Local resources " + locaResources)
+    return locaResources
+  }
+  
+  def prepareEnvironment: HashMap[String, String] = {
+    val env = new HashMap[String, String]()
+    // should we add this ?
+    Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment())
+
+    // 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, "./")
+    }
+
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
+    Client.populateHadoopClasspath(yarnConf, 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)
+    return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager]
+  }
+  
+}
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..61dd72a6518ca9bebbb5b30c4c6d88ed540173be
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -0,0 +1,547 @@
+package spark.deploy.yarn
+
+import spark.{Logging, Utils}
+import spark.scheduler.SplitInfo
+import scala.collection
+import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container}
+import spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
+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") {
+  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.
+// 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])
+  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]
+
+  private val numWorkersRunning = new AtomicInteger()
+  // Used to generate a unique id per worker
+  private val workerIdCounter = new AtomicInteger()
+  private val lastResponseId = new AtomicInteger()
+
+  def getNumWorkersRunning: Int = numWorkersRunning.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.
+
+    // 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)
+
+      val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+      // ignore if not satisfying constraints      {
+      for (container <- _allocatedContainers) {
+        if (isResourceConstraintSatisfied(container)) {
+          // allocatedContainers += container
+
+          val host = container.getNodeId.getHost
+          val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]())
+
+          containers += container
+        }
+        // Add all ignored containers to released list
+        else releasedContainerList.add(container.getId())
+      }
+
+      // Find the appropriate containers to use
+      // Slightly non trivial groupBy I guess ...
+      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)
+
+        var remainingContainers = hostToContainers.get(candidateHost).getOrElse(null)
+        assert(remainingContainers != null)
+
+        if (requiredHostCount >= remainingContainers.size){
+          // Since we got <= required containers, add all to dataLocalContainers
+          dataLocalContainers.put(candidateHost, remainingContainers)
+          // all consumed
+          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)
+          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)
+          for (container <- remaining) releasedContainerList.add(container.getId())
+          remainingContainers = null
+        }
+
+        // now rack local
+        if (remainingContainers != null){
+          val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+
+          if (rack != null){
+            val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0)
+            val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - 
+              rackLocalContainers.get(rack).getOrElse(List()).size
+
+
+            if (requiredRackCount >= remainingContainers.size){
+              // Add all to dataLocalContainers
+              dataLocalContainers.put(rack, remainingContainers)
+              // 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]())
+
+              existingRackLocal ++= rackLocal
+              remainingContainers = remaining
+            }
+          }
+        }
+
+        // If still not consumed, then it is off rack host - add to that list.
+        if (remainingContainers != null){
+          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 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.
+
+      val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size)
+      allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
+      allocatedContainers ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
+      allocatedContainers ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+
+      // Run each of the allocated containers
+      for (container <- allocatedContainers) {
+        val numWorkersRunningNow = numWorkersRunning.incrementAndGet()
+        val workerHostname = container.getNodeId.getHost
+        val containerId = container.getId
+
+        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")
+          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)
+          val workerId = workerIdCounter.incrementAndGet().toString
+          val driverUrl = "akka://spark@%s:%s/user/%s".format(
+            System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+            StandaloneSchedulerBackend.ACTOR_NAME)
+
+          logInfo("launching container on " + containerId + " host " + workerHostname)
+          // 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]())
+
+            containerSet += containerId
+            allocatedContainerToHostMap.put(containerId, workerHostname)
+            if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+          }
+
+          new Thread(
+            new WorkerRunnable(container, conf, driverUrl, workerId,
+              workerHostname, workerMemory, workerCores)
+          ).start()
+        }
+      }
+      logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + 
+        _allocatedContainers.size + "), current count " + numWorkersRunning.get() +
+        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
+    }
+
+
+    val completedContainers = amResp.getCompletedContainersStatuses()
+    if (completedContainers.size > 0){
+      logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() +
+        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
+
+      for (completedContainer <- completedContainers){
+        val containerId = completedContainer.getContainerId
+
+        // Was this released by us ? If yes, then simply remove from containerSet and move on.
+        if (pendingReleaseContainers.containsKey(containerId)) {
+          pendingReleaseContainers.remove(containerId)
+        }
+        else {
+          // 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)
+        }
+
+        allocatedHostToContainersMap.synchronized {
+          if (allocatedContainerToHostMap.containsKey(containerId)) {
+            val host = allocatedContainerToHostMap.get(containerId).getOrElse(null)
+            assert (host != null)
+
+            val containerSet = allocatedHostToContainersMap.get(host).getOrElse(null)
+            assert (containerSet != null)
+
+            containerSet -= containerId
+            if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host)
+            else allocatedHostToContainersMap.update(host, containerSet)
+
+            allocatedContainerToHostMap -= containerId
+
+            // 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
+              if (rackCount > 0) allocatedRackCount.put(rack, rackCount)
+              else allocatedRackCount.remove(rack)
+            }
+          }
+        }
+      }
+      logDebug("After completed " + completedContainers.size + " containers, current count " + 
+        numWorkersRunning.get() + ", to-be-released " + releasedContainerList + 
+        ", pendingReleaseContainers : " + pendingReleaseContainers)
+    }
+  }
+
+  def createRackResourceRequests(hostContainers: List[ResourceRequest]): List[ResourceRequest] = {
+    // First generate modified racks and new set of hosts under it : then issue requests
+    val rackToCounts = new HashMap[String, Int]()
+
+    // Within this lock - used to read/write to the rack related maps too.
+    for (container <- hostContainers) {
+      val candidateHost = container.getHostName
+      val candidateNumContainers = container.getNumContainers
+      assert(YarnAllocationHandler.ANY_HOST != candidateHost)
+
+      val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+      if (rack != null) {
+        var count = rackToCounts.getOrElse(rack, 0)
+        count += candidateNumContainers
+        rackToCounts.put(rack, count)
+      }
+    }
+
+    val requestedContainers: ArrayBuffer[ResourceRequest] = 
+      new ArrayBuffer[ResourceRequest](rackToCounts.size)
+    for ((rack, count) <- rackToCounts){
+      requestedContainers += 
+        createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY)
+    }
+
+    requestedContainers.toList
+  }
+
+  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
+  }
+
+  private def allocateWorkerResources(numWorkers: Int): AllocateResponse = {
+
+    var resourceRequests: List[ResourceRequest] = null
+
+      // default.
+    if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
+      logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty)
+      resourceRequests = List(
+        createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY))
+    }
+    else {
+      // request for all hosts in preferred nodes and for numWorkers - 
+      // candidates.size, request by default allocation policy.
+      val hostContainerRequests: ArrayBuffer[ResourceRequest] = 
+        new ArrayBuffer[ResourceRequest](preferredHostToCount.size)
+      for ((candidateHost, candidateCount) <- preferredHostToCount) {
+        val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
+
+        if (requiredCount > 0) {
+          hostContainerRequests += 
+            createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY)
+        }
+      }
+      val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList)
+
+      val anyContainerRequests: ResourceRequest = 
+        createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)
+
+      val containerRequests: ArrayBuffer[ResourceRequest] =
+        new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1)
+
+      containerRequests ++= hostContainerRequests
+      containerRequests ++= rackContainerRequests
+      containerRequests += anyContainerRequests
+
+      resourceRequests = containerRequests.toList
+    }
+
+    val req = Records.newRecord(classOf[AllocateRequest])
+    req.setResponseId(lastResponseId.incrementAndGet)
+    req.setApplicationAttemptId(appAttemptId)
+
+    req.addAllAsks(resourceRequests)
+
+    val releasedContainerList = createReleasedContainerList()
+    req.addAllReleases(releasedContainerList)
+
+
+
+    if (numWorkers > 0) {
+      logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.")
+    }
+    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)
+    }
+    resourceManager.allocate(req)
+  }
+
+
+  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)
+
+        val hostname = resource
+        val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority)
+
+        // 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)
+    }
+  }
+
+  private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = {
+
+    val rsrcRequest = Records.newRecord(classOf[ResourceRequest])
+    val memCapability = Records.newRecord(classOf[Resource])
+    // There probably is some overhead here, let's reserve a bit more memory.
+    memCapability.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+    rsrcRequest.setCapability(memCapability)
+
+    val pri = Records.newRecord(classOf[Priority])
+    pri.setPriority(priority)
+    rsrcRequest.setPriority(pri)
+
+    rsrcRequest.setHostName(hostname)
+
+    rsrcRequest.setNumContainers(java.lang.Math.max(numWorkers, 0))
+    rsrcRequest
+  }
+
+  def createReleasedContainerList(): ArrayBuffer[ContainerId] = {
+
+    val retval = new ArrayBuffer[ContainerId](1)
+    // iterator on COW list ...
+    for (container <- releasedContainerList.iterator()){
+      retval += container
+    }
+    // remove from the original list.
+    if (! retval.isEmpty) {
+      releasedContainerList.removeAll(retval)
+      for (v <- retval) pendingReleaseContainers.put(v, true)
+      logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + 
+        pendingReleaseContainers)
+    }
+
+    retval
+  }
+}
+
+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 RM 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,
+                   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)
+  }
+
+  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)
+  }
+
+  // A simple method to copy the split info map.
+  private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) :
+  // host to count, rack to count
+  (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]] = {
+    val set = rackToHostSet.get(rack)
+    if (set == null) return None
+
+    // No better way to get a Set[String] from JSet ?
+    val convertedSet: collection.mutable.Set[String] = set
+    Some(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)
+
+        // 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/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ed732d36bfc7c6e1d110d6544676f0278835319c
--- /dev/null
+++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -0,0 +1,42 @@
+package spark.scheduler.cluster
+
+import spark._
+import spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+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) {
+
+  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
+  }
+
+  // By default, if rack is unknown, return nothing
+  override def getCachedHostsForRack(rack: String): Option[Set[String]] = {
+    if (rack == None || rack == null) return None
+
+    YarnAllocationHandler.fetchCachedHostsForRack(rack)
+  }
+
+  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/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
index 35300cea5883521b98b44baa792b5c1638c671d0..a0652d7fc78a9ead97b2a2fec8fbd986f29e25aa 100644
--- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -4,4 +4,7 @@ trait HadoopMapRedUtil {
   def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
 
   def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
+    jobId, isMap, taskId, attemptId)
 }
diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
index 7afdbff3205c36b7c920ffa20851dfde06db3eec..7fdbe322fdf0ba50d1f62de502247f4991fbf16c 100644
--- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -7,4 +7,7 @@ trait HadoopMapReduceUtil {
   def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
 
   def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+
+  def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
+    jobId, isMap, taskId, attemptId)
 }
diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a0fb4fe25d188b02f1ac62245eaaa0de86b5b46c
--- /dev/null
+++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
@@ -0,0 +1,23 @@
+package spark.deploy
+import org.apache.hadoop.conf.Configuration
+
+
+/**
+ * Contains util methods to interact with Hadoop from spark.
+ */
+object SparkHadoopUtil {
+
+  def getUserNameFromEnvironment(): String = {
+    // defaulting to -D ...
+    System.getProperty("user.name")
+  }
+
+  def runAsUser(func: (Product) => Unit, args: Product) {
+
+    // Add support, if exists - for now, simply run func !
+    func(args)
+  }
+
+  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
+  def newConfiguration(): Configuration = new Configuration()
+}
diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java
new file mode 100644
index 0000000000000000000000000000000000000000..3a62dacbc8e89521c8f440e035eeb0920293266d
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileClient.java
@@ -0,0 +1,82 @@
+package spark.network.netty;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.oio.OioEventLoopGroup;
+import io.netty.channel.socket.oio.OioSocketChannel;
+
+
+class FileClient {
+
+  private FileClientHandler handler = null;
+  private Channel channel = null;
+  private Bootstrap bootstrap = null;
+
+  public FileClient(FileClientHandler handler) {
+    this.handler = handler;
+  }
+
+  public void init() {
+    bootstrap = new Bootstrap();
+    bootstrap.group(new OioEventLoopGroup())
+      .channel(OioSocketChannel.class)
+      .option(ChannelOption.SO_KEEPALIVE, true)
+      .option(ChannelOption.TCP_NODELAY, true)
+      .handler(new FileClientChannelInitializer(handler));
+  }
+
+  public static final class ChannelCloseListener implements ChannelFutureListener {
+    private FileClient fc = null;
+
+    public ChannelCloseListener(FileClient fc){
+      this.fc = fc;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) {
+      if (fc.bootstrap!=null){
+        fc.bootstrap.shutdown();
+        fc.bootstrap = null;
+      }
+    }
+  }
+
+  public void connect(String host, int port) {
+    try {
+      // Start the connection attempt.
+      channel = bootstrap.connect(host, port).sync().channel();
+      // ChannelFuture cf = channel.closeFuture();
+      //cf.addListener(new ChannelCloseListener(this));
+    } catch (InterruptedException e) {
+      close();
+    }
+  }
+
+  public void waitForClose() {
+    try {
+      channel.closeFuture().sync();
+    } catch (InterruptedException e){
+      e.printStackTrace();
+    }
+  }
+
+  public void sendRequest(String file) {
+    //assert(file == null);
+    //assert(channel == null);
+    channel.write(file + "\r\n");
+  }
+
+  public void close() {
+    if(channel != null) {
+      channel.close();
+      channel = null;
+    }
+    if ( bootstrap!=null) {
+      bootstrap.shutdown();
+      bootstrap = null;
+    }
+  }
+}
diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..af25baf641110cc5d820c1fa067e47c6b16b76ec
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
@@ -0,0 +1,24 @@
+package spark.network.netty;
+
+import io.netty.buffer.BufType;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.codec.string.StringEncoder;
+
+
+class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
+
+  private FileClientHandler fhandler;
+
+  public FileClientChannelInitializer(FileClientHandler handler) {
+    fhandler = handler;
+  }
+
+  @Override
+  public void initChannel(SocketChannel channel) {
+    // file no more than 2G
+    channel.pipeline()
+      .addLast("encoder", new StringEncoder(BufType.BYTE))
+      .addLast("handler", fhandler);
+  }
+}
diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java
new file mode 100644
index 0000000000000000000000000000000000000000..2069dee5caa597b1a9b59d5cd341dcde3bf974e9
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileClientHandler.java
@@ -0,0 +1,35 @@
+package spark.network.netty;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundByteHandlerAdapter;
+
+
+abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
+
+  private FileHeader currentHeader = null;
+
+  public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header);
+
+  @Override
+  public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) {
+    // Use direct buffer if possible.
+    return ctx.alloc().ioBuffer();
+  }
+
+  @Override
+  public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) {
+    // get header
+    if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) {
+      currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE()));
+    }
+    // get file
+    if(in.readableBytes() >= currentHeader.fileLen()) {
+      handle(ctx, in, currentHeader);
+      currentHeader = null;
+      ctx.close();
+    }
+  }
+
+}
+
diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java
new file mode 100644
index 0000000000000000000000000000000000000000..647b26bf8a155566c443a4d36e2418389c24f426
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileServer.java
@@ -0,0 +1,51 @@
+package spark.network.netty;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.Channel;
+import io.netty.channel.oio.OioEventLoopGroup;
+import io.netty.channel.socket.oio.OioServerSocketChannel;
+
+
+/**
+ * Server that accept the path of a file an echo back its content.
+ */
+class FileServer {
+
+  private ServerBootstrap bootstrap = null;
+  private Channel channel = null;
+  private PathResolver pResolver;
+
+  public FileServer(PathResolver pResolver) {
+    this.pResolver = pResolver;
+  }
+
+  public void run(int port) {
+    // Configure the server.
+    bootstrap = new ServerBootstrap();
+    try {
+      bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup())
+        .channel(OioServerSocketChannel.class)
+        .option(ChannelOption.SO_BACKLOG, 100)
+        .option(ChannelOption.SO_RCVBUF, 1500)
+        .childHandler(new FileServerChannelInitializer(pResolver));
+      // Start the server.
+      channel = bootstrap.bind(port).sync().channel();
+      channel.closeFuture().sync();
+    } catch (InterruptedException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    } finally{
+      bootstrap.shutdown();
+    }
+  }
+
+  public void stop() {
+    if (channel!=null) {
+      channel.close();
+    }
+    if (bootstrap != null) {
+      bootstrap.shutdown();
+    }
+  }
+}
diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
new file mode 100644
index 0000000000000000000000000000000000000000..8f1f5c65cd757891ad5f0809d36e529c508af07d
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
@@ -0,0 +1,25 @@
+package spark.network.netty;
+
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.codec.DelimiterBasedFrameDecoder;
+import io.netty.handler.codec.Delimiters;
+import io.netty.handler.codec.string.StringDecoder;
+
+
+class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
+
+  PathResolver pResolver;
+
+  public FileServerChannelInitializer(PathResolver pResolver) {
+    this.pResolver = pResolver;
+  }
+
+  @Override
+  public void initChannel(SocketChannel channel) {
+    channel.pipeline()
+      .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter()))
+      .addLast("strDecoder", new StringDecoder())
+      .addLast("handler", new FileServerHandler(pResolver));
+  }
+}
diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java
new file mode 100644
index 0000000000000000000000000000000000000000..a78eddb1b58b94d439d71cec229bb38df7328ff7
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/FileServerHandler.java
@@ -0,0 +1,65 @@
+package spark.network.netty;
+
+import java.io.File;
+import java.io.FileInputStream;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+import io.netty.channel.DefaultFileRegion;
+
+
+class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
+
+  PathResolver pResolver;
+
+  public FileServerHandler(PathResolver pResolver){
+    this.pResolver = pResolver;
+  }
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, String blockId) {
+    String path = pResolver.getAbsolutePath(blockId);
+    // if getFilePath returns null, close the channel
+    if (path == null) {
+      //ctx.close();
+      return;
+    }
+    File file = new File(path);
+    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();
+      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()));
+      } 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();
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+    cause.printStackTrace();
+    ctx.close();
+  }
+}
diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java
new file mode 100755
index 0000000000000000000000000000000000000000..302411672cf96a4dd55f1b81e938b1007d295c57
--- /dev/null
+++ b/core/src/main/java/spark/network/netty/PathResolver.java
@@ -0,0 +1,12 @@
+package spark.network.netty;
+
+
+public interface PathResolver {
+  /**
+   * Get the absolute path of the file
+   *
+   * @param fileId
+   * @return the absolute path of file
+   */
+  public String getAbsolutePath(String fileId);
+}
diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
index 53b0389c3a67373394e8a30a9a12a2401718c45a..e1fb02157aafecd591d8dd2631c8bc88ed52eb60 100644
--- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
@@ -1,14 +1,19 @@
 package spark
 
-import executor.{ShuffleReadMetrics, TaskMetrics}
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-import spark.storage.{DelegateBlockFetchTracker, BlockManagerId}
-import util.{CompletionIterator, TimedIterator}
+import spark.executor.{ShuffleReadMetrics, TaskMetrics}
+import spark.serializer.Serializer
+import spark.storage.BlockManagerId
+import spark.util.CompletionIterator
+
 
 private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
-  override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = {
+
+  override def fetch[K, V](
+    shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) = {
+
     logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId))
     val blockManager = SparkEnv.get.blockManager
 
@@ -48,18 +53,17 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
       }
     }
 
-    val blockFetcherItr = blockManager.getMultiple(blocksByAddress)
-    val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker
-    itr.setDelegate(blockFetcherItr)
+    val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer)
+    val itr = blockFetcherItr.flatMap(unpackBlock)
+
     CompletionIterator[(K,V), Iterator[(K,V)]](itr, {
       val shuffleMetrics = new ShuffleReadMetrics
-      shuffleMetrics.shuffleReadMillis = itr.getNetMillis
-      shuffleMetrics.remoteFetchTime = itr.remoteFetchTime
-      shuffleMetrics.remoteFetchWaitTime = itr.remoteFetchWaitTime
-      shuffleMetrics.remoteBytesRead = itr.remoteBytesRead
-      shuffleMetrics.totalBlocksFetched = itr.totalBlocks
-      shuffleMetrics.localBlocksFetched = itr.numLocalBlocks
-      shuffleMetrics.remoteBlocksFetched = itr.numRemoteBlocks
+      shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime
+      shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime
+      shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead
+      shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks
+      shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks
+      shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks
       metrics.shuffleReadMetrics = Some(shuffleMetrics)
     })
   }
diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala
index c7b379a3fbe2d8da307693ec12f20de00832e339..f7a2b7e8027ef16c608b9a55e87db4ad9d1139c2 100644
--- a/core/src/main/scala/spark/CacheManager.scala
+++ b/core/src/main/scala/spark/CacheManager.scala
@@ -27,7 +27,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
           if (loading.contains(key)) {
             logInfo("Loading contains " + key + ", waiting...")
             while (loading.contains(key)) {
-              try {loading.wait()} catch {case _ =>}
+              try {loading.wait()} catch {case _ : Throwable =>}
             }
             logInfo("Loading no longer contains " + key + ", so returning cached result")
             // See whether someone else has successfully loaded it. The main way this would fail
diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala
index 98525b99c84201ba2b9728b3b07a18ffd2c4041e..50d6a1c5c9fb93559bbe470498c72952f2993085 100644
--- a/core/src/main/scala/spark/ClosureCleaner.scala
+++ b/core/src/main/scala/spark/ClosureCleaner.scala
@@ -8,12 +8,20 @@ import scala.collection.mutable.Set
 import org.objectweb.asm.{ClassReader, MethodVisitor, Type}
 import org.objectweb.asm.commons.EmptyVisitor
 import org.objectweb.asm.Opcodes._
+import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream}
 
 private[spark] object ClosureCleaner extends Logging {
   // Get an ASM class reader for a given class from the JAR that loaded it
   private def getClassReader(cls: Class[_]): ClassReader = {
-    new ClassReader(cls.getResourceAsStream(
-      cls.getName.replaceFirst("^.*\\.", "") + ".class"))
+    // Copy data over, before delegating to ClassReader - else we can run out of open file handles.
+    val className = cls.getName.replaceFirst("^.*\\.", "") + ".class"
+    val resourceStream = cls.getResourceAsStream(className)
+    // todo: Fixme - continuing with earlier behavior ...
+    if (resourceStream == null) return new ClassReader(resourceStream)
+
+    val baos = new ByteArrayOutputStream(128)
+    Utils.copyStream(resourceStream, baos, true)
+    new ClassReader(new ByteArrayInputStream(baos.toByteArray))
   }
 
   // Check whether a class represents a Scala closure
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala
index 5eea9073220f1259c61f1082eb78996294fc434f..2af44aa3834449216d01435d245425af74a12f0c 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/spark/Dependency.scala
@@ -25,10 +25,12 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
  * @param shuffleId the shuffle id
  * @param rdd the parent RDD
  * @param partitioner partitioner used to partition the shuffle output
+ * @param serializerClass class name of the serializer to use
  */
 class ShuffleDependency[K, V](
     @transient rdd: RDD[(K, V)],
-    val partitioner: Partitioner)
+    val partitioner: Partitioner,
+    val serializerClass: String = null)
   extends Dependency(rdd) {
 
   val shuffleId: Int = rdd.context.newShuffleId()
diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala
index a953081d245fab139e944e7217b1d00c7ef37c2b..40b0193f1994fb9099a13d3ac4db1e701b085920 100644
--- a/core/src/main/scala/spark/FetchFailedException.scala
+++ b/core/src/main/scala/spark/FetchFailedException.scala
@@ -3,18 +3,25 @@ package spark
 import spark.storage.BlockManagerId
 
 private[spark] class FetchFailedException(
-    val bmAddress: BlockManagerId,
-    val shuffleId: Int,
-    val mapId: Int,
-    val reduceId: Int,
+    taskEndReason: TaskEndReason,
+    message: String,
     cause: Throwable)
   extends Exception {
-  
-  override def getMessage(): String = 
-    "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId)
+
+  def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, cause: Throwable) =
+    this(FetchFailed(bmAddress, shuffleId, mapId, reduceId),
+      "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId),
+      cause)
+
+  def this (shuffleId: Int, reduceId: Int, cause: Throwable) =
+    this(FetchFailed(null, shuffleId, -1, reduceId),
+      "Unable to fetch locations from master: %d %d".format(shuffleId, reduceId), cause)
+
+  override def getMessage(): String = message
+
 
   override def getCause(): Throwable = cause
 
-  def toTaskEndReason: TaskEndReason =
-    FetchFailed(bmAddress, shuffleId, mapId, reduceId)
+  def toTaskEndReason: TaskEndReason = taskEndReason
+
 }
diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala
index afcf9f6db4a2be26ebe334b098c09a861b98a37d..5e8396edb92235de51e2ff6c33d29027ca670c9e 100644
--- a/core/src/main/scala/spark/HadoopWriter.scala
+++ b/core/src/main/scala/spark/HadoopWriter.scala
@@ -2,14 +2,10 @@ package org.apache.hadoop.mapred
 
 import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.fs.Path
-import org.apache.hadoop.util.ReflectionUtils
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
 
 import java.text.SimpleDateFormat
 import java.text.NumberFormat
 import java.io.IOException
-import java.net.URI
 import java.util.Date
 
 import spark.Logging
@@ -24,7 +20,7 @@ import spark.SerializableWritable
  * a filename to write to, etc, exactly like in a Hadoop MapReduce job.
  */
 class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable {
-  
+
   private val now = new Date()
   private val conf = new SerializableWritable(jobConf)
   
@@ -106,6 +102,12 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe
     }
   }
 
+  def commitJob() {
+    // always ? Or if cmtr.needsTaskCommit ?
+    val cmtr = getOutputCommitter()
+    cmtr.commitJob(getJobContext())
+  }
+
   def cleanup() {
     getOutputCommitter().cleanupJob(getJobContext())
   }
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala
index 7c1c1bb1440bcbd263947481ee8a0d762d1587be..0fc8c314630bc9bc5f9223ae7948665b0c7b1e32 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/spark/Logging.scala
@@ -68,6 +68,10 @@ trait Logging {
     if (log.isErrorEnabled) log.error(msg, throwable)
   }
 
+  protected def isTraceEnabled(): Boolean = {
+    log.isTraceEnabled
+  }
+
   // Method for ensuring that logging is initialized, to avoid having multiple
   // threads do it concurrently (as SLF4J initialization is not thread safe).
   protected def initLogging() { log }
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index 866d630a6d27b8b41b2965c97146d8087d9f450b..fde597ffd1a30fd1891e016b9decc87abf474118 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -1,7 +1,6 @@
 package spark
 
 import java.io._
-import java.util.concurrent.ConcurrentHashMap
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 
 import scala.collection.mutable.HashMap
@@ -12,8 +11,7 @@ import akka.dispatch._
 import akka.pattern.ask
 import akka.remote._
 import akka.util.Duration
-import akka.util.Timeout
-import akka.util.duration._
+
 
 import spark.scheduler.MapStatus
 import spark.storage.BlockManagerId
@@ -40,10 +38,12 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac
 
 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 mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
+  private var 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.
@@ -52,7 +52,7 @@ private[spark] class MapOutputTracker extends Logging {
 
   // Cache a serialized version of the output statuses for each shuffle to send them out faster
   var cacheGeneration = generation
-  val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
+  private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
 
   val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
 
@@ -60,7 +60,6 @@ private[spark] class MapOutputTracker extends Logging {
   // throw a SparkException if this fails.
   def askTracker(message: Any): Any = {
     try {
-      val timeout = 10.seconds
       val future = trackerActor.ask(message)(timeout)
       return Await.result(future, timeout)
     } catch {
@@ -77,10 +76,9 @@ private[spark] class MapOutputTracker extends Logging {
   }
 
   def registerShuffle(shuffleId: Int, numMaps: Int) {
-    if (mapStatuses.get(shuffleId) != None) {
+    if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) {
       throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
     }
-    mapStatuses.put(shuffleId, new Array[MapStatus](numMaps))
   }
 
   def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
@@ -101,8 +99,9 @@ private[spark] class MapOutputTracker extends Logging {
   }
 
   def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
-    var array = mapStatuses(shuffleId)
-    if (array != null) {
+    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
@@ -115,13 +114,14 @@ private[spark] class MapOutputTracker extends Logging {
   }
 
   // Remembers which map output locations are currently being fetched on a worker
-  val fetching = new HashSet[Int]
+  private val fetching = new HashSet[Int]
 
   // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle
   def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = {
     val statuses = mapStatuses.get(shuffleId).orNull
     if (statuses == null) {
       logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
+      var fetchedStatuses: Array[MapStatus] = null
       fetching.synchronized {
         if (fetching.contains(shuffleId)) {
           // Someone else is fetching it; wait for them to be done
@@ -132,31 +132,48 @@ private[spark] class MapOutputTracker extends Logging {
               case e: InterruptedException =>
             }
           }
-          return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, mapStatuses(shuffleId))
-        } else {
+        }
+
+        // Either while we waited the fetch happened successfully, or
+        // someone fetched it in between the get and the fetching.synchronized.
+        fetchedStatuses = mapStatuses.get(shuffleId).orNull
+        if (fetchedStatuses == null) {
+          // We have to do the fetch, get others to wait for us.
           fetching += shuffleId
         }
       }
-      // We won the race to fetch the output locs; do so
-      logInfo("Doing the fetch; tracker actor = " + trackerActor)
-      val host = System.getProperty("spark.hostname", Utils.localHostName)
-      // This try-finally prevents hangs due to timeouts:
-      var fetchedStatuses: Array[MapStatus] = null
-      try {
-        val fetchedBytes =
-          askTracker(GetMapOutputStatuses(shuffleId, host)).asInstanceOf[Array[Byte]]
-        fetchedStatuses = deserializeStatuses(fetchedBytes)
-        logInfo("Got the output locations")
-        mapStatuses.put(shuffleId, fetchedStatuses)
-      } finally {
-        fetching.synchronized {
-          fetching -= shuffleId
-          fetching.notifyAll()
+      
+      if (fetchedStatuses == null) {
+        // We won the race to fetch the output locs; do so
+        logInfo("Doing the fetch; tracker actor = " + trackerActor)
+        val hostPort = Utils.localHostPort()
+        // This try-finally prevents hangs due to timeouts:
+        try {
+          val fetchedBytes =
+            askTracker(GetMapOutputStatuses(shuffleId, hostPort)).asInstanceOf[Array[Byte]]
+          fetchedStatuses = deserializeStatuses(fetchedBytes)
+          logInfo("Got the output locations")
+          mapStatuses.put(shuffleId, fetchedStatuses)
+        } finally {
+          fetching.synchronized {
+            fetching -= shuffleId
+            fetching.notifyAll()
+          }
+        }
+      }
+      if (fetchedStatuses != null) {
+        fetchedStatuses.synchronized {
+          return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses)
         }
       }
-      return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses)
+      else{
+        throw new FetchFailedException(null, shuffleId, -1, reduceId,
+          new Exception("Missing all output locations for shuffle " + shuffleId))
+      }      
     } else {
-      return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses)
+      statuses.synchronized {
+        return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses)
+      }
     }
   }
 
@@ -194,7 +211,8 @@ private[spark] class MapOutputTracker extends Logging {
     generationLock.synchronized {
       if (newGen > generation) {
         logInfo("Updating generation to " + newGen + " and clearing cache")
-        mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
+        // mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
+        mapStatuses.clear()
         generation = newGen
       }
     }
@@ -232,10 +250,13 @@ private[spark] class MapOutputTracker extends Logging {
   // 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.
-  def serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = {
+  private def serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = {
     val out = new ByteArrayOutputStream
     val objOut = new ObjectOutputStream(new GZIPOutputStream(out))
-    objOut.writeObject(statuses)
+    // Since statuses can be modified in parallel, sync on it
+    statuses.synchronized {
+      objOut.writeObject(statuses)
+    }
     objOut.close()
     out.toByteArray
   }
@@ -243,7 +264,10 @@ private[spark] class MapOutputTracker extends Logging {
   // Opposite of serializeStatuses.
   def deserializeStatuses(bytes: Array[Byte]): Array[MapStatus] = {
     val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes)))
-    objIn.readObject().asInstanceOf[Array[MapStatus]]
+    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 )
   }
 }
 
@@ -253,14 +277,11 @@ private[spark] object MapOutputTracker {
   // 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),
   // throw a FetchFailedException.
-  def convertMapStatuses(
+  private def convertMapStatuses(
         shuffleId: Int,
         reduceId: Int,
         statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = {
-    if (statuses == null) {
-      throw new FetchFailedException(null, shuffleId, -1, reduceId,
-        new Exception("Missing all output locations for shuffle " + shuffleId))
-    }
+    assert (statuses != null)
     statuses.map {
       status => 
         if (status == null) {
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index e7408e4352abfc2e53e20f91039e186ff72c139f..2b0e697337610f8fee2670262b6274bc1834afc9 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -52,7 +52,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       mergeValue: (C, V) => C,
       mergeCombiners: (C, C) => C,
       partitioner: Partitioner,
-      mapSideCombine: Boolean = true): RDD[(K, C)] = {
+      mapSideCombine: Boolean = true,
+      serializerClass: String = null): RDD[(K, C)] = {
     if (getKeyClass().isArray) {
       if (mapSideCombine) {
         throw new SparkException("Cannot use map-side combining with array keys.")
@@ -67,13 +68,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       self.mapPartitions(aggregator.combineValuesByKey(_), true)
     } else if (mapSideCombine) {
       val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true)
-      val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner)
+      val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner, serializerClass)
       partitioned.mapPartitions(aggregator.combineCombinersByKey(_), 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](self, partitioner)
+      val values = new ShuffledRDD[K, V](self, partitioner, serializerClass)
       values.mapPartitions(aggregator.combineValuesByKey(_), true)
     }
   }
@@ -88,6 +89,33 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions))
   }
 
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = {
+    combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner)
+  }
+
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = {
+    foldByKey(zeroValue, new HashPartitioner(numPartitions))(func)
+  }
+
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = {
+    foldByKey(zeroValue, defaultPartitioner(self))(func)
+  }
+
   /**
    * Merge the values for each key using an associative reduce function. This will also perform
    * the merging locally on each mapper before sending results to a reducer, similarly to a
@@ -440,6 +468,23 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     cogroup(other1, other2, defaultPartitioner(self, other1, other2))
   }
 
+  /**
+   * Return an RDD with the pairs from `this` whose keys are not in `other`.
+   *
+   * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
+   * RDD will be <= us.
+   */
+  def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] =
+    subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size)))
+
+  /** Return an RDD with the pairs from `this` whose keys are not in `other`. */
+  def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
+    subtractByKey(other, new HashPartitioner(numPartitions))
+
+  /** Return an RDD with the pairs from `this` whose keys are not in `other`. */
+  def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
+    new SubtractedRDD[K, V, W](self, other, p)
+
   /**
    * Return the list of values in the RDD for key `key`. This operation is done efficiently if the
    * RDD has a known partitioner by only searching the partition that the key maps to.
@@ -501,8 +546,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       // 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 = new TaskAttemptID(jobtrackerID,
-        stageId, false, context.splitId, attemptNumber)
+      val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber)
       val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
       val format = outputFormatClass.newInstance
       val committer = format.getOutputCommitter(hadoopContext)
@@ -521,11 +565,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
      * however we're only going to use this local OutputCommitter for
      * setupJob/commitJob, so we just use a dummy "map" task.
      */
-    val jobAttemptId = new TaskAttemptID(jobtrackerID, stageId, true, 0, 0)
+    val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0)
     val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
     val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
     jobCommitter.setupJob(jobTaskContext)
     val count = self.context.runJob(self, writeShard _).sum
+    jobCommitter.commitJob(jobTaskContext)
     jobCommitter.cleanupJob(jobTaskContext)
   }
 
@@ -593,6 +638,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     }
 
     self.context.runJob(self, writeToFile _)
+    writer.commitJob()
     writer.cleanup()
   }
 
@@ -600,7 +646,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * Return an RDD with the keys of each tuple.
    */
   def keys: RDD[K] = self.map(_._1)
-  
+
   /**
    * Return an RDD with the values of each tuple.
    */
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index 584efa8adf5f3877c914cf54b0fd22eefc325963..dde131696fdf2bf489c772d1c5eac8d4ad246790 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -31,9 +31,13 @@ import spark.rdd.MapPartitionsRDD
 import spark.rdd.MapPartitionsWithIndexRDD
 import spark.rdd.PipedRDD
 import spark.rdd.SampledRDD
+import spark.rdd.ShuffledRDD
 import spark.rdd.SubtractedRDD
 import spark.rdd.UnionRDD
 import spark.rdd.ZippedRDD
+import spark.rdd.ZippedPartitionsRDD2
+import spark.rdd.ZippedPartitionsRDD3
+import spark.rdd.ZippedPartitionsRDD4
 import spark.storage.StorageLevel
 
 import SparkContext._
@@ -103,7 +107,7 @@ abstract class RDD[T: ClassManifest](
   // =======================================================================
 
   /** A unique ID for this RDD (within its SparkContext). */
-  val id = sc.newRddId()
+  val id: Int = sc.newRddId()
 
   /** A friendly name for this RDD */
   var name: String = null
@@ -116,7 +120,8 @@ abstract class RDD[T: ClassManifest](
 
   /**
    * Set this RDD's storage level to persist its values across operations after the first time
-   * it is computed. Can only be called once on each RDD.
+   * it is computed. This can only be used to assign a new storage level if the RDD does not
+   * have a storage level set yet..
    */
   def persist(newLevel: StorageLevel): RDD[T] = {
     // TODO: Handle changes of StorageLevel
@@ -136,6 +141,15 @@ abstract class RDD[T: ClassManifest](
   /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
   def cache(): RDD[T] = persist()
 
+  /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */
+  def unpersist(): RDD[T] = {
+    logInfo("Removing RDD " + id + " from persistence list")
+    sc.env.blockManager.master.removeRdd(id)
+    sc.persistentRdds.remove(id)
+    storageLevel = StorageLevel.NONE
+    this
+  }
+
   /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
   def getStorageLevel = storageLevel
 
@@ -237,7 +251,14 @@ abstract class RDD[T: ClassManifest](
   /**
    * Return a new RDD that is reduced into `numPartitions` partitions.
    */
-  def coalesce(numPartitions: Int): RDD[T] = new CoalescedRDD(this, numPartitions)
+  def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = {
+    if (shuffle) {
+      // include a shuffle step so that our upstream tasks are still distributed
+      new CoalescedRDD(new ShuffledRDD(map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys
+    } else {
+      new CoalescedRDD(this, numPartitions)
+    }
+  }
 
   /**
    * Return a sampled subset of this RDD.
@@ -358,12 +379,68 @@ abstract class RDD[T: ClassManifest](
    * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
    * of the original partition.
    */
-  @deprecated("use mapPartitionsWithIndex")
+  @deprecated("use mapPartitionsWithIndex", "0.7.0")
   def mapPartitionsWithSplit[U: ClassManifest](
     f: (Int, Iterator[T]) => Iterator[U],
     preservesPartitioning: Boolean = false): RDD[U] =
     new MapPartitionsWithIndexRDD(this, sc.clean(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: ClassManifest, U: ClassManifest](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)
+  }
+
+  /**
+   * FlatMaps 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 flatMapWith[A: ClassManifest, U: ClassManifest](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)
+  }
+
+  /**
+   * Applies f to each element of 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 foreachWith[A: ClassManifest](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(_ => {})
+  }
+
+  /**
+   * Filters this RDD with p, where p 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 filterWith[A: ClassManifest](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)
+  }
+
   /**
    * Zips this RDD with another one, returning key-value pairs with the first element in each RDD,
    * second element in each RDD, etc. Assumes that the two RDDs have the *same number of
@@ -372,6 +449,31 @@ abstract class RDD[T: ClassManifest](
    */
   def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other)
 
+  /**
+   * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by
+   * applying a function to the zipped partitions. Assumes that all the RDDs have the
+   * *same number of partitions*, but does *not* require them to have the same number
+   * of elements in each partition.
+   */
+  def zipPartitions[B: ClassManifest, V: ClassManifest](
+      f: (Iterator[T], Iterator[B]) => Iterator[V],
+      rdd2: RDD[B]): RDD[V] =
+    new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2)
+
+  def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest](
+      f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V],
+      rdd2: RDD[B],
+      rdd3: RDD[C]): RDD[V] =
+    new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3)
+
+  def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest](
+      f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V],
+      rdd2: RDD[B],
+      rdd3: RDD[C],
+      rdd4: RDD[D]): RDD[V] =
+    new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4)
+
+
   // Actions (launch a job to return a value to the user program)
 
   /**
@@ -382,6 +484,14 @@ abstract class RDD[T: ClassManifest](
     sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF))
   }
 
+  /**
+   * 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))
+  }
+
   /**
    * Return an array that contains all of the elements in this RDD.
    */
@@ -404,7 +514,7 @@ abstract class RDD[T: ClassManifest](
 
   /**
    * Return an RDD with the elements from `this` that are not in `other`.
-   * 
+   *
    * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
    * RDD will be <= us.
    */
@@ -420,7 +530,23 @@ abstract class RDD[T: ClassManifest](
   /**
    * Return an RDD with the elements from `this` that are not in `other`.
    */
-  def subtract(other: RDD[T], p: Partitioner): RDD[T] = new SubtractedRDD[T](this, other, p)
+  def subtract(other: RDD[T], p: Partitioner): RDD[T] = {
+    if (partitioner == Some(p)) {
+      // Our partitioner knows how to handle T (which, since we have a partitioner, is
+      // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples
+      val p2 = new Partitioner() {
+        override def numPartitions = p.numPartitions
+        override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1)
+      }
+      // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies
+      // anyway, and when calling .keys, will not have a partitioner set, even though
+      // the SubtractedRDD will, thanks to p2's de-tupled partitioning, already be
+      // partitioned by the right/real keys (e.g. p).
+      this.map(x => (x, null)).subtractByKey(other.map((_, null)), p2).keys
+    } else {
+      this.map(x => (x, null)).subtractByKey(other.map((_, null)), p).keys
+    }
+  }
 
   /**
    * Reduces the elements of this RDD using the specified commutative and associative binary operator.
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala
index d00092e9845e2a888de7d7ca03ba23d90ae50ba9..57e0405fb4185a3bcd6336deb45f659170c1f2bb 100644
--- a/core/src/main/scala/spark/RDDCheckpointData.scala
+++ b/core/src/main/scala/spark/RDDCheckpointData.scala
@@ -1,6 +1,7 @@
 package spark
 
 import org.apache.hadoop.fs.Path
+import org.apache.hadoop.conf.Configuration
 import rdd.{CheckpointRDD, CoalescedRDD}
 import scheduler.{ResultTask, ShuffleMapTask}
 
@@ -62,14 +63,20 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
       }
     }
 
+    // Create the output path for the checkpoint
+    val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id)
+    val fs = path.getFileSystem(new Configuration())
+    if (!fs.mkdirs(path)) {
+      throw new SparkException("Failed to create checkpoint path " + path)
+    }
+
     // Save to file, and reload it as an RDD
-    val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id).toString
-    rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _)
-    val newRDD = new CheckpointRDD[T](rdd.context, path)
+    rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _)
+    val newRDD = new CheckpointRDD[T](rdd.context, path.toString)
 
     // Change the dependencies and partitions of the RDD
     RDDCheckpointData.synchronized {
-      cpFile = Some(path)
+      cpFile = Some(path.toString)
       cpRDD = Some(newRDD)
       rdd.markCheckpointed(newRDD)   // Update the RDD's dependencies and partitions
       cpState = Checkpointed
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index 6b4a11d6d3f9efc994ab0ccced9b529348542677..518034e07bf172ac96cd515b43543415c274cbf9 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -36,17 +36,17 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla
     self: RDD[(K, V)])
   extends Logging
   with Serializable {
-  
+
   private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
     val c = {
-      if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { 
+      if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) {
         classManifest[T].erasure
       } else {
         // We get the type of the Writable class by looking at the apply method which converts
         // from T to Writable. Since we have two apply methods we filter out the one which
-        // is of the form "java.lang.Object apply(java.lang.Object)"
+        // is not of the form "java.lang.Object apply(java.lang.Object)"
         implicitly[T => Writable].getClass.getDeclaredMethods().filter(
-            m => m.getReturnType().toString != "java.lang.Object" &&
+            m => m.getReturnType().toString != "class java.lang.Object" &&
                  m.getName() == "apply")(0).getReturnType
 
       }
@@ -69,17 +69,17 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla
     val valueClass = getWritableClass[V]
     val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass)
     val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass)
-  
-    logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) 
+
+    logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" )
     val format = classOf[SequenceFileOutputFormat[Writable, Writable]]
     if (!convertKey && !convertValue) {
-      self.saveAsHadoopFile(path, keyClass, valueClass, format) 
+      self.saveAsHadoopFile(path, keyClass, valueClass, format)
     } else if (!convertKey && convertValue) {
-      self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) 
+      self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format)
     } else if (convertKey && !convertValue) {
-      self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format) 
+      self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format)
     } else if (convertKey && convertValue) {
-      self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) 
-    } 
+      self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format)
+    }
   }
 }
diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala
index 442e9f0269dc48284c8e20411e1f72427f627b96..9513a001263f10b3cc2007efbaa0bfc931e6ab8e 100644
--- a/core/src/main/scala/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/spark/ShuffleFetcher.scala
@@ -1,13 +1,16 @@
 package spark
 
-import executor.TaskMetrics
+import spark.executor.TaskMetrics
+import spark.serializer.Serializer
+
 
 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[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : Iterator[(K,V)]
+  def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics,
+    serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[(K,V)]
 
   /** Stop the fetcher */
   def stop() {}
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index eef25ef588386bca5a994f43e6b2bdb9ef6601cb..b30ca600e29b25873afbfc56ddc7b0604cd4a741 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -1,49 +1,54 @@
 package spark
 
 import java.io._
-import java.util.concurrent.atomic.AtomicInteger
 import java.net.URI
 import java.util.Properties
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
 
+import scala.collection.JavaConversions._
 import scala.collection.Map
 import scala.collection.generic.Growable
 import scala.collection.mutable.HashMap
 import scala.collection.JavaConversions._
 import scala.util.DynamicVariable
+import scala.collection.mutable.{ConcurrentMap, HashMap}
+
+import akka.actor.Actor._
 
-import org.apache.hadoop.fs.Path
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapred.InputFormat
-import org.apache.hadoop.mapred.SequenceFileInputFormat
-import org.apache.hadoop.io.Writable
-import org.apache.hadoop.io.IntWritable
-import org.apache.hadoop.io.LongWritable
-import org.apache.hadoop.io.FloatWritable
-import org.apache.hadoop.io.DoubleWritable
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.ArrayWritable
 import org.apache.hadoop.io.BooleanWritable
 import org.apache.hadoop.io.BytesWritable
-import org.apache.hadoop.io.ArrayWritable
+import org.apache.hadoop.io.DoubleWritable
+import org.apache.hadoop.io.FloatWritable
+import org.apache.hadoop.io.IntWritable
+import org.apache.hadoop.io.LongWritable
 import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.Text
+import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapred.FileInputFormat
+import org.apache.hadoop.mapred.InputFormat
 import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapred.SequenceFileInputFormat
 import org.apache.hadoop.mapred.TextInputFormat
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
 import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
+import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
+
 import org.apache.mesos.MesosNativeLibrary
 
-import spark.deploy.LocalSparkCluster
-import spark.partial.ApproximateEvaluator
-import spark.partial.PartialResult
+import spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
+import spark.partial.{ApproximateEvaluator, PartialResult}
 import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
-import spark.scheduler._
+import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler}
+import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler}
 import spark.scheduler.local.LocalScheduler
-import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
 import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
-import spark.storage.BlockManagerUI
+import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo}
 import spark.util.{MetadataCleaner, TimeStampedHashMap}
-import spark.storage.{StorageStatus, StorageUtils, RDDInfo}
+
 
 /**
  * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@@ -61,7 +66,10 @@ class SparkContext(
     val appName: String,
     val sparkHome: String = null,
     val jars: Seq[String] = Nil,
-    val environment: Map[String, String] = Map())
+    val environment: Map[String, String] = Map(),
+    // 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]] = scala.collection.immutable.Map())
   extends Logging {
 
   // Ensure logging is initialized before we spawn any threads
@@ -69,7 +77,7 @@ class SparkContext(
 
   // Set Spark driver host and port system properties
   if (System.getProperty("spark.driver.host") == null) {
-    System.setProperty("spark.driver.host", Utils.localIpAddress)
+    System.setProperty("spark.driver.host", Utils.localHostName())
   }
   if (System.getProperty("spark.driver.port") == null) {
     System.setProperty("spark.driver.port", "0")
@@ -96,12 +104,14 @@ class SparkContext(
   private[spark] val addedJars = HashMap[String, Long]()
 
   // Keeps track of all persisted RDDs
-  private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]()
+  private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
   private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
 
 
   // Add each JAR given through the constructor
-  jars.foreach { addJar(_) }
+  if (jars != null) {
+    jars.foreach { addJar(_) }
+  }
 
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
@@ -113,7 +123,9 @@ class SparkContext(
       executorEnvs(key) = value
     }
   }
-  executorEnvs ++= environment
+  if (environment != null) {
+    executorEnvs ++= environment
+  }
 
   // Create and start the scheduler
   private var taskScheduler: TaskScheduler = {
@@ -166,6 +178,22 @@ class SparkContext(
         }
         scheduler
 
+      case "yarn-standalone" =>
+        val scheduler = try {
+          val clazz = Class.forName("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))
@@ -185,12 +213,12 @@ class SparkContext(
   }
   taskScheduler.start()
 
-  private var dagScheduler = new DAGScheduler(taskScheduler)
+  @volatile private var dagScheduler = new DAGScheduler(taskScheduler)
   dagScheduler.start()
 
   /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
   val hadoopConfiguration = {
-    val conf = new Configuration()
+    val conf = SparkHadoopUtil.newConfiguration()
     // Explicitly check for S3 environment variables
     if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
       conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
@@ -222,6 +250,8 @@ class SparkContext(
     }
     localProperties.value.setProperty(key,value)
   }
+  // Post init
+  taskScheduler.postStartHook()
 
   // Methods for creating RDDs
 
@@ -487,7 +517,7 @@ class SparkContext(
    */
   def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
     env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
-      (blockManagerId.ip + ":" + blockManagerId.port, mem)
+      (blockManagerId.host + ":" + blockManagerId.port, mem)
     }
   }
 
@@ -495,7 +525,7 @@ class SparkContext(
    * Return information about what RDDs are cached, if they are in mem or on disk, how much space
    * they take, etc.
    */
-  def getRDDStorageInfo : Array[RDDInfo] = {
+  def getRDDStorageInfo: Array[RDDInfo] = {
     StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
   }
 
@@ -506,7 +536,7 @@ class SparkContext(
   /**
    * Return information about blocks stored in all of the slaves
    */
-  def getExecutorStorageStatus : Array[StorageStatus] = {
+  def getExecutorStorageStatus: Array[StorageStatus] = {
     env.blockManager.master.getStorageStatus
   }
 
@@ -543,10 +573,13 @@ class SparkContext(
 
   /** Shut down the SparkContext. */
   def stop() {
-    if (dagScheduler != null) {
+    // Do this only if not stopped already - best case effort.
+    // prevent NPE if stopped more than once.
+    val dagSchedulerCopy = dagScheduler
+    dagScheduler = null
+    if (dagSchedulerCopy != null) {
       metadataCleaner.cancel()
-      dagScheduler.stop()
-      dagScheduler = null
+      dagSchedulerCopy.stop()
       taskScheduler = null
       // TODO: Cache.stop()?
       env.stop()
@@ -562,6 +595,7 @@ class SparkContext(
     }
   }
 
+
   /**
    * Get Spark's home location from either a value set through the constructor,
    * or the spark.home Java property, or the SPARK_HOME environment variable
@@ -700,7 +734,7 @@ class SparkContext(
    */
   def setCheckpointDir(dir: String, useExisting: Boolean = false) {
     val path = new Path(dir)
-    val fs = path.getFileSystem(new Configuration())
+    val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration())
     if (!useExisting) {
       if (fs.exists(path)) {
         throw new Exception("Checkpoint directory '" + path + "' already exists.")
diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala
index 7157fd26883d3a3f7b29fb71fc272886a92ecfd5..be1a04d619c1b2a53682d56d38a2bd1d2e30b59f 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/spark/SparkEnv.scala
@@ -3,13 +3,14 @@ package spark
 import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
 import akka.remote.RemoteActorRefProvider
 
-import serializer.Serializer
 import spark.broadcast.BroadcastManager
 import spark.storage.BlockManager
 import spark.storage.BlockManagerMaster
 import spark.network.ConnectionManager
+import spark.serializer.{Serializer, SerializerManager}
 import spark.util.AkkaUtils
 
+
 /**
  * Holds all the runtime environment objects for a running Spark instance (either master or worker),
  * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently
@@ -20,6 +21,7 @@ import spark.util.AkkaUtils
 class SparkEnv (
     val executorId: String,
     val actorSystem: ActorSystem,
+    val serializerManager: SerializerManager,
     val serializer: Serializer,
     val closureSerializer: Serializer,
     val cacheManager: CacheManager,
@@ -29,8 +31,11 @@ class SparkEnv (
     val blockManager: BlockManager,
     val connectionManager: ConnectionManager,
     val httpFileServer: HttpFileServer,
-    val sparkFilesDir: String
-  ) {
+    val sparkFilesDir: String,
+    // To be set only as part of initialization of SparkContext.
+    // (executorId, defaultHostPort) => executorHostPort
+    // If executorId is NOT found, return defaultHostPort
+    var executorIdToHostPort: Option[(String, String) => String]) {
 
   def stop() {
     httpFileServer.stop()
@@ -44,6 +49,17 @@ class SparkEnv (
     // down, but let's call it anyway in case it gets fixed in a later release
     actorSystem.awaitTermination()
   }
+
+
+  def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = {
+    val env = SparkEnv.get
+    if (env.executorIdToHostPort.isEmpty) {
+      // default to using host, not host port. Relevant to non cluster modes.
+      return defaultHostPort
+    }
+
+    env.executorIdToHostPort.get(executorId, defaultHostPort)
+  }
 }
 
 object SparkEnv extends Logging {
@@ -72,6 +88,16 @@ object SparkEnv extends Logging {
       System.setProperty("spark.driver.port", boundPort.toString)
     }
 
+    // set only if unset until now.
+    if (System.getProperty("spark.hostPort", null) == null) {
+      if (!isDriver){
+        // unexpected
+        Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set")
+      }
+      Utils.checkHost(hostname)
+      System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+    }
+
     val classLoader = Thread.currentThread.getContextClassLoader
 
     // Create an instance of the class named by the given Java system property, or by
@@ -81,16 +107,23 @@ object SparkEnv extends Logging {
       Class.forName(name, true, classLoader).newInstance().asInstanceOf[T]
     }
 
-    val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer")
-    
+    val serializerManager = new SerializerManager
+
+    val serializer = serializerManager.setDefault(
+      System.getProperty("spark.serializer", "spark.JavaSerializer"))
+
+    val closureSerializer = serializerManager.get(
+      System.getProperty("spark.closure.serializer", "spark.JavaSerializer"))
+
     def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
       if (isDriver) {
         logInfo("Registering " + name)
         actorSystem.actorOf(Props(newActor), name = name)
       } else {
-        val driverIp: String = System.getProperty("spark.driver.host", "localhost")
+        val driverHost: String = System.getProperty("spark.driver.host", "localhost")
         val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
-        val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, name)
+        Utils.checkHost(driverHost, "Expected hostname")
+        val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
         logInfo("Connecting to " + name + ": " + url)
         actorSystem.actorFor(url)
       }
@@ -105,9 +138,6 @@ object SparkEnv extends Logging {
 
     val broadcastManager = new BroadcastManager(isDriver)
 
-    val closureSerializer = instantiateClass[Serializer](
-      "spark.closure.serializer", "spark.JavaSerializer")
-
     val cacheManager = new CacheManager(blockManager)
 
     // Have to assign trackerActor after initialization as MapOutputTrackerActor
@@ -142,6 +172,7 @@ object SparkEnv extends Logging {
     new SparkEnv(
       executorId,
       actorSystem,
+      serializerManager,
       serializer,
       closureSerializer,
       cacheManager,
@@ -151,7 +182,7 @@ object SparkEnv extends Logging {
       blockManager,
       connectionManager,
       httpFileServer,
-      sparkFilesDir)
+      sparkFilesDir,
+      None)
   }
-  
 }
diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala
index 420c54bc9a0d4b56f0b8c466f77de0a15a0f8be9..ca793eb4021a64dccddfb73c6ada2d5feaeeed59 100644
--- a/core/src/main/scala/spark/TaskEndReason.scala
+++ b/core/src/main/scala/spark/TaskEndReason.scala
@@ -14,9 +14,17 @@ private[spark] case object Success extends TaskEndReason
 private[spark] 
 case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it
 
-private[spark] 
-case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason
+private[spark] case class FetchFailed(
+    bmAddress: BlockManagerId,
+    shuffleId: Int,
+    mapId: Int,
+    reduceId: Int)
+  extends TaskEndReason
 
-private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason
+private[spark] case class ExceptionFailure(
+    className: String,
+    description: String,
+    stackTrace: Array[StackTraceElement])
+  extends TaskEndReason
 
 private[spark] case class OtherFailure(message: String) extends TaskEndReason
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index 81daacf958b5a03d3135f4587f2d6e411b0c6a3c..c1495d531714a8c38b3309f36096b165dceb6b0c 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -1,18 +1,18 @@
 package spark
 
 import java.io._
-import java.net._
+import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket}
 import java.util.{Locale, Random, UUID}
-import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
-import org.apache.hadoop.conf.Configuration
+import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor}
 import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, HashMap}
 import scala.collection.JavaConversions._
 import scala.io.Source
 import com.google.common.io.Files
 import com.google.common.util.concurrent.ThreadFactoryBuilder
-import scala.Some
 import spark.serializer.SerializerInstance
+import spark.deploy.SparkHadoopUtil
+import java.util.regex.Pattern
 
 /**
  * Various utility methods used by Spark.
@@ -68,6 +68,41 @@ private object Utils extends Logging {
     return buf
   }
 
+
+  private val shutdownDeletePaths = new collection.mutable.HashSet[String]()
+
+  // Register the path to be deleted via shutdown hook
+  def registerShutdownDeleteDir(file: File) {
+    val absolutePath = file.getAbsolutePath()
+    shutdownDeletePaths.synchronized {
+      shutdownDeletePaths += absolutePath
+    }
+  }
+
+  // Is the path already registered to be deleted via a shutdown hook ?
+  def hasShutdownDeleteDir(file: File): Boolean = {
+    val absolutePath = file.getAbsolutePath()
+    shutdownDeletePaths.synchronized {
+      shutdownDeletePaths.contains(absolutePath)
+    }
+  }
+
+  // Note: if file is child of some registered path, while not equal to it, then return true; else false
+  // This is to ensure that two shutdown hooks do not try to delete each others paths - resulting in IOException
+  // and incomplete cleanup
+  def hasRootAsShutdownDeleteDir(file: File): Boolean = {
+
+    val absolutePath = file.getAbsolutePath()
+
+    val retval = shutdownDeletePaths.synchronized {
+      shutdownDeletePaths.find(path => ! absolutePath.equals(path) && absolutePath.startsWith(path) ).isDefined
+    }
+
+    if (retval) logInfo("path = " + file + ", already present as root for deletion.")
+
+    retval
+  }
+
   /** Create a temporary directory inside the given parent directory */
   def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = {
     var attempts = 0
@@ -86,10 +121,14 @@ private object Utils extends Logging {
         }
       } catch { case e: IOException => ; }
     }
+
+    registerShutdownDeleteDir(dir)
+
     // Add a shutdown hook to delete the temp dir when the JVM exits
     Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) {
       override def run() {
-        Utils.deleteRecursively(dir)
+        // Attempt to delete if some patch which is parent of this is not already registered.
+        if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir)
       }
     })
     return dir
@@ -168,7 +207,7 @@ private object Utils extends Logging {
       case _ =>
         // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
         val uri = new URI(url)
-        val conf = new Configuration()
+        val conf = SparkHadoopUtil.newConfiguration()
         val fs = FileSystem.get(uri, conf)
         val in = fs.open(new Path(uri))
         val out = new FileOutputStream(tempFile)
@@ -227,8 +266,10 @@ private object Utils extends Logging {
 
   /**
    * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4).
+   * Note, this is typically not used from within core spark.
    */
   lazy val localIpAddress: String = findLocalIpAddress()
+  lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress)
 
   private def findLocalIpAddress(): String = {
     val defaultIpOverride = System.getenv("SPARK_LOCAL_IP")
@@ -266,6 +307,8 @@ private object Utils extends Logging {
    * hostname it reports to the master.
    */
   def setCustomHostname(hostname: String) {
+    // DEBUG code
+    Utils.checkHost(hostname)
     customHostname = Some(hostname)
   }
 
@@ -273,7 +316,102 @@ private object Utils extends Logging {
    * Get the local machine's hostname.
    */
   def localHostName(): String = {
-    customHostname.getOrElse(InetAddress.getLocalHost.getHostName)
+    customHostname.getOrElse(localIpAddressHostname)
+  }
+
+  def getAddressHostName(address: String): String = {
+    InetAddress.getByName(address).getHostName
+  }
+
+
+
+  def localHostPort(): String = {
+    val retval = System.getProperty("spark.hostPort", null)
+    if (retval == null) {
+      logErrorWithStack("spark.hostPort not set but invoking localHostPort")
+      return localHostName()
+    }
+
+    retval
+  }
+
+/*
+  // Used by DEBUG code : remove when all testing done
+  private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$")
+  def checkHost(host: String, message: String = "") {
+    // Currently catches only ipv4 pattern, this is just a debugging tool - not rigourous !
+    // if (host.matches("^[0-9]+(\\.[0-9]+)*$")) {
+    if (ipPattern.matcher(host).matches()) {
+      Utils.logErrorWithStack("Unexpected to have host " + host + " which matches IP pattern. Message " + message)
+    }
+    if (Utils.parseHostPort(host)._2 != 0){
+      Utils.logErrorWithStack("Unexpected to have host " + host + " which has port in it. Message " + message)
+    }
+  }
+
+  // Used by DEBUG code : remove when all testing done
+  def checkHostPort(hostPort: String, message: String = "") {
+    val (host, port) = Utils.parseHostPort(hostPort)
+    checkHost(host)
+    if (port <= 0){
+      Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message)
+    }
+  }
+
+  // Used by DEBUG code : remove when all testing done
+  def logErrorWithStack(msg: String) {
+    try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+    // temp code for debug
+    System.exit(-1)
+  }
+*/
+
+  // Once testing is complete in various modes, replace with this ?
+  def checkHost(host: String, message: String = "") {}
+  def checkHostPort(hostPort: String, message: String = "") {}
+
+  // Used by DEBUG code : remove when all testing done
+  def logErrorWithStack(msg: String) {
+    try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+  }
+
+  def getUserNameFromEnvironment(): String = {
+    SparkHadoopUtil.getUserNameFromEnvironment
+  }
+
+  // Typically, this will be of order of number of nodes in cluster
+  // If not, we should change it to LRUCache or something.
+  private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]()
+  def parseHostPort(hostPort: String): (String,  Int) = {
+    {
+      // Check cache first.
+      var cached = hostPortParseResults.get(hostPort)
+      if (cached != null) return cached
+    }
+
+    val indx: Int = hostPort.lastIndexOf(':')
+    // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now.
+    // For now, we assume that if port exists, then it is valid - not check if it is an int > 0
+    if (-1 == indx) {
+      val retval = (hostPort, 0)
+      hostPortParseResults.put(hostPort, retval)
+      return retval
+    }
+
+    val retval = (hostPort.substring(0, indx).trim(), hostPort.substring(indx + 1).trim().toInt)
+    hostPortParseResults.putIfAbsent(hostPort, retval)
+    hostPortParseResults.get(hostPort)
+  }
+
+  def addIfNoPort(hostPort: String,  port: Int): String = {
+    if (port <= 0) throw new IllegalArgumentException("Invalid port specified " + port)
+
+    // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now.
+    // For now, we assume that if port exists, then it is valid - not check if it is an int > 0
+    val indx: Int = hostPort.lastIndexOf(':')
+    if (-1 != indx) return hostPort
+
+    hostPort + ":" + port
   }
 
   private[spark] val daemonThreadFactory: ThreadFactory =
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index ba00b6a8448f1d28d2bd4d257aca9a62db8b7539..16692c044037c236f5c8c9469183837826df9f23 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
@@ -57,6 +57,12 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
    */
   def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions))
 
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  def coalesce(numPartitions: Int, shuffle: Boolean): JavaDoubleRDD =
+    fromRDD(srdd.coalesce(numPartitions, shuffle))
+
   /**
    * Return an RDD with the elements from `this` that are not in `other`.
    * 
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index c1bd13c49a9e64a0dcbbad35fa2c1e9f02c32d81..30084df4e20fc99e5ed00a69f76d661b8047b61d 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -66,7 +66,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   /**
    * Return a new RDD that is reduced into `numPartitions` partitions.
    */
-  def coalesce(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numPartitions))
+  def coalesce(numPartitions: Int): JavaPairRDD[K, V] = fromRDD(rdd.coalesce(numPartitions))
+
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  def coalesce(numPartitions: Int, shuffle: Boolean): JavaPairRDD[K, V] =
+    fromRDD(rdd.coalesce(numPartitions, shuffle))
 
   /**
    * Return a sampled subset of this RDD.
@@ -160,6 +166,30 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   : PartialResult[java.util.Map[K, BoundedDouble]] =
     rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap)
 
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] =
+    fromRDD(rdd.foldByKey(zeroValue, partitioner)(func))
+
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] =
+    fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func))
+
+  /**
+   * Merge the values for each key using an associative function and a neutral "zero value" which may
+   * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for
+   * list concatenation, 0 for addition, or 1 for multiplication.).
+   */
+  def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] =
+    fromRDD(rdd.foldByKey(zeroValue)(func))
+
   /**
    * Merge the values for each key using an associative reduce function. This will also perform
    * the merging locally on each mapper before sending results to a reducer, similarly to a
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index 301688889898e169e52b75951c159fb1b7a3159d..eb81ed64cd62caaff652b1a99fc3733b7f9e6057 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -14,12 +14,18 @@ JavaRDDLike[T, JavaRDD[T]] {
   /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
   def cache(): JavaRDD[T] = wrapRDD(rdd.cache())
 
-  /** 
+  /**
    * Set this RDD's storage level to persist its values across operations after the first time
-   * it is computed. Can only be called once on each RDD.
+   * it is computed. This can only be used to assign a new storage level if the RDD does not
+   * have a storage level set yet..
    */
   def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel))
 
+  /**
+   * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+   */
+  def unpersist(): JavaRDD[T] = wrapRDD(rdd.unpersist())
+
   // Transformations (return a new RDD)
 
   /**
@@ -31,7 +37,7 @@ JavaRDDLike[T, JavaRDD[T]] {
    * Return a new RDD containing the distinct elements in this RDD.
    */
   def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions))
-  
+
   /**
    * Return a new RDD containing only the elements that satisfy a predicate.
    */
@@ -43,12 +49,18 @@ JavaRDDLike[T, JavaRDD[T]] {
    */
   def coalesce(numPartitions: Int): JavaRDD[T] = rdd.coalesce(numPartitions)
 
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  def coalesce(numPartitions: Int, shuffle: Boolean): JavaRDD[T] =
+    rdd.coalesce(numPartitions, shuffle)
+
   /**
    * Return a sampled subset of this RDD.
    */
   def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] =
     wrapRDD(rdd.sample(withReplacement, fraction, seed))
-    
+
   /**
    * Return the union of this RDD and another one. Any identical elements will appear multiple
    * times (use `.distinct()` to eliminate them).
@@ -57,7 +69,7 @@ JavaRDDLike[T, JavaRDD[T]] {
 
   /**
    * Return an RDD with the elements from `this` that are not in `other`.
-   * 
+   *
    * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
    * RDD will be <= us.
    */
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index d884529d7a6f552227deb3989912efeff13cd5f2..9b74d1226f1d9c994a1bcf13e5e5919710fd0877 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -182,6 +182,21 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
     JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest)
   }
 
+  /**
+   * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by
+   * applying a function to the zipped partitions. Assumes that all the RDDs have the
+   * *same number of partitions*, but does *not* require them to have the same number
+   * of elements in each partition.
+   */
+  def zipPartitions[U, V](
+      f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V],
+      other: JavaRDDLike[U, _]): JavaRDD[V] = {
+    def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator(
+      f.apply(asJavaIterator(x), asJavaIterator(y)).iterator())
+    JavaRDD.fromRDD(
+      rdd.zipPartitions(fn, other.rdd)(other.classManifest, f.elementType()))(f.elementType())
+  }
+
   // Actions (launch a job to return a value to the user program)
 
   /**
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6044043add31611aece67b7489ff79aa5b1aee1f
--- /dev/null
+++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
@@ -0,0 +1,11 @@
+package spark.api.java.function
+
+/**
+ * A function that takes two inputs and returns zero or more output records.
+ */
+abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] {
+  @throws(classOf[Exception])
+  def call(a: A, b:B) : java.lang.Iterable[C]
+
+  def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]]
+}
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index 9b4d54ab4e0461364d643bb857b267ed4e05bed6..807119ca8c08a1430d3db582a16f02b148812983 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -277,6 +277,8 @@ private class BytesToString extends spark.api.java.function.Function[Array[Byte]
  */
 class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int)
   extends AccumulatorParam[JList[Array[Byte]]] {
+
+  Utils.checkHost(serverHost, "Expected hostname")
   
   override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
 
diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
index 6659e53b25f370f1b6747a03b7a42ec147b121de..02193c7008d74202277d393bbf33dc626f21908f 100644
--- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
@@ -2,10 +2,11 @@ package spark.deploy
 
 private[spark] class ApplicationDescription(
     val name: String,
-    val cores: Int,
+    val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */
     val memoryPerSlave: Int,
     val command: Command,
-    val sparkHome: String)
+    val sparkHome: String,
+    val appUiUrl: String)
   extends Serializable {
 
   val user = System.getProperty("user.name", "<unknown>")
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index 8a3e64e4c22fa60ef4bf05a02a14ff399aea2632..51274acb1ed3b8e167e30d874514ca3dfb9ca22b 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -4,6 +4,7 @@ import spark.deploy.ExecutorState.ExecutorState
 import spark.deploy.master.{WorkerInfo, ApplicationInfo}
 import spark.deploy.worker.ExecutorRunner
 import scala.collection.immutable.List
+import spark.Utils
 
 
 private[spark] sealed trait DeployMessage extends Serializable
@@ -19,7 +20,10 @@ case class RegisterWorker(
     memory: Int,
     webUiPort: Int,
     publicAddress: String)
-  extends DeployMessage
+  extends DeployMessage {
+  Utils.checkHost(host, "Required hostname")
+  assert (port > 0)
+}
 
 private[spark] 
 case class ExecutorStateChanged(
@@ -58,7 +62,9 @@ private[spark]
 case class RegisteredApplication(appId: String) extends DeployMessage
 
 private[spark] 
-case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int)
+case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
+  Utils.checkHostPort(hostPort, "Required hostport")
+}
 
 private[spark]
 case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String],
@@ -81,6 +87,9 @@ private[spark]
 case class MasterState(host: String, port: Int, workers: Array[WorkerInfo],
   activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
 
+  Utils.checkHost(host, "Required hostname")
+  assert (port > 0)
+
   def uri = "spark://" + host + ":" + port
 }
 
@@ -92,4 +101,8 @@ private[spark] case object RequestWorkerState
 private[spark]
 case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner],
   finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, 
-  coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String)
+  coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {
+
+  Utils.checkHost(host, "Required hostname")
+  assert (port > 0)
+}
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala
index 38a6ebfc242c163a1f5be26c2c46dbacfe45da12..ea832101d217746fd30c3000332655ab792a565e 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala
@@ -12,6 +12,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
     def write(obj: WorkerInfo) = JsObject(
       "id" -> JsString(obj.id),
       "host" -> JsString(obj.host),
+      "port" -> JsNumber(obj.port),
       "webuiaddress" -> JsString(obj.webUiAddress),
       "cores" -> JsNumber(obj.cores),
       "coresused" -> JsNumber(obj.coresUsed),
@@ -25,7 +26,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
       "starttime" -> JsNumber(obj.startTime),
       "id" -> JsString(obj.id),
       "name" -> JsString(obj.desc.name),
-      "cores" -> JsNumber(obj.desc.cores),
+      "cores" -> JsNumber(obj.desc.maxCores),
       "user" -> JsString(obj.desc.user),
       "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave),
       "submitdate" -> JsString(obj.submitDate.toString))
@@ -34,7 +35,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
   implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] {
     def write(obj: ApplicationDescription) = JsObject(
       "name" -> JsString(obj.name),
-      "cores" -> JsNumber(obj.cores),
+      "cores" -> JsNumber(obj.maxCores),
       "memoryperslave" -> JsNumber(obj.memoryPerSlave),
       "user" -> JsString(obj.user)
     )
diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
index 22319a96caef7ff80f97259bccd8381b0f6514bd..55bb61b0ccf88918a691684871c7627d064d99f6 100644
--- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
@@ -18,7 +18,7 @@ import scala.collection.mutable.ArrayBuffer
 private[spark]
 class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging {
   
-  private val localIpAddress = Utils.localIpAddress
+  private val localHostname = Utils.localHostName()
   private val masterActorSystems = ArrayBuffer[ActorSystem]()
   private val workerActorSystems = ArrayBuffer[ActorSystem]()
   
@@ -26,13 +26,13 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
     logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
-    val (masterSystem, masterPort) = Master.startSystemAndActor(localIpAddress, 0, 0)
+    val (masterSystem, masterPort) = Master.startSystemAndActor(localHostname, 0, 0)
     masterActorSystems += masterSystem
-    val masterUrl = "spark://" + localIpAddress + ":" + masterPort
+    val masterUrl = "spark://" + localHostname + ":" + masterPort
 
     /* Start the Workers */
     for (workerNum <- 1 to numWorkers) {
-      val (workerSystem, _) = Worker.startSystemAndActor(localIpAddress, 0, 0, coresPerWorker,
+      val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker,
         memoryPerWorker, masterUrl, null, Some(workerNum))
       workerActorSystems += workerSystem
     }
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index 2fc5e657f96926a1363e4cef96c4837e855cac3a..4af44f9c164c5ff7052d58ec6aefa40dc7dca9e2 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -3,6 +3,7 @@ package spark.deploy.client
 import spark.deploy._
 import akka.actor._
 import akka.pattern.ask
+import akka.util.Duration
 import akka.util.duration._
 import akka.pattern.AskTimeoutException
 import spark.{SparkException, Logging}
@@ -59,10 +60,10 @@ private[spark] class Client(
         markDisconnected()
         context.stop(self)
 
-      case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) =>
+      case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) =>
         val fullId = appId + "/" + id
-        logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores))
-        listener.executorAdded(fullId, workerId, host, cores, memory)
+        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
@@ -112,7 +113,7 @@ private[spark] class Client(
   def stop() {
     if (actor != null) {
       try {
-        val timeout = 5.seconds
+        val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
         val future = actor.ask(StopClient)(timeout)
         Await.result(future, timeout)
       } catch {
diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala
index b7008321df564976d37ca9f428d7d47920a30bf3..e8c4083f9dfef8ec8acf0a3d3b5931cbd768f66c 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala
@@ -12,7 +12,7 @@ private[spark] trait ClientListener {
 
   def disconnected(): Unit
 
-  def executorAdded(fullId: String, workerId: String, host: String, cores: Int, memory: Int): 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/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala
index dc004b59ca5ac247d4e7c8125b775a1f1698e7ea..f195082808aab54bd6a30bbd923b58efc89103dc 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -16,7 +16,7 @@ private[spark] object TestClient {
       System.exit(0)
     }
 
-    def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) {}
+    def executorAdded(id: String, workerId: String, hostPort: String, cores: Int, memory: Int) {}
 
     def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {}
   }
@@ -25,7 +25,7 @@ private[spark] object TestClient {
     val url = args(0)
     val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
     val desc = new ApplicationDescription(
-      "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home")
+      "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)
     client.start()
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
index 3591a9407237a765003af9d59d99603c70cf06a8..785c16e2bef1a09a94075957c036821396afc107 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
@@ -10,7 +10,8 @@ private[spark] class ApplicationInfo(
     val id: String,
     val desc: ApplicationDescription,
     val submitDate: Date,
-    val driver: ActorRef)
+    val driver: ActorRef,
+    val appUiUrl: String)
 {
   var state = ApplicationState.WAITING
   var executors = new mutable.HashMap[Int, ExecutorInfo]
@@ -37,7 +38,7 @@ private[spark] class ApplicationInfo(
     coresGranted -= exec.cores
   }
 
-  def coresLeft: Int = desc.cores - coresGranted
+  def coresLeft: Int = desc.maxCores - coresGranted
 
   private var _retryCount = 0
 
@@ -60,4 +61,5 @@ private[spark] class ApplicationInfo(
       System.currentTimeMillis() - startTime
     }
   }
+
 }
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 71b9d0801d59407559fdb44e82f282842fe9b733..3e965e82ac6b733f0d3c9c697fbd4a5812275c0e 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -15,7 +15,7 @@ import spark.{Logging, SparkException, Utils}
 import spark.util.AkkaUtils
 
 
-private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
+private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
   val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
 
@@ -35,9 +35,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
 
   var firstApp: Option[ApplicationInfo] = None
 
+  Utils.checkHost(host, "Expected hostname")
+
   val masterPublicAddress = {
     val envVar = System.getenv("SPARK_PUBLIC_DNS")
-    if (envVar != null) envVar else ip
+    if (envVar != null) envVar else host
   }
 
   // As a temporary workaround before better ways of configuring memory, we allow users to set
@@ -46,7 +48,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
   val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
-    logInfo("Starting Spark master at spark://" + ip + ":" + port)
+    logInfo("Starting Spark master at spark://" + host + ":" + port)
     // Listen for remote client disconnection events, since they don't go through Akka's watch()
     context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
     startWebUi()
@@ -145,7 +147,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
     }
 
     case RequestMasterState => {
-      sender ! MasterState(ip, port, workers.toArray, apps.toArray, completedApps.toArray)
+      sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray)
     }
   }
 
@@ -211,13 +213,13 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
     logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
     worker.addExecutor(exec)
     worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
-    exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory)
+    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 = {
     // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them.
-    workers.filter(w => (w.host == host) && (w.state == WorkerState.DEAD)).foreach(workers -= _)
+    workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _)
     val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
     workers += worker
     idToWorker(worker.id) = worker
@@ -242,7 +244,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
   def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
-    val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver)
+    val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
     apps += app
     idToApp(app.id) = app
     actorToApp(driver) = app
@@ -273,6 +275,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
       for (exec <- app.executors.values) {
         exec.worker.removeExecutor(exec)
         exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
+        exec.state = ExecutorState.KILLED
       }
       app.markFinished(state)
       app.driver ! ApplicationRemoved(state.toString)
@@ -307,7 +310,7 @@ private[spark] object Master {
 
   def main(argStrings: Array[String]) {
     val args = new MasterArguments(argStrings)
-    val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort)
+    val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
     actorSystem.awaitTermination()
   }
 
diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
index 4ceab3fc036da556e359e3e6db24f6c7b79a8a3d..3d28ecabb4d0e95ce843ab434b45c297dc17de45 100644
--- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
@@ -7,13 +7,13 @@ import spark.Utils
  * Command-line parser for the master.
  */
 private[spark] class MasterArguments(args: Array[String]) {
-  var ip = Utils.localHostName()
+  var host = Utils.localHostName()
   var port = 7077
   var webUiPort = 8080
   
   // Check for settings in environment variables 
-  if (System.getenv("SPARK_MASTER_IP") != null) {
-    ip = System.getenv("SPARK_MASTER_IP")
+  if (System.getenv("SPARK_MASTER_HOST") != null) {
+    host = System.getenv("SPARK_MASTER_HOST")
   }
   if (System.getenv("SPARK_MASTER_PORT") != null) {
     port = System.getenv("SPARK_MASTER_PORT").toInt
@@ -26,7 +26,13 @@ private[spark] class MasterArguments(args: Array[String]) {
 
   def parse(args: List[String]): Unit = args match {
     case ("--ip" | "-i") :: value :: tail =>
-      ip = value
+      Utils.checkHost(value, "ip no longer supported, please use hostname " + value)
+      host = value
+      parse(tail)
+
+    case ("--host" | "-h") :: value :: tail =>
+      Utils.checkHost(value, "Please use hostname " + value)
+      host = value
       parse(tail)
 
     case ("--port" | "-p") :: IntParam(value) :: tail =>
@@ -54,7 +60,8 @@ private[spark] class MasterArguments(args: Array[String]) {
       "Usage: Master [options]\n" +
       "\n" +
       "Options:\n" +
-      "  -i IP, --ip IP         IP address or DNS name to listen on\n" +
+      "  -i HOST, --ip HOST     Hostname to listen on (deprecated, please use --host or -h) \n" +
+      "  -h HOST, --host HOST   Hostname to listen on\n" +
       "  -p PORT, --port PORT   Port to listen on (default: 7077)\n" +
       "  --webui-port PORT      Port for web UI (default: 8080)")
     System.exit(exitCode)
diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
index 54faa375fbd468e4ea05ce3fda9a9e41dfcea166..a4e21c81308e5b34c57eaa6dc1f604f47524d487 100644
--- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
@@ -3,7 +3,7 @@ package spark.deploy.master
 import akka.actor.{ActorRef, ActorSystem}
 import akka.dispatch.Await
 import akka.pattern.ask
-import akka.util.Timeout
+import akka.util.{Duration, Timeout}
 import akka.util.duration._
 import cc.spray.Directives
 import cc.spray.directives._
@@ -22,7 +22,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
   val RESOURCE_DIR = "spark/deploy/master/webui"
   val STATIC_RESOURCE_DIR = "spark/deploy/static"
   
-  implicit val timeout = Timeout(10 seconds)
+  implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds"))
   
   val handler = {
     get {
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
index 23df1bb463288721e38379023b3fd01c4c8632d8..0c08c5f417238c88c9a07c56dba778cc9e1a287f 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
@@ -2,6 +2,7 @@ package spark.deploy.master
 
 import akka.actor.ActorRef
 import scala.collection.mutable
+import spark.Utils
 
 private[spark] class WorkerInfo(
   val id: String,
@@ -13,6 +14,9 @@ private[spark] class WorkerInfo(
   val webUiPort: Int,
   val publicAddress: String) {
 
+  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
@@ -23,6 +27,11 @@ private[spark] class WorkerInfo(
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
 
+  def hostPort: String = {
+    assert (port > 0)
+    host + ":" + port
+  }
+
   def addExecutor(exec: ExecutorInfo) {
     executors(exec.fullId) = exec
     coresUsed += exec.cores
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index de11771c8e62d7cdb0b629a105bb8d1ca21e544c..04a774658e4260ab212bdad06e68b8d3dcdf320f 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -21,11 +21,13 @@ private[spark] class ExecutorRunner(
     val memory: Int,
     val worker: ActorRef,
     val workerId: String,
-    val hostname: String,
+    val hostPort: String,
     val sparkHome: File,
     val workDir: File)
   extends Logging {
 
+  Utils.checkHostPort(hostPort, "Expected hostport")
+
   val fullId = appId + "/" + execId
   var workerThread: Thread = null
   var process: Process = null
@@ -68,7 +70,7 @@ private[spark] class ExecutorRunner(
   /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */
   def substituteVariables(argument: String): String = argument match {
     case "{{EXECUTOR_ID}}" => execId.toString
-    case "{{HOSTNAME}}" => hostname
+    case "{{HOSTNAME}}" => Utils.parseHostPort(hostPort)._1
     case "{{CORES}}" => cores.toString
     case other => other
   }
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index 2bbc931316291fae8c911b3c24ca3105a0f799ec..3dc2207170730529843c141fab4f49f3652e33cc 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -16,7 +16,7 @@ import spark.deploy.master.Master
 import java.io.File
 
 private[spark] class Worker(
-    ip: String,
+    host: String,
     port: Int,
     webUiPort: Int,
     cores: Int,
@@ -25,6 +25,9 @@ private[spark] class Worker(
     workDirPath: String = null)
   extends Actor with Logging {
 
+  Utils.checkHost(host, "Expected hostname")
+  assert (port > 0)
+
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For worker and executor IDs
 
   // Send a heartbeat every (heartbeat timeout) / 4 milliseconds
@@ -39,7 +42,7 @@ private[spark] class Worker(
   val finishedExecutors = new HashMap[String, ExecutorRunner]
   val publicAddress = {
     val envVar = System.getenv("SPARK_PUBLIC_DNS")
-    if (envVar != null) envVar else ip
+    if (envVar != null) envVar else host
   }
 
   var coresUsed = 0
@@ -51,10 +54,14 @@ private[spark] class Worker(
   def createWorkDir() {
     workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work"))
     try {
-      if (!workDir.exists() && !workDir.mkdirs()) {
+      // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs()
+      // So attempting to create and then check if directory was created or not.
+      workDir.mkdirs()
+      if ( !workDir.exists() || !workDir.isDirectory) {
         logError("Failed to create work directory " + workDir)
         System.exit(1)
       }
+      assert (workDir.isDirectory)
     } catch {
       case e: Exception =>
         logError("Failed to create work directory " + workDir, e)
@@ -64,7 +71,7 @@ private[spark] class Worker(
 
   override def preStart() {
     logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format(
-      ip, port, cores, Utils.memoryMegabytesToString(memory)))
+      host, port, cores, Utils.memoryMegabytesToString(memory)))
     sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
     logInfo("Spark home: " + sparkHome)
     createWorkDir()
@@ -74,20 +81,14 @@ private[spark] class Worker(
 
   def connectToMaster() {
     logInfo("Connecting to master " + masterUrl)
-    try {
-      master = context.actorFor(Master.toAkkaUrl(masterUrl))
-      master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress)
-      context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
-      context.watch(master) // Doesn't work with remote actors, but useful for testing
-    } catch {
-      case e: Exception =>
-        logError("Failed to connect to master", e)
-        System.exit(1)
-    }
+    master = context.actorFor(Master.toAkkaUrl(masterUrl))
+    master ! RegisterWorker(workerId, host, port, cores, memory, webUiPort, publicAddress)
+    context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+    context.watch(master) // Doesn't work with remote actors, but useful for testing
   }
 
   def startWebUi() {
-    val webUi = new WorkerWebUI(context.system, self)
+    val webUi = new WorkerWebUI(context.system, self, workDir)
     try {
       AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler)
     } catch {
@@ -112,7 +113,7 @@ private[spark] class Worker(
     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, ip, new File(execSparkHome_), workDir)
+        appId, execId, appDesc, cores_, memory_, self, workerId, host + ":" + port, new File(execSparkHome_), workDir)
       executors(appId + "/" + execId) = manager
       manager.start()
       coresUsed += cores_
@@ -147,7 +148,7 @@ private[spark] class Worker(
       masterDisconnected()
       
     case RequestWorkerState => {
-      sender ! WorkerState(ip, port, workerId, executors.values.toList,
+      sender ! WorkerState(host, port, workerId, executors.values.toList,
         finishedExecutors.values.toList, masterUrl, cores, memory, 
         coresUsed, memoryUsed, masterWebUiUrl)
     }
@@ -162,7 +163,7 @@ private[spark] class Worker(
   }
 
   def generateWorkerId(): String = {
-    "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), ip, port)
+    "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), host, port)
   }
 
   override def postStop() {
@@ -173,7 +174,7 @@ private[spark] class Worker(
 private[spark] object Worker {
   def main(argStrings: Array[String]) {
     val args = new WorkerArguments(argStrings)
-    val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort, args.cores,
+    val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
       args.memory, args.master, args.workDir)
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 08f02bad80d7f47b2f019745216538eda2640223..2b96611ee3372426edb58e1a0bb34a68be352bb1 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -9,7 +9,7 @@ import java.lang.management.ManagementFactory
  * Command-line parser for the master.
  */
 private[spark] class WorkerArguments(args: Array[String]) {
-  var ip = Utils.localHostName()
+  var host = Utils.localHostName()
   var port = 0
   var webUiPort = 8081
   var cores = inferDefaultCores()
@@ -38,7 +38,13 @@ private[spark] class WorkerArguments(args: Array[String]) {
 
   def parse(args: List[String]): Unit = args match {
     case ("--ip" | "-i") :: value :: tail =>
-      ip = value
+      Utils.checkHost(value, "ip no longer supported, please use hostname " + value)
+      host = value
+      parse(tail)
+
+    case ("--host" | "-h") :: value :: tail =>
+      Utils.checkHost(value, "Please use hostname " + value)
+      host = value
       parse(tail)
 
     case ("--port" | "-p") :: IntParam(value) :: tail =>
@@ -93,7 +99,8 @@ private[spark] class WorkerArguments(args: Array[String]) {
       "  -c CORES, --cores CORES  Number of cores to use\n" +
       "  -m MEM, --memory MEM     Amount of memory to use (e.g. 1000M, 2G)\n" +
       "  -d DIR, --work-dir DIR   Directory to run apps in (default: SPARK_HOME/work)\n" +
-      "  -i IP, --ip IP           IP address or DNS name to listen on\n" +
+      "  -i HOST, --ip IP         Hostname to listen on (deprecated, please use --host or -h)\n" +
+      "  -h HOST, --host HOST     Hostname to listen on\n" +
       "  -p PORT, --port PORT     Port to listen on (default: random)\n" +
       "  --webui-port PORT        Port for web UI (default: 8081)")
     System.exit(exitCode)
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
index 135cc2e86cc9267661775a915f57f2fe470dec00..3235c50d1bd31ae87a972a084a03f94bfec829f4 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
@@ -3,7 +3,7 @@ package spark.deploy.worker
 import akka.actor.{ActorRef, ActorSystem}
 import akka.dispatch.Await
 import akka.pattern.ask
-import akka.util.Timeout
+import akka.util.{Duration, Timeout}
 import akka.util.duration._
 import cc.spray.Directives
 import cc.spray.typeconversion.TwirlSupport._
@@ -12,16 +12,17 @@ import cc.spray.typeconversion.SprayJsonSupport._
 
 import spark.deploy.{WorkerState, RequestWorkerState}
 import spark.deploy.JsonProtocol._
+import java.io.File
 
 /**
  * Web UI server for the standalone worker.
  */
 private[spark]
-class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives {
+class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) extends Directives {
   val RESOURCE_DIR = "spark/deploy/worker/webui"
   val STATIC_RESOURCE_DIR = "spark/deploy/static"
   
-  implicit val timeout = Timeout(10 seconds)
+  implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds"))
   
   val handler = {
     get {
@@ -43,7 +44,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct
       path("log") {
         parameters("appId", "executorId", "logType") { (appId, executorId, logType) =>
           respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) {
-            getFromFileName("work/" + appId + "/" + executorId + "/" + logType)
+            getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType)
           }
         }
       } ~
diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala
index 4474ef4593703423215a172d103d7b98a3952b36..da20b8454441c737c34cd6c561702bc45db367e2 100644
--- a/core/src/main/scala/spark/executor/Executor.scala
+++ b/core/src/main/scala/spark/executor/Executor.scala
@@ -16,66 +16,66 @@ import java.nio.ByteBuffer
 /**
  * The Mesos executor for Spark.
  */
-private[spark] class Executor extends Logging {
-  var urlClassLoader : ExecutorURLClassLoader = null
-  var threadPool: ExecutorService = null
-  var env: SparkEnv = null
+private[spark] class Executor(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) extends Logging {
 
   // 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]()
+  private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]()
+  private val currentJars: HashMap[String, Long] = new HashMap[String, Long]()
 
-  val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
+  private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
 
   initLogging()
 
-  def initialize(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) {
-    // Make sure the local hostname we report matches the cluster scheduler's name for this host
-    Utils.setCustomHostname(slaveHostname)
+  // No ip or host:port - just hostname
+  Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname")
+  // must not have port specified.
+  assert (0 == Utils.parseHostPort(slaveHostname)._2)
 
-    // Set spark.* system properties from executor arg
-    for ((key, value) <- properties) {
-      System.setProperty(key, value)
-    }
+  // Make sure the local hostname we report matches the cluster scheduler's name for this host
+  Utils.setCustomHostname(slaveHostname)
+
+  // Set spark.* system properties from executor arg
+  for ((key, value) <- properties) {
+    System.setProperty(key, value)
+  }
+
+  // Create our ClassLoader and set it on this thread
+  private val urlClassLoader = createClassLoader()
+  Thread.currentThread.setContextClassLoader(urlClassLoader)
 
-    // Create our ClassLoader and set it on this thread
-    urlClassLoader = createClassLoader()
-    Thread.currentThread.setContextClassLoader(urlClassLoader)
-
-    // 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)
-              }
+  // 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)
         }
       }
-    )
+    }
+  )
 
-    // Initialize Spark environment (using system properties read above)
-    env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false)
-    SparkEnv.set(env)
+  // Initialize Spark environment (using system properties read above)
+  val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false)
+  SparkEnv.set(env)
 
-    // Start worker thread pool
-    threadPool = new ThreadPoolExecutor(
-      1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable])
-  }
+  // Start worker thread pool
+  val threadPool = new ThreadPoolExecutor(
+    1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable])
 
   def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) {
     threadPool.execute(new TaskRunner(context, taskId, serializedTask))
@@ -122,7 +122,7 @@ private[spark] class Executor extends Logging {
         }
 
         case t: Throwable => {
-          val reason = ExceptionFailure(t)
+          val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace)
           context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
 
           // TODO: Should we exit the whole executor here? On the one hand, the failed task may
diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
index 818d6d1dda7f9774c60abfbf6ee0dff7730983a2..10f3531df0d350ee15487a0fc73ca6122a4150fd 100644
--- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
@@ -8,11 +8,12 @@ import com.google.protobuf.ByteString
 import spark.{Utils, Logging}
 import spark.TaskState
 
-private[spark] class MesosExecutorBackend(executor: Executor)
+private[spark] class MesosExecutorBackend
   extends MesosExecutor
   with ExecutorBackend
   with Logging {
 
+  var executor: Executor = null
   var driver: ExecutorDriver = null
 
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) {
@@ -32,16 +33,19 @@ private[spark] class MesosExecutorBackend(executor: Executor)
     logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue)
     this.driver = driver
     val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray)
-    executor.initialize(
+    executor = new Executor(
       executorInfo.getExecutorId.getValue,
       slaveInfo.getHostname,
-      properties
-    )
+      properties)
   }
 
   override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
     val taskId = taskInfo.getTaskId.getValue.toLong
-    executor.launchTask(this, taskId, taskInfo.getData.asReadOnlyByteBuffer)
+    if (executor == null) {
+      logError("Received launchTask but executor was null")
+    } else {
+      executor.launchTask(this, taskId, taskInfo.getData.asReadOnlyByteBuffer)
+    }
   }
 
   override def error(d: ExecutorDriver, message: String) {
@@ -68,7 +72,7 @@ private[spark] object MesosExecutorBackend {
   def main(args: Array[String]) {
     MesosNativeLibrary.load()
     // Create a new Executor and start it running
-    val runner = new MesosExecutorBackend(new Executor)
+    val runner = new MesosExecutorBackend()
     new MesosExecutorDriver(runner).run()
   }
 }
diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
index 9a82c3054c0fcf3ff6d0ecf5dc36c54d658fdaa0..ebe2ac68d8b2527656003aa91229365507d3352a 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
@@ -12,23 +12,27 @@ import spark.scheduler.cluster.RegisteredExecutor
 import spark.scheduler.cluster.LaunchTask
 import spark.scheduler.cluster.RegisterExecutorFailed
 import spark.scheduler.cluster.RegisterExecutor
+import spark.Utils
+import spark.deploy.SparkHadoopUtil
 
 private[spark] class StandaloneExecutorBackend(
-    executor: Executor,
     driverUrl: String,
     executorId: String,
-    hostname: String,
+    hostPort: String,
     cores: Int)
   extends Actor
   with ExecutorBackend
   with Logging {
 
+  Utils.checkHostPort(hostPort, "Expected hostport")
+
+  var executor: Executor = null
   var driver: ActorRef = null
 
   override def preStart() {
     logInfo("Connecting to driver: " + driverUrl)
     driver = context.actorFor(driverUrl)
-    driver ! RegisterExecutor(executorId, hostname, cores)
+    driver ! RegisterExecutor(executorId, hostPort, cores)
     context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
     context.watch(driver) // Doesn't work with remote actors, but useful for testing
   }
@@ -36,7 +40,8 @@ private[spark] class StandaloneExecutorBackend(
   override def receive = {
     case RegisteredExecutor(sparkProperties) =>
       logInfo("Successfully registered with driver")
-      executor.initialize(executorId, hostname, sparkProperties)
+      // Make this host instead of hostPort ?
+      executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties)
 
     case RegisterExecutorFailed(message) =>
       logError("Slave registration failed: " + message)
@@ -44,7 +49,12 @@ private[spark] class StandaloneExecutorBackend(
 
     case LaunchTask(taskDesc) =>
       logInfo("Got assigned task " + taskDesc.taskId)
-      executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
+      if (executor == null) {
+        logError("Received launchTask but executor was null")
+        System.exit(1)
+      } else {
+        executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
+      }
 
     case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
       logError("Driver terminated or disconnected! Shutting down.")
@@ -58,11 +68,30 @@ private[spark] class StandaloneExecutorBackend(
 
 private[spark] object StandaloneExecutorBackend {
   def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
+    SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores))
+  }
+
+  // This will be run 'as' the user
+  def run0(args: Product) {
+    assert(4 == args.productArity)
+    runImpl(args.productElement(0).asInstanceOf[String], 
+      args.productElement(1).asInstanceOf[String],
+      args.productElement(2).asInstanceOf[String],
+      args.productElement(3).asInstanceOf[Int])
+  }
+  
+  private def runImpl(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)
+    // set it
+    val sparkHostPort = hostname + ":" + boundPort
+    System.setProperty("spark.hostPort", sparkHostPort)
     val actor = actorSystem.actorOf(
-      Props(new StandaloneExecutorBackend(new Executor, driverUrl, executorId, hostname, cores)),
+      Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
       name = "Executor")
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala
index b9c07830f5c94a62440a7b53fb81721a5c0400e6..a7c56c237199b2cb98366219282d02bcc521389b 100644
--- a/core/src/main/scala/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/spark/executor/TaskMetrics.scala
@@ -49,14 +49,9 @@ class ShuffleReadMetrics extends Serializable {
   var localBlocksFetched: Int = _
 
   /**
-   * Total time to read shuffle data
+   * Total time that is spent blocked waiting for shuffle to fetch data
    */
-  var shuffleReadMillis: Long = _
-
-  /**
-   * Total time that is spent blocked waiting for shuffle to fetch remote data
-   */
-  var remoteFetchWaitTime: Long = _
+  var fetchWaitTime: Long = _
 
   /**
    * The total amount of time for all the shuffle fetches.  This adds up time from overlapping
diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7b0e489a6c0cc222afa1e8581ba4ccce3f71c93b
--- /dev/null
+++ b/core/src/main/scala/spark/network/BufferMessage.scala
@@ -0,0 +1,94 @@
+package spark.network
+
+import java.nio.ByteBuffer
+
+import scala.collection.mutable.ArrayBuffer
+
+import spark.storage.BlockManager
+
+
+private[spark]
+class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int)
+  extends Message(Message.BUFFER_MESSAGE, id_) {
+
+  val initialSize = currentSize()
+  var gotChunkForSendingOnce = false
+
+  def size = initialSize
+
+  def currentSize() = {
+    if (buffers == null || buffers.isEmpty) {
+      0
+    } else {
+      buffers.map(_.remaining).reduceLeft(_ + _)
+    }
+  }
+
+  def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = {
+    if (maxChunkSize <= 0) {
+      throw new Exception("Max chunk size is " + maxChunkSize)
+    }
+
+    if (size == 0 && gotChunkForSendingOnce == false) {
+      val newChunk = new MessageChunk(
+        new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null)
+      gotChunkForSendingOnce = true
+      return Some(newChunk)
+    }
+
+    while(!buffers.isEmpty) {
+      val buffer = buffers(0)
+      if (buffer.remaining == 0) {
+        BlockManager.dispose(buffer)
+        buffers -= buffer
+      } else {
+        val newBuffer = if (buffer.remaining <= maxChunkSize) {
+          buffer.duplicate()
+        } else {
+          buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer]
+        }
+        buffer.position(buffer.position + newBuffer.remaining)
+        val newChunk = new MessageChunk(new MessageChunkHeader(
+            typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer)
+        gotChunkForSendingOnce = true
+        return Some(newChunk)
+      }
+    }
+    None
+  }
+
+  def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = {
+    // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer
+    if (buffers.size > 1) {
+      throw new Exception("Attempting to get chunk from message with multiple data buffers")
+    }
+    val buffer = buffers(0)
+    if (buffer.remaining > 0) {
+      if (buffer.remaining < chunkSize) {
+        throw new Exception("Not enough space in data buffer for receiving chunk")
+      }
+      val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer]
+      buffer.position(buffer.position + newBuffer.remaining)
+      val newChunk = new MessageChunk(new MessageChunkHeader(
+          typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer)
+      return Some(newChunk)
+    }
+    None
+  }
+
+  def flip() {
+    buffers.foreach(_.flip)
+  }
+
+  def hasAckId() = (ackId != 0)
+
+  def isCompletelyReceived() = !buffers(0).hasRemaining
+
+  override def toString = {
+    if (hasAckId) {
+      "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")"
+    } else {
+      "BufferMessage(id = " + id + ", size = " + size + ")"
+    }
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala
index d1451bc2124c581eff01dfae5277612ea5c995c7..6e28f677a31aff4df52ab02bc9c48434e2fdd0ac 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/spark/network/Connection.scala
@@ -13,12 +13,13 @@ import java.net._
 
 private[spark]
 abstract class Connection(val channel: SocketChannel, val selector: Selector,
-                          val remoteConnectionManagerId: ConnectionManagerId) extends Logging {
+    val socketRemoteConnectionManagerId: ConnectionManagerId)
+  extends Logging {
+
   def this(channel_ : SocketChannel, selector_ : Selector) = {
     this(channel_, selector_,
-         ConnectionManagerId.fromSocketAddress(
-            channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]
-         ))
+      ConnectionManagerId.fromSocketAddress(
+        channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]))
   }
 
   channel.configureBlocking(false)
@@ -33,16 +34,47 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
 
   val remoteAddress = getRemoteAddress()
 
+  // Read channels typically do not register for write and write does not for read
+  // Now, we do have write registering for read too (temporarily), but this is to detect
+  // channel close NOT to actually read/consume data on it !
+  // How does this work if/when we move to SSL ?
+
+  // What is the interest to register with selector for when we want this connection to be selected
+  def registerInterest()
+
+  // What is the interest to register with selector for when we want this connection to
+  // be de-selected
+  // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack,
+  // it will be SelectionKey.OP_READ (until we fix it properly)
+  def unregisterInterest()
+
+  // On receiving a read event, should we change the interest for this channel or not ?
+  // Will be true for ReceivingConnection, false for SendingConnection.
+  def changeInterestForRead(): Boolean
+
+  // On receiving a write event, should we change the interest for this channel or not ?
+  // Will be false for ReceivingConnection, true for SendingConnection.
+  // Actually, for now, should not get triggered for ReceivingConnection
+  def changeInterestForWrite(): Boolean
+
+  def getRemoteConnectionManagerId(): ConnectionManagerId = {
+    socketRemoteConnectionManagerId
+  }
+
   def key() = channel.keyFor(selector)
 
   def getRemoteAddress() = channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]
 
-  def read() { 
-    throw new UnsupportedOperationException("Cannot read on connection of type " + this.getClass.toString) 
+  // Returns whether we have to register for further reads or not.
+  def read(): Boolean = {
+    throw new UnsupportedOperationException(
+      "Cannot read on connection of type " + this.getClass.toString)
   }
-  
-  def write() { 
-    throw new UnsupportedOperationException("Cannot write on connection of type " + this.getClass.toString) 
+
+  // Returns whether we have to register for further writes or not.
+  def write(): Boolean = {
+    throw new UnsupportedOperationException(
+      "Cannot write on connection of type " + this.getClass.toString)
   }
 
   def close() {
@@ -54,26 +86,32 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
     callOnCloseCallback()
   }
 
-  def onClose(callback: Connection => Unit) {onCloseCallback = callback}
+  def onClose(callback: Connection => Unit) {
+    onCloseCallback = callback
+  }
 
-  def onException(callback: (Connection, Exception) => Unit) {onExceptionCallback = callback}
+  def onException(callback: (Connection, Exception) => Unit) {
+    onExceptionCallback = callback
+  }
 
-  def onKeyInterestChange(callback: (Connection, Int) => Unit) {onKeyInterestChangeCallback = callback}
+  def onKeyInterestChange(callback: (Connection, Int) => Unit) {
+    onKeyInterestChangeCallback = callback
+  }
 
   def callOnExceptionCallback(e: Exception) {
     if (onExceptionCallback != null) {
       onExceptionCallback(this, e)
     } else {
-      logError("Error in connection to " + remoteConnectionManagerId + 
+      logError("Error in connection to " + getRemoteConnectionManagerId() +
         " and OnExceptionCallback not registered", e)
     }
   }
-  
+
   def callOnCloseCallback() {
     if (onCloseCallback != null) {
       onCloseCallback(this)
     } else {
-      logWarning("Connection to " + remoteConnectionManagerId + 
+      logWarning("Connection to " + getRemoteConnectionManagerId() +
         " closed and OnExceptionCallback not registered")
     }
 
@@ -81,7 +119,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
 
   def changeConnectionKeyInterest(ops: Int) {
     if (onKeyInterestChangeCallback != null) {
-      onKeyInterestChangeCallback(this, ops) 
+      onKeyInterestChangeCallback(this, ops)
     } else {
       throw new Exception("OnKeyInterestChangeCallback not registered")
     }
@@ -105,24 +143,25 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
     print(" (" + position + ", " + length + ")")
     buffer.position(curPosition)
   }
-
 }
 
 
-private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector,
-                                       remoteId_ : ConnectionManagerId)
-extends Connection(SocketChannel.open, selector_, remoteId_) {
+private[spark]
+class SendingConnection(val address: InetSocketAddress, selector_ : Selector,
+    remoteId_ : ConnectionManagerId)
+  extends Connection(SocketChannel.open, selector_, remoteId_) {
 
   class Outbox(fair: Int = 0) {
     val messages = new Queue[Message]()
-    val defaultChunkSize = 65536  //32768 //16384 
+    val defaultChunkSize = 65536  //32768 //16384
     var nextMessageToBeUsed = 0
 
     def addMessage(message: Message) {
-      messages.synchronized{ 
+      messages.synchronized{
         /*messages += message*/
         messages.enqueue(message)
-        logDebug("Added [" + message + "] to outbox for sending to [" + remoteConnectionManagerId + "]")
+        logDebug("Added [" + message + "] to outbox for sending to " +
+          "[" + getRemoteConnectionManagerId() + "]")
       }
     }
 
@@ -147,18 +186,18 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
               message.started = true
               message.startTime = System.currentTimeMillis
             }
-            return chunk 
+            return chunk
           } else {
-            /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/
+            /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/
             message.finishTime = System.currentTimeMillis
-            logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId +
+            logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() +
               "] in "  + message.timeTaken )
           }
         }
       }
       None
     }
-    
+
     private def getChunkRR(): Option[MessageChunk] = {
       messages.synchronized {
         while (!messages.isEmpty) {
@@ -170,15 +209,17 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
             messages.enqueue(message)
             nextMessageToBeUsed = nextMessageToBeUsed + 1
             if (!message.started) {
-              logDebug("Starting to send [" + message + "] to [" + remoteConnectionManagerId + "]")
+              logDebug(
+                "Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]")
               message.started = true
               message.startTime = System.currentTimeMillis
             }
-            logTrace("Sending chunk from [" + message+ "] to [" + remoteConnectionManagerId + "]")
-            return chunk 
+            logTrace(
+              "Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]")
+            return chunk
           } else {
             message.finishTime = System.currentTimeMillis
-            logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId +
+            logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() +
               "] in "  + message.timeTaken )
           }
         }
@@ -186,27 +227,40 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
       None
     }
   }
-  
-  val outbox = new Outbox(1) 
+
+  private val outbox = new Outbox(1)
   val currentBuffers = new ArrayBuffer[ByteBuffer]()
 
   /*channel.socket.setSendBufferSize(256 * 1024)*/
 
-  override def getRemoteAddress() = address 
+  override def getRemoteAddress() = address
+
+  val DEFAULT_INTEREST = SelectionKey.OP_READ
+
+  override def registerInterest() {
+    // Registering read too - does not really help in most cases, but for some
+    // it does - so let us keep it for now.
+    changeConnectionKeyInterest(SelectionKey.OP_WRITE | DEFAULT_INTEREST)
+  }
+
+  override def unregisterInterest() {
+    changeConnectionKeyInterest(DEFAULT_INTEREST)
+  }
 
   def send(message: Message) {
     outbox.synchronized {
       outbox.addMessage(message)
       if (channel.isConnected) {
-        changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
+        registerInterest()
       }
     }
   }
 
+  // MUST be called within the selector loop
   def connect() {
     try{
-      channel.connect(address)
       channel.register(selector, SelectionKey.OP_CONNECT)
+      channel.connect(address)
       logInfo("Initiating connection to [" + address + "]")
     } catch {
       case e: Exception => {
@@ -216,36 +270,52 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
     }
   }
 
-  def finishConnect() {
+  def finishConnect(force: Boolean): Boolean = {
     try {
-      channel.finishConnect
-      changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
+      // Typically, this should finish immediately since it was triggered by a connect
+      // selection - though need not necessarily always complete successfully.
+      val connected = channel.finishConnect
+      if (!force && !connected) {
+        logInfo(
+          "finish connect failed [" + address + "], " + outbox.messages.size + " messages pending")
+        return false
+      }
+
+      // Fallback to previous behavior - assume finishConnect completed
+      // This will happen only when finishConnect failed for some repeated number of times
+      // (10 or so)
+      // Is highly unlikely unless there was an unclean close of socket, etc
+      registerInterest()
       logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending")
+      return true
     } catch {
       case e: Exception => {
         logWarning("Error finishing connection to " + address, e)
         callOnExceptionCallback(e)
+        // ignore
+        return true
       }
     }
   }
 
-  override def write() {
-    try{
-      while(true) {
+  override def write(): Boolean = {
+    try {
+      while (true) {
         if (currentBuffers.size == 0) {
           outbox.synchronized {
             outbox.getChunk() match {
               case Some(chunk) => {
-                currentBuffers ++= chunk.buffers 
+                currentBuffers ++= chunk.buffers
               }
               case None => {
-                changeConnectionKeyInterest(SelectionKey.OP_READ)
-                return
+                // changeConnectionKeyInterest(0)
+                /*key.interestOps(0)*/
+                return false
               }
             }
           }
         }
-        
+
         if (currentBuffers.size > 0) {
           val buffer = currentBuffers(0)
           val remainingBytes = buffer.remaining
@@ -254,69 +324,109 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
             currentBuffers -= buffer
           }
           if (writtenBytes < remainingBytes) {
-            return
+            // re-register for write.
+            return true
           }
         }
       }
     } catch {
-      case e: Exception => { 
-        logWarning("Error writing in connection to " + remoteConnectionManagerId, e)
+      case e: Exception => {
+        logWarning("Error writing in connection to " + getRemoteConnectionManagerId(), e)
         callOnExceptionCallback(e)
         close()
+        return false
       }
     }
+    // should not happen - to keep scala compiler happy
+    return true
   }
 
-  override def read() {
+  // This is a hack to determine if remote socket was closed or not.
+  // SendingConnection DOES NOT expect to receive any data - if it does, it is an error
+  // For a bunch of cases, read will return -1 in case remote socket is closed : hence we
+  // register for reads to determine that.
+  override def read(): Boolean = {
     // We don't expect the other side to send anything; so, we just read to detect an error or EOF.
     try {
       val length = channel.read(ByteBuffer.allocate(1))
       if (length == -1) { // EOF
         close()
       } else if (length > 0) {
-        logWarning("Unexpected data read from SendingConnection to " + remoteConnectionManagerId)
+        logWarning(
+          "Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId())
       }
     } catch {
       case e: Exception =>
-        logError("Exception while reading SendingConnection to " + remoteConnectionManagerId, e)
+        logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), e)
         callOnExceptionCallback(e)
         close()
     }
+
+    false
   }
+
+  override def changeInterestForRead(): Boolean = false
+
+  override def changeInterestForWrite(): Boolean = true
 }
 
 
-private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) 
-extends Connection(channel_, selector_) {
-  
+// Must be created within selector loop - else deadlock
+private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector)
+  extends Connection(channel_, selector_) {
+
   class Inbox() {
     val messages = new HashMap[Int, BufferMessage]()
-    
+
     def getChunk(header: MessageChunkHeader): Option[MessageChunk] = {
-      
+
       def createNewMessage: BufferMessage = {
         val newMessage = Message.create(header).asInstanceOf[BufferMessage]
         newMessage.started = true
         newMessage.startTime = System.currentTimeMillis
-        logDebug("Starting to receive [" + newMessage + "] from [" + remoteConnectionManagerId + "]") 
+        logDebug(
+          "Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]")
         messages += ((newMessage.id, newMessage))
         newMessage
       }
-      
+
       val message = messages.getOrElseUpdate(header.id, createNewMessage)
-      logTrace("Receiving chunk of [" + message + "] from [" + remoteConnectionManagerId + "]")
+      logTrace(
+        "Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]")
       message.getChunkForReceiving(header.chunkSize)
     }
-    
+
     def getMessageForChunk(chunk: MessageChunk): Option[BufferMessage] = {
-      messages.get(chunk.header.id) 
+      messages.get(chunk.header.id)
     }
 
     def removeMessage(message: Message) {
       messages -= message.id
     }
   }
-  
+
+  @volatile private var inferredRemoteManagerId: ConnectionManagerId = null
+
+  override def getRemoteConnectionManagerId(): ConnectionManagerId = {
+    val currId = inferredRemoteManagerId
+    if (currId != null) currId else super.getRemoteConnectionManagerId()
+  }
+
+  // The reciever's remote address is the local socket on remote side : which is NOT
+  // the connection manager id of the receiver.
+  // We infer that from the messages we receive on the receiver socket.
+  private def processConnectionManagerId(header: MessageChunkHeader) {
+    val currId = inferredRemoteManagerId
+    if (header.address == null || currId != null) return
+
+    val managerId = ConnectionManagerId.fromSocketAddress(header.address)
+
+    if (managerId != null) {
+      inferredRemoteManagerId = managerId
+    }
+  }
+
+
   val inbox = new Inbox()
   val headerBuffer: ByteBuffer = ByteBuffer.allocate(MessageChunkHeader.HEADER_SIZE)
   var onReceiveCallback: (Connection , Message) => Unit = null
@@ -324,24 +434,29 @@ extends Connection(channel_, selector_) {
 
   channel.register(selector, SelectionKey.OP_READ)
 
-  override def read() {
+  override def read(): Boolean = {
     try {
       while (true) {
         if (currentChunk == null) {
           val headerBytesRead = channel.read(headerBuffer)
           if (headerBytesRead == -1) {
             close()
-            return
+            return false
           }
           if (headerBuffer.remaining > 0) {
-            return
+            // re-register for read event ...
+            return true
           }
           headerBuffer.flip
           if (headerBuffer.remaining != MessageChunkHeader.HEADER_SIZE) {
-            throw new Exception("Unexpected number of bytes (" + headerBuffer.remaining + ") in the header")
+            throw new Exception(
+              "Unexpected number of bytes (" + headerBuffer.remaining + ") in the header")
           }
           val header = MessageChunkHeader.create(headerBuffer)
           headerBuffer.clear()
+
+          processConnectionManagerId(header)
+
           header.typ match {
             case Message.BUFFER_MESSAGE => {
               if (header.totalSize == 0) {
@@ -349,7 +464,8 @@ extends Connection(channel_, selector_) {
                   onReceiveCallback(this, Message.create(header))
                 }
                 currentChunk = null
-                return
+                // re-register for read event ...
+                return true
               } else {
                 currentChunk = inbox.getChunk(header).orNull
               }
@@ -357,26 +473,28 @@ extends Connection(channel_, selector_) {
             case _ => throw new Exception("Message of unknown type received")
           }
         }
-        
+
         if (currentChunk == null) throw new Exception("No message chunk to receive data")
-       
+
         val bytesRead = channel.read(currentChunk.buffer)
         if (bytesRead == 0) {
-          return
+          // re-register for read event ...
+          return true
         } else if (bytesRead == -1) {
           close()
-          return
+          return false
         }
 
         /*logDebug("Read " + bytesRead + " bytes for the buffer")*/
-        
+
         if (currentChunk.buffer.remaining == 0) {
           /*println("Filled buffer at " + System.currentTimeMillis)*/
           val bufferMessage = inbox.getMessageForChunk(currentChunk).get
           if (bufferMessage.isCompletelyReceived) {
             bufferMessage.flip
             bufferMessage.finishTime = System.currentTimeMillis
-            logDebug("Finished receiving [" + bufferMessage + "] from [" + remoteConnectionManagerId + "] in " + bufferMessage.timeTaken) 
+            logDebug("Finished receiving [" + bufferMessage + "] from " +
+              "[" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken)
             if (onReceiveCallback != null) {
               onReceiveCallback(this, bufferMessage)
             }
@@ -386,13 +504,32 @@ extends Connection(channel_, selector_) {
         }
       }
     } catch {
-      case e: Exception  => { 
-        logWarning("Error reading from connection to " + remoteConnectionManagerId, e)
+      case e: Exception  => {
+        logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e)
         callOnExceptionCallback(e)
         close()
+        return false
       }
     }
+    // should not happen - to keep scala compiler happy
+    return true
   }
-  
+
   def onReceive(callback: (Connection, Message) => Unit) {onReceiveCallback = callback}
+
+  override def changeInterestForRead(): Boolean = true
+
+  override def changeInterestForWrite(): Boolean = {
+    throw new IllegalStateException("Unexpected invocation right now")
+  }
+
+  override def registerInterest() {
+    // Registering read too - does not really help in most cases, but for some
+    // it does - so let us keep it for now.
+    changeConnectionKeyInterest(SelectionKey.OP_READ)
+  }
+
+  override def unregisterInterest() {
+    changeConnectionKeyInterest(0)
+  }
 }
diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala
index b6ec664d7e81bb581f1be553b3b1dfaafca61865..624a094856778877326f47ec6c13a6b68fe0cc0a 100644
--- a/core/src/main/scala/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/spark/network/ConnectionManager.scala
@@ -6,28 +6,19 @@ import java.nio._
 import java.nio.channels._
 import java.nio.channels.spi._
 import java.net._
-import java.util.concurrent.Executors
+import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor}
 
+import scala.collection.mutable.HashSet
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.SynchronizedMap
 import scala.collection.mutable.SynchronizedQueue
-import scala.collection.mutable.Queue
 import scala.collection.mutable.ArrayBuffer
 
 import akka.dispatch.{Await, Promise, ExecutionContext, Future}
 import akka.util.Duration
 import akka.util.duration._
 
-private[spark] case class ConnectionManagerId(host: String, port: Int) {
-  def toSocketAddress() = new InetSocketAddress(host, port)
-}
 
-private[spark] object ConnectionManagerId {
-  def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = {
-    new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort())
-  }
-}
-  
 private[spark] class ConnectionManager(port: Int) extends Logging {
 
   class MessageStatus(
@@ -41,73 +32,263 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
 
     def markDone() { completionHandler(this) }
   }
-  
-  val selector = SelectorProvider.provider.openSelector()
-  val handleMessageExecutor = Executors.newFixedThreadPool(System.getProperty("spark.core.connection.handler.threads","20").toInt)
-  val serverChannel = ServerSocketChannel.open()
-  val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] 
-  val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection]
-  val messageStatuses = new HashMap[Int, MessageStatus] 
-  val connectionRequests = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection]
-  val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
-  val sendMessageRequests = new Queue[(Message, SendingConnection)]
+
+  private val selector = SelectorProvider.provider.openSelector()
+
+  private val handleMessageExecutor = new ThreadPoolExecutor(
+    System.getProperty("spark.core.connection.handler.threads.min","20").toInt,
+    System.getProperty("spark.core.connection.handler.threads.max","60").toInt,
+    System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    new LinkedBlockingDeque[Runnable]())
+
+  private val handleReadWriteExecutor = new ThreadPoolExecutor(
+    System.getProperty("spark.core.connection.io.threads.min","4").toInt,
+    System.getProperty("spark.core.connection.io.threads.max","32").toInt,
+    System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    new LinkedBlockingDeque[Runnable]())
+
+  // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap
+  private val handleConnectExecutor = new ThreadPoolExecutor(
+    System.getProperty("spark.core.connection.connect.threads.min","1").toInt,
+    System.getProperty("spark.core.connection.connect.threads.max","8").toInt,
+    System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    new LinkedBlockingDeque[Runnable]())
+
+  private val serverChannel = ServerSocketChannel.open()
+  private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection]
+  private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection]
+  private val messageStatuses = new HashMap[Int, MessageStatus]
+  private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
+  private val registerRequests = new SynchronizedQueue[SendingConnection]
 
   implicit val futureExecContext = ExecutionContext.fromExecutor(Utils.newDaemonCachedThreadPool())
 
-  var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null
+  private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null
 
   serverChannel.configureBlocking(false)
   serverChannel.socket.setReuseAddress(true)
-  serverChannel.socket.setReceiveBufferSize(256 * 1024) 
+  serverChannel.socket.setReceiveBufferSize(256 * 1024)
 
   serverChannel.socket.bind(new InetSocketAddress(port))
   serverChannel.register(selector, SelectionKey.OP_ACCEPT)
 
   val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort)
   logInfo("Bound socket to port " + serverChannel.socket.getLocalPort() + " with id = " + id)
-  
-  val selectorThread = new Thread("connection-manager-thread") {
+
+  private val selectorThread = new Thread("connection-manager-thread") {
     override def run() = ConnectionManager.this.run()
   }
   selectorThread.setDaemon(true)
   selectorThread.start()
 
-  private def run() {
-    try {
-      while(!selectorThread.isInterrupted) {
-        for ((connectionManagerId, sendingConnection) <- connectionRequests) {
-          sendingConnection.connect() 
-          addConnection(sendingConnection)
-          connectionRequests -= connectionManagerId
+  private val writeRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]()
+
+  private def triggerWrite(key: SelectionKey) {
+    val conn = connectionsByKey.getOrElse(key, null)
+    if (conn == null) return
+
+    writeRunnableStarted.synchronized {
+      // So that we do not trigger more write events while processing this one.
+      // The write method will re-register when done.
+      if (conn.changeInterestForWrite()) conn.unregisterInterest()
+      if (writeRunnableStarted.contains(key)) {
+        // key.interestOps(key.interestOps() & ~ SelectionKey.OP_WRITE)
+        return
+      }
+
+      writeRunnableStarted += key
+    }
+    handleReadWriteExecutor.execute(new Runnable {
+      override def run() {
+        var register: Boolean = false
+        try {
+          register = conn.write()
+        } finally {
+          writeRunnableStarted.synchronized {
+            writeRunnableStarted -= key
+            if (register && conn.changeInterestForWrite()) {
+              conn.registerInterest()
+            }
+          }
         }
-        sendMessageRequests.synchronized {
-          while (!sendMessageRequests.isEmpty) {
-            val (message, connection) = sendMessageRequests.dequeue
-            connection.send(message)
+      }
+    } )
+  }
+
+  private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]()
+
+  private def triggerRead(key: SelectionKey) {
+    val conn = connectionsByKey.getOrElse(key, null)
+    if (conn == null) return
+
+    readRunnableStarted.synchronized {
+      // So that we do not trigger more read events while processing this one.
+      // The read method will re-register when done.
+      if (conn.changeInterestForRead())conn.unregisterInterest()
+      if (readRunnableStarted.contains(key)) {
+        return
+      }
+
+      readRunnableStarted += key
+    }
+    handleReadWriteExecutor.execute(new Runnable {
+      override def run() {
+        var register: Boolean = false
+        try {
+          register = conn.read()
+        } finally {
+          readRunnableStarted.synchronized {
+            readRunnableStarted -= key
+            if (register && conn.changeInterestForRead()) {
+              conn.registerInterest()
+            }
           }
         }
+      }
+    } )
+  }
+
+  private def triggerConnect(key: SelectionKey) {
+    val conn = connectionsByKey.getOrElse(key, null).asInstanceOf[SendingConnection]
+    if (conn == null) return
+
+    // prevent other events from being triggered
+    // Since we are still trying to connect, we do not need to do the additional steps in triggerWrite
+    conn.changeConnectionKeyInterest(0)
+
+    handleConnectExecutor.execute(new Runnable {
+      override def run() {
+
+        var tries: Int = 10
+        while (tries >= 0) {
+          if (conn.finishConnect(false)) return
+          // Sleep ?
+          Thread.sleep(1)
+          tries -= 1
+        }
+
+        // fallback to previous behavior : we should not really come here since this method was
+        // triggered since channel became connectable : but at times, the first finishConnect need not
+        // succeed : hence the loop to retry a few 'times'.
+        conn.finishConnect(true)
+      }
+    } )
+  }
+
+  // MUST be called within selector loop - else deadlock.
+  private def triggerForceCloseByException(key: SelectionKey, e: Exception) {
+    try {
+      key.interestOps(0)
+    } catch {
+      // ignore exceptions
+      case e: Exception => logDebug("Ignoring exception", e)
+    }
+
+    val conn = connectionsByKey.getOrElse(key, null)
+    if (conn == null) return
+
+    // Pushing to connect threadpool
+    handleConnectExecutor.execute(new Runnable {
+      override def run() {
+        try {
+          conn.callOnExceptionCallback(e)
+        } catch {
+          // ignore exceptions
+          case e: Exception => logDebug("Ignoring exception", e)
+        }
+        try {
+          conn.close()
+        } catch {
+          // ignore exceptions
+          case e: Exception => logDebug("Ignoring exception", e)
+        }
+      }
+    })
+  }
+
 
-        while (!keyInterestChangeRequests.isEmpty) {
+  def run() {
+    try {
+      while(!selectorThread.isInterrupted) {
+        while (! registerRequests.isEmpty) {
+          val conn: SendingConnection = registerRequests.dequeue
+          addListeners(conn)
+          conn.connect()
+          addConnection(conn)
+        }
+
+        while(!keyInterestChangeRequests.isEmpty) {
           val (key, ops) = keyInterestChangeRequests.dequeue
-          val connection = connectionsByKey(key)
-          val lastOps = key.interestOps()
-          key.interestOps(ops)
-          
-          def intToOpStr(op: Int): String = {
-            val opStrs = ArrayBuffer[String]()
-            if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ"
-            if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE"
-            if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT"
-            if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT"
-            if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " "
+
+          try {
+            if (key.isValid) {
+              val connection = connectionsByKey.getOrElse(key, null)
+              if (connection != null) {
+                val lastOps = key.interestOps()
+                key.interestOps(ops)
+
+                // hot loop - prevent materialization of string if trace not enabled.
+                if (isTraceEnabled()) {
+                  def intToOpStr(op: Int): String = {
+                    val opStrs = ArrayBuffer[String]()
+                    if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ"
+                    if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE"
+                    if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT"
+                    if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT"
+                    if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " "
+                  }
+
+                  logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId()  +
+                    "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]")
+                }
+              }
+            } else {
+              logInfo("Key not valid ? " + key)
+              throw new CancelledKeyException()
+            }
+          } catch {
+            case e: CancelledKeyException => {
+              logInfo("key already cancelled ? " + key, e)
+              triggerForceCloseByException(key, e)
+            }
+            case e: Exception => {
+              logError("Exception processing key " + key, e)
+              triggerForceCloseByException(key, e)
+            }
           }
-          
-          logTrace("Changed key for connection to [" + connection.remoteConnectionManagerId  + 
-            "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]")
-          
         }
 
-        val selectedKeysCount = selector.select()
+        val selectedKeysCount =
+          try {
+            selector.select()
+          } catch {
+            // Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently.
+            case e: CancelledKeyException => {
+              // Some keys within the selectors list are invalid/closed. clear them.
+              val allKeys = selector.keys().iterator()
+
+              while (allKeys.hasNext()) {
+                val key = allKeys.next()
+                try {
+                  if (! key.isValid) {
+                    logInfo("Key not valid ? " + key)
+                    throw new CancelledKeyException()
+                  }
+                } catch {
+                  case e: CancelledKeyException => {
+                    logInfo("key already cancelled ? " + key, e)
+                    triggerForceCloseByException(key, e)
+                  }
+                  case e: Exception => {
+                    logError("Exception processing key " + key, e)
+                    triggerForceCloseByException(key, e)
+                  }
+                }
+              }
+            }
+            0
+          }
+
         if (selectedKeysCount == 0) {
           logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys")
         }
@@ -115,20 +296,40 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
           logInfo("Selector thread was interrupted!")
           return
         }
-        
-        val selectedKeys = selector.selectedKeys().iterator()
-        while (selectedKeys.hasNext()) {
-          val key = selectedKeys.next
-          selectedKeys.remove()
-          if (key.isValid) {
-            if (key.isAcceptable) {
-              acceptConnection(key)
-            } else if (key.isConnectable) {
-              connectionsByKey(key).asInstanceOf[SendingConnection].finishConnect()
-            } else if (key.isReadable) {
-              connectionsByKey(key).read()
-            } else if (key.isWritable) {
-              connectionsByKey(key).write()
+
+        if (0 != selectedKeysCount) {
+          val selectedKeys = selector.selectedKeys().iterator()
+          while (selectedKeys.hasNext()) {
+            val key = selectedKeys.next
+            selectedKeys.remove()
+            try {
+              if (key.isValid) {
+                if (key.isAcceptable) {
+                  acceptConnection(key)
+                } else
+                if (key.isConnectable) {
+                  triggerConnect(key)
+                } else
+                if (key.isReadable) {
+                  triggerRead(key)
+                } else
+                if (key.isWritable) {
+                  triggerWrite(key)
+                }
+              } else {
+                logInfo("Key not valid ? " + key)
+                throw new CancelledKeyException()
+              }
+            } catch {
+              // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException.
+              case e: CancelledKeyException => {
+                logInfo("key already cancelled ? " + key, e)
+                triggerForceCloseByException(key, e)
+              }
+              case e: Exception => {
+                logError("Exception processing key " + key, e)
+                triggerForceCloseByException(key, e)
+              }
             }
           }
         }
@@ -137,97 +338,119 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
       case e: Exception => logError("Error in select loop", e)
     }
   }
-  
-  private def acceptConnection(key: SelectionKey) {
+
+  def acceptConnection(key: SelectionKey) {
     val serverChannel = key.channel.asInstanceOf[ServerSocketChannel]
-    val newChannel = serverChannel.accept()
-    val newConnection = new ReceivingConnection(newChannel, selector)
-    newConnection.onReceive(receiveMessage)
-    newConnection.onClose(removeConnection)
-    addConnection(newConnection)
-    logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]")
-  }
 
-  private def addConnection(connection: Connection) {
-    connectionsByKey += ((connection.key, connection))
-    if (connection.isInstanceOf[SendingConnection]) {
-      val sendingConnection = connection.asInstanceOf[SendingConnection]
-      connectionsById += ((sendingConnection.remoteConnectionManagerId, sendingConnection))
+    var newChannel = serverChannel.accept()
+
+    // accept them all in a tight loop. non blocking accept with no processing, should be fine
+    while (newChannel != null) {
+      try {
+        val newConnection = new ReceivingConnection(newChannel, selector)
+        newConnection.onReceive(receiveMessage)
+        addListeners(newConnection)
+        addConnection(newConnection)
+        logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]")
+      } catch {
+        // might happen in case of issues with registering with selector
+        case e: Exception => logError("Error in accept loop", e)
+      }
+
+      newChannel = serverChannel.accept()
     }
+  }
+
+  private def addListeners(connection: Connection) {
     connection.onKeyInterestChange(changeConnectionKeyInterest)
     connection.onException(handleConnectionError)
     connection.onClose(removeConnection)
   }
 
-  private def removeConnection(connection: Connection) {
+  def addConnection(connection: Connection) {
+    connectionsByKey += ((connection.key, connection))
+  }
+
+  def removeConnection(connection: Connection) {
     connectionsByKey -= connection.key
-    if (connection.isInstanceOf[SendingConnection]) {
-      val sendingConnection = connection.asInstanceOf[SendingConnection]
-      val sendingConnectionManagerId = sendingConnection.remoteConnectionManagerId
-      logInfo("Removing SendingConnection to " + sendingConnectionManagerId)
-      
-      connectionsById -= sendingConnectionManagerId
-
-      messageStatuses.synchronized {
-        messageStatuses
-          .values.filter(_.connectionManagerId == sendingConnectionManagerId).foreach(status => {
-            logInfo("Notifying " + status)
-            status.synchronized {
-            status.attempted = true 
-             status.acked = false
-             status.markDone()
-            }
+
+    try {
+      if (connection.isInstanceOf[SendingConnection]) {
+        val sendingConnection = connection.asInstanceOf[SendingConnection]
+        val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId()
+        logInfo("Removing SendingConnection to " + sendingConnectionManagerId)
+
+        connectionsById -= sendingConnectionManagerId
+
+        messageStatuses.synchronized {
+          messageStatuses
+            .values.filter(_.connectionManagerId == sendingConnectionManagerId).foreach(status => {
+              logInfo("Notifying " + status)
+              status.synchronized {
+              status.attempted = true
+               status.acked = false
+               status.markDone()
+              }
+            })
+
+          messageStatuses.retain((i, status) => {
+            status.connectionManagerId != sendingConnectionManagerId
           })
+        }
+      } else if (connection.isInstanceOf[ReceivingConnection]) {
+        val receivingConnection = connection.asInstanceOf[ReceivingConnection]
+        val remoteConnectionManagerId = receivingConnection.getRemoteConnectionManagerId()
+        logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId)
+
+        val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId)
+          if (! sendingConnectionOpt.isDefined) {
+          logError("Corresponding SendingConnectionManagerId not found")
+          return
+        }
 
-        messageStatuses.retain((i, status) => { 
-          status.connectionManagerId != sendingConnectionManagerId 
-        })
-      }
-    } else if (connection.isInstanceOf[ReceivingConnection]) {
-      val receivingConnection = connection.asInstanceOf[ReceivingConnection]
-      val remoteConnectionManagerId = receivingConnection.remoteConnectionManagerId
-      logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId)
-      
-      val sendingConnectionManagerId = connectionsById.keys.find(_.host == remoteConnectionManagerId.host).orNull
-      if (sendingConnectionManagerId == null) {
-        logError("Corresponding SendingConnectionManagerId not found")
-        return
-      }
-      logInfo("Corresponding SendingConnectionManagerId is " + sendingConnectionManagerId)
-      
-      val sendingConnection = connectionsById(sendingConnectionManagerId)
-      sendingConnection.close()
-      connectionsById -= sendingConnectionManagerId
-      
-      messageStatuses.synchronized {
-        for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) {
-          logInfo("Notifying " + s)
-          s.synchronized {
-            s.attempted = true
-            s.acked = false
-            s.markDone()
+        val sendingConnection = sendingConnectionOpt.get
+        connectionsById -= remoteConnectionManagerId
+        sendingConnection.close()
+
+        val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId()
+
+        assert (sendingConnectionManagerId == remoteConnectionManagerId)
+
+        messageStatuses.synchronized {
+          for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) {
+            logInfo("Notifying " + s)
+            s.synchronized {
+              s.attempted = true
+              s.acked = false
+              s.markDone()
+            }
           }
-        }
 
-        messageStatuses.retain((i, status) => { 
-          status.connectionManagerId != sendingConnectionManagerId 
-        })
+          messageStatuses.retain((i, status) => {
+            status.connectionManagerId != sendingConnectionManagerId
+          })
+        }
       }
+    } finally {
+      // So that the selection keys can be removed.
+      wakeupSelector()
     }
   }
 
-  private def handleConnectionError(connection: Connection, e: Exception) {
-    logInfo("Handling connection error on connection to " + connection.remoteConnectionManagerId)
+  def handleConnectionError(connection: Connection, e: Exception) {
+    logInfo("Handling connection error on connection to " + connection.getRemoteConnectionManagerId())
     removeConnection(connection)
   }
 
-  private def changeConnectionKeyInterest(connection: Connection, ops: Int) {
-    keyInterestChangeRequests += ((connection.key, ops))  
+  def changeConnectionKeyInterest(connection: Connection, ops: Int) {
+    keyInterestChangeRequests += ((connection.key, ops))
+    // so that registerations happen !
+    wakeupSelector()
   }
 
-  private def receiveMessage(connection: Connection, message: Message) {
+  def receiveMessage(connection: Connection, message: Message) {
     val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress)
-    logDebug("Received [" + message + "] from [" + connectionManagerId + "]") 
+    logDebug("Received [" + message + "] from [" + connectionManagerId + "]")
     val runnable = new Runnable() {
       val creationTime = System.currentTimeMillis
       def run() {
@@ -247,11 +470,11 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
         if (bufferMessage.hasAckId) {
           val sentMessageStatus = messageStatuses.synchronized {
             messageStatuses.get(bufferMessage.ackId) match {
-              case Some(status) => { 
-                messageStatuses -= bufferMessage.ackId 
+              case Some(status) => {
+                messageStatuses -= bufferMessage.ackId
                 status
               }
-              case None => { 
+              case None => {
                 throw new Exception("Could not find reference for received ack message " + message.id)
                 null
               }
@@ -271,7 +494,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
             logDebug("Not calling back as callback is null")
             None
           }
-          
+
           if (ackMessage.isDefined) {
             if (!ackMessage.get.isInstanceOf[BufferMessage]) {
               logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass())
@@ -281,7 +504,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
             }
           }
 
-          sendMessage(connectionManagerId, ackMessage.getOrElse { 
+          sendMessage(connectionManagerId, ackMessage.getOrElse {
             Message.createBufferMessage(bufferMessage.id)
           })
         }
@@ -293,18 +516,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
   private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) {
     def startNewConnection(): SendingConnection = {
       val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port)
-      val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId,
-          new SendingConnection(inetSocketAddress, selector, connectionManagerId))
-      newConnection   
+      val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId)
+      registerRequests.enqueue(newConnection)
+
+      newConnection
     }
-    val lookupKey = ConnectionManagerId.fromSocketAddress(connectionManagerId.toSocketAddress)
-    val connection = connectionsById.getOrElse(lookupKey, startNewConnection())
+    // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it useful in our test-env ...
+    // If we do re-add it, we should consistently use it everywhere I guess ?
+    val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection())
     message.senderAddress = id.toSocketAddress()
     logDebug("Sending [" + message + "] to [" + connectionManagerId + "]")
-    /*connection.send(message)*/
-    sendMessageRequests.synchronized {
-      sendMessageRequests += ((message, connection))
-    }
+    connection.send(message)
+
+    wakeupSelector()
+  }
+
+  private def wakeupSelector() {
     selector.wakeup()
   }
 
@@ -337,6 +564,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
       logWarning("All connections not cleaned up")
     }
     handleMessageExecutor.shutdown()
+    handleReadWriteExecutor.shutdown()
+    handleConnectExecutor.shutdown()
     logInfo("ConnectionManager stopped")
   }
 }
@@ -346,17 +575,17 @@ private[spark] object ConnectionManager {
 
   def main(args: Array[String]) {
     val manager = new ConnectionManager(9999)
-    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
+    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       println("Received [" + msg + "] from [" + id + "]")
       None
     })
-    
+
     /*testSequentialSending(manager)*/
     /*System.gc()*/
 
     /*testParallelSending(manager)*/
     /*System.gc()*/
-    
+
     /*testParallelDecreasingSending(manager)*/
     /*System.gc()*/
 
@@ -368,9 +597,9 @@ private[spark] object ConnectionManager {
     println("--------------------------")
     println("Sequential Sending")
     println("--------------------------")
-    val size = 10 * 1024 * 1024 
+    val size = 10 * 1024 * 1024
     val count = 10
-    
+
     val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
     buffer.flip
 
@@ -386,7 +615,7 @@ private[spark] object ConnectionManager {
     println("--------------------------")
     println("Parallel Sending")
     println("--------------------------")
-    val size = 10 * 1024 * 1024 
+    val size = 10 * 1024 * 1024
     val count = 10
 
     val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
@@ -401,12 +630,12 @@ private[spark] object ConnectionManager {
       if (!g.isDefined) println("Failed")
     })
     val finishTime = System.currentTimeMillis
-    
+
     val mb = size * count / 1024.0 / 1024.0
     val ms = finishTime - startTime
     val tput = mb * 1000.0 / ms
     println("--------------------------")
-    println("Started at " + startTime + ", finished at " + finishTime) 
+    println("Started at " + startTime + ", finished at " + finishTime)
     println("Sent " + count + " messages of size " + size + " in " + ms + " ms (" + tput + " MB/s)")
     println("--------------------------")
     println()
@@ -416,7 +645,7 @@ private[spark] object ConnectionManager {
     println("--------------------------")
     println("Parallel Decreasing Sending")
     println("--------------------------")
-    val size = 10 * 1024 * 1024 
+    val size = 10 * 1024 * 1024
     val count = 10
     val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put(Array.tabulate[Byte](size * (i + 1))(x => x.toByte)))
     buffers.foreach(_.flip)
@@ -431,7 +660,7 @@ private[spark] object ConnectionManager {
       if (!g.isDefined) println("Failed")
     })
     val finishTime = System.currentTimeMillis
-    
+
     val ms = finishTime - startTime
     val tput = mb * 1000.0 / ms
     println("--------------------------")
@@ -445,7 +674,7 @@ private[spark] object ConnectionManager {
     println("--------------------------")
     println("Continuous Sending")
     println("--------------------------")
-    val size = 10 * 1024 * 1024 
+    val size = 10 * 1024 * 1024
     val count = 10
 
     val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b554e84251c14f8886198dd86c942d6dd80f4c34
--- /dev/null
+++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala
@@ -0,0 +1,21 @@
+package spark.network
+
+import java.net.InetSocketAddress
+
+import spark.Utils
+
+
+private[spark] case class ConnectionManagerId(host: String, port: Int) {
+  // DEBUG code
+  Utils.checkHost(host)
+  assert (port > 0)
+
+  def toSocketAddress() = new InetSocketAddress(host, port)
+}
+
+
+private[spark] object ConnectionManagerId {
+  def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = {
+    new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort())
+  }
+}
diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala
index 525751b5bf915e814aaa3960a5404d542605b434..d4f03610ebfdfb5f282728e3b9b481fb5c7b18d2 100644
--- a/core/src/main/scala/spark/network/Message.scala
+++ b/core/src/main/scala/spark/network/Message.scala
@@ -1,55 +1,10 @@
 package spark.network
 
-import spark._
-
-import scala.collection.mutable.ArrayBuffer
-
 import java.nio.ByteBuffer
-import java.net.InetAddress
 import java.net.InetSocketAddress
-import storage.BlockManager
-
-private[spark] class MessageChunkHeader(
-    val typ: Long,
-    val id: Int,
-    val totalSize: Int,
-    val chunkSize: Int,
-    val other: Int,
-    val address: InetSocketAddress) {
-  lazy val buffer = {
-    val ip = address.getAddress.getAddress() 
-    val port = address.getPort()
-    ByteBuffer.
-      allocate(MessageChunkHeader.HEADER_SIZE).
-      putLong(typ).
-      putInt(id).
-      putInt(totalSize).
-      putInt(chunkSize).
-      putInt(other).
-      putInt(ip.size).
-      put(ip).
-      putInt(port).
-      position(MessageChunkHeader.HEADER_SIZE).
-      flip.asInstanceOf[ByteBuffer]
-  }
-
-  override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + 
-      " and sizes " + totalSize + " / " + chunkSize + " bytes"
-}
 
-private[spark] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
-  val size = if (buffer == null) 0 else buffer.remaining
-  lazy val buffers = {
-    val ab = new ArrayBuffer[ByteBuffer]()
-    ab += header.buffer
-    if (buffer != null) { 
-      ab += buffer
-    }
-    ab
-  }
+import scala.collection.mutable.ArrayBuffer
 
-  override def toString = "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")"
-}
 
 private[spark] abstract class Message(val typ: Long, val id: Int) {
   var senderAddress: InetSocketAddress = null
@@ -58,120 +13,16 @@ private[spark] abstract class Message(val typ: Long, val id: Int) {
   var finishTime = -1L
 
   def size: Int
-  
+
   def getChunkForSending(maxChunkSize: Int): Option[MessageChunk]
-  
+
   def getChunkForReceiving(chunkSize: Int): Option[MessageChunk]
- 
+
   def timeTaken(): String = (finishTime - startTime).toString + " ms"
 
   override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")"
 }
 
-private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) 
-extends Message(Message.BUFFER_MESSAGE, id_) {
-  
-  val initialSize = currentSize() 
-  var gotChunkForSendingOnce = false
-  
-  def size = initialSize 
-
-  def currentSize() = {
-    if (buffers == null || buffers.isEmpty) {
-      0 
-    } else {
-      buffers.map(_.remaining).reduceLeft(_ + _)
-    }
-  }
-  
-  def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = {
-    if (maxChunkSize <= 0) {
-      throw new Exception("Max chunk size is " + maxChunkSize)
-    }
-
-    if (size == 0 && gotChunkForSendingOnce == false) {
-      val newChunk = new MessageChunk(new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null)
-      gotChunkForSendingOnce = true
-      return Some(newChunk)
-    }
-
-    while(!buffers.isEmpty) {
-      val buffer = buffers(0)
-      if (buffer.remaining == 0) {
-        BlockManager.dispose(buffer)
-        buffers -= buffer
-      } else {
-        val newBuffer = if (buffer.remaining <= maxChunkSize) {
-          buffer.duplicate()
-        } else {
-          buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer]
-        }
-        buffer.position(buffer.position + newBuffer.remaining)
-        val newChunk = new MessageChunk(new MessageChunkHeader(
-            typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer)
-        gotChunkForSendingOnce = true
-        return Some(newChunk)
-      }
-    }
-    None
-  }
-
-  def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = {
-    // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer
-    if (buffers.size > 1) {
-      throw new Exception("Attempting to get chunk from message with multiple data buffers")
-    }
-    val buffer = buffers(0)
-    if (buffer.remaining > 0) {
-      if (buffer.remaining < chunkSize) {
-        throw new Exception("Not enough space in data buffer for receiving chunk")
-      }
-      val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer]
-      buffer.position(buffer.position + newBuffer.remaining)
-      val newChunk = new MessageChunk(new MessageChunkHeader(
-          typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer)
-      return Some(newChunk)
-    }
-    None 
-  }
-
-  def flip() {
-    buffers.foreach(_.flip)
-  }
-
-  def hasAckId() = (ackId != 0)
-
-  def isCompletelyReceived() = !buffers(0).hasRemaining
-  
-  override def toString = {
-    if (hasAckId) {
-      "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")"
-    } else {
-      "BufferMessage(id = " + id + ", size = " + size + ")"
-    }
-  }
-}
-
-private[spark] object MessageChunkHeader {
-  val HEADER_SIZE = 40 
-  
-  def create(buffer: ByteBuffer): MessageChunkHeader = {
-    if (buffer.remaining != HEADER_SIZE) {
-      throw new IllegalArgumentException("Cannot convert buffer data to Message")
-    }
-    val typ = buffer.getLong()
-    val id = buffer.getInt()
-    val totalSize = buffer.getInt()
-    val chunkSize = buffer.getInt()
-    val other = buffer.getInt()
-    val ipSize = buffer.getInt()
-    val ipBytes = new Array[Byte](ipSize)
-    buffer.get(ipBytes)
-    val ip = InetAddress.getByAddress(ipBytes)
-    val port = buffer.getInt()
-    new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port))
-  }
-}
 
 private[spark] object Message {
   val BUFFER_MESSAGE = 1111111111L
@@ -180,14 +31,16 @@ private[spark] object Message {
 
   def getNewId() = synchronized {
     lastId += 1
-    if (lastId == 0) lastId += 1
+    if (lastId == 0) {
+      lastId += 1
+    }
     lastId
   }
 
   def createBufferMessage(dataBuffers: Seq[ByteBuffer], ackId: Int): BufferMessage = {
     if (dataBuffers == null) {
       return new BufferMessage(getNewId(), new ArrayBuffer[ByteBuffer], ackId)
-    } 
+    }
     if (dataBuffers.exists(_ == null)) {
       throw new Exception("Attempting to create buffer message with null buffer")
     }
@@ -196,7 +49,7 @@ private[spark] object Message {
 
   def createBufferMessage(dataBuffers: Seq[ByteBuffer]): BufferMessage =
     createBufferMessage(dataBuffers, 0)
-  
+
   def createBufferMessage(dataBuffer: ByteBuffer, ackId: Int): BufferMessage = {
     if (dataBuffer == null) {
       return createBufferMessage(Array(ByteBuffer.allocate(0)), ackId)
@@ -204,15 +57,18 @@ private[spark] object Message {
       return createBufferMessage(Array(dataBuffer), ackId)
     }
   }
- 
-  def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage = 
+
+  def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage =
     createBufferMessage(dataBuffer, 0)
-  
-  def createBufferMessage(ackId: Int): BufferMessage = createBufferMessage(new Array[ByteBuffer](0), ackId)
+
+  def createBufferMessage(ackId: Int): BufferMessage = {
+    createBufferMessage(new Array[ByteBuffer](0), ackId)
+  }
 
   def create(header: MessageChunkHeader): Message = {
     val newMessage: Message = header.typ match {
-      case BUFFER_MESSAGE => new BufferMessage(header.id, ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other)
+      case BUFFER_MESSAGE => new BufferMessage(header.id,
+        ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other)
     }
     newMessage.senderAddress = header.address
     newMessage
diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala
new file mode 100644
index 0000000000000000000000000000000000000000..aaf9204d0e61a363c7d7859e45fcd8f934b4ca79
--- /dev/null
+++ b/core/src/main/scala/spark/network/MessageChunk.scala
@@ -0,0 +1,25 @@
+package spark.network
+
+import java.nio.ByteBuffer
+
+import scala.collection.mutable.ArrayBuffer
+
+
+private[network]
+class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
+
+  val size = if (buffer == null) 0 else buffer.remaining
+
+  lazy val buffers = {
+    val ab = new ArrayBuffer[ByteBuffer]()
+    ab += header.buffer
+    if (buffer != null) {
+      ab += buffer
+    }
+    ab
+  }
+
+  override def toString = {
+    "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")"
+  }
+}
diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3693d509d68593aa0867178059fc20e80a6a79c9
--- /dev/null
+++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala
@@ -0,0 +1,58 @@
+package spark.network
+
+import java.net.InetAddress
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+
+
+private[spark] class MessageChunkHeader(
+    val typ: Long,
+    val id: Int,
+    val totalSize: Int,
+    val chunkSize: Int,
+    val other: Int,
+    val address: InetSocketAddress) {
+  lazy val buffer = {
+    // No need to change this, at 'use' time, we do a reverse lookup of the hostname.
+    // Refer to network.Connection
+    val ip = address.getAddress.getAddress()
+    val port = address.getPort()
+    ByteBuffer.
+      allocate(MessageChunkHeader.HEADER_SIZE).
+      putLong(typ).
+      putInt(id).
+      putInt(totalSize).
+      putInt(chunkSize).
+      putInt(other).
+      putInt(ip.size).
+      put(ip).
+      putInt(port).
+      position(MessageChunkHeader.HEADER_SIZE).
+      flip.asInstanceOf[ByteBuffer]
+  }
+
+  override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ +
+      " and sizes " + totalSize + " / " + chunkSize + " bytes"
+}
+
+
+private[spark] object MessageChunkHeader {
+  val HEADER_SIZE = 40
+
+  def create(buffer: ByteBuffer): MessageChunkHeader = {
+    if (buffer.remaining != HEADER_SIZE) {
+      throw new IllegalArgumentException("Cannot convert buffer data to Message")
+    }
+    val typ = buffer.getLong()
+    val id = buffer.getInt()
+    val totalSize = buffer.getInt()
+    val chunkSize = buffer.getInt()
+    val other = buffer.getInt()
+    val ipSize = buffer.getInt()
+    val ipBytes = new Array[Byte](ipSize)
+    buffer.get(ipBytes)
+    val ip = InetAddress.getByAddress(ipBytes)
+    val port = buffer.getInt()
+    new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port))
+  }
+}
diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala
new file mode 100644
index 0000000000000000000000000000000000000000..aed4254234e391361192b53aecebf947dfd2c0a8
--- /dev/null
+++ b/core/src/main/scala/spark/network/netty/FileHeader.scala
@@ -0,0 +1,57 @@
+package spark.network.netty
+
+import io.netty.buffer._
+
+import spark.Logging
+
+private[spark] class FileHeader (
+  val fileLen: Int,
+  val blockId: String) 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))
+    //padding the rest of header
+    if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) {
+      buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes)
+    } else {
+      throw new Exception("too long header " + buf.readableBytes) 
+      logInfo("too long header") 
+    }
+    buf
+  }
+
+}
+
+private[spark] object FileHeader {
+
+  val HEADER_SIZE = 40
+
+  def getFileLenOffset = 0
+  def getFileLenSize = Integer.SIZE/8
+
+  def create(buf: ByteBuf): FileHeader = {
+    val length = buf.readInt
+    val idLength = buf.readInt
+    val idBuilder = new StringBuilder(idLength)
+    for (i <- 1 to idLength) {
+      idBuilder += buf.readByte().asInstanceOf[Char]
+    }
+    val 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)
+
+  }
+}
+
diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a91f5a886d732234b2640b6305e0ecd54baec1c8
--- /dev/null
+++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
@@ -0,0 +1,84 @@
+package spark.network.netty
+
+import java.util.concurrent.Executors
+
+import io.netty.buffer.ByteBuf
+import io.netty.channel.ChannelHandlerContext
+import io.netty.util.CharsetUtil
+
+import spark.Logging
+import spark.network.ConnectionManagerId
+
+
+private[spark] class ShuffleCopier extends Logging {
+
+  def getBlock(cmId: ConnectionManagerId, blockId: String,
+      resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+
+    val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
+    val fc = new FileClient(handler)
+    fc.init()
+    fc.connect(cmId.host, cmId.port)
+    fc.sendRequest(blockId)
+    fc.waitForClose()
+    fc.close()
+  }
+
+  def getBlocks(cmId: ConnectionManagerId,
+    blocks: Seq[(String, Long)],
+    resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+
+    for ((blockId, size) <- blocks) {
+      getBlock(cmId, blockId, resultCollectCallback)
+    }
+  }
+}
+
+
+private[spark] object ShuffleCopier extends Logging {
+
+  private class ShuffleClientHandler(resultCollectCallBack: (String, Long, ByteBuf) => Unit)
+    extends FileClientHandler with Logging {
+
+    override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) {
+      logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)");
+      resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen))
+    }
+  }
+
+  def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) {
+    logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"")
+  }
+
+  def runGetBlock(host:String, port:Int, file:String){
+    val handler = new ShuffleClientHandler(echoResultCollectCallBack)
+    val fc = new FileClient(handler)
+    fc.init();
+    fc.connect(host, port)
+    fc.sendRequest(file)
+    fc.waitForClose();
+    fc.close()
+  }
+
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: ShuffleCopier <host> <port> <shuffle_block_id> <threads>")
+      System.exit(1)
+    }
+    val host = args(0)
+    val port = args(1).toInt
+    val file = args(2)
+    val threads = if (args.length > 3) args(3).toInt else 10
+
+    val copiers = Executors.newFixedThreadPool(80)
+    for (i <- Range(0, threads)) {
+      val runnable = new Runnable() {
+        def run() {
+          runGetBlock(host, port, file)
+        }
+      }
+      copiers.execute(runnable)
+    }
+    copiers.shutdown
+  }
+}
diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dc87fefc567949b4bc05e4de2c6002ce61752070
--- /dev/null
+++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala
@@ -0,0 +1,56 @@
+package spark.network.netty
+
+import java.io.File
+
+import spark.Logging
+
+
+private[spark] class ShuffleSender(val port: Int, val pResolver: PathResolver) extends Logging {
+  val server = new FileServer(pResolver)
+
+  Runtime.getRuntime().addShutdownHook(
+    new Thread() {
+      override def run() {
+        server.stop()
+      }
+    }
+  )
+
+  def start() {
+    server.run(port)
+  }
+}
+
+
+private[spark] object ShuffleSender {
+
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: ShuffleSender <port> <subDirsPerLocalDir> <list of shuffle_block_directories>")
+      System.exit(1)
+    }
+
+    val port = args(0).toInt
+    val subDirsPerLocalDir = args(1).toInt
+    val localDirs = args.drop(2).map(new File(_))
+
+    val pResovler = new PathResolver {
+      override def getAbsolutePath(blockId: String): String = {
+        if (!blockId.startsWith("shuffle_")) {
+          throw new Exception("Block " + blockId + " is not a shuffle block")
+        }
+        // Figure out which local directory it hashes to, and which subdirectory in that
+        val hash = math.abs(blockId.hashCode)
+        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 sender = new ShuffleSender(port, pResovler)
+
+    sender.start()
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index 7348c4f15bad60a94e736a2e10cec2a018233e99..719d4bf03e34b80b3d8a5661abfed12b1d986299 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,7 +1,7 @@
 package spark.rdd
 
-import scala.collection.mutable.HashMap
 import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
+import spark.storage.BlockManager
 
 private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
   val index = idx
@@ -11,12 +11,7 @@ private[spark]
 class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
   extends RDD[T](sc, Nil) {
 
-  @transient lazy val locations_  = {
-    val blockManager = SparkEnv.get.blockManager
-    /*val locations = blockIds.map(id => blockManager.getLocations(id))*/
-    val locations = blockManager.getLocations(blockIds)
-    HashMap(blockIds.zip(locations):_*)
-  }
+  @transient lazy val locations_ = BlockManager.blockIdsToExecutorLocations(blockIds, SparkEnv.get)
 
   override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => {
     new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 9e37bdf659201c2ec7be7dd84de83881d34d8704..43ee39c993a3c0b254fe35bbbf9bcd8fb167e10c 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -8,6 +8,7 @@ import org.apache.hadoop.util.ReflectionUtils
 import org.apache.hadoop.fs.Path
 import java.io.{File, IOException, EOFException}
 import java.text.NumberFormat
+import spark.deploy.SparkHadoopUtil
 
 private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
 
@@ -21,13 +22,20 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
   @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration)
 
   override def getPartitions: Array[Partition] = {
-    val dirContents = fs.listStatus(new Path(checkpointPath))
-    val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted
-    val numPartitions =  partitionFiles.size
-    if (numPartitions > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
-        ! partitionFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1)))) {
-      throw new SparkException("Invalid checkpoint directory: " + checkpointPath)
-    }
+    val cpath = new Path(checkpointPath)
+    val numPartitions =
+    // listStatus can throw exception if path does not exist.
+    if (fs.exists(cpath)) {
+      val dirContents = fs.listStatus(cpath)
+      val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted
+      val numPart =  partitionFiles.size
+      if (numPart > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
+          ! partitionFiles(numPart-1).endsWith(CheckpointRDD.splitIdToFile(numPart-1)))) {
+        throw new SparkException("Invalid checkpoint directory: " + checkpointPath)
+      }
+      numPart
+    } else 0
+
     Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i))
   }
 
@@ -58,7 +66,7 @@ private[spark] object CheckpointRDD extends Logging {
 
   def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) {
     val outputDir = new Path(path)
-    val fs = outputDir.getFileSystem(new Configuration())
+    val fs = outputDir.getFileSystem(SparkHadoopUtil.newConfiguration())
 
     val finalOutputName = splitIdToFile(ctx.splitId)
     val finalOutputPath = new Path(outputDir, finalOutputName)
@@ -83,6 +91,7 @@ private[spark] object CheckpointRDD extends Logging {
 
     if (!fs.rename(tempOutputPath, finalOutputPath)) {
       if (!fs.exists(finalOutputPath)) {
+        logInfo("Deleting tempOutputPath " + tempOutputPath)
         fs.delete(tempOutputPath, false)
         throw new IOException("Checkpoint failed: failed to save output of task: "
           + ctx.attemptId + " and final output path does not exist")
@@ -95,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging {
   }
 
   def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
-    val fs = path.getFileSystem(new Configuration())
+    val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration())
     val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
     val fileInputStream = fs.open(path, bufferSize)
     val serializer = SparkEnv.get.serializer.newInstance()
@@ -117,11 +126,11 @@ 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(new Configuration())
+    val fs = path.getFileSystem(SparkHadoopUtil.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")
     assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same")
-    fs.delete(path)
+    fs.delete(path, true)
   }
 }
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index 65b4621b87ed0370a82e8a73769a7b3e28ef33c2..7599ba1a0224b207c2b524bdea11805c35b9a67b 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -2,10 +2,11 @@ package spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}
 import java.util.{HashMap => JHashMap}
+
 import scala.collection.JavaConversions
 import scala.collection.mutable.ArrayBuffer
 
-import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Partition, TaskContext}
+import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext}
 import spark.{Dependency, OneToOneDependency, ShuffleDependency}
 
 
@@ -28,7 +29,8 @@ private[spark] case class NarrowCoGroupSplitDep(
 private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
 
 private[spark]
-class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Partition with Serializable {
+class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
+  extends Partition with Serializable {
   override val index: Int = idx
   override def hashCode(): Int = idx
 }
@@ -40,7 +42,20 @@ private[spark] class CoGroupAggregator
     { (b1, b2) => b1 ++ b2 })
   with Serializable
 
-class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
+
+/**
+ * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a
+ * tuple with the list of values for that key.
+ *
+ * @param rdds parent RDDs.
+ * @param part partitioner used to partition the shuffle output.
+ * @param mapSideCombine flag indicating whether to merge values before shuffle step.
+ */
+class CoGroupedRDD[K](
+  @transient var rdds: Seq[RDD[(K, _)]],
+  part: Partitioner,
+  val mapSideCombine: Boolean = true,
+  val serializerClass: String = null)
   extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) {
 
   private val aggr = new CoGroupAggregator
@@ -52,8 +67,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
         new OneToOneDependency(rdd)
       } else {
         logInfo("Adding shuffle dependency with " + rdd)
-        val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
-        new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
+        if (mapSideCombine) {
+          val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
+          new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part, serializerClass)
+        } else {
+          new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass)
+        }
       }
     }
   }
@@ -70,7 +89,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
           case _ =>
             new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
         }
-      }.toList)
+      }.toArray)
     }
     array
   }
@@ -82,6 +101,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
     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]]]
+
     def getSeq(k: K): Seq[ArrayBuffer[Any]] = {
       val seq = map.get(k)
       if (seq != null) {
@@ -92,6 +112,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
         seq
       }
     }
+
+    val ser = SparkEnv.get.serializerManager.get(serializerClass)
     for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
       case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
         // Read them from the parent
@@ -102,9 +124,16 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
       case ShuffleCoGroupSplitDep(shuffleId) => {
         // Read map outputs of shuffle
         val fetcher = SparkEnv.get.shuffleFetcher
-        val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics)
-        for ((k, vs) <- fetchItr) {
-          getSeq(k)(depNum) ++= vs
+        if (mapSideCombine) {
+          // With map side combine on, for each key, the shuffle fetcher returns a list of values.
+          fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics, ser).foreach {
+            case (key, values) => getSeq(key)(depNum) ++= values
+          }
+        } else {
+          // With map side combine off, for each key the shuffle fetcher returns a single value.
+          fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach {
+            case (key, value) => getSeq(key)(depNum) += value
+          }
         }
       }
     }
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
index 0d16cf6e85459156392ed2d168a7a5a06d4acf0a..6d862c0c2898a6b94a43ff8085bab6edc685066a 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -37,8 +37,8 @@ class CoalescedRDD[T: ClassManifest](
       prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) }
     } else {
       (0 until maxPartitions).map { i =>
-        val rangeStart = (i * prevSplits.length) / maxPartitions
-        val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions
+        val rangeStart = ((i.toLong * prevSplits.length) / maxPartitions).toInt
+        val rangeEnd = (((i.toLong + 1) * prevSplits.length) / maxPartitions).toInt
         new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray)
       }.toArray
     }
diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e4dd3a7fa7dce14b1641539412098d13bb0f8ea7
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala
@@ -0,0 +1,16 @@
+package spark.rdd
+
+import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
+
+
+/**
+ * An RDD that is empty, i.e. has no element in it.
+ */
+class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) {
+
+  override def getPartitions: Array[Partition] = Array.empty
+
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    throw new UnsupportedOperationException("empty RDD")
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index 78097502bca48d207a65563718079c638490d987..cbf5512e247a46773294a428b8f2d6f9d5034876 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -16,6 +16,8 @@ import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
 import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
+import spark.util.NextIterator
+import org.apache.hadoop.conf.Configurable
 
 
 /**
@@ -49,6 +51,9 @@ class HadoopRDD[K, V](
 
   override def getPartitions: Array[Partition] = {
     val inputFormat = createInputFormat(conf)
+    if (inputFormat.isInstanceOf[Configurable]) {
+      inputFormat.asInstanceOf[Configurable].setConf(conf)
+    }
     val inputSplits = inputFormat.getSplits(conf, minSplits)
     val array = new Array[Partition](inputSplits.size)
     for (i <- 0 until inputSplits.size) {
@@ -62,47 +67,34 @@ class HadoopRDD[K, V](
       .asInstanceOf[InputFormat[K, V]]
   }
 
-  override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
+  override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] {
     val split = theSplit.asInstanceOf[HadoopPartition]
     var reader: RecordReader[K, V] = null
 
     val conf = confBroadcast.value.value
     val fmt = createInputFormat(conf)
+    if (fmt.isInstanceOf[Configurable]) {
+      fmt.asInstanceOf[Configurable].setConf(conf)
+    }
     reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL)
 
     // Register an on-task-completion callback to close the input stream.
-    context.addOnCompleteCallback{ () => close() }
+    context.addOnCompleteCallback{ () => closeIfNeeded() }
 
     val key: K = reader.createKey()
     val value: V = reader.createValue()
-    var gotNext = false
-    var finished = false
-
-    override def hasNext: Boolean = {
-      if (!gotNext) {
-        try {
-          finished = !reader.next(key, value)
-        } catch {
-          case eof: EOFException =>
-            finished = true
-        }
-        gotNext = true
-      }
-      !finished
-    }
 
-    override def next: (K, V) = {
-      if (!gotNext) {
+    override def getNext() = {
+      try {
         finished = !reader.next(key, value)
+      } catch {
+        case eof: EOFException =>
+          finished = true
       }
-      if (finished) {
-        throw new NoSuchElementException("End of stream")
-      }
-      gotNext = false
       (key, value)
     }
 
-    private def close() {
+    override def close() {
       try {
         reader.close()
       } catch {
diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a50f40773738dd9bad1ef98e2af611f5758e5f94
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala
@@ -0,0 +1,103 @@
+package spark.rdd
+
+import java.sql.{Connection, ResultSet}
+
+import spark.{Logging, Partition, RDD, SparkContext, TaskContext}
+import spark.util.NextIterator
+
+private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition {
+  override def index = idx
+}
+
+/**
+ * An RDD that executes an SQL query on a JDBC connection and reads results.
+ * For usage example, see test case JdbcRDDSuite.
+ *
+ * @param getConnection a function that returns an open Connection.
+ *   The RDD takes care of closing the connection.
+ * @param sql the text of the query.
+ *   The query must contain two ? placeholders for parameters used to partition the results.
+ *   E.g. "select title, author from books where ? <= id and id <= ?"
+ * @param lowerBound the minimum value of the first placeholder
+ * @param upperBound the maximum value of the second placeholder
+ *   The lower and upper bounds are inclusive.
+ * @param numPartitions the number of partitions.
+ *   Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2,
+ *   the query would be executed twice, once with (1, 10) and once with (11, 20)
+ * @param mapRow a function from a ResultSet to a single row of the desired result type(s).
+ *   This should only call getInt, getString, etc; the RDD takes care of calling next.
+ *   The default maps a ResultSet to an array of Object.
+ */
+class JdbcRDD[T: ClassManifest](
+    sc: SparkContext,
+    getConnection: () => Connection,
+    sql: String,
+    lowerBound: Long,
+    upperBound: Long,
+    numPartitions: Int,
+    mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray _)
+  extends RDD[T](sc, Nil) with Logging {
+
+  override def getPartitions: Array[Partition] = {
+    // bounds are inclusive, hence the + 1 here and - 1 on end
+    val length = 1 + upperBound - lowerBound
+    (0 until numPartitions).map(i => {
+      val start = lowerBound + ((i * length) / numPartitions).toLong
+      val end = lowerBound + (((i + 1) * length) / numPartitions).toLong - 1
+      new JdbcPartition(i, start, end)
+    }).toArray
+  }
+
+  override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] {
+    context.addOnCompleteCallback{ () => closeIfNeeded() }
+    val part = thePart.asInstanceOf[JdbcPartition]
+    val conn = getConnection()
+    val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
+
+    // setFetchSize(Integer.MIN_VALUE) is a mysql driver specific way to force streaming results,
+    // rather than pulling entire resultset into memory.
+    // see http://dev.mysql.com/doc/refman/5.0/en/connector-j-reference-implementation-notes.html
+    if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) {
+      stmt.setFetchSize(Integer.MIN_VALUE)
+      logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ")
+    }
+
+    stmt.setLong(1, part.lower)
+    stmt.setLong(2, part.upper)
+    val rs = stmt.executeQuery()
+
+    override def getNext: T = {
+      if (rs.next()) {
+        mapRow(rs)
+      } else {
+        finished = true
+        null.asInstanceOf[T]
+      }
+    }
+
+    override def close() {
+      try {
+        if (null != rs && ! rs.isClosed()) rs.close()
+      } catch {
+        case e: Exception => logWarning("Exception closing resultset", e)
+      }
+      try {
+        if (null != stmt && ! stmt.isClosed()) stmt.close()
+      } catch {
+        case e: Exception => logWarning("Exception closing statement", e)
+      }
+      try {
+        if (null != conn && ! stmt.isClosed()) conn.close()
+        logInfo("closed connection")
+      } catch {
+        case e: Exception => logWarning("Exception closing connection", e)
+      }
+    }
+  }
+}
+
+object JdbcRDD {
+  def resultSetToObjectArray(rs: ResultSet) = {
+    Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1))
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index df2361025c75327837c4e13184e762261e4b7509..901d01ef3084f628839a4af8dcab2251b92cd953 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -3,7 +3,7 @@ package spark.rdd
 import java.text.SimpleDateFormat
 import java.util.Date
 
-import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce._
 
@@ -42,6 +42,9 @@ class NewHadoopRDD[K, V](
 
   override def getPartitions: Array[Partition] = {
     val inputFormat = inputFormatClass.newInstance
+    if (inputFormat.isInstanceOf[Configurable]) {
+      inputFormat.asInstanceOf[Configurable].setConf(conf)
+    }
     val jobContext = newJobContext(conf, jobId)
     val rawSplits = inputFormat.getSplits(jobContext).toArray
     val result = new Array[Partition](rawSplits.size)
@@ -54,9 +57,12 @@ class NewHadoopRDD[K, V](
   override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
     val split = theSplit.asInstanceOf[NewHadoopPartition]
     val conf = confBroadcast.value.value
-    val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0)
+    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)
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
index 51f02409b6a75d689159970e59af52f887fc8626..c7d1926b83d822866ed55fb4404fee6e9c77e4e1 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -3,6 +3,7 @@ package spark.rdd
 import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
 import spark.SparkContext._
 
+
 private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
   override val index = idx
   override def hashCode(): Int = idx
@@ -12,13 +13,15 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
  * The resulting RDD from a shuffle (e.g. repartitioning of data).
  * @param prev the parent RDD.
  * @param part the partitioner used to partition the RDD
+ * @param serializerClass class name of the serializer to use.
  * @tparam K the key class.
  * @tparam V the value class.
  */
 class ShuffledRDD[K, V](
-    prev: RDD[(K, V)],
-    part: Partitioner)
-  extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) {
+    @transient prev: RDD[(K, V)],
+    part: Partitioner,
+    serializerClass: String = null)
+  extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part, serializerClass))) {
 
   override val partitioner = Some(part)
 
@@ -28,6 +31,7 @@ class ShuffledRDD[K, V](
 
   override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
     val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
-    SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics)
+    SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics,
+      SparkEnv.get.serializerManager.get(serializerClass))
   }
 }
diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
index 43ec90cac5a95111534a7e15d9638c438b71dde5..8a9efc5da21f2d94a27732534e79354fa0ce5e28 100644
--- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
@@ -1,7 +1,8 @@
 package spark.rdd
 
-import java.util.{HashSet => JHashSet}
+import java.util.{HashMap => JHashMap}
 import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
 import spark.RDD
 import spark.Partitioner
 import spark.Dependency
@@ -11,6 +12,7 @@ import spark.SparkEnv
 import spark.ShuffleDependency
 import spark.OneToOneDependency
 
+
 /**
  * An optimized version of cogroup for set difference/subtraction.
  *
@@ -27,10 +29,12 @@ import spark.OneToOneDependency
  * you can use `rdd1`'s partitioner/partition size and not worry about running
  * out of memory because of the size of `rdd2`.
  */
-private[spark] class SubtractedRDD[T: ClassManifest](
-    @transient var rdd1: RDD[T],
-    @transient var rdd2: RDD[T],
-    part: Partitioner) extends RDD[T](rdd1.context, Nil) {
+private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest](
+    @transient var rdd1: RDD[(K, V)],
+    @transient var rdd2: RDD[(K, W)],
+    part: Partitioner,
+    val serializerClass: String = null)
+  extends RDD[(K, V)](rdd1.context, Nil) {
 
   override def getDependencies: Seq[Dependency[_]] = {
     Seq(rdd1, rdd2).map { rdd =>
@@ -39,26 +43,7 @@ private[spark] class SubtractedRDD[T: ClassManifest](
         new OneToOneDependency(rdd)
       } else {
         logInfo("Adding shuffle dependency with " + rdd)
-        val mapSideCombinedRDD = rdd.mapPartitions(i => {
-          val set = new JHashSet[T]()
-          while (i.hasNext) {
-            set.add(i.next)
-          }
-          set.iterator
-        }, true)
-        // ShuffleDependency requires a tuple (k, v), which it will partition by k.
-        // We need this to partition to map to the same place as the k for
-        // OneToOneDependency, which means:
-        // - for already-tupled RDD[(A, B)], into getPartition(a)
-        // - for non-tupled RDD[C], into getPartition(c)
-        val part2 = new Partitioner() {
-          def numPartitions = part.numPartitions
-          def getPartition(key: Any) = key match {
-            case (k, v) => part.getPartition(k)
-            case k => part.getPartition(k)
-          }
-        }
-        new ShuffleDependency(mapSideCombinedRDD.map((_, null)), part2)
+        new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part, serializerClass)
       }
     }
   }
@@ -74,29 +59,44 @@ private[spark] class SubtractedRDD[T: ClassManifest](
           case _ =>
             new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
         }
-      }.toList)
+      }.toArray)
     }
     array
   }
 
   override val partitioner = Some(part)
 
-  override def compute(p: Partition, context: TaskContext): Iterator[T] = {
+  override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = {
     val partition = p.asInstanceOf[CoGroupPartition]
-    val set = new JHashSet[T]
-    def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match {
-      case NarrowCoGroupSplitDep(rdd, _, itsSplit) =>
-        for (k <- rdd.iterator(itsSplit, context))
-          op(k.asInstanceOf[T])
-      case ShuffleCoGroupSplitDep(shuffleId) =>
-        for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics))
-          op(k.asInstanceOf[T])
+    val serializer = SparkEnv.get.serializerManager.get(serializerClass)
+    val map = new JHashMap[K, ArrayBuffer[V]]
+    def getSeq(k: K): ArrayBuffer[V] = {
+      val seq = map.get(k)
+      if (seq != null) {
+        seq
+      } else {
+        val seq = new ArrayBuffer[V]()
+        map.put(k, seq)
+        seq
+      }
+    }
+    def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match {
+      case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
+        for (t <- rdd.iterator(itsSplit, context))
+          op(t.asInstanceOf[(K, V)])
+      }
+      case ShuffleCoGroupSplitDep(shuffleId) => {
+        val iter = SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index,
+          context.taskMetrics, serializer)
+        for (t <- iter)
+          op(t.asInstanceOf[(K, V)])
+      }
     }
-    // the first dep is rdd1; add all keys to the set
-    integrate(partition.deps(0), set.add)
-    // the second dep is rdd2; remove all of its keys from the set
-    integrate(partition.deps(1), set.remove)
-    set.iterator
+    // the first dep is rdd1; add all values to the map
+    integrate(partition.deps(0), t => getSeq(t._1) += t._2)
+    // the second dep is rdd2; remove all of its keys
+    integrate(partition.deps(1), t => map.remove(t._1))
+    map.iterator.map { t =>  t._2.iterator.map { (t._1, _) } }.flatten
   }
 
   override def clearDependencies() {
@@ -105,4 +105,4 @@ private[spark] class SubtractedRDD[T: ClassManifest](
     rdd2 = null
   }
 
-}
\ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dd9f3c26809742816ecfce79d7c44fca3acae4c0
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
@@ -0,0 +1,142 @@
+package spark.rdd
+
+import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
+
+private[spark] class ZippedPartitionsPartition(
+    idx: Int,
+    @transient rdds: Seq[RDD[_]])
+  extends Partition {
+
+  override val index: Int = idx
+  var partitionValues = rdds.map(rdd => rdd.partitions(idx))
+  def partitions = partitionValues
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent split at the time of task serialization
+    partitionValues = rdds.map(rdd => rdd.partitions(idx))
+    oos.defaultWriteObject()
+  }
+}
+
+abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
+    sc: SparkContext,
+    var rdds: Seq[RDD[_]])
+  extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) {
+
+  override def getPartitions: Array[Partition] = {
+    val sizes = rdds.map(x => x.partitions.size)
+    if (!sizes.forall(x => x == sizes(0))) {
+      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
+  }
+
+  override def getPreferredLocations(s: Partition): Seq[String] = {
+    // Note that as number of rdd's increase and/or number of slaves in cluster increase, the computed preferredLocations below
+    // become diminishingly small : so we might need to look at alternate strategies to alleviate this.
+    // If there are no (or very small number of preferred locations), we will end up transferred the blocks to 'any' node in the
+    // cluster - paying with n/w and cache cost.
+    // Maybe pick a node which figures max amount of time ?
+    // Choose node which is hosting 'larger' of some subset of blocks ?
+    // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible)
+    val splits = s.asInstanceOf[ZippedPartitionsPartition].partitions
+    val rddSplitZip = rdds.zip(splits)
+
+    // exact match.
+    val exactMatchPreferredLocations = rddSplitZip.map(x => x._1.preferredLocations(x._2))
+    val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y))
+
+    // Remove exact match and then do host local match.
+    val otherNodePreferredLocations = rddSplitZip.map(x => {
+      x._1.preferredLocations(x._2).map(hostPort => {
+        val host = Utils.parseHostPort(hostPort)._1
+
+        if (exactMatchLocations.contains(host)) null else host
+      }).filter(_ != null)
+    })
+    val otherNodeLocalLocations = otherNodePreferredLocations.reduce((x, y) => x.intersect(y))
+
+    otherNodeLocalLocations ++ exactMatchLocations
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdds = null
+  }
+}
+
+class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest](
+    sc: SparkContext,
+    f: (Iterator[A], Iterator[B]) => Iterator[V],
+    var rdd1: RDD[A],
+    var rdd2: RDD[B])
+  extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) {
+
+  override def compute(s: Partition, context: TaskContext): Iterator[V] = {
+    val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions
+    f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context))
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+  }
+}
+
+class ZippedPartitionsRDD3
+  [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest](
+    sc: SparkContext,
+    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)) {
+
+  override def compute(s: Partition, context: TaskContext): Iterator[V] = {
+    val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions
+    f(rdd1.iterator(partitions(0), context),
+      rdd2.iterator(partitions(1), context),
+      rdd3.iterator(partitions(2), context))
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+    rdd3 = null
+  }
+}
+
+class ZippedPartitionsRDD4
+  [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest](
+    sc: SparkContext,
+    f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V],
+    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)) {
+
+  override def compute(s: Partition, context: TaskContext): Iterator[V] = {
+    val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions
+    f(rdd1.iterator(partitions(0), context),
+      rdd2.iterator(partitions(1), context),
+      rdd3.iterator(partitions(2), context),
+      rdd4.iterator(partitions(3), context))
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+    rdd3 = null
+    rdd4 = null
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala
index e80ec17aa5e0a4bbffd94b835fb18cff597c1f00..f728e93d2452e4e4c74d8192b15ad860a51708dd 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala
@@ -1,6 +1,6 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
+import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
 import java.io.{ObjectOutputStream, IOException}
 
 
@@ -10,17 +10,17 @@ private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
     @transient rdd2: RDD[U]
   ) extends Partition {
 
-  var split1 = rdd1.partitions(idx)
-  var split2 = rdd1.partitions(idx)
+  var partition1 = rdd1.partitions(idx)
+  var partition2 = rdd2.partitions(idx)
   override val index: Int = idx
 
-  def splits = (split1, split2)
+  def partitions = (partition1, partition2)
 
   @throws(classOf[IOException])
   private def writeObject(oos: ObjectOutputStream) {
-    // Update the reference to parent split at the time of task serialization
-    split1 = rdd1.partitions(idx)
-    split2 = rdd2.partitions(idx)
+    // Update the reference to parent partition at the time of task serialization
+    partition1 = rdd1.partitions(idx)
+    partition2 = rdd2.partitions(idx)
     oos.defaultWriteObject()
   }
 }
@@ -43,13 +43,32 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest](
   }
 
   override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = {
-    val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits
-    rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
+    val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions
+    rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context))
   }
 
   override def getPreferredLocations(s: Partition): Seq[String] = {
-    val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits
-    rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
+    // Note that as number of slaves in cluster increase, the computed preferredLocations can become small : so we might need
+    // to look at alternate strategies to alleviate this. (If there are no (or very small number of preferred locations), we
+    // will end up transferred the blocks to 'any' node in the cluster - paying with n/w and cache cost.
+    // Maybe pick one or the other ? (so that atleast one block is local ?).
+    // Choose node which is hosting 'larger' of the blocks ?
+    // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible)
+    val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions
+    val pref1 = rdd1.preferredLocations(partition1)
+    val pref2 = rdd2.preferredLocations(partition2)
+
+    // exact match - instance local and host local.
+    val exactMatchLocations = pref1.intersect(pref2)
+
+    // remove locations which are already handled via exactMatchLocations, and intersect where both partitions are node local.
+    val otherNodeLocalPref1 = pref1.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1)
+    val otherNodeLocalPref2 = pref2.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1)
+    val otherNodeLocalLocations = otherNodeLocalPref1.intersect(otherNodeLocalPref2)
+
+
+    // Can have mix of instance local (hostPort) and node local (host) locations as preference !
+    exactMatchLocations ++ otherNodeLocalLocations
   }
 
   override def clearDependencies() {
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index abc24c027090e4e537e76854325827f83b692896..7feeb9754289cfb294b195f5579089725aca6b55 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -13,7 +13,7 @@ import spark.executor.TaskMetrics
 import spark.partial.ApproximateActionListener
 import spark.partial.ApproximateEvaluator
 import spark.partial.PartialResult
-import spark.storage.BlockManagerMaster
+import spark.storage.{BlockManager, BlockManagerMaster}
 import spark.util.{MetadataCleaner, TimeStampedHashMap}
 
 /**
@@ -51,6 +51,11 @@ class DAGScheduler(
     eventQueue.put(ExecutorLost(execId))
   }
 
+  // Called by TaskScheduler when a host is added
+  override def executorGained(execId: String, hostPort: String) {
+    eventQueue.put(ExecutorGained(execId, hostPort))
+  }
+
   // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures.
   override def taskSetFailed(taskSet: TaskSet, reason: String) {
     eventQueue.put(TaskSetFailed(taskSet, reason))
@@ -115,9 +120,8 @@ class DAGScheduler(
   private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = {
     if (!cacheLocs.contains(rdd.id)) {
       val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
-      cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map {
-        locations => locations.map(_.ip).toList
-      }.toArray
+      val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env, blockManagerMaster)
+      cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil))
     }
     cacheLocs(rdd.id)
   }
@@ -300,6 +304,9 @@ class DAGScheduler(
           submitStage(finalStage)
         }
 
+      case ExecutorGained(execId, hostPort) =>
+        handleExecutorGained(execId, hostPort)
+
       case ExecutorLost(execId) =>
         handleExecutorLost(execId)
 
@@ -638,6 +645,14 @@ class DAGScheduler(
                "(generation " + currentGeneration + ")")
     }
   }
+  
+  private def handleExecutorGained(execId: String, hostPort: String) {
+    // remove from failedGeneration(execId) ?
+    if (failedGeneration.contains(execId)) {
+      logInfo("Host gained which was in lost list earlier: " + hostPort)
+      failedGeneration -= execId
+    }
+  }
 
   /**
    * Aborts all jobs depending on a particular Stage. This is called in response to a task set
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
index 303c211e2a17260f49603fc7bce816bc4f78a09f..acad915f13f3ca0c2bb5237e88889f712f3236dd 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
@@ -35,6 +35,10 @@ private[spark] case class CompletionEvent(
     taskMetrics: TaskMetrics)
   extends DAGSchedulerEvent
 
+private[spark] case class ExecutorGained(execId: String, hostPort: String) extends DAGSchedulerEvent {
+  Utils.checkHostPort(hostPort, "Required hostport")
+}
+
 private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
 
 private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
new file mode 100644
index 0000000000000000000000000000000000000000..287f731787f1b184d4e2781dbd28e1f4463f5866
--- /dev/null
+++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
@@ -0,0 +1,156 @@
+package spark.scheduler
+
+import spark.Logging
+import scala.collection.immutable.Set
+import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
+import org.apache.hadoop.util.ReflectionUtils
+import org.apache.hadoop.mapreduce.Job
+import org.apache.hadoop.conf.Configuration
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.JavaConversions._
+
+
+/**
+ * Parses and holds information about inputFormat (and files) specified as a parameter.
+ */
+class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], 
+                      val path: String) extends Logging {
+
+  var mapreduceInputFormat: Boolean = false
+  var mapredInputFormat: Boolean = false
+
+  validate()
+
+  override def toString(): String = {
+    "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path
+  }
+
+  override def hashCode(): Int = {
+    var hashCode = inputFormatClazz.hashCode
+    hashCode = hashCode * 31 + path.hashCode
+    hashCode
+  }
+
+  // Since we are not doing canonicalization of path, this can be wrong : like relative vs absolute path
+  // .. which is fine, this is best case effort to remove duplicates - right ?
+  override def equals(other: Any): Boolean = other match {
+    case that: InputFormatInfo => {
+      // not checking config - that should be fine, right ?
+      this.inputFormatClazz == that.inputFormatClazz &&
+        this.path == that.path
+    }
+    case _ => false
+  }
+
+  private def validate() {
+    logDebug("validate InputFormatInfo : " + inputFormatClazz + ", path  " + path)
+
+    try {
+      if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) {
+        logDebug("inputformat is from mapreduce package")
+        mapreduceInputFormat = true
+      }
+      else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) {
+        logDebug("inputformat is from mapred package")
+        mapredInputFormat = true
+      }
+      else {
+        throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz +
+          " is NOT a supported input format ? does not implement either of the supported hadoop api's")
+      }
+    }
+    catch {
+      case e: ClassNotFoundException => {
+        throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e)
+      }
+    }
+  }
+
+
+  // This method does not expect failures, since validate has already passed ...
+  private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = {
+    val conf = new JobConf(configuration)
+    FileInputFormat.setInputPaths(conf, path)
+
+    val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] =
+      ReflectionUtils.newInstance(inputFormatClazz.asInstanceOf[Class[_]], conf).asInstanceOf[
+        org.apache.hadoop.mapreduce.InputFormat[_, _]]
+    val job = new Job(conf)
+
+    val retval = new ArrayBuffer[SplitInfo]()
+    val list = instance.getSplits(job)
+    for (split <- list) {
+      retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, split)
+    }
+
+    return retval.toSet
+  }
+
+  // This method does not expect failures, since validate has already passed ...
+  private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = {
+    val jobConf = new JobConf(configuration)
+    FileInputFormat.setInputPaths(jobConf, path)
+
+    val instance: org.apache.hadoop.mapred.InputFormat[_, _] =
+      ReflectionUtils.newInstance(inputFormatClazz.asInstanceOf[Class[_]], jobConf).asInstanceOf[
+        org.apache.hadoop.mapred.InputFormat[_, _]]
+
+    val retval = new ArrayBuffer[SplitInfo]()
+    instance.getSplits(jobConf, jobConf.getNumMapTasks()).foreach(
+        elem => retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, elem)
+    )
+
+    return retval.toSet
+   }
+
+  private def findPreferredLocations(): Set[SplitInfo] = {
+    logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + 
+      ", inputFormatClazz : " + inputFormatClazz)
+    if (mapreduceInputFormat) {
+      return prefLocsFromMapreduceInputFormat()
+    }
+    else {
+      assert(mapredInputFormat)
+      return prefLocsFromMapredInputFormat()
+    }
+  }
+}
+
+
+
+
+object InputFormatInfo {
+  /**
+    Computes the preferred locations based on input(s) and returned a location to block map.
+    Typical use of this method for allocation would follow some algo like this 
+    (which is what we currently do in YARN branch) :
+    a) For each host, count number of splits hosted on that host.
+    b) Decrement the currently allocated containers on that host.
+    c) Compute rack info for each host and update rack -> count map based on (b).
+    d) Allocate nodes based on (c)
+    e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node 
+       (even if data locality on that is very high) : this is to prevent fragility of job if a single 
+       (or small set of) hosts go down.
+
+    go to (a) until required nodes are allocated.
+
+    If a node 'dies', follow same procedure.
+
+    PS: I know the wording here is weird, hopefully it makes some sense !
+  */
+  def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] = {
+
+    val nodeToSplit = new HashMap[String, HashSet[SplitInfo]]
+    for (inputSplit <- formats) {
+      val splits = inputSplit.findPreferredLocations()
+
+      for (split <- splits){
+        val location = split.hostLocation
+        val set = nodeToSplit.getOrElseUpdate(location, new HashSet[SplitInfo])
+        set += split
+      }
+    }
+
+    nodeToSplit
+  }
+}
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index beb21a76fe5c8247b3373e05cb09355141d49ed8..83166bce224f3b638a6de227321d5eeb4b42106d 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -70,6 +70,13 @@ private[spark] class ResultTask[T, U](
     rdd.partitions(partition)
   }
 
+  private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq
+
+  {
+    // DEBUG code
+    preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs))
+  }
+
   override def run(attemptId: Long): U = {
     val context = new TaskContext(stageId, partition, attemptId)
     metrics = Some(context.taskMetrics)
@@ -80,7 +87,7 @@ private[spark] class ResultTask[T, U](
     }
   }
 
-  override def preferredLocations: Seq[String] = locs
+  override def preferredLocations: Seq[String] = preferredLocs
 
   override def toString = "ResultTask(" + stageId + ", " + partition + ")"
 
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index 36d087a4d009c8e6a4fdf85bfc5d6e6e6e3926bf..95647389c3aae82075ec1eee3defec5beb0fb548 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -13,9 +13,10 @@ import com.ning.compress.lzf.LZFInputStream
 import com.ning.compress.lzf.LZFOutputStream
 
 import spark._
-import executor.ShuffleWriteMetrics
+import spark.executor.ShuffleWriteMetrics
 import spark.storage._
-import util.{TimeStampedHashMap, MetadataCleaner}
+import spark.util.{TimeStampedHashMap, MetadataCleaner}
+
 
 private[spark] object ShuffleMapTask {
 
@@ -77,13 +78,20 @@ private[spark] class ShuffleMapTask(
     var rdd: RDD[_],
     var dep: ShuffleDependency[_,_],
     var partition: Int,
-    @transient var locs: Seq[String])
+    @transient private var locs: Seq[String])
   extends Task[MapStatus](stageId)
   with Externalizable
   with Logging {
 
   protected def this() = this(0, null, null, 0, null)
 
+  @transient private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq
+
+  {
+    // DEBUG code
+    preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs))
+  }
+
   var split = if (rdd == null) {
     null
   } else {
@@ -121,40 +129,58 @@ private[spark] class ShuffleMapTask(
 
     val taskContext = new TaskContext(stageId, partition, attemptId)
     metrics = Some(taskContext.taskMetrics)
+
+    val blockManager = SparkEnv.get.blockManager
+    var shuffle: ShuffleBlocks = null
+    var buckets: ShuffleWriterGroup = null
+
     try {
-      // Partition the map output.
-      val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)])
+      // 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)
+
+      // Write the map output to its associated buckets.
       for (elem <- rdd.iterator(split, taskContext)) {
         val pair = elem.asInstanceOf[(Any, Any)]
         val bucketId = dep.partitioner.getPartition(pair._1)
-        buckets(bucketId) += pair
+        buckets.writers(bucketId).write(pair)
       }
 
-      val compressedSizes = new Array[Byte](numOutputSplits)
-
-      var totalBytes = 0l
-
-      val blockManager = SparkEnv.get.blockManager
-      for (i <- 0 until numOutputSplits) {
-        val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i
-        // Get a Scala iterator from Java map
-        val iter: Iterator[(Any, Any)] = buckets(i).iterator
-        val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false)
+      // 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 =>
+        writer.commit()
+        writer.close()
+        val size = writer.size()
         totalBytes += size
-        compressedSizes(i) = MapOutputTracker.compressSize(size)
+        MapOutputTracker.compressSize(size)
       }
+
+      // Update shuffle metrics.
       val shuffleMetrics = new ShuffleWriteMetrics
       shuffleMetrics.shuffleBytesWritten = totalBytes
       metrics.get.shuffleWriteMetrics = Some(shuffleMetrics)
 
       return 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())
+      }
+      throw e
     } finally {
+      // Release the writers back to the shuffle block manager.
+      if (shuffle != null && buckets != null) {
+        shuffle.releaseWriters(buckets)
+      }
       // Execute the callbacks on task completion.
       taskContext.executeOnCompleteCallbacks()
     }
   }
 
-  override def preferredLocations: Seq[String] = locs
+  override def preferredLocations: Seq[String] = preferredLocs
 
   override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition)
 }
diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala
index 21185227ab0c6ff822c73e4e91c135434a92d726..a65140b145833c70e584936785ac8fe94d6b0bc2 100644
--- a/core/src/main/scala/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/spark/scheduler/SparkListener.scala
@@ -31,7 +31,7 @@ class StatsReportListener extends SparkListener with Logging {
     showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten})
 
     //fetch & io
-    showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.remoteFetchWaitTime})
+    showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.fetchWaitTime})
     showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead})
     showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize))
 
@@ -137,7 +137,7 @@ case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], othe
 object RuntimePercentage {
   def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = {
     val denom = totalTime.toDouble
-    val fetchTime = metrics.shuffleReadMetrics.map{_.remoteFetchWaitTime}
+    val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime}
     val fetch = fetchTime.map{_ / denom}
     val exec = (metrics.executorRunTime - fetchTime.getOrElse(0l)) / denom
     val other = 1.0 - (exec + fetch.getOrElse(0d))
diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6abfb7a1f7150122523370abc03fa6546d5e96a1
--- /dev/null
+++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala
@@ -0,0 +1,61 @@
+package spark.scheduler
+
+import collection.mutable.ArrayBuffer
+
+// information about a specific split instance : handles both split instances.
+// So that we do not need to worry about the differences.
+class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String,
+                val length: Long, val underlyingSplit: Any) {
+  override def toString(): String = {
+    "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz +
+      ", hostLocation : " + hostLocation + ", path : " + path +
+      ", length : " + length + ", underlyingSplit " + underlyingSplit
+  }
+
+  override def hashCode(): Int = {
+    var hashCode = inputFormatClazz.hashCode
+    hashCode = hashCode * 31 + hostLocation.hashCode
+    hashCode = hashCode * 31 + path.hashCode
+    // ignore overflow ? It is hashcode anyway !
+    hashCode = hashCode * 31 + (length & 0x7fffffff).toInt
+    hashCode
+  }
+
+  // This is practically useless since most of the Split impl's dont seem to implement equals :-(
+  // So unless there is identity equality between underlyingSplits, it will always fail even if it
+  // is pointing to same block.
+  override def equals(other: Any): Boolean = other match {
+    case that: SplitInfo => {
+      this.hostLocation == that.hostLocation &&
+        this.inputFormatClazz == that.inputFormatClazz &&
+        this.path == that.path &&
+        this.length == that.length &&
+        // other split specific checks (like start for FileSplit)
+        this.underlyingSplit == that.underlyingSplit
+    }
+    case _ => false
+  }
+}
+
+object SplitInfo {
+
+  def toSplitInfo(inputFormatClazz: Class[_], path: String,
+                  mapredSplit: org.apache.hadoop.mapred.InputSplit): Seq[SplitInfo] = {
+    val retval = new ArrayBuffer[SplitInfo]()
+    val length = mapredSplit.getLength
+    for (host <- mapredSplit.getLocations) {
+      retval += new SplitInfo(inputFormatClazz, host, path, length, mapredSplit)
+    }
+    retval
+  }
+
+  def toSplitInfo(inputFormatClazz: Class[_], path: String,
+                  mapreduceSplit: org.apache.hadoop.mapreduce.InputSplit): Seq[SplitInfo] = {
+    val retval = new ArrayBuffer[SplitInfo]()
+    val length = mapreduceSplit.getLength
+    for (host <- mapreduceSplit.getLocations) {
+      retval += new SplitInfo(inputFormatClazz, host, path, length, mapreduceSplit)
+    }
+    retval
+  }
+}
diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
index d549b184b088d65472302b8523d5d609efe36a6b..7787b547620f6bea6cbb8f1d7a626638b7b6edb8 100644
--- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
@@ -10,6 +10,10 @@ package spark.scheduler
 private[spark] trait TaskScheduler {
   def start(): Unit
 
+  // Invoked after system has successfully initialized (typically in spark context).
+  // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc.
+  def postStartHook() { }
+
   // Disconnect from the cluster.
   def stop(): Unit
 
diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
index 771518dddfacaaf2916a1f6cd834983725cbc533..b75d3736cf5d0b064babcc100e3c63ac0f31c5c4 100644
--- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
@@ -14,6 +14,9 @@ private[spark] trait TaskSchedulerListener {
   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, hostPort: String): Unit
+
   // A node was lost from the cluster.
   def executorLost(execId: String): Unit
 
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
index e6399a3547255e9cde652c7770978a01521a07d0..9547f4f6dd188652e9df4089a12fa457a6bc3df9 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
@@ -1,6 +1,6 @@
 package spark.scheduler.cluster
 
-import java.io.{File, FileInputStream, FileOutputStream}
+import java.lang.{Boolean => JBoolean}
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
@@ -25,6 +25,35 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong
   // Threshold above which we warn user initial TaskSet may be starved
   val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong
+  // How often to revive offers in case there are pending tasks - that is how often to try to get
+  // tasks scheduled in case there are nodes available : default 0 is to disable it - to preserve existing behavior
+  // Note that this is required due to delayed scheduling due to data locality waits, etc.
+  // TODO: rename property ?
+  val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong
+
+  /*
+   This property controls how aggressive we should be to modulate waiting for node local task scheduling.
+   To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for node locality of tasks before
+   scheduling on other nodes. We have modified this in yarn branch such that offers to task set happen in prioritized order :
+   node-local, rack-local and then others
+   But once all available node local (and no pref) tasks are scheduled, instead of waiting for 3 sec before
+   scheduling to other nodes (which degrades performance for time sensitive tasks and on larger clusters), we can
+   modulate that : to also allow rack local nodes or any node. The default is still set to HOST - so that previous behavior is
+   maintained. This is to allow tuning the tension between pulling rdd data off node and scheduling computation asap.
+
+   TODO: rename property ? The value is one of
+   - NODE_LOCAL (default, no change w.r.t current behavior),
+   - RACK_LOCAL and
+   - ANY
+
+   Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective.
+
+   Additional Note: For non trivial clusters, there is a 4x - 5x reduction in running time (in some of our experiments) based on whether
+   it is left at default NODE_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY.
+   If cluster is rack aware, then setting it to RACK_LOCAL gives best tradeoff and a 3x - 4x performance improvement while minimizing IO impact.
+   Also, it brings down the variance in running time drastically.
+    */
+  val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL"))
 
   val activeTaskSets = new HashMap[String, TaskSetManager]
 
@@ -32,9 +61,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   val taskIdToExecutorId = new HashMap[Long, String]
   val taskSetTaskIds = new HashMap[String, HashSet[Long]]
 
-  var hasReceivedTask = false
-  var hasLaunchedTask = false
-  val starvationTimer = new Timer(true)
+  @volatile private var hasReceivedTask = false
+  @volatile private var hasLaunchedTask = false
+  private val starvationTimer = new Timer(true)
 
   // Incrementing Mesos task IDs
   val nextTaskId = new AtomicLong(0)
@@ -42,11 +71,16 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   // Which executor IDs we have executors on
   val activeExecutorIds = new HashSet[String]
 
+  // TODO: We might want to remove this and merge it with execId datastructures - but later.
+  // Which hosts in the cluster are alive (contains hostPort's) - used for process local and node local task locality.
+  private val hostPortsAlive = new HashSet[String]
+  private val hostToAliveHostPorts = new HashMap[String, HashSet[String]]
+
   // The set of executors we have on each host; this is used to compute hostsAlive, which
   // in turn is used to decide when we can attain data locality on a given host
-  val executorsByHost = new HashMap[String, HashSet[String]]
+  private val executorsByHostPort = new HashMap[String, HashSet[String]]
 
-  val executorIdToHost = new HashMap[String, String]
+  private val executorIdToHostPort = new HashMap[String, String]
 
   // JAR server, if any JARs were added by the user to the SparkContext
   var jarServer: HttpServer = null
@@ -83,6 +117,14 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       }
     }
     schedulableBuilder.buildPools()
+    // resolve executorId to hostPort mapping.
+    def executorToHostPort(executorId: String, defaultHostPort: String): String = {
+      executorIdToHostPort.getOrElse(executorId, defaultHostPort)
+    }
+
+    // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler
+    // Will that be a design violation ?
+    SparkEnv.get.executorIdToHostPort = Some(executorToHostPort)
   }
 
 
@@ -91,11 +133,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   override def start() {
     backend.start()
 
-    if (System.getProperty("spark.speculation", "false") == "true") {
+    if (JBoolean.getBoolean("spark.speculation")) {
       new Thread("ClusterScheduler speculation check") {
         setDaemon(true)
 
         override def run() {
+          logInfo("Starting speculative execution thread")
           while (true) {
             try {
               Thread.sleep(SPECULATION_INTERVAL)
@@ -107,6 +150,27 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
         }
       }.start()
     }
+
+
+    // Change to always run with some default if TASK_REVIVAL_INTERVAL <= 0 ?
+    if (TASK_REVIVAL_INTERVAL > 0) {
+      new Thread("ClusterScheduler task offer revival check") {
+        setDaemon(true)
+
+        override def run() {
+          logInfo("Starting speculative task offer revival thread")
+          while (true) {
+            try {
+              Thread.sleep(TASK_REVIVAL_INTERVAL)
+            } catch {
+              case e: InterruptedException => {}
+            }
+
+            if (hasPendingTasks()) backend.reviveOffers()
+          }
+        }
+      }.start()
+    }
   }
 
   override def submitTasks(taskSet: TaskSet) {
@@ -156,14 +220,37 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       SparkEnv.set(sc.env)
       // Mark each slave as alive and remember its hostname
       for (o <- offers) {
-        executorIdToHost(o.executorId) = o.hostname
-        if (!executorsByHost.contains(o.hostname)) {
-          executorsByHost(o.hostname) = new HashSet()
+        // DEBUG Code
+        Utils.checkHostPort(o.hostPort)
+
+        executorIdToHostPort(o.executorId) = o.hostPort
+        if (! executorsByHostPort.contains(o.hostPort)) {
+          executorsByHostPort(o.hostPort) = new HashSet[String]()
         }
+
+        hostPortsAlive += o.hostPort
+        hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(o.hostPort)._1, new HashSet[String]).add(o.hostPort)
+        executorGained(o.executorId, o.hostPort)
       }
       // Build a list of tasks to assign to each slave
       val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
+      // merge availableCpus into nodeToAvailableCpus block ?
       val availableCpus = offers.map(o => o.cores).toArray
+      val nodeToAvailableCpus = {
+        val map = new HashMap[String, Int]()
+        for (offer <- offers) {
+          val hostPort = offer.hostPort
+          val cores = offer.cores
+          // DEBUG code
+          Utils.checkHostPort(hostPort)
+
+          val host = Utils.parseHostPort(hostPort)._1
+
+          map.put(host, map.getOrElse(host, 0) + cores)
+        }
+
+        map
+      }
       var launchedTask = false
       val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue()
       for (manager <- sortedTaskSetQueue)
@@ -171,13 +258,90 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
         logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks))
       }
       for (manager <- sortedTaskSetQueue) {
+
+        // Split offers based on node local, rack local and off-rack tasks.
+        val processLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
+        val nodeLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
+        val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
+        val otherOffers = new HashMap[String, ArrayBuffer[Int]]()
+
+        for (i <- 0 until offers.size) {
+          val hostPort = offers(i).hostPort
+          // DEBUG code
+          Utils.checkHostPort(hostPort)
+
+          val numProcessLocalTasks =  math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i)))
+          if (numProcessLocalTasks > 0){
+            val list = processLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int])
+            for (j <- 0 until numProcessLocalTasks) list += i
+          }
+
+          val host = Utils.parseHostPort(hostPort)._1
+          val numNodeLocalTasks =  math.max(0,
+            // Remove process local tasks (which are also host local btw !) from this
+            math.min(manager.numPendingTasksForHost(hostPort) - numProcessLocalTasks, nodeToAvailableCpus(host)))
+          if (numNodeLocalTasks > 0){
+            val list = nodeLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
+            for (j <- 0 until numNodeLocalTasks) list += i
+          }
+
+          val numRackLocalTasks =  math.max(0,
+            // Remove node local tasks (which are also rack local btw !) from this
+            math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numProcessLocalTasks - numNodeLocalTasks, nodeToAvailableCpus(host)))
+          if (numRackLocalTasks > 0){
+            val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
+            for (j <- 0 until numRackLocalTasks) list += i
+          }
+          if (numNodeLocalTasks <= 0 && numRackLocalTasks <= 0){
+            // add to others list - spread even this across cluster.
+            val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
+            list += i
+          }
+        }
+
+        val offersPriorityList = new ArrayBuffer[Int](
+          processLocalOffers.size + nodeLocalOffers.size + rackLocalOffers.size + otherOffers.size)
+
+        // First process local, then host local, then rack, then others
+
+        // numNodeLocalOffers contains count of both process local and host offers.
+        val numNodeLocalOffers = {
+          val processLocalPriorityList = ClusterScheduler.prioritizeContainers(processLocalOffers)
+          offersPriorityList ++= processLocalPriorityList
+
+          val nodeLocalPriorityList = ClusterScheduler.prioritizeContainers(nodeLocalOffers)
+          offersPriorityList ++= nodeLocalPriorityList
+
+          processLocalPriorityList.size + nodeLocalPriorityList.size
+        }
+        val numRackLocalOffers = {
+          val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers)
+          offersPriorityList ++= rackLocalPriorityList
+          rackLocalPriorityList.size
+        }
+        offersPriorityList ++= ClusterScheduler.prioritizeContainers(otherOffers)
+
+        var lastLoop = false
+        val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match {
+          case TaskLocality.NODE_LOCAL => numNodeLocalOffers
+          case TaskLocality.RACK_LOCAL => numRackLocalOffers + numNodeLocalOffers
+          case TaskLocality.ANY => offersPriorityList.size
+        }
+
         do {
           launchedTask = false
-          for (i <- 0 until offers.size) {
-            var launchedTask = true
+          var loopCount = 0
+          for (i <- offersPriorityList) {
             val execId = offers(i).executorId
-            val host = offers(i).hostname
-            manager.slaveOffer(execId,host,availableCpus(i)) match {
+            val hostPort = offers(i).hostPort
+
+            // If last loop and within the lastLoopIndex, expand scope - else use null (which will use default/existing)
+            val overrideLocality = if (lastLoop && loopCount < lastLoopIndex) TASK_SCHEDULING_AGGRESSION else null
+
+            // If last loop, override waiting for host locality - we scheduled all local tasks already and there might be more available ...
+            loopCount += 1
+
+            manager.slaveOffer(execId, hostPort, availableCpus(i), overrideLocality) match {
               case Some(task) =>
                 tasks(i) += task
                 val tid = task.taskId
@@ -185,15 +349,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
                 taskSetTaskIds(manager.taskSet.id) += tid
                 taskIdToExecutorId(tid) = execId
                 activeExecutorIds += execId
-                executorsByHost(host) += execId
+                executorsByHostPort(hostPort) += execId
                 availableCpus(i) -= 1
                 launchedTask = true
-
+                
               case None => {}
               }
             }
-        } while(launchedTask)
+          // Loop once more - when lastLoop = true, then we try to schedule task on all nodes irrespective of
+          // data locality (we still go in order of priority : but that would not change anything since
+          // if data local tasks had been available, we would have scheduled them already)
+          if (lastLoop) {
+            // prevent more looping
+            launchedTask = false
+          } else if (!lastLoop && !launchedTask) {
+            // Do this only if TASK_SCHEDULING_AGGRESSION != NODE_LOCAL
+            if (TASK_SCHEDULING_AGGRESSION != TaskLocality.NODE_LOCAL) {
+              // fudge launchedTask to ensure we loop once more
+              launchedTask = true
+              // dont loop anymore
+              lastLoop = true
+            }
+          }
+        } while (launchedTask)
       }
+      
       if (tasks.size > 0) {
         hasLaunchedTask = true
       }
@@ -280,10 +460,15 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     if (jarServer != null) {
       jarServer.stop()
     }
+
+    // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out.
+    // TODO: Do something better !
+    Thread.sleep(5000L)
   }
 
   override def defaultParallelism() = backend.defaultParallelism()
 
+
   // Check for speculatable tasks in all our active jobs.
   def checkSpeculatableTasks() {
     var shouldRevive = false
@@ -295,12 +480,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
   }
 
+  // Check for pending tasks in all our active jobs.
+  def hasPendingTasks(): Boolean = {
+    synchronized {
+      rootPool.hasPendingTasks()
+    }
+  }
+
   def executorLost(executorId: String, reason: ExecutorLossReason) {
     var failedExecutor: Option[String] = None
+
     synchronized {
       if (activeExecutorIds.contains(executorId)) {
-        val host = executorIdToHost(executorId)
-        logError("Lost executor %s on %s: %s".format(executorId, host, reason))
+        val hostPort = executorIdToHostPort(executorId)
+        logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason))
         removeExecutor(executorId)
         failedExecutor = Some(executorId)
       } else {
@@ -318,19 +511,104 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
   }
 
-  /** Get a list of hosts that currently have executors */
-  def hostsAlive: scala.collection.Set[String] = executorsByHost.keySet
-
   /** Remove an executor from all our data structures and mark it as lost */
   private def removeExecutor(executorId: String) {
     activeExecutorIds -= executorId
-    val host = executorIdToHost(executorId)
-    val execs = executorsByHost.getOrElse(host, new HashSet)
+    val hostPort = executorIdToHostPort(executorId)
+    if (hostPortsAlive.contains(hostPort)) {
+      // DEBUG Code
+      Utils.checkHostPort(hostPort)
+
+      hostPortsAlive -= hostPort
+      hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort)
+    }
+      
+    val execs = executorsByHostPort.getOrElse(hostPort, new HashSet)
     execs -= executorId
     if (execs.isEmpty) {
-      executorsByHost -= host
+      executorsByHostPort -= hostPort
     }
-    executorIdToHost -= executorId
-    rootPool.executorLost(executorId, host)
+    executorIdToHostPort -= executorId
+    rootPool.executorLost(executorId, hostPort)
+  }
+
+  def executorGained(execId: String, hostPort: String) {
+    listener.executorGained(execId, hostPort)
+  }
+
+  def getExecutorsAliveOnHost(host: String): Option[Set[String]] = {
+    Utils.checkHost(host)
+
+    val retval = hostToAliveHostPorts.get(host)
+    if (retval.isDefined) {
+      return Some(retval.get.toSet)
+    }
+
+    None
+  }
+
+  def isExecutorAliveOnHostPort(hostPort: String): Boolean = {
+    // Even if hostPort is a host, it does not matter - it is just a specific check.
+    // But we do have to ensure that only hostPort get into hostPortsAlive !
+    // So no check against Utils.checkHostPort
+    hostPortsAlive.contains(hostPort)
+  }
+
+  // By default, rack is unknown
+  def getRackForHost(value: String): Option[String] = None
+
+  // By default, (cached) hosts for rack is unknown
+  def getCachedHostsForRack(rack: String): Option[Set[String]] = None
+}
+
+
+object ClusterScheduler {
+
+  // Used to 'spray' available containers across the available set to ensure too many containers on same host
+  // are not used up. Used in yarn mode and in task scheduling (when there are multiple containers available
+  // to execute a task)
+  // For example: yarn can returns more containers than we would have requested under ANY, this method
+  // prioritizes how to use the allocated containers.
+  // flatten the map such that the array buffer entries are spread out across the returned value.
+  // given <host, list[container]> == <h1, [c1 .. c5]>, <h2, [c1 .. c3]>, <h3, [c1, c2]>, <h4, c1>, <h5, c1>, i
+  // the return value would be something like : h1c1, h2c1, h3c1, h4c1, h5c1, h1c2, h2c2, h3c2, h1c3, h2c3, h1c4, h1c5
+  // We then 'use' the containers in this order (consuming only the top K from this list where
+  // K = number to be user). This is to ensure that if we have multiple eligible allocations,
+  // they dont end up allocating all containers on a small number of hosts - increasing probability of
+  // multiple container failure when a host goes down.
+  // Note, there is bias for keys with higher number of entries in value to be picked first (by design)
+  // Also note that invocation of this method is expected to have containers of same 'type'
+  // (host-local, rack-local, off-rack) and not across types : so that reordering is simply better from
+  // the available list - everything else being same.
+  // That is, we we first consume data local, then rack local and finally off rack nodes. So the
+  // prioritization from this method applies to within each category
+  def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = {
+    val _keyList = new ArrayBuffer[K](map.size)
+    _keyList ++= map.keys
+
+    // order keyList based on population of value in map
+    val keyList = _keyList.sortWith(
+      (left, right) => map.get(left).getOrElse(Set()).size > map.get(right).getOrElse(Set()).size
+    )
+
+    val retval = new ArrayBuffer[T](keyList.size * 2)
+    var index = 0
+    var found = true
+
+    while (found){
+      found = false
+      for (key <- keyList) {
+        val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null)
+        assert(containerList != null)
+        // Get the index'th entry for this host - if present
+        if (index < containerList.size){
+          retval += containerList.apply(index)
+          found = true
+        }
+      }
+      index += 1
+    }
+
+    retval.toList
   }
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala
index 4dc15f413c6606cd469c4be0eb4c8786323005c6..941ba7a3f185a420e1fa54bcbc2050851f7d47a3 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala
@@ -97,4 +97,8 @@ private[spark] class Pool(
       parent.decreaseRunningTasks(taskNum)
     }
   }
+
+  override def hasPendingTasks(): Boolean = {
+    schedulableQueue.exists(_.hasPendingTasks())
+  }
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
index 6bb7525b49735f6a43ddcf4751964f12fc342b14..2dd9c0564fc3e7376e61804949036de3903285a4 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
@@ -23,4 +23,5 @@ private[spark] trait Schedulable {
   def executorLost(executorId: String, host: String): Unit
   def checkSpeculatableTasks(): Boolean
   def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager]
+  def hasPendingTasks(): Boolean
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index bb289c9cf391b4d1dcc6f94113117b9468c4a578..170ede0f4422685ee29bfa14fec9a7a535be514b 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -31,7 +31,8 @@ private[spark] class SparkDeploySchedulerBackend(
     val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(
       throw new IllegalArgumentException("must supply spark home for spark standalone"))
-    val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome)
+    val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
+        sc.ui.appUIAddress)
 
     client = new Client(sc.env.actorSystem, master, appDesc, this)
     client.start()
@@ -57,9 +58,9 @@ private[spark] class SparkDeploySchedulerBackend(
     }
   }
 
-  override def executorAdded(executorId: String, workerId: String, host: String, cores: Int, memory: Int) {
-    logInfo("Granted executor ID %s on host %s with %d cores, %s RAM".format(
-       executorId, host, cores, Utils.memoryMegabytesToString(memory)))
+  override def executorAdded(executorId: String, workerId: String, hostPort: String, cores: Int, memory: Int) {
+    logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format(
+       executorId, hostPort, cores, Utils.memoryMegabytesToString(memory)))
   }
 
   override def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) {
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
index d7660678248b2d9f028d8364bd3caa8cbd47660c..333529484460e3dae9865d9f105edcca766e81a7 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -3,6 +3,7 @@ package spark.scheduler.cluster
 import spark.TaskState.TaskState
 import java.nio.ByteBuffer
 import spark.util.SerializableBuffer
+import spark.Utils
 
 private[spark] sealed trait StandaloneClusterMessage extends Serializable
 
@@ -19,8 +20,10 @@ case class RegisterExecutorFailed(message: String) extends StandaloneClusterMess
 
 // Executors to driver
 private[spark]
-case class RegisterExecutor(executorId: String, host: String, cores: Int)
-  extends StandaloneClusterMessage
+case class RegisterExecutor(executorId: String, hostPort: String, cores: Int)
+  extends StandaloneClusterMessage {
+  Utils.checkHostPort(hostPort, "Expected host port")
+}
 
 private[spark]
 case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer)
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 7a428e3361976b3a8d78b8a8c1c1a7f771399c1a..004592a54043857fe1102bcb6b4161823fd0fc14 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -5,8 +5,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import akka.actor._
 import akka.util.duration._
 import akka.pattern.ask
+import akka.util.Duration
 
-import spark.{SparkException, Logging, TaskState}
+import spark.{Utils, SparkException, Logging, TaskState}
 import akka.dispatch.Await
 import java.util.concurrent.atomic.AtomicInteger
 import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
@@ -24,12 +25,12 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
   var totalCoreCount = new AtomicInteger(0)
 
   class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
-    val executorActor = new HashMap[String, ActorRef]
-    val executorAddress = new HashMap[String, Address]
-    val executorHost = new HashMap[String, String]
-    val freeCores = new HashMap[String, Int]
-    val actorToExecutorId = new HashMap[ActorRef, String]
-    val addressToExecutorId = new HashMap[Address, String]
+    private val executorActor = new HashMap[String, ActorRef]
+    private val executorAddress = new HashMap[String, Address]
+    private val executorHostPort = 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() {
       // Listen for remote client disconnection events, since they don't go through Akka's watch()
@@ -37,7 +38,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     }
 
     def receive = {
-      case RegisterExecutor(executorId, host, cores) =>
+      case RegisterExecutor(executorId, hostPort, cores) =>
+        Utils.checkHostPort(hostPort, "Host port expected " + hostPort)
         if (executorActor.contains(executorId)) {
           sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId)
         } else {
@@ -45,7 +47,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
           sender ! RegisteredExecutor(sparkProperties)
           context.watch(sender)
           executorActor(executorId) = sender
-          executorHost(executorId) = host
+          executorHostPort(executorId) = hostPort
           freeCores(executorId) = cores
           executorAddress(executorId) = sender.path.address
           actorToExecutorId(sender) = executorId
@@ -85,13 +87,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     // Make fake resource offers on all executors
     def makeOffers() {
       launchTasks(scheduler.resourceOffers(
-        executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))}))
+        executorHostPort.toArray.map {case (id, hostPort) => new WorkerOffer(id, hostPort, freeCores(id))}))
     }
 
     // Make fake resource offers on just one executor
     def makeOffers(executorId: String) {
       launchTasks(scheduler.resourceOffers(
-        Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId)))))
+        Seq(new WorkerOffer(executorId, executorHostPort(executorId), freeCores(executorId)))))
     }
 
     // Launch tasks returned by a set of resource offers
@@ -110,9 +112,9 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
         actorToExecutorId -= executorActor(executorId)
         addressToExecutorId -= executorAddress(executorId)
         executorActor -= executorId
-        executorHost -= executorId
+        executorHostPort -= executorId
         freeCores -= executorId
-        executorHost -= executorId
+        executorHostPort -= executorId
         totalCoreCount.addAndGet(-numCores)
         scheduler.executorLost(executorId, SlaveLost(reason))
       }
@@ -128,7 +130,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     while (iterator.hasNext) {
       val entry = iterator.next
       val (key, value) = (entry.getKey.toString, entry.getValue.toString)
-      if (key.startsWith("spark.")) {
+      if (key.startsWith("spark.") && !key.equals("spark.hostPort")) {
         properties += ((key, value))
       }
     }
@@ -136,10 +138,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
       Props(new DriverActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME)
   }
 
+  private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+
   override def stop() {
     try {
       if (driverActor != null) {
-        val timeout = 5.seconds
         val future = driverActor.ask(StopDriver)(timeout)
         Await.result(future, timeout)
       }
@@ -159,7 +162,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
   // Called by subclasses when notified of a lost worker
   def removeExecutor(executorId: String, reason: String) {
     try {
-      val timeout = 5.seconds
       val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout)
       Await.result(future, timeout)
     } catch {
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
index dfe3c5a85bc25f47a85a0da31a4649b27ef2c3be..718f26bfbd74e96f38d4fa5a7b3418022bb8ca34 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
@@ -1,5 +1,7 @@
 package spark.scheduler.cluster
 
+import spark.Utils
+
 /**
  * Information about a running task attempt inside a TaskSet.
  */
@@ -9,8 +11,11 @@ class TaskInfo(
     val index: Int,
     val launchTime: Long,
     val executorId: String,
-    val host: String,
-    val preferred: Boolean) {
+    val hostPort: String,
+    val taskLocality: TaskLocality.TaskLocality) {
+
+  Utils.checkHostPort(hostPort, "Expected hostport")
+
   var finishTime: Long = 0
   var failed = false
 
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
index b9d2dbf4874549403290bc5c892213e869ec60bc..1c403ef3234fd8e35dfd506b587829b18e235717 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
@@ -1,7 +1,6 @@
 package spark.scheduler.cluster
 
-import java.util.Arrays
-import java.util.{HashMap => JHashMap}
+import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays}
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
@@ -14,6 +13,44 @@ import spark.scheduler._
 import spark.TaskState.TaskState
 import java.nio.ByteBuffer
 
+private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging {
+
+  // process local is expected to be used ONLY within tasksetmanager for now.
+  val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value
+
+  type TaskLocality = Value
+
+  def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = {
+
+    // Must not be the constraint.
+    assert (constraint != TaskLocality.PROCESS_LOCAL)
+
+    constraint match {
+      case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL
+      case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL
+      // For anything else, allow
+      case _ => true
+    }
+  }
+
+  def parse(str: String): TaskLocality = {
+    // better way to do this ?
+    try {
+      val retval = TaskLocality.withName(str)
+      // Must not specify PROCESS_LOCAL !
+      assert (retval != TaskLocality.PROCESS_LOCAL)
+
+      retval
+    } catch {
+      case nEx: NoSuchElementException => {
+        logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL");
+        // default to preserve earlier behavior
+        NODE_LOCAL
+      }
+    }
+  }
+}
+
 /**
  * Schedules the tasks within a single TaskSet in the ClusterScheduler.
  */
@@ -58,14 +95,22 @@ private[spark] class TaskSetManager(
   // Last time when we launched a preferred task (for delay scheduling)
   var lastPreferredLaunchTime = System.currentTimeMillis
 
-  // List of pending tasks for each node. These collections are actually
+  // List of pending tasks for each node (process local to container). These collections are actually
   // treated as stacks, in which new tasks are added to the end of the
   // ArrayBuffer and removed from the end. This makes it faster to detect
   // tasks that repeatedly fail because whenever a task failed, it is put
   // back at the head of the stack. They are also only cleaned up lazily;
   // when a task is launched, it remains in all the pending lists except
   // the one that it was launched from, but gets removed from them later.
-  val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
+  private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]]
+
+  // List of pending tasks for each node.
+  // Essentially, similar to pendingTasksForHostPort, except at host level
+  private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
+
+  // List of pending tasks for each node based on rack locality.
+  // Essentially, similar to pendingTasksForHost, except at rack level
+  private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]]
 
   // List containing pending tasks with no locality preferences
   val pendingTasksWithNoPrefs = new ArrayBuffer[Int]
@@ -107,26 +152,150 @@ private[spark] class TaskSetManager(
     addPendingTask(i)
   }
 
+  // Note that it follows the hierarchy.
+  // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and
+  // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL
+  private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler,
+                                     taskLocality: TaskLocality.TaskLocality): HashSet[String] = {
+
+    if (TaskLocality.PROCESS_LOCAL == taskLocality) {
+      // straight forward comparison ! Special case it.
+      val retval = new HashSet[String]()
+      scheduler.synchronized {
+        for (location <- _taskPreferredLocations) {
+          if (scheduler.isExecutorAliveOnHostPort(location)) {
+            retval += location
+          }
+        }
+      }
+
+      return retval
+    }
+
+    val taskPreferredLocations =
+      if (TaskLocality.NODE_LOCAL == taskLocality) {
+        _taskPreferredLocations
+      } else {
+        assert (TaskLocality.RACK_LOCAL == taskLocality)
+        // Expand set to include all 'seen' rack local hosts.
+        // This works since container allocation/management happens within master - so any rack locality information is updated in msater.
+        // Best case effort, and maybe sort of kludge for now ... rework it later ?
+        val hosts = new HashSet[String]
+        _taskPreferredLocations.foreach(h => {
+          val rackOpt = scheduler.getRackForHost(h)
+          if (rackOpt.isDefined) {
+            val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get)
+            if (hostsOpt.isDefined) {
+              hosts ++= hostsOpt.get
+            }
+          }
+
+          // Ensure that irrespective of what scheduler says, host is always added !
+          hosts += h
+        })
+
+        hosts
+      }
+
+    val retval = new HashSet[String]
+    scheduler.synchronized {
+      for (prefLocation <- taskPreferredLocations) {
+        val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1)
+        if (aliveLocationsOpt.isDefined) {
+          retval ++= aliveLocationsOpt.get
+        }
+      }
+    }
+
+    retval
+  }
+
   // Add a task to all the pending-task lists that it should be on.
   private def addPendingTask(index: Int) {
-    val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive
-    if (locations.size == 0) {
+    // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate
+    // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it.
+    val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL)
+    val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL)
+    val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL)
+
+    if (rackLocalLocations.size == 0) {
+      // Current impl ensures this.
+      assert (processLocalLocations.size == 0)
+      assert (hostLocalLocations.size == 0)
       pendingTasksWithNoPrefs += index
     } else {
-      for (host <- locations) {
-        val list = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer())
+
+      // process local locality
+      for (hostPort <- processLocalLocations) {
+        // DEBUG Code
+        Utils.checkHostPort(hostPort)
+
+        val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer())
+        hostPortList += index
+      }
+
+      // host locality (includes process local)
+      for (hostPort <- hostLocalLocations) {
+        // DEBUG Code
+        Utils.checkHostPort(hostPort)
+
+        val host = Utils.parseHostPort(hostPort)._1
+        val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer())
+        hostList += index
+      }
+
+      // rack locality (includes process local and host local)
+      for (rackLocalHostPort <- rackLocalLocations) {
+        // DEBUG Code
+        Utils.checkHostPort(rackLocalHostPort)
+
+        val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1
+        val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer())
         list += index
       }
     }
+
     allPendingTasks += index
   }
 
+  // Return the pending tasks list for a given host port (process local), or an empty list if
+  // there is no map entry for that host
+  private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = {
+    // DEBUG Code
+    Utils.checkHostPort(hostPort)
+    pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer())
+  }
+
   // Return the pending tasks list for a given host, or an empty list if
   // there is no map entry for that host
-  private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
+  private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = {
+    val host = Utils.parseHostPort(hostPort)._1
     pendingTasksForHost.getOrElse(host, ArrayBuffer())
   }
 
+  // Return the pending tasks (rack level) list for a given host, or an empty list if
+  // there is no map entry for that host
+  private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = {
+    val host = Utils.parseHostPort(hostPort)._1
+    pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer())
+  }
+
+  // Number of pending tasks for a given host Port (which would be process local)
+  def numPendingTasksForHostPort(hostPort: String): Int = {
+    getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) )
+  }
+
+  // Number of pending tasks for a given host (which would be data local)
+  def numPendingTasksForHost(hostPort: String): Int = {
+    getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) )
+  }
+
+  // Number of pending rack local tasks for a given host
+  def numRackLocalPendingTasksForHost(hostPort: String): Int = {
+    getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) )
+  }
+
+
   // Dequeue a pending task from the given list and return its index.
   // Return None if the list is empty.
   // This method also cleans up any tasks in the list that have already
@@ -143,26 +312,49 @@ private[spark] class TaskSetManager(
   }
 
   // Return a speculative task for a given host if any are available. The task should not have an
-  // attempt running on this host, in case the host is slow. In addition, if localOnly is set, the
-  // task must have a preference for this host (or no preferred locations at all).
-  private def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = {
-    val hostsAlive = sched.hostsAlive
+  // attempt running on this host, in case the host is slow. In addition, if locality is set, the
+  // task must have a preference for this host/rack/no preferred locations at all.
+  private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = {
+
+    assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL))
     speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set
-    val localTask = speculatableTasks.find {
-        index =>
-          val locations = tasks(index).preferredLocations.toSet & hostsAlive
-          val attemptLocs = taskAttempts(index).map(_.host)
-          (locations.size == 0 || locations.contains(host)) && !attemptLocs.contains(host)
+
+    if (speculatableTasks.size > 0) {
+      val localTask = speculatableTasks.find {
+          index =>
+            val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL)
+            val attemptLocs = taskAttempts(index).map(_.hostPort)
+            (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort)
+        }
+
+      if (localTask != None) {
+        speculatableTasks -= localTask.get
+        return localTask
       }
-    if (localTask != None) {
-      speculatableTasks -= localTask.get
-      return localTask
-    }
-    if (!localOnly && speculatableTasks.size > 0) {
-      val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.host).contains(host))
-      if (nonLocalTask != None) {
-        speculatableTasks -= nonLocalTask.get
-        return nonLocalTask
+
+      // check for rack locality
+      if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
+        val rackTask = speculatableTasks.find {
+          index =>
+            val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL)
+            val attemptLocs = taskAttempts(index).map(_.hostPort)
+            locations.contains(hostPort) && !attemptLocs.contains(hostPort)
+        }
+
+        if (rackTask != None) {
+          speculatableTasks -= rackTask.get
+          return rackTask
+        }
+      }
+
+      // Any task ...
+      if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
+        // Check for attemptLocs also ?
+        val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort))
+        if (nonLocalTask != None) {
+          speculatableTasks -= nonLocalTask.get
+          return nonLocalTask
+        }
       }
     }
     return None
@@ -170,59 +362,112 @@ private[spark] class TaskSetManager(
 
   // Dequeue a pending task for a given node and return its index.
   // If localOnly is set to false, allow non-local tasks as well.
-  private def findTask(host: String, localOnly: Boolean): Option[Int] = {
-    val localTask = findTaskFromList(getPendingTasksForHost(host))
+  private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = {
+    val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort))
+    if (processLocalTask != None) {
+      return processLocalTask
+    }
+
+    val localTask = findTaskFromList(getPendingTasksForHost(hostPort))
     if (localTask != None) {
       return localTask
     }
+
+    if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
+      val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort))
+      if (rackLocalTask != None) {
+        return rackLocalTask
+      }
+    }
+
+    // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner.
+    // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down).
     val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs)
     if (noPrefTask != None) {
       return noPrefTask
     }
-    if (!localOnly) {
+
+    if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
       val nonLocalTask = findTaskFromList(allPendingTasks)
       if (nonLocalTask != None) {
         return nonLocalTask
       }
     }
+
     // Finally, if all else has failed, find a speculative task
-    return findSpeculativeTask(host, localOnly)
+    return findSpeculativeTask(hostPort, locality)
   }
 
-  // Does a host count as a preferred location for a task? This is true if
-  // either the task has preferred locations and this host is one, or it has
+  private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = {
+    Utils.checkHostPort(hostPort)
+
+    val locs = task.preferredLocations
+
+    locs.contains(hostPort)
+  }
+
+  private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = {
+    val locs = task.preferredLocations
+
+    // If no preference, consider it as host local
+    if (locs.isEmpty) return true
+
+    val host = Utils.parseHostPort(hostPort)._1
+    locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined
+  }
+
+  // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location).
+  // This is true if either the task has preferred locations and this host is one, or it has
   // no preferred locations (in which we still count the launch as preferred).
-  private def isPreferredLocation(task: Task[_], host: String): Boolean = {
+  private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = {
+
     val locs = task.preferredLocations
-    return (locs.contains(host) || locs.isEmpty)
+
+    val preferredRacks = new HashSet[String]()
+    for (preferredHost <- locs) {
+      val rack = sched.getRackForHost(preferredHost)
+      if (None != rack) preferredRacks += rack.get
+    }
+
+    if (preferredRacks.isEmpty) return false
+
+    val hostRack = sched.getRackForHost(hostPort)
+
+    return None != hostRack && preferredRacks.contains(hostRack.get)
   }
 
   // Respond to an offer of a single slave from the scheduler by finding a task
-  def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = {
+  def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = {
+
     if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) {
-      val time = System.currentTimeMillis
-      val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT)
+      // If explicitly specified, use that
+      val locality = if (overrideLocality != null) overrideLocality else {
+        // expand only if we have waited for more than LOCALITY_WAIT for a host local task ...
+        val time = System.currentTimeMillis
+        if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY
+      }
 
-      findTask(host, localOnly) match {
+      findTask(hostPort, locality) match {
         case Some(index) => {
           // Found a task; do some bookkeeping and return a Mesos task for it
           val task = tasks(index)
           val taskId = sched.newTaskId()
           // Figure out whether this should count as a preferred launch
-          val preferred = isPreferredLocation(task, host)
-          val prefStr = if (preferred) {
-            "preferred"
-          } else {
-            "non-preferred, not one of " + task.preferredLocations.mkString(", ")
-          }
-          logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format(
-            taskSet.id, index, taskId, execId, host, prefStr))
+          val taskLocality =
+            if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else
+            if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else
+            if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else
+              TaskLocality.ANY
+          val prefStr = taskLocality.toString
+          logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format(
+            taskSet.id, index, taskId, execId, hostPort, prefStr))
           // Do various bookkeeping
           copiesRunning(index) += 1
-          val info = new TaskInfo(taskId, index, time, execId, host, preferred)
+          val time = System.currentTimeMillis
+          val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality)
           taskInfos(taskId) = info
           taskAttempts(index) = info :: taskAttempts(index)
-          if (preferred) {
+          if (TaskLocality.NODE_LOCAL == taskLocality) {
             lastPreferredLaunchTime = time
           }
           // Serialize and return the task
@@ -313,7 +558,7 @@ private[spark] class TaskSetManager(
             return
 
           case ef: ExceptionFailure =>
-            val key = ef.exception.toString
+            val key = ef.description
             val now = System.currentTimeMillis
             val (printFull, dupCount) = {
               if (recentExceptions.contains(key)) {
@@ -331,10 +576,11 @@ private[spark] class TaskSetManager(
               }
             }
             if (printFull) {
-              val locs = ef.exception.getStackTrace.map(loc => "\tat %s".format(loc.toString))
-              logInfo("Loss was due to %s\n%s".format(ef.exception.toString, locs.mkString("\n")))
+              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.exception.toString, dupCount))
+              logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount))
             }
 
           case _ => {}
@@ -404,19 +650,24 @@ private[spark] class TaskSetManager(
     return sortedTaskSetQueue
   }
 
-  override def executorLost(execId: String, hostname: String) {
+  override def executorLost(execId: String, hostPort: String) {
     logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id)
-    val newHostsAlive = sched.hostsAlive
+
     // If some task has preferred locations only on hostname, and there are no more executors there,
     // put it in the no-prefs list to avoid the wait from delay scheduling
-    if (!newHostsAlive.contains(hostname)) {
-      for (index <- getPendingTasksForHost(hostname)) {
-        val newLocs = tasks(index).preferredLocations.toSet & newHostsAlive
-        if (newLocs.isEmpty) {
-          pendingTasksWithNoPrefs += index
-        }
+
+    // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to
+    // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc.
+    // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if
+    // there is no host local node for the task (not if there is no process local node for the task)
+    for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) {
+      // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL)
+      val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL)
+      if (newLocs.isEmpty) {
+        pendingTasksWithNoPrefs += index
       }
     }
+
     // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage
     if (tasks(0).isInstanceOf[ShuffleMapTask]) {
       for ((tid, info) <- taskInfos if info.executorId == execId) {
@@ -468,7 +719,7 @@ private[spark] class TaskSetManager(
           !speculatableTasks.contains(index)) {
           logInfo(
             "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format(
-              taskSet.id, index, info.host, threshold))
+              taskSet.id, index, info.hostPort, threshold))
           speculatableTasks += index
           foundTasks = true
         }
@@ -476,4 +727,8 @@ private[spark] class TaskSetManager(
     }
     return foundTasks
   }
+
+  override def hasPendingTasks(): Boolean = {
+    numTasks > 0 && tasksFinished < numTasks
+  }
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
index 3c3afcbb14d3f7f677be8aae91128e35eef01c7f..c47824315ccbd6c61d2fafa27c7288cbc2d548e1 100644
--- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
@@ -4,5 +4,5 @@ package spark.scheduler.cluster
  * Represents free resources available on an executor.
  */
 private[spark]
-class WorkerOffer(val executorId: String, val hostname: String, val cores: Int) {
+class WorkerOffer(val executorId: String, val hostPort: String, val cores: Int) {
 }
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
index a76253ea14d3e108f251501c948383d143b666f6..37a67f9b1b977a6f2ea5ac750a0bf4031060e2a8 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -7,7 +7,7 @@ import scala.collection.mutable.HashMap
 import spark._
 import spark.executor.ExecutorURLClassLoader
 import spark.scheduler._
-import spark.scheduler.cluster.TaskInfo
+import spark.scheduler.cluster.{TaskLocality, TaskInfo}
 
 /**
  * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally
@@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
 
     def runTask(task: Task[_], idInJob: Int, attemptId: Int) {
       logInfo("Running " + task)
-      val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local", true)
+      val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL)
       // Set the Spark execution environment for the worker thread
       SparkEnv.set(env)
       try {
@@ -67,8 +67,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
         logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes")
         val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes)
         updateDependencies(taskFiles, taskJars)   // Download any files added with addFile
+        val deserStart = System.currentTimeMillis()
         val deserializedTask = ser.deserialize[Task[_]](
             taskBytes, Thread.currentThread.getContextClassLoader)
+        val deserTime = System.currentTimeMillis() - deserStart
 
         // Run it
         val result: Any = deserializedTask.run(attemptId)
@@ -77,15 +79,19 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
         // 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 resultToReturn = ser.deserialize[Any](ser.serialize(result))
+        val serResult = ser.serialize(result)
+        deserializedTask.metrics.get.resultSize = serResult.limit()
+        val resultToReturn = ser.deserialize[Any](serResult)
         val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
           ser.serialize(Accumulators.values))
         logInfo("Finished " + task)
         info.markSuccessful()
+        deserializedTask.metrics.get.executorRunTime = info.duration.toInt  //close enough
+        deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt
 
         // If the threadpool has not already been shutdown, notify DAGScheduler
         if (!Thread.currentThread().isInterrupted)
-          listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, null)
+          listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, deserializedTask.metrics.getOrElse(null))
       } catch {
         case t: Throwable => {
           logError("Exception in task " + idInJob, t)
@@ -95,8 +101,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
               submitTask(task, idInJob)
             } else {
               // TODO: Do something nicer here to return all the way to the user
-              if (!Thread.currentThread().isInterrupted)
-                listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null)
+              if (!Thread.currentThread().isInterrupted) {
+                val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace)
+                listener.taskEnded(task, failure, null, null, info, null)
+              }
             }
           }
         }
diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala
index 50b086125a55e2217f5877c54c708fae8f4e8a3b..2ad73b711de741255ecb1ca7d9836675c0484e3f 100644
--- a/core/src/main/scala/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/spark/serializer/Serializer.scala
@@ -1,10 +1,13 @@
 package spark.serializer
 
-import java.nio.ByteBuffer
 import java.io.{EOFException, InputStream, OutputStream}
+import java.nio.ByteBuffer
+
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+
 import spark.util.ByteBufferInputStream
 
+
 /**
  * A serializer. Because some serialization libraries are not thread safe, this class is used to
  * create [[spark.serializer.SerializerInstance]] objects that do the actual serialization and are
@@ -14,6 +17,7 @@ trait Serializer {
   def newInstance(): SerializerInstance
 }
 
+
 /**
  * An instance of a serializer, for use by one thread at a time.
  */
@@ -45,6 +49,7 @@ trait SerializerInstance {
   }
 }
 
+
 /**
  * A stream for writing serialized objects.
  */
@@ -61,6 +66,7 @@ trait SerializationStream {
   }
 }
 
+
 /**
  * A stream for reading serialized objects.
  */
@@ -72,40 +78,18 @@ trait DeserializationStream {
    * Read the elements of this stream through an iterator. This can only be called once, as
    * reading each element will consume data from the input source.
    */
-  def asIterator: Iterator[Any] = new Iterator[Any] {
-    var gotNext = false
-    var finished = false
-    var nextValue: Any = null
-
-    private def getNext() {
+  def asIterator: Iterator[Any] = new spark.util.NextIterator[Any] {
+    override protected def getNext() = {
       try {
-        nextValue = readObject[Any]()
+        readObject[Any]()
       } catch {
         case eof: EOFException =>
           finished = true
       }
-      gotNext = true
     }
 
-    override def hasNext: Boolean = {
-      if (!gotNext) {
-        getNext()
-      }
-      if (finished) {
-        close()
-      }
-      !finished
-    }
-
-    override def next(): Any = {
-      if (!gotNext) {
-        getNext()
-      }
-      if (finished) {
-        throw new NoSuchElementException("End of stream")
-      }
-      gotNext = false
-      nextValue
+    override protected def close() {
+      DeserializationStream.this.close()
     }
   }
 }
diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..60b2aac797a130bfbde9e83f804811a0e53aafe4
--- /dev/null
+++ b/core/src/main/scala/spark/serializer/SerializerManager.scala
@@ -0,0 +1,45 @@
+package spark.serializer
+
+import java.util.concurrent.ConcurrentHashMap
+
+
+/**
+ * A service that returns a serializer object given the serializer's class name. If a previous
+ * instance of the serializer object has been created, the get method returns that instead of
+ * creating a new one.
+ */
+private[spark] class SerializerManager {
+
+  private val serializers = new ConcurrentHashMap[String, Serializer]
+  private var _default: Serializer = _
+
+  def default = _default
+
+  def setDefault(clsName: String): Serializer = {
+    _default = get(clsName)
+    _default
+  }
+
+  def get(clsName: String): Serializer = {
+    if (clsName == null) {
+      default
+    } else {
+      var serializer = serializers.get(clsName)
+      if (serializer != null) {
+        // If the serializer has been created previously, reuse that.
+        serializer
+      } else this.synchronized {
+        // Otherwise, create a new one. But make sure no other thread has attempted
+        // to create another new one at the same time.
+        serializer = serializers.get(clsName)
+        if (serializer == null) {
+          val clsLoader = Thread.currentThread.getContextClassLoader
+          serializer =
+            Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer]
+          serializers.put(clsName, serializer)
+        }
+        serializer
+      }
+    }
+  }
+}
diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f275d476df9f145755ef1286f71c1fdd13dd1b79
--- /dev/null
+++ b/core/src/main/scala/spark/storage/BlockException.scala
@@ -0,0 +1,5 @@
+package spark.storage
+
+private[spark]
+case class BlockException(blockId: String, message: String) extends Exception(message)
+
diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
index ababb04305cde82f18665b6b65298af2857faf0c..0718156b1ba2f1dacc0925aa4add58123cdc8834 100644
--- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala
+++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
@@ -1,10 +1,10 @@
 package spark.storage
 
 private[spark] trait BlockFetchTracker {
-    def totalBlocks : Int
-    def numLocalBlocks: Int
-    def numRemoteBlocks: Int
-    def remoteFetchTime : Long
-    def remoteFetchWaitTime: Long
-    def remoteBytesRead : Long
+  def totalBlocks : Int
+  def numLocalBlocks: Int
+  def numRemoteBlocks: Int
+  def remoteFetchTime : Long
+  def fetchWaitTime: Long
+  def remoteBytesRead : Long
 }
diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..43f835237ca32662740ee4029063ee4a4bc5fc26
--- /dev/null
+++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
@@ -0,0 +1,360 @@
+package spark.storage
+
+import java.nio.ByteBuffer
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashSet
+import scala.collection.mutable.Queue
+
+import io.netty.buffer.ByteBuf
+
+import spark.Logging
+import spark.Utils
+import spark.SparkException
+import spark.network.BufferMessage
+import spark.network.ConnectionManagerId
+import spark.network.netty.ShuffleCopier
+import spark.serializer.Serializer
+
+
+/**
+ * A block fetcher iterator interface. There are two implementations:
+ *
+ * BasicBlockFetcherIterator: uses a custom-built NIO communication layer.
+ * NettyBlockFetcherIterator: uses Netty (OIO) as the communication layer.
+ *
+ * Eventually we would like the two to converge and use a single NIO-based communication layer,
+ * but extensive tests show that under some circumstances (e.g. large shuffles with lots of cores),
+ * NIO would perform poorly and thus the need for the Netty OIO one.
+ */
+
+private[storage]
+trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])]
+  with Logging with BlockFetchTracker {
+  def initialize()
+}
+
+
+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)]) {
+    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]) {
+    def failed: Boolean = size == -1
+  }
+
+  class BasicBlockFetcherIterator(
+      private val blockManager: BlockManager,
+      val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+      serializer: Serializer)
+    extends BlockFetcherIterator {
+
+    import blockManager._
+
+    private var _remoteBytesRead = 0l
+    private var _remoteFetchTime = 0l
+    private var _fetchWaitTime = 0l
+
+    if (blocksByAddress == null) {
+      throw new IllegalArgumentException("BlocksByAddress is null")
+    }
+
+    protected var _totalBlocks = blocksByAddress.map(_._2.size).sum
+    logDebug("Getting " + _totalBlocks + " blocks")
+    protected var startTime = System.currentTimeMillis
+    protected val localBlockIds = new ArrayBuffer[String]()
+    protected val remoteBlockIds = new HashSet[String]()
+
+    // A queue to hold our results.
+    protected val results = new LinkedBlockingQueue[FetchResult]
+
+    // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that
+    // the number of bytes in flight is limited to maxBytesInFlight
+    private val fetchRequests = new Queue[FetchRequest]
+
+    // Current bytes in flight from our requests
+    private var bytesInFlight = 0L
+
+    protected def sendRequest(req: FetchRequest) {
+      logDebug("Sending request for %d blocks (%s) from %s".format(
+        req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort))
+      val cmId = new ConnectionManagerId(req.address.host, req.address.port)
+      val blockMessageArray = new BlockMessageArray(req.blocks.map {
+        case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId))
+      })
+      bytesInFlight += req.size
+      val sizeMap = req.blocks.toMap  // so we can look up the size of each blockID
+      val fetchStart = System.currentTimeMillis()
+      val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
+      future.onSuccess {
+        case Some(message) => {
+          val fetchDone = System.currentTimeMillis()
+          _remoteFetchTime += fetchDone - fetchStart
+          val bufferMessage = message.asInstanceOf[BufferMessage]
+          val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage)
+          for (blockMessage <- blockMessageArray) {
+            if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) {
+              throw new SparkException(
+                "Unexpected message " + blockMessage.getType + " received from " + cmId)
+            }
+            val blockId = blockMessage.getId
+            results.put(new FetchResult(blockId, sizeMap(blockId),
+              () => dataDeserialize(blockId, blockMessage.getData, serializer)))
+            _remoteBytesRead += req.size
+            logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
+          }
+        }
+        case None => {
+          logError("Could not get block(s) from " + cmId)
+          for ((blockId, size) <- req.blocks) {
+            results.put(new FetchResult(blockId, -1, null))
+          }
+        }
+      }
+    }
+
+    protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = {
+      // Split local and remote blocks. Remote blocks are further split into FetchRequests of size
+      // at most maxBytesInFlight in order to limit the amount of data in flight.
+      val remoteRequests = new ArrayBuffer[FetchRequest]
+      for ((address, blockInfos) <- blocksByAddress) {
+        if (address == blockManagerId) {
+          localBlockIds ++= blockInfos.map(_._1)
+        } else {
+          remoteBlockIds ++= blockInfos.map(_._1)
+          // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them
+          // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5
+          // nodes, rather than blocking on reading output from one node.
+          val minRequestSize = math.max(maxBytesInFlight / 5, 1L)
+          logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
+          val iterator = blockInfos.iterator
+          var curRequestSize = 0L
+          var curBlocks = new ArrayBuffer[(String, Long)]
+          while (iterator.hasNext) {
+            val (blockId, size) = iterator.next()
+            curBlocks += ((blockId, size))
+            curRequestSize += size
+            if (curRequestSize >= minRequestSize) {
+              // Add this FetchRequest
+              remoteRequests += new FetchRequest(address, curBlocks)
+              curRequestSize = 0
+              curBlocks = new ArrayBuffer[(String, Long)]
+            }
+          }
+          // Add in the final request
+          if (!curBlocks.isEmpty) {
+            remoteRequests += new FetchRequest(address, curBlocks)
+          }
+        }
+      }
+      remoteRequests
+    }
+
+    protected def getLocalBlocks() {
+      // Get the local blocks while remote blocks are being fetched. Note that it's okay to do
+      // these all at once because they will just memory-map some files, so they won't consume
+      // any memory that might exceed our maxBytesInFlight
+      for (id <- localBlockIds) {
+        getLocal(id) match {
+          case Some(iter) => {
+            // Pass 0 as size since it's not in flight
+            results.put(new FetchResult(id, 0, () => iter))
+            logDebug("Got local block " + id)
+          }
+          case None => {
+            throw new BlockException(id, "Could not get block " + id + " from local machine")
+          }
+        }
+      }
+    }
+
+    override def initialize() {
+      // Split local and remote blocks.
+      val remoteRequests = splitLocalRemoteBlocks()
+      // Add the remote requests into our queue in a random order
+      fetchRequests ++= Utils.randomize(remoteRequests)
+
+      // Send out initial requests for blocks, up to our maxBytesInFlight
+      while (!fetchRequests.isEmpty &&
+        (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
+        sendRequest(fetchRequests.dequeue())
+      }
+
+      val numGets = remoteBlockIds.size - fetchRequests.size
+      logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime))
+
+      // Get Local Blocks
+      startTime = System.currentTimeMillis
+      getLocalBlocks()
+      logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
+    }
+
+    //an iterator that will read fetched blocks off the queue as they arrive.
+    @volatile protected var resultsGotten = 0
+
+    override def hasNext: Boolean = resultsGotten < _totalBlocks
+
+    override def next(): (String, Option[Iterator[Any]]) = {
+      resultsGotten += 1
+      val startFetchWait = System.currentTimeMillis()
+      val result = results.take()
+      val stopFetchWait = System.currentTimeMillis()
+      _fetchWaitTime += (stopFetchWait - startFetchWait)
+      if (! result.failed) bytesInFlight -= result.size
+      while (!fetchRequests.isEmpty &&
+        (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
+        sendRequest(fetchRequests.dequeue())
+      }
+      (result.blockId, if (result.failed) None else Some(result.deserialize()))
+    }
+
+    // Implementing BlockFetchTracker trait.
+    override def totalBlocks: Int = _totalBlocks
+    override def numLocalBlocks: Int = localBlockIds.size
+    override def numRemoteBlocks: Int = remoteBlockIds.size
+    override def remoteFetchTime: Long = _remoteFetchTime
+    override def fetchWaitTime: Long = _fetchWaitTime
+    override def remoteBytesRead: Long = _remoteBytesRead
+  }
+  // End of BasicBlockFetcherIterator
+
+  class NettyBlockFetcherIterator(
+      blockManager: BlockManager,
+      blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+      serializer: Serializer)
+    extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) {
+
+    import blockManager._
+
+    val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest]
+
+    private def startCopiers(numCopiers: Int): List[_ <: Thread] = {
+      (for ( i <- Range(0,numCopiers) ) yield {
+        val copier = new Thread {
+          override def run(){
+            try {
+              while(!isInterrupted && !fetchRequestsSync.isEmpty) {
+                sendRequest(fetchRequestsSync.take())
+              }
+            } catch {
+              case x: InterruptedException => logInfo("Copier Interrupted")
+              //case _ => throw new SparkException("Exception Throw in Shuffle Copier")
+            }
+          }
+        }
+        copier.start
+        copier
+      }).toList
+    }
+
+    //keep this to interrupt the threads when necessary
+    private def stopCopiers() {
+      for (copier <- copiers) {
+        copier.interrupt()
+      }
+    }
+
+    override protected def sendRequest(req: FetchRequest) {
+
+      def putResult(blockId: String, blockSize: Long, blockData: ByteBuf) {
+        val fetchResult = new FetchResult(blockId, blockSize,
+          () => dataDeserialize(blockId, blockData.nioBuffer, serializer))
+        results.put(fetchResult)
+      }
+
+      logDebug("Sending request for %d blocks (%s) from %s".format(
+        req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host))
+      val cmId = new ConnectionManagerId(
+        req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt)
+      val cpier = new ShuffleCopier
+      cpier.getBlocks(cmId, req.blocks, putResult)
+      logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host )
+    }
+
+    override protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = {
+      // Split local and remote blocks. Remote blocks are further split into FetchRequests of size
+      // at most maxBytesInFlight in order to limit the amount of data in flight.
+      val originalTotalBlocks = _totalBlocks;
+      val remoteRequests = new ArrayBuffer[FetchRequest]
+      for ((address, blockInfos) <- blocksByAddress) {
+        if (address == blockManagerId) {
+          localBlockIds ++= blockInfos.map(_._1)
+        } else {
+          remoteBlockIds ++= blockInfos.map(_._1)
+          // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them
+          // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5
+          // nodes, rather than blocking on reading output from one node.
+          val minRequestSize = math.max(maxBytesInFlight / 5, 1L)
+          logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
+          val iterator = blockInfos.iterator
+          var curRequestSize = 0L
+          var curBlocks = new ArrayBuffer[(String, Long)]
+          while (iterator.hasNext) {
+            val (blockId, size) = iterator.next()
+            if (size > 0) {
+              curBlocks += ((blockId, size))
+              curRequestSize += size
+            } else if (size == 0) {
+              //here we changes the totalBlocks
+              _totalBlocks -= 1
+            } else {
+              throw new BlockException(blockId, "Negative block size " + size)
+            }
+            if (curRequestSize >= minRequestSize) {
+              // Add this FetchRequest
+              remoteRequests += new FetchRequest(address, curBlocks)
+              curRequestSize = 0
+              curBlocks = new ArrayBuffer[(String, Long)]
+            }
+          }
+          // Add in the final request
+          if (!curBlocks.isEmpty) {
+            remoteRequests += new FetchRequest(address, curBlocks)
+          }
+        }
+      }
+      logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " +
+        originalTotalBlocks + " blocks")
+      remoteRequests
+    }
+
+    private var copiers: List[_ <: Thread] = null
+
+    override def initialize() {
+      // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks
+      val remoteRequests = splitLocalRemoteBlocks()
+      // Add the remote requests into our queue in a random order
+      for (request <- Utils.randomize(remoteRequests)) {
+        fetchRequestsSync.put(request)
+      }
+
+      copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt)
+      logInfo("Started " + fetchRequestsSync.size + " remote gets in " +
+        Utils.getUsedTimeMs(startTime))
+
+      // Get Local Blocks
+      startTime = System.currentTimeMillis
+      getLocalBlocks()
+      logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
+    }
+
+    override def next(): (String, Option[Iterator[Any]]) = {
+      resultsGotten += 1
+      val result = results.take()
+      // if all the results has been retrieved, shutdown the copiers
+      if (resultsGotten == _totalBlocks && copiers != null) {
+        stopCopiers()
+      }
+      (result.blockId, if (result.failed) None else Some(result.deserialize()))
+    }
+  }
+  // End of NettyBlockFetcherIterator
+}
diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala
index 3118d3d412b2f207f502c0cbbbd8479837217fb3..40d608628e9c272f3df630266c39c468ae58ce61 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/spark/storage/BlockManager.scala
@@ -2,9 +2,8 @@ package spark.storage
 
 import java.io.{InputStream, OutputStream}
 import java.nio.{ByteBuffer, MappedByteBuffer}
-import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue}
 
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
+import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue}
 import scala.collection.JavaConversions._
 
 import akka.actor.{ActorSystem, Cancellable, Props}
@@ -24,30 +23,35 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam
 import sun.nio.ch.DirectBuffer
 
 
-private[spark]
-case class BlockException(blockId: String, message: String, ex: Exception = null)
-extends Exception(message)
-
-private[spark]
-class BlockManager(
+private[spark] class BlockManager(
     executorId: String,
     actorSystem: ActorSystem,
     val master: BlockManagerMaster,
-    val serializer: Serializer,
+    val defaultSerializer: Serializer,
     maxMemory: Long)
   extends Logging {
 
-  class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
-    var pending: Boolean = true
-    var size: Long = -1L
-    var failed: Boolean = false
+  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 (pending) {
+      if (initThread != Thread.currentThread() && pending) {
         synchronized {
           while (pending) this.wait()
         }
@@ -57,28 +61,37 @@ class BlockManager(
 
     /** 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 {
-        pending = false
-        failed = false
-        size = sizeInBytes
         this.notifyAll()
       }
     }
 
     /** Mark this BlockInfo as ready but failed */
     def markFailure() {
+      assert (pending)
+      size = 0
+      initThread = null
+      failed = true
+      initThread = null
+      pending = false
       synchronized {
-        failed = true
-        pending = false
         this.notifyAll()
       }
     }
   }
 
+  val shuffleBlockManager = new ShuffleBlockManager(this)
+
   private val blockInfo = new TimeStampedHashMap[String, BlockInfo]
 
   private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
-  private[storage] val diskStore: BlockStore =
+  private[storage] val diskStore: DiskStore =
     new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
 
   val connectionManager = new ConnectionManager(0)
@@ -101,7 +114,7 @@ class BlockManager(
 
   val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties
 
-  val host = System.getProperty("spark.hostname", Utils.localHostName())
+  val hostPort = Utils.localHostPort()
 
   val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
     name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
@@ -212,9 +225,12 @@ class BlockManager(
    * Tell the master about the current storage status of a block. This will send a block update
    * message reflecting the current status, *not* the desired storage level in its block info.
    * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk.
+   *
+   * 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) {
-    val needReregister = !tryToReportBlockStatus(blockId, info)
+  def reportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L) {
+    val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize)
     if (needReregister) {
       logInfo("Got told to reregister updating block " + blockId)
       // Reregistering will report our new block for free.
@@ -228,7 +244,7 @@ class BlockManager(
    * 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): Boolean = {
+  private def tryToReportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = {
     val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized {
       info.level match {
         case null =>
@@ -237,7 +253,7 @@ class BlockManager(
           val inMem = level.useMemory && memoryStore.contains(blockId)
           val onDisk = level.useDisk && diskStore.contains(blockId)
           val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication)
-          val memSize = if (inMem) memoryStore.getSize(blockId) else 0L
+          val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize
           val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L
           (storageLevel, memSize, diskSize, info.tellMaster)
       }
@@ -252,24 +268,23 @@ class BlockManager(
 
 
   /**
-   * Get locations of the block.
+   * Get locations of an array of blocks.
    */
-  def getLocations(blockId: String): Seq[String] = {
+  def getLocationBlockIds(blockIds: Array[String]): Array[Seq[BlockManagerId]] = {
     val startTimeMs = System.currentTimeMillis
-    var managers = master.getLocations(blockId)
-    val locations = managers.map(_.ip)
-    logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs))
+    val locations = master.getLocations(blockIds).toArray
+    logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
     return locations
   }
 
   /**
-   * Get locations of an array of blocks.
+   * A short-circuited method to get blocks directly from disk. This is used for getting
+   * shuffle blocks. It is safe to do so without a lock on block info since disk store
+   * never deletes (recent) items.
    */
-  def getLocations(blockIds: Array[String]): Array[Seq[String]] = {
-    val startTimeMs = System.currentTimeMillis
-    val locations = master.getLocations(blockIds).map(_.map(_.ip).toSeq).toArray
-    logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
-    return locations
+  def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = {
+    diskStore.getValues(blockId, serializer).orElse(
+      sys.error("Block " + blockId + " not found on disk, though it should be"))
   }
 
   /**
@@ -277,18 +292,6 @@ class BlockManager(
    */
   def getLocal(blockId: String): Option[Iterator[Any]] = {
     logDebug("Getting local block " + blockId)
-
-    // As an optimization for map output fetches, if the block is for a shuffle, return it
-    // without acquiring a lock; the disk store never deletes (recent) items so this should work
-    if (blockId.startsWith("shuffle_")) {
-      return diskStore.getValues(blockId) match {
-        case Some(iterator) =>
-          Some(iterator)
-        case None =>
-          throw new Exception("Block " + blockId + " not found on disk, though it should be")
-      }
-    }
-
     val info = blockInfo.get(blockId).orNull
     if (info != null) {
       info.synchronized {
@@ -339,6 +342,8 @@ class BlockManager(
               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)
@@ -372,7 +377,7 @@ class BlockManager(
 
     // As an optimization for map output fetches, if the block is for a shuffle, return it
     // without acquiring a lock; the disk store never deletes (recent) items so this should work
-    if (blockId.startsWith("shuffle_")) {
+    if (ShuffleBlockManager.isShuffle(blockId)) {
       return diskStore.getBytes(blockId) match {
         case Some(bytes) =>
           Some(bytes)
@@ -411,6 +416,7 @@ class BlockManager(
           // 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)
@@ -450,7 +456,7 @@ class BlockManager(
     for (loc <- locations) {
       logDebug("Getting remote block " + blockId + " from " + loc)
       val data = BlockManagerWorker.syncGetBlock(
-          GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port))
+          GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
       if (data != null) {
         return Some(dataDeserialize(blockId, data))
       }
@@ -473,9 +479,19 @@ class BlockManager(
    * fashion as they're received. Expects a size in bytes to be provided for each block fetched,
    * so that we can control the maxMegabytesInFlight for the fetch.
    */
-  def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])])
+  def getMultiple(
+    blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer)
       : BlockFetcherIterator = {
-    return new BlockFetcherIterator(this, blocksByAddress)
+
+    val iter =
+      if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) {
+        new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer)
+      } else {
+        new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer)
+      }
+
+    iter.initialize()
+    iter
   }
 
   def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
@@ -485,6 +501,22 @@ class BlockManager(
     put(blockId, elements, level, tellMaster)
   }
 
+  /**
+   * A short circuited method to get a block writer that can write data directly to disk.
+   * This is currently used for writing shuffle files out. Callers should handle error
+   * cases.
+   */
+  def getDiskBlockWriter(blockId: String, 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
+  }
+
   /**
    * Put a new block of values to the block manager. Returns its (estimated) size in bytes.
    */
@@ -501,17 +533,26 @@ class BlockManager(
       throw new IllegalArgumentException("Storage level is null or invalid")
     }
 
-    val oldBlock = blockInfo.get(blockId).orNull
-    if (oldBlock != null && oldBlock.waitForReady()) {
-      logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
-      return oldBlock.size
-    }
-
     // 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 = new BlockInfo(level, tellMaster)
-    blockInfo.put(blockId, myInfo)
+    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 oldBlockOpt.get.size
+        }
+
+        // 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
 
@@ -531,6 +572,7 @@ class BlockManager(
       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
@@ -555,26 +597,25 @@ class BlockManager(
 
         // 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(size)
         if (tellMaster) {
           reportBlockStatus(blockId, myInfo)
         }
-      } catch {
+      } 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.
-        case e: Exception => {
+        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", e)
-          throw e
+          logWarning("Putting block " + blockId + " failed")
         }
       }
     }
     logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs))
 
-
     // Replicate block if required
     if (level.replication > 1) {
       val remoteStartTime = System.currentTimeMillis
@@ -611,16 +652,26 @@ class BlockManager(
       throw new IllegalArgumentException("Storage level is null or invalid")
     }
 
-    if (blockInfo.contains(blockId)) {
-      logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
-      return
-    }
-
     // 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 = new BlockInfo(level, tellMaster)
-    blockInfo.put(blockId, myInfo)
+    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
 
@@ -639,6 +690,7 @@ class BlockManager(
       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
@@ -649,22 +701,24 @@ class BlockManager(
           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)
         }
-      } catch {
+      } 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.
-        case e: Exception => {
+        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", e)
-          throw e
+          logWarning("Putting block " + blockId + " failed")
         }
       }
     }
@@ -698,7 +752,7 @@ class BlockManager(
       logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is "
         + data.limit() + " Bytes. To node: " + peer)
       if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel),
-        new ConnectionManagerId(peer.ip, peer.port))) {
+        new ConnectionManagerId(peer.host, peer.port))) {
         logError("Failed to call syncPutBlock to " + peer)
       }
       logDebug("Replicated BlockId " + blockId + " once used " +
@@ -730,6 +784,14 @@ class BlockManager(
     val info = blockInfo.get(blockId).orNull
     if (info != null)  {
       info.synchronized {
+        // required ? As of now, this will be invoked only for blocks which are ready
+        // But in case this changes in future, adding for consistency sake.
+        if (! info.waitForReady() ) {
+          // If we get here, the block write failed.
+          logWarning("Block " + blockId + " was marked as failure. Nothing to drop")
+          return
+        }
+
         val level = info.level
         if (level.useDisk && !diskStore.contains(blockId)) {
           logInfo("Writing block " + blockId + " to disk")
@@ -740,12 +802,13 @@ class BlockManager(
               diskStore.putBytes(blockId, bytes, level)
           }
         }
+        val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
         val blockWasRemoved = memoryStore.remove(blockId)
         if (!blockWasRemoved) {
           logWarning("Block " + blockId + " could not be dropped from memory as it does not exist")
         }
         if (info.tellMaster) {
-          reportBlockStatus(blockId, info)
+          reportBlockStatus(blockId, info, droppedMemorySize)
         }
         if (!level.useDisk) {
           // The block is completely gone from this node; forget it so we can put() it again later.
@@ -805,7 +868,7 @@ class BlockManager(
   }
 
   def shouldCompress(blockId: String): Boolean = {
-    if (blockId.startsWith("shuffle_")) {
+    if (ShuffleBlockManager.isShuffle(blockId)) {
       compressShuffle
     } else if (blockId.startsWith("broadcast_")) {
       compressBroadcast
@@ -820,7 +883,11 @@ class BlockManager(
    * Wrap an output stream for compression if block compression is enabled for its block type
    */
   def wrapForCompression(blockId: String, s: OutputStream): OutputStream = {
-    if (shouldCompress(blockId)) new LZFOutputStream(s) else s
+    if (shouldCompress(blockId)) {
+      (new LZFOutputStream(s)).setFinishBlockOnFlush(true)
+    } else {
+      s
+    }
   }
 
   /**
@@ -830,7 +897,10 @@ class BlockManager(
     if (shouldCompress(blockId)) new LZFInputStream(s) else s
   }
 
-  def dataSerialize(blockId: String, values: Iterator[Any]): ByteBuffer = {
+  def dataSerialize(
+      blockId: String,
+      values: Iterator[Any],
+      serializer: Serializer = defaultSerializer): ByteBuffer = {
     val byteStream = new FastByteArrayOutputStream(4096)
     val ser = serializer.newInstance()
     ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close()
@@ -842,7 +912,10 @@ class BlockManager(
    * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of
    * the iterator is reached.
    */
-  def dataDeserialize(blockId: String, bytes: ByteBuffer): Iterator[Any] = {
+  def dataDeserialize(
+      blockId: String,
+      bytes: ByteBuffer,
+      serializer: Serializer = defaultSerializer): Iterator[Any] = {
     bytes.rewind()
     val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true))
     serializer.newInstance().deserializeStream(stream).asIterator
@@ -862,8 +935,8 @@ class BlockManager(
   }
 }
 
-private[spark]
-object BlockManager extends Logging {
+
+private[spark] object BlockManager extends Logging {
 
   val ID_GENERATOR = new IdGenerator
 
@@ -892,177 +965,45 @@ object BlockManager extends Logging {
       }
     }
   }
-}
-
-class BlockFetcherIterator(
-    private val blockManager: BlockManager,
-    val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]
-) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker {
-
-  import blockManager._
-
-  private var _remoteBytesRead = 0l
-  private var _remoteFetchTime = 0l
-  private var _remoteFetchWaitTime = 0l
-
-  if (blocksByAddress == null) {
-    throw new IllegalArgumentException("BlocksByAddress is null")
-  }
-  val totalBlocks = blocksByAddress.map(_._2.size).sum
-  logDebug("Getting " + totalBlocks + " blocks")
-  var startTime = System.currentTimeMillis
-  val localBlockIds = new ArrayBuffer[String]()
-  val remoteBlockIds = new HashSet[String]()
-
-  // 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]) {
-    def failed: Boolean = size == -1
-  }
 
-  // A queue to hold our results.
-  val results = new LinkedBlockingQueue[FetchResult]
-
-  // A request to fetch one or more blocks, complete with their sizes
-  class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) {
-    val size = blocks.map(_._2).sum
-  }
+  def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv, blockManagerMaster: BlockManagerMaster = null): HashMap[String, List[String]] = {
+    // env == null and blockManagerMaster != null is used in tests
+    assert (env != null || blockManagerMaster != null)
+    val locationBlockIds: Seq[Seq[BlockManagerId]] =
+      if (env != null) {
+        val blockManager = env.blockManager
+        blockManager.getLocationBlockIds(blockIds)
+      } else {
+        blockManagerMaster.getLocations(blockIds)
+      }
 
-  // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that
-  // the number of bytes in flight is limited to maxBytesInFlight
-  val fetchRequests = new Queue[FetchRequest]
+    // Convert from block master locations to executor locations (we need that for task scheduling)
+    val executorLocations = new HashMap[String, List[String]]()
+    for (i <- 0 until blockIds.length) {
+      val blockId = blockIds(i)
+      val blockLocations = locationBlockIds(i)
 
-  // Current bytes in flight from our requests
-  var bytesInFlight = 0L
+      val executors = new HashSet[String]()
 
-  def sendRequest(req: FetchRequest) {
-    logDebug("Sending request for %d blocks (%s) from %s".format(
-      req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip))
-    val cmId = new ConnectionManagerId(req.address.ip, req.address.port)
-    val blockMessageArray = new BlockMessageArray(req.blocks.map {
-      case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId))
-    })
-    bytesInFlight += req.size
-    val sizeMap = req.blocks.toMap  // so we can look up the size of each blockID
-    val fetchStart = System.currentTimeMillis()
-    val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
-    future.onSuccess {
-      case Some(message) => {
-        val fetchDone = System.currentTimeMillis()
-        _remoteFetchTime += fetchDone - fetchStart
-        val bufferMessage = message.asInstanceOf[BufferMessage]
-        val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage)
-        for (blockMessage <- blockMessageArray) {
-          if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) {
-            throw new SparkException(
-              "Unexpected message " + blockMessage.getType + " received from " + cmId)
-          }
-          val blockId = blockMessage.getId
-          results.put(new FetchResult(
-            blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData)))
-          _remoteBytesRead += req.size
-          logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
+      if (env != null) {
+        for (bkLocation <- blockLocations) {
+          val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host)
+          executors += executorHostPort
+          // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort)
         }
-      }
-      case None => {
-        logError("Could not get block(s) from " + cmId)
-        for ((blockId, size) <- req.blocks) {
-          results.put(new FetchResult(blockId, -1, null))
+      } else {
+        // Typically while testing, etc - revert to simply using host.
+        for (bkLocation <- blockLocations) {
+          executors += bkLocation.host
+          // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort)
         }
       }
-    }
-  }
 
-  // Split local and remote blocks. Remote blocks are further split into FetchRequests of size
-  // at most maxBytesInFlight in order to limit the amount of data in flight.
-  val remoteRequests = new ArrayBuffer[FetchRequest]
-  for ((address, blockInfos) <- blocksByAddress) {
-    if (address == blockManagerId) {
-      localBlockIds ++= blockInfos.map(_._1)
-    } else {
-      remoteBlockIds ++= blockInfos.map(_._1)
-      // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them
-      // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5
-      // nodes, rather than blocking on reading output from one node.
-      val minRequestSize = math.max(maxBytesInFlight / 5, 1L)
-      logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
-      val iterator = blockInfos.iterator
-      var curRequestSize = 0L
-      var curBlocks = new ArrayBuffer[(String, Long)]
-      while (iterator.hasNext) {
-        val (blockId, size) = iterator.next()
-        curBlocks += ((blockId, size))
-        curRequestSize += size
-        if (curRequestSize >= minRequestSize) {
-          // Add this FetchRequest
-          remoteRequests += new FetchRequest(address, curBlocks)
-          curRequestSize = 0
-          curBlocks = new ArrayBuffer[(String, Long)]
-        }
-      }
-      // Add in the final request
-      if (!curBlocks.isEmpty) {
-        remoteRequests += new FetchRequest(address, curBlocks)
-      }
+      executorLocations.put(blockId, executors.toSeq.toList)
     }
-  }
-  // Add the remote requests into our queue in a random order
-  fetchRequests ++= Utils.randomize(remoteRequests)
-
-  // Send out initial requests for blocks, up to our maxBytesInFlight
-  while (!fetchRequests.isEmpty &&
-    (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
-    sendRequest(fetchRequests.dequeue())
-  }
 
-  val numGets = remoteBlockIds.size - fetchRequests.size
-  logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime))
-
-  // Get the local blocks while remote blocks are being fetched. Note that it's okay to do
-  // these all at once because they will just memory-map some files, so they won't consume
-  // any memory that might exceed our maxBytesInFlight
-  startTime = System.currentTimeMillis
-  for (id <- localBlockIds) {
-    getLocal(id) match {
-      case Some(iter) => {
-        results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight
-        logDebug("Got local block " + id)
-      }
-      case None => {
-        throw new BlockException(id, "Could not get block " + id + " from local machine")
-      }
-    }
-  }
-  logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
-
-  //an iterator that will read fetched blocks off the queue as they arrive.
-  var resultsGotten = 0
-
-  def hasNext: Boolean = resultsGotten < totalBlocks
-
-  def next(): (String, Option[Iterator[Any]]) = {
-    resultsGotten += 1
-    val startFetchWait = System.currentTimeMillis()
-    val result = results.take()
-    val stopFetchWait = System.currentTimeMillis()
-    _remoteFetchWaitTime += (stopFetchWait - startFetchWait)
-    bytesInFlight -= result.size
-    while (!fetchRequests.isEmpty &&
-      (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
-      sendRequest(fetchRequests.dequeue())
-    }
-    (result.blockId, if (result.failed) None else Some(result.deserialize()))
+    executorLocations
   }
 
-
-  //methods to profile the block fetching
-  def numLocalBlocks = localBlockIds.size
-  def numRemoteBlocks = remoteBlockIds.size
-
-  def remoteFetchTime = _remoteFetchTime
-  def remoteFetchWaitTime = _remoteFetchWaitTime
-
-  def remoteBytesRead = _remoteBytesRead
-
 }
+
diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala
index f2f1e77d41a65cb810fd58904626dccb7b5caaff..f4a2181490f9fe9aceb67e2bc7c7aeb2198b54ea 100644
--- a/core/src/main/scala/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerId.scala
@@ -2,6 +2,7 @@ package spark.storage
 
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
 import java.util.concurrent.ConcurrentHashMap
+import spark.Utils
 
 /**
  * This class represent an unique identifier for a BlockManager.
@@ -13,7 +14,7 @@ import java.util.concurrent.ConcurrentHashMap
  */
 private[spark] class BlockManagerId private (
     private var executorId_ : String,
-    private var ip_ : String,
+    private var host_ : String,
     private var port_ : Int
   ) extends Externalizable {
 
@@ -21,32 +22,45 @@ private[spark] class BlockManagerId private (
 
   def executorId: String = executorId_
 
-  def ip: String = ip_
+  if (null != host_){
+    Utils.checkHost(host_, "Expected hostname")
+    assert (port_ > 0)
+  }
+
+  def hostPort: String = {
+    // DEBUG code
+    Utils.checkHost(host)
+    assert (port > 0)
+
+    host + ":" + port
+  }
+
+  def host: String = host_
 
   def port: Int = port_
 
   override def writeExternal(out: ObjectOutput) {
     out.writeUTF(executorId_)
-    out.writeUTF(ip_)
+    out.writeUTF(host_)
     out.writeInt(port_)
   }
 
   override def readExternal(in: ObjectInput) {
     executorId_ = in.readUTF()
-    ip_ = in.readUTF()
+    host_ = in.readUTF()
     port_ = in.readInt()
   }
 
   @throws(classOf[IOException])
   private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this)
 
-  override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, ip, port)
+  override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, host, port)
 
-  override def hashCode: Int = (executorId.hashCode * 41 + ip.hashCode) * 41 + port
+  override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port
 
   override def equals(that: Any) = that match {
     case id: BlockManagerId =>
-      executorId == id.executorId && port == id.port && ip == id.ip
+      executorId == id.executorId && port == id.port && host == id.host
     case _ =>
       false
   }
@@ -55,8 +69,8 @@ private[spark] class BlockManagerId private (
 
 private[spark] object BlockManagerId {
 
-  def apply(execId: String, ip: String, port: Int) =
-    getCachedBlockManagerId(new BlockManagerId(execId, ip, port))
+  def apply(execId: String, host: String, port: Int) =
+    getCachedBlockManagerId(new BlockManagerId(execId, host, port))
 
   def apply(in: ObjectInput) = {
     val obj = new BlockManagerId()
@@ -67,11 +81,7 @@ private[spark] object BlockManagerId {
   val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]()
 
   def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = {
-    if (blockManagerIdCache.containsKey(id)) {
-      blockManagerIdCache.get(id)
-    } else {
-      blockManagerIdCache.put(id, id)
-      id
-    }
+    blockManagerIdCache.putIfAbsent(id, id)
+    blockManagerIdCache.get(id)
   }
 }
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
index 036fdc3480119307f9c01094bfd47fd5e75c06e2..ac26c168670456d7631651c6f1676f26c5805145 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
@@ -15,6 +15,7 @@ import akka.util.duration._
 
 import spark.{Logging, SparkException, Utils}
 
+
 private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
 
   val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
@@ -22,7 +23,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
 
   val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
 
-  val timeout = 10.seconds
+  val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
 
   /** Remove a dead executor from the driver actor. This is only called on the driver side. */
   def removeExecutor(execId: String) {
@@ -87,6 +88,21 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
     askDriverWithReply(RemoveBlock(blockId))
   }
 
+  /**
+   * Remove all blocks belonging to the given RDD.
+   */
+  def removeRdd(rddId: Int) {
+    val rddBlockPrefix = "rdd_" + rddId + "_"
+    // Get the list of blocks in block manager, and remove ones that are part of this RDD.
+    // The runtime complexity is linear to the number of blocks persisted in the cluster.
+    // It could be expensive if the cluster is large and has a lot of blocks persisted.
+    getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) =>
+      if (blockId.startsWith(rddBlockPrefix)) {
+        removeBlock(blockId)
+      }
+    }
+  }
+
   /**
    * Return the memory status for each block manager, in the form of a map from
    * the block manager's id to two long values. The first value is the maximum
diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
index 2830bc62975ae53599eb9f645b7cfa52e30c6572..9b64f95df80731cd8188edb5a2a159410ccda153 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
@@ -121,7 +121,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
     val toRemove = new HashSet[BlockManagerId]
     for (info <- blockManagerInfo.values) {
       if (info.lastSeenMs < minSeenTime) {
-        logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats")
+        logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " +
+          (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms")
         toRemove += info.blockManagerId
       }
     }
@@ -332,8 +333,8 @@ object BlockManagerMasterActor {
     // Mapping from block id to its status.
     private val _blocks = new JHashMap[String, BlockStatus]
 
-    logInfo("Registering block manager %s:%d with %s RAM".format(
-      blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem)))
+    logInfo("Registering block manager %s with %s RAM".format(
+      blockManagerId.hostPort, Utils.memoryBytesToString(maxMem)))
 
     def updateLastSeenMs() {
       _lastSeenMs = System.currentTimeMillis()
@@ -358,13 +359,13 @@ object BlockManagerMasterActor {
         _blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize))
         if (storageLevel.useMemory) {
           _remainingMem -= memSize
-          logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format(
-            blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize),
+          logInfo("Added %s in memory on %s (size: %s, free: %s)".format(
+            blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize),
             Utils.memoryBytesToString(_remainingMem)))
         }
         if (storageLevel.useDisk) {
-          logInfo("Added %s on disk on %s:%d (size: %s)".format(
-            blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize)))
+          logInfo("Added %s on disk on %s (size: %s)".format(
+            blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize)))
         }
       } else if (_blocks.containsKey(blockId)) {
         // If isValid is not true, drop the block.
@@ -372,13 +373,13 @@ object BlockManagerMasterActor {
         _blocks.remove(blockId)
         if (blockStatus.storageLevel.useMemory) {
           _remainingMem += blockStatus.memSize
-          logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format(
-            blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize),
+          logInfo("Removed %s on %s in memory (size: %s, free: %s)".format(
+            blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize),
             Utils.memoryBytesToString(_remainingMem)))
         }
         if (blockStatus.storageLevel.useDisk) {
-          logInfo("Removed %s on %s:%d on disk (size: %s)".format(
-            blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize)))
+          logInfo("Removed %s on %s on disk (size: %s)".format(
+            blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize)))
         }
       }
     }
diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
index 1494f901037c9d2ff74c8922dae4113f3c047cb8..cff48d9909a161047a11cffa5056aee6ecefd751 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
@@ -49,16 +49,16 @@ class UpdateBlockInfo(
     blockManagerId.writeExternal(out)
     out.writeUTF(blockId)
     storageLevel.writeExternal(out)
-    out.writeInt(memSize.toInt)
-    out.writeInt(diskSize.toInt)
+    out.writeLong(memSize)
+    out.writeLong(diskSize)
   }
 
   override def readExternal(in: ObjectInput) {
     blockManagerId = BlockManagerId(in)
     blockId = in.readUTF()
     storageLevel = StorageLevel(in)
-    memSize = in.readInt()
-    diskSize = in.readInt()
+    memSize = in.readLong()
+    diskSize = in.readLong()
   }
 }
 
diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala
index 9e6721ec17169a8ca33753393329049467b105dd..e02281344a8f816fbc254eb0d93b2a69a73b74b2 100644
--- a/core/src/main/scala/spark/storage/BlockManagerUI.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala
@@ -1,7 +1,7 @@
 package spark.storage
 
 import akka.actor.{ActorRef, ActorSystem}
-import akka.util.Timeout
+import akka.util.Duration
 import akka.util.duration._
 import cc.spray.typeconversion.TwirlSupport._
 import cc.spray.Directives
@@ -19,20 +19,21 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef,
 
   val STATIC_RESOURCE_DIR = "spark/deploy/static"
 
-  implicit val timeout = Timeout(10 seconds)
+  implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+  val host = Utils.localHostName()
+  val port = if (System.getProperty("spark.ui.port") != null) {
+    System.getProperty("spark.ui.port").toInt
+  } else {
+    // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which
+    // random port it bound to, so we have to try to find a local one by creating a socket.
+    Utils.findFreePort()
+  }
 
   /** Start a HTTP server to run the Web interface */
   def start() {
     try {
-      val port = if (System.getProperty("spark.ui.port") != null) {
-        System.getProperty("spark.ui.port").toInt
-      } else {
-        // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which
-        // random port it bound to, so we have to try to find a local one by creating a socket.
-        Utils.findFreePort()
-      }
       AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer")
-      logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port))
+      logInfo("Started BlockManager web UI at http://%s:%d".format(host, port))
     } catch {
       case e: Exception =>
         logError("Failed to create BlockManager WebUI", e)
@@ -73,4 +74,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef,
       }
     }
   }
+
+  private[spark] def appUIAddress = "http://" + host + ":" + port
 }
diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
index d2985559c13f8aa13679f8325f2a69d1ec1f14e6..15225f93a626d05759a8fd94e4f2089a27899176 100644
--- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
@@ -19,7 +19,7 @@ import spark.network._
  */
 private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
   initLogging()
-  
+
   blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive)
 
   def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = {
@@ -51,7 +51,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
         logDebug("Received [" + pB + "]")
         putBlock(pB.id, pB.data, pB.level)
         return None
-      } 
+      }
       case BlockMessage.TYPE_GET_BLOCK => {
         val gB = new GetBlock(blockMessage.getId)
         logDebug("Received [" + gB + "]")
@@ -90,28 +90,26 @@ private[spark] object BlockManagerWorker extends Logging {
   private var blockManagerWorker: BlockManagerWorker = null
   private val DATA_TRANSFER_TIME_OUT_MS: Long = 500
   private val REQUEST_RETRY_INTERVAL_MS: Long = 1000
-  
+
   initLogging()
-  
+
   def startBlockManagerWorker(manager: BlockManager) {
     blockManagerWorker = new BlockManagerWorker(manager)
   }
-  
+
   def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = {
     val blockManager = blockManagerWorker.blockManager
-    val connectionManager = blockManager.connectionManager 
-    val serializer = blockManager.serializer
+    val connectionManager = blockManager.connectionManager
     val blockMessage = BlockMessage.fromPutBlock(msg)
     val blockMessageArray = new BlockMessageArray(blockMessage)
     val resultMessage = connectionManager.sendMessageReliablySync(
         toConnManagerId, blockMessageArray.toBufferMessage)
     return (resultMessage != None)
   }
-  
+
   def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = {
     val blockManager = blockManagerWorker.blockManager
-    val connectionManager = blockManager.connectionManager 
-    val serializer = blockManager.serializer
+    val connectionManager = blockManager.connectionManager
     val blockMessage = BlockMessage.fromGetBlock(msg)
     val blockMessageArray = new BlockMessageArray(blockMessage)
     val responseMessage = connectionManager.sendMessageReliablySync(
diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala
index a25decb123d8651a7359d81c83d2b84d58334e9e..ee0c5ff9a2d1cdd97f8158f766e1949b73ca5a83 100644
--- a/core/src/main/scala/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala
@@ -115,6 +115,7 @@ private[spark] object BlockMessageArray {
     val newBuffer = ByteBuffer.allocate(totalSize)
     newBuffer.clear()
     bufferMessage.buffers.foreach(buffer => {
+      assert (0 == buffer.position())
       newBuffer.put(buffer)
       buffer.rewind()
     })
diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala
new file mode 100644
index 0000000000000000000000000000000000000000..42e2b07d5ca5a6f2d906afcbcd5ee1ec8b56fe40
--- /dev/null
+++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala
@@ -0,0 +1,50 @@
+package spark.storage
+
+import java.nio.ByteBuffer
+
+
+/**
+ * An interface for writing JVM objects to some underlying storage. This interface allows
+ * appending data to an existing block, and can guarantee atomicity in the case of faults
+ * as it allows the caller to revert partial writes.
+ *
+ * This interface does not support concurrent writes.
+ */
+abstract class BlockObjectWriter(val blockId: String) {
+
+  var closeEventHandler: () => Unit = _
+
+  def open(): BlockObjectWriter
+
+  def close() {
+    closeEventHandler()
+  }
+
+  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.
+   */
+  def commit(): Long
+
+  /**
+   * Reverts writes that haven't been flushed yet. Callers should invoke this function
+   * when there are runtime exceptions.
+   */
+  def revertPartialWrites()
+
+  /**
+   * Writes an object.
+   */
+  def write(value: Any)
+
+  /**
+   * Size of the valid writes, in bytes.
+   */
+  def size(): Long
+}
diff --git a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala
deleted file mode 100644
index 5c491877bad1929cbda15eb414b252f99e00e382..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala
+++ /dev/null
@@ -1,12 +0,0 @@
-package spark.storage
-
-private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker {
-  var delegate : BlockFetchTracker = _
-  def setDelegate(d: BlockFetchTracker) {delegate = d}
-  def totalBlocks = delegate.totalBlocks
-  def numLocalBlocks = delegate.numLocalBlocks
-  def numRemoteBlocks = delegate.numRemoteBlocks
-  def remoteFetchTime = delegate.remoteFetchTime
-  def remoteFetchWaitTime = delegate.remoteFetchWaitTime
-  def remoteBytesRead = delegate.remoteBytesRead
-}
diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala
index ddbf8821ad15aa172cb248ef3aed45edb3e38f33..933eeaa2162c66e7c8c43fa312d8958bfd667e87 100644
--- a/core/src/main/scala/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/spark/storage/DiskStore.scala
@@ -1,41 +1,117 @@
 package spark.storage
 
+import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile}
 import java.nio.ByteBuffer
-import java.io.{File, FileOutputStream, RandomAccessFile}
+import java.nio.channels.FileChannel
 import java.nio.channels.FileChannel.MapMode
 import java.util.{Random, Date}
 import java.text.SimpleDateFormat
 
-import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
-
 import scala.collection.mutable.ArrayBuffer
 
-import spark.executor.ExecutorExitCode
+import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
 import spark.Utils
+import spark.executor.ExecutorExitCode
+import spark.serializer.{Serializer, SerializationStream}
+import spark.Logging
+import spark.network.netty.ShuffleSender
+import spark.network.netty.PathResolver
+
 
 /**
  * Stores BlockManager blocks on disk.
  */
 private class DiskStore(blockManager: BlockManager, rootDirs: String)
-  extends BlockStore(blockManager) {
+  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
+
+    override def open(): DiskBlockObjectWriter = {
+      val fos = new FileOutputStream(f, true)
+      channel = fos.getChannel()
+      bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos))
+      objOut = serializer.newInstance().serializeStream(bs)
+      this
+    }
+
+    override def close() {
+      objOut.close()
+      bs.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 = {
+      bs.flush()
+      val prevPos = lastValidPosition
+      lastValidPosition = channel.position()
+      lastValidPosition - prevPos
+    }
+
+    override def revertPartialWrites() {
+      // Discard current writes. We do this by flushing the outstanding writes and
+      // truncate the file to the last valid position.
+      bs.flush()
+      channel.truncate(lastValidPosition)
+    }
+
+    override def write(value: Any) {
+      objOut.writeObject(value)
+    }
+
+    override def size(): Long = lastValidPosition
+  }
 
   val MAX_DIR_CREATION_ATTEMPTS: Int = 10
   val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt
 
+  var shuffleSender : Thread = null
+  val thisInstance = this
   // 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.
   val localDirs = createLocalDirs()
   val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean
+
   addShutdownHook()
 
+  if(useNetty){
+    startShuffleBlockSender()
+  }
+
+  def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int)
+    : BlockObjectWriter = {
+    new DiskBlockObjectWriter(blockId, serializer, bufferSize)
+  }
+
+
   override def getSize(blockId: String): Long = {
     getFile(blockId).length()
   }
 
-  override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
+  override def putBytes(blockId: String, _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)
@@ -49,6 +125,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
       blockId, Utils.memoryBytesToString(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
+  }
+
   override def putValues(
       blockId: String,
       values: ArrayBuffer[Any],
@@ -61,18 +149,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
     val file = createFile(blockId)
     val fileOut = blockManager.wrapForCompression(blockId,
       new FastBufferedOutputStream(new FileOutputStream(file)))
-    val objOut = blockManager.serializer.newInstance().serializeStream(fileOut)
+    val objOut = blockManager.defaultSerializer.newInstance().serializeStream(fileOut)
     objOut.writeAll(values.iterator)
     objOut.close()
     val length = file.length()
+
+    val timeTaken = System.currentTimeMillis - startTime
     logDebug("Block %s stored as %s file on disk in %d ms".format(
-      blockId, Utils.memoryBytesToString(length), (System.currentTimeMillis - startTime)))
+      blockId, Utils.memoryBytesToString(length), timeTaken))
 
     if (returnValues) {
       // Return a byte buffer for the contents of the file
-      val channel = new RandomAccessFile(file, "r").getChannel()
-      val buffer = channel.map(MapMode.READ_ONLY, 0, length)
-      channel.close()
+      val buffer = getFileBytes(file)
       PutResult(length, Right(buffer))
     } else {
       PutResult(length, null)
@@ -81,10 +169,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
 
   override def getBytes(blockId: String): Option[ByteBuffer] = {
     val file = getFile(blockId)
-    val length = file.length().toInt
-    val channel = new RandomAccessFile(file, "r").getChannel()
-    val bytes = channel.map(MapMode.READ_ONLY, 0, length)
-    channel.close()
+    val bytes = getFileBytes(file)
     Some(bytes)
   }
 
@@ -92,11 +177,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
     getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes))
   }
 
+  /**
+   * 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]] = {
+    getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer))
+  }
+
   override def remove(blockId: String): Boolean = {
     val file = getFile(blockId)
     if (file.exists()) {
       file.delete()
-      true
     } else {
       false
     }
@@ -106,9 +198,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
     getFile(blockId).exists()
   }
 
-  private def createFile(blockId: String): File = {
+  private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = {
     val file = getFile(blockId)
-    if (file.exists()) {
+    if (!allowAppendExisting && file.exists()) {
       throw new Exception("File for block " + blockId + " already exists on disk: " + file)
     }
     file
@@ -156,8 +248,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
           localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
           localDir = new File(rootDir, "spark-local-" + localDirId)
           if (!localDir.exists) {
-            localDir.mkdirs()
-            foundLocalDir = true
+            foundLocalDir = localDir.mkdirs()
           }
         } catch {
           case e: Exception =>
@@ -175,15 +266,56 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
   }
 
   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")
         try {
-          localDirs.foreach(localDir => Utils.deleteRecursively(localDir))
+          localDirs.foreach { localDir =>
+            if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)
+          }
+          if (useNetty && shuffleSender != null)
+            shuffleSender.stop
         } catch {
           case t: Throwable => logError("Exception while deleting local spark dirs", t)
         }
       }
     })
   }
+
+  private def startShuffleBlockSender() {
+    try {
+      val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt
+
+      val pResolver = new PathResolver {
+        override def getAbsolutePath(blockId: String): String = {
+          if (!blockId.startsWith("shuffle_")) {
+            return null
+          }
+          thisInstance.getFile(blockId).getAbsolutePath()
+        }
+      }
+      shuffleSender = new Thread {
+        override def run() = {
+          val sender = new ShuffleSender(port, pResolver)
+          logInfo("Created ShuffleSender binding to port : "+ port)
+          sender.start
+        }
+      }
+      shuffleSender.setDaemon(true)
+      shuffleSender.start
+
+    } catch {
+      case interrupted: InterruptedException =>
+        logInfo("Runner thread for ShuffleBlockSender interrupted")
+
+      case e: Exception => {
+        logError("Error running ShuffleBlockSender ", e)
+        if (shuffleSender != null) {
+          shuffleSender.stop
+          shuffleSender = null
+        }
+      }
+    }
+  }
 }
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
index 949588476c20150b1dd5c73f4303dbf85d2ad518..eba5ee507ff0c1d8e4fcaa7aef3eda716736a4ba 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -31,7 +31,9 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     }
   }
 
-  override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
+  override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) {
+    // Work on a duplicate - since the original input might be used elsewhere.
+    val bytes = _bytes.duplicate()
     bytes.rewind()
     if (level.deserialized) {
       val values = blockManager.dataDeserialize(blockId, bytes)
diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..49eabfb0d21505616be4f72531fe074118b3a53c
--- /dev/null
+++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
@@ -0,0 +1,50 @@
+package spark.storage
+
+import spark.serializer.Serializer
+
+
+private[spark]
+class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter])
+
+
+private[spark]
+trait ShuffleBlocks {
+  def acquireWriters(mapId: Int): ShuffleWriterGroup
+  def releaseWriters(group: ShuffleWriterGroup)
+}
+
+
+private[spark]
+class ShuffleBlockManager(blockManager: BlockManager) {
+
+  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).open()
+        }
+        new ShuffleWriterGroup(mapId, writers)
+      }
+
+      override def releaseWriters(group: ShuffleWriterGroup) = {
+        // Nothing really to release here.
+      }
+    }
+  }
+}
+
+
+private[spark]
+object ShuffleBlockManager {
+
+  // Returns the block id for a given shuffle block.
+  def blockId(shuffleId: Int, bucketId: Int, groupId: Int): String = {
+    "shuffle_" + shuffleId + "_" + groupId + "_" + bucketId
+  }
+
+  // Returns true if the block is a shuffle block.
+  def isShuffle(blockId: String): Boolean = blockId.startsWith("shuffle_")
+}
diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala
index 3b5a77ab228bb2df833cd04fa6e22e8ba93dd6dd..cc0c354e7e9ea204135c77daa91e1179f63f61ac 100644
--- a/core/src/main/scala/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/spark/storage/StorageLevel.scala
@@ -123,11 +123,7 @@ object StorageLevel {
   val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]()
 
   private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = {
-    if (storageLevelCache.containsKey(level)) {
-      storageLevelCache.get(level)
-    } else {
-      storageLevelCache.put(level, level)
-      level
-    }
+    storageLevelCache.putIfAbsent(level, level)
+    storageLevelCache.get(level)
   }
 }
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
index dec47a9d4113b40dcd4fe25f95a021b3cc1f681f..8f52168c241b3893bb6f6b68b42c6e0419cfece3 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -4,9 +4,9 @@ import spark.{Utils, SparkContext}
 import BlockManagerMasterActor.BlockStatus
 
 private[spark]
-case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, 
+case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
   blocks: Map[String, BlockStatus]) {
-  
+
   def memUsed(blockPrefix: String = "") = {
     blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize).
       reduceOption(_+_).getOrElse(0l)
@@ -22,35 +22,40 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
 }
 
 case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
-  numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) {
+  numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long)
+  extends Ordered[RDDInfo] {
   override def toString = {
     import Utils.memoryBytesToString
     "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id,
       storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize))
   }
+
+  override def compare(that: RDDInfo) = {
+    this.id - that.id
+  }
 }
 
 /* Helper methods for storage-related objects */
 private[spark]
 object StorageUtils {
 
-  /* Given the current storage status of the BlockManager, returns information for each RDD */ 
-  def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], 
+  /* Given the current storage status of the BlockManager, returns information for each RDD */
+  def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus],
     sc: SparkContext) : Array[RDDInfo] = {
-    rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) 
+    rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc)
   }
 
-  /* Given a list of BlockStatus objets, returns information for each RDD */ 
-  def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], 
+  /* Given a list of BlockStatus objets, returns information for each RDD */
+  def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus],
     sc: SparkContext) : Array[RDDInfo] = {
 
     // Group by rddId, ignore the partition name
-    val groupedRddBlocks = infos.groupBy { case(k, v) =>
+    val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) =>
       k.substring(0,k.lastIndexOf('_'))
     }.mapValues(_.values.toArray)
 
     // For each RDD, generate an RDDInfo object
-    groupedRddBlocks.map { case(rddKey, rddBlocks) =>
+    val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) =>
 
       // Add up memory and disk sizes
       val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
@@ -65,10 +70,14 @@ object StorageUtils {
 
       RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize)
     }.toArray
+
+    scala.util.Sorting.quickSort(rddInfos)
+
+    rddInfos
   }
 
-  /* Removes all BlockStatus object that are not part of a block prefix */ 
-  def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], 
+  /* Removes all BlockStatus object that are not part of a block prefix */
+  def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus],
     prefix: String) : Array[StorageStatus] = {
 
     storageStatusList.map { status =>
diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index 3e805b78314c78267e86fa2c6f9a0e359637f624..9fb7e001badcbf5156a7755fa047b69764a8f586 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -11,7 +11,7 @@ import cc.spray.{SprayCanRootService, HttpService}
 import cc.spray.can.server.HttpServer
 import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler
 import akka.dispatch.Await
-import spark.SparkException
+import spark.{Utils, SparkException}
 import java.util.concurrent.TimeoutException
 
 /**
@@ -31,7 +31,10 @@ private[spark] object AkkaUtils {
     val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
     val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt
     val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
-    val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
+    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 akkaConf = ConfigFactory.parseString("""
       akka.daemonic = on
       akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
@@ -45,8 +48,9 @@ private[spark] object AkkaUtils {
       akka.remote.netty.execution-pool-size = %d
       akka.actor.default-dispatcher.throughput = %d
       akka.remote.log-remote-lifecycle-events = %s
+      akka.remote.netty.write-timeout = %ds
       """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
-                 if (lifecycleEvents) "on" else "off"))
+        lifecycleEvents, akkaWriteTimeout))
 
     val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader)
 
@@ -60,8 +64,9 @@ private[spark] object AkkaUtils {
   /**
    * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to
    * handle requests. Returns the bound port or throws a SparkException on failure.
+   * TODO: Not changing ip to host here - is it required ?
    */
-  def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, 
+  def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route,
       name: String = "HttpServer"): ActorRef = {
     val ioWorker = new IoWorker(actorSystem).start()
     val httpService = actorSystem.actorOf(Props(new HttpService(route)))
diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..48b5018dddbdd1326a7e7ff7bda771bc44c76ce0
--- /dev/null
+++ b/core/src/main/scala/spark/util/NextIterator.scala
@@ -0,0 +1,71 @@
+package spark.util
+
+/** Provides a basic/boilerplate Iterator implementation. */
+private[spark] abstract class NextIterator[U] extends Iterator[U] {
+  
+  private var gotNext = false
+  private var nextValue: U = _
+  private var closed = false
+  protected var finished = false
+
+  /**
+   * Method for subclasses to implement to provide the next element.
+   *
+   * If no next element is available, the subclass should set `finished`
+   * to `true` and may return any value (it will be ignored).
+   *
+   * This convention is required because `null` may be a valid value,
+   * and using `Option` seems like it might create unnecessary Some/None
+   * instances, given some iterators might be called in a tight loop.
+   * 
+   * @return U, or set 'finished' when done
+   */
+  protected def getNext(): U
+
+  /**
+   * Method for subclasses to implement when all elements have been successfully
+   * iterated, and the iteration is done.
+   *
+   * <b>Note:</b> `NextIterator` cannot guarantee that `close` will be
+   * called because it has no control over what happens when an exception
+   * happens in the user code that is calling hasNext/next.
+   *
+   * Ideally you should have another try/catch, as in HadoopRDD, that
+   * ensures any resources are closed should iteration fail.
+   */
+  protected def close()
+
+  /**
+   * Calls the subclass-defined close method, but only once.
+   *
+   * Usually calling `close` multiple times should be fine, but historically
+   * there have been issues with some InputFormats throwing exceptions.
+   */
+  def closeIfNeeded() {
+    if (!closed) {
+      close()
+      closed = true
+    }
+  }
+
+  override def hasNext: Boolean = {
+    if (!finished) {
+      if (!gotNext) {
+        nextValue = getNext()
+        if (finished) {
+          closeIfNeeded()
+        }
+        gotNext = true
+      }
+    }
+    !finished
+  }
+
+  override def next(): U = {
+    if (!hasNext) {
+      throw new NoSuchElementException("End of stream")
+    }
+    gotNext = false
+    nextValue
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
index 188f8910da8d54f1b43a5a8d24fa08348a6d95cb..92dfaa6e6f3d1c8491df5ee933d634aa4199f2bc 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
@@ -3,6 +3,7 @@ package spark.util
 import java.util.concurrent.ConcurrentHashMap
 import scala.collection.JavaConversions
 import scala.collection.mutable.Map
+import spark.scheduler.MapStatus
 
 /**
  * This is a custom implementation of scala.collection.mutable.Map which stores the insertion
@@ -42,6 +43,13 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging {
     this
   }
 
+  // Should we return previous value directly or as Option ?
+  def putIfAbsent(key: A, value: B): Option[B] = {
+    val prev = internalMap.putIfAbsent(key, (value, currentTime))
+    if (prev != null) Some(prev._1) else None
+  }
+
+
   override def -= (key: A): this.type = {
     internalMap.remove(key)
     this
diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala
deleted file mode 100644
index 539b01f4ce47d3ff7237ca619d220aded7b04ee1..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/util/TimedIterator.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-package spark.util
-
-/**
- * A utility for tracking the total time an iterator takes to iterate through its elements.
- *
- * In general, this should only be used if you expect it to take a considerable amount of time
- * (eg. milliseconds) to get each element -- otherwise, the timing won't be very accurate,
- * and you are probably just adding more overhead
- */
-class TimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] {
-  private var netMillis = 0l
-  private var nElems = 0
-  def hasNext = {
-    val start = System.currentTimeMillis()
-    val r = sub.hasNext
-    val end = System.currentTimeMillis()
-    netMillis += (end - start)
-    r
-  }
-  def next = {
-    val start = System.currentTimeMillis()
-    val r = sub.next
-    val end = System.currentTimeMillis()
-    netMillis += (end - start)
-    nElems += 1
-    r
-  }
-
-  def getNetMillis = netMillis
-  def getAverageTimePerItem = netMillis / nElems.toDouble
-
-}
diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html
index 301a7e212495d5fd9454be3f8620d41e34a58e24..5e5e5de5513fbd0cfd90e8c84dc3b76f30805ea6 100644
--- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html
@@ -9,19 +9,17 @@
         <li><strong>ID:</strong> @app.id</li>
         <li><strong>Description:</strong> @app.desc.name</li>
         <li><strong>User:</strong> @app.desc.user</li>
-        <li><strong>Cores:</strong> 
-          @app.desc.cores 
-          (@app.coresGranted Granted 
-          @if(app.desc.cores == Integer.MAX_VALUE) {
-
+        <li><strong>Cores:</strong>
+          @if(app.desc.maxCores == Integer.MAX_VALUE) {
+            Unlimited (@app.coresGranted granted)
           } else {
-            , @app.coresLeft
+            @app.desc.maxCores (@app.coresGranted granted, @app.coresLeft left)
           }
-          )
         </li>
         <li><strong>Memory per Slave:</strong> @app.desc.memoryPerSlave</li>
         <li><strong>Submit Date:</strong> @app.submitDate</li>
         <li><strong>State:</strong> @app.state</li>
+        <li><strong><a href="@app.appUiUrl">Application Detail UI</a></strong></li>
       </ul>
     </div>
   </div>
diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
index d2d80fad489920d1276742cc4097b37e1a639563..21e72c7aabf7ceb17103ff9ab913bd5fba357670 100644
--- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
@@ -3,7 +3,7 @@
 <tr>
   <td>@executor.id</td>
   <td>
-    <a href="@executor.worker.webUiAddress">@executor.worker.id</href>
+    <a href="@executor.worker.webUiAddress">@executor.worker.id</a>
   </td>
   <td>@executor.cores</td>
   <td>@executor.memory</td>
diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html
index ac51a39a5199d4cd96636f602ba0a9e7ce8b23c0..b9b9f08810930a0e452e523e680bb7c44897be65 100644
--- a/core/src/main/twirl/spark/deploy/master/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/index.scala.html
@@ -2,7 +2,7 @@
 @import spark.deploy.master._
 @import spark.Utils
 
-@spark.common.html.layout(title = "Spark Master on " + state.host) {
+@spark.common.html.layout(title = "Spark Master on " + state.host + ":" + state.port) {
 
   <!-- Cluster Details -->
   <div class="row">
diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html
index be69e9bf028b0678d7f6acdaa8cb3b131cee4575..46277ca421bd22df2cc68c637ec3dd3a7a3ae620 100644
--- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html
@@ -4,7 +4,7 @@
 
 <tr>
   <td>
-    <a href="@worker.webUiAddress">@worker.id</href>
+    <a href="@worker.webUiAddress">@worker.id</a>
   </td>
   <td>@{worker.host}:@{worker.port}</td>
   <td>@worker.state</td>
diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html
index c39f769a7387f96113c3f088a88fd6d1ac19351e..0e66af9284762abebd4acc8dbb0b8edd3aa58e9a 100644
--- a/core/src/main/twirl/spark/deploy/worker/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html
@@ -1,7 +1,7 @@
 @(worker: spark.deploy.WorkerState)
 @import spark.Utils
 
-@spark.common.html.layout(title = "Spark Worker on " + worker.host) {
+@spark.common.html.layout(title = "Spark Worker on " + worker.host + ":" + worker.port) {
 
   <!-- Worker Details -->
   <div class="row">
diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html
index d54b8de4cc81394149cceac0d52b965483e95a7c..cd72a688c1ffbd71e93f66e18b143d248b814ca3 100644
--- a/core/src/main/twirl/spark/storage/worker_table.scala.html
+++ b/core/src/main/twirl/spark/storage/worker_table.scala.html
@@ -12,7 +12,7 @@
   <tbody>
     @for(status <- workersStatusList) {
       <tr>
-        <td>@(status.blockManagerId.ip + ":" + status.blockManagerId.port)</td>
+        <td>@(status.blockManagerId.host + ":" + status.blockManagerId.port)</td>
         <td>
           @(Utils.memoryBytesToString(status.memUsed(prefix)))
           (@(Utils.memoryBytesToString(status.memRemaining)) Total Available)
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
index 4104b33c8b6815ddebbe50ea595e633fb0cba46e..06a94ed24c0b9da703682d7924000ba119debb2a 100644
--- a/core/src/test/scala/spark/DistributedSuite.scala
+++ b/core/src/test/scala/spark/DistributedSuite.scala
@@ -3,8 +3,10 @@ package spark
 import network.ConnectionManagerId
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
+import org.scalatest.concurrent.Timeouts._
 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._
@@ -16,7 +18,13 @@ import scala.collection.mutable.ArrayBuffer
 import SparkContext._
 import storage.{GetBlock, BlockManagerWorker, StorageLevel}
 
-class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext {
+
+class NotSerializableClass
+class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {}
+
+
+class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
+  with LocalSparkContext {
 
   val clusterUrl = "local-cluster[2,1,512]"
 
@@ -25,6 +33,24 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     System.clearProperty("spark.storage.memoryFraction")
   }
 
+  test("task throws not serializable exception") {
+    // Ensures that executors do not crash when an exn is not serializable. If executors crash,
+    // this test will hang. Correct behavior is that executors don't crash but fail tasks
+    // and the scheduler throws a SparkException.
+
+    // numSlaves must be less than numPartitions
+    val numSlaves = 3
+    val numPartitions = 10
+
+    sc = new SparkContext("local-cluster[%s,1,512]".format(numSlaves), "test")
+    val data = sc.parallelize(1 to 100, numPartitions).
+      map(x => throw new NotSerializableExn(new NotSerializableClass))
+    intercept[SparkException] {
+      data.count()
+    }
+    resetSparkContext()
+  }
+
   test("local-cluster format") {
     sc = new SparkContext("local-cluster[2,1,512]", "test")
     assert(sc.parallelize(1 to 2, 2).count() == 2)
@@ -153,7 +179,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     val blockManager = SparkEnv.get.blockManager
     blockManager.master.getLocations(blockId).foreach(id => {
       val bytes = BlockManagerWorker.syncGetBlock(
-        GetBlock(blockId), ConnectionManagerId(id.ip, id.port))
+        GetBlock(blockId), ConnectionManagerId(id.host, id.port))
       val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList
       assert(deserialized === (1 to 100).toList)
     })
@@ -252,12 +278,35 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
       assert(data2.count === 2)
     }
   }
+
+  test("unpersist RDDs") {
+    DistributedSuite.amMaster = true
+    sc = new SparkContext("local-cluster[3,1,512]", "test")
+    val data = sc.parallelize(Seq(true, false, false, false), 4)
+    data.persist(StorageLevel.MEMORY_ONLY_2)
+    data.count
+    assert(sc.persistentRdds.isEmpty === false)
+    data.unpersist()
+    assert(sc.persistentRdds.isEmpty === true)
+
+    failAfter(Span(3000, Millis)) {
+      try {
+        while (! sc.getRDDStorageInfo.isEmpty) {
+          Thread.sleep(200)
+        }
+      } catch {
+        case _ => { Thread.sleep(10) }
+          // Do nothing. We might see exceptions because block manager
+          // is racing this thread to remove entries from the driver.
+      }
+    }
+  }
 }
 
 object DistributedSuite {
   // Indicates whether this JVM is marked for failure.
   var mark = false
-  
+
   // Set by test to remember if we are in the driver program so we can assert
   // that we are not.
   var amMaster = false
@@ -274,9 +323,9 @@ object DistributedSuite {
   // Act like an identity function, but if mark was set to true previously, fail,
   // crashing the entire JVM.
   def failOnMarkedIdentity(item: Boolean): Boolean = {
-    if (mark) { 
+    if (mark) {
       System.exit(42)
-    } 
+    }
     item
-  } 
+  }
 }
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index 26e3ab72c0c2b233c31b9f2ecf5dd5ea292af678..93bb69b41c7907cf3645e3b310d5200b1d9a3808 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -196,6 +196,28 @@ public class JavaAPISuite implements Serializable {
     Assert.assertEquals(33, sum);
   }
 
+  @Test
+  public void foldByKey() {
+    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
+      new Tuple2<Integer, Integer>(2, 1),
+      new Tuple2<Integer, Integer>(2, 1),
+      new Tuple2<Integer, Integer>(1, 1),
+      new Tuple2<Integer, Integer>(3, 2),
+      new Tuple2<Integer, Integer>(3, 1)
+    );
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0,
+      new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer a, Integer b) {
+          return a + b;
+        }
+    });
+    Assert.assertEquals(1, sums.lookup(1).get(0).intValue());
+    Assert.assertEquals(2, sums.lookup(2).get(0).intValue());
+    Assert.assertEquals(3, sums.lookup(3).get(0).intValue());
+  }
+
   @Test
   public void reduceByKey() {
     List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
@@ -610,6 +632,32 @@ public class JavaAPISuite implements Serializable {
     zipped.count();
   }
 
+  @Test
+  public void zipPartitions() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2);
+    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2);
+    FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn =
+      new FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer>() {
+        @Override
+        public Iterable<Integer> call(Iterator<Integer> i, Iterator<String> s) {
+          int sizeI = 0;
+          int sizeS = 0;
+          while (i.hasNext()) {
+            sizeI += 1;
+            i.next();
+          }
+          while (s.hasNext()) {
+            sizeS += 1;
+            s.next();
+          }
+          return Arrays.asList(sizeI, sizeS);
+        }
+      };
+
+    JavaRDD<Integer> sizes = rdd1.zipPartitions(sizesFn, rdd2);
+    Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString());
+  }
+
   @Test
   public void accumulators() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala
index ff00dd05dd7875e78f426b4e4f9b6bec20cfd079..76d5258b02b9712fae6d5a5935f6036a1a23fb44 100644
--- a/core/src/test/scala/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/spark/LocalSparkContext.scala
@@ -27,6 +27,7 @@ object LocalSparkContext {
     sc.stop()
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */
@@ -38,4 +39,4 @@ object LocalSparkContext {
     }
   }
 
-}
\ No newline at end of file
+}
diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
index 3abc584b6a177e0159ddecea2dc7e85ac87eb2cc..b5cedc0b688c2aa5759d24f5617dede93db3fbc0 100644
--- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
@@ -80,12 +80,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
   }
 
   test("remote fetch") {
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0)
+    val hostname = "localhost"
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0)
+    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 (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0)
+    val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
     val slaveTracker = new MapOutputTracker()
     slaveTracker.trackerActor = slaveSystem.actorFor(
         "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index 9739ba869b3182d0fbdc17ae3e9aaa63d859882c..a761dd77c5118f6ae2a37e2a3cd5d55a3fbb2ffe 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -2,8 +2,10 @@ package spark
 
 import scala.collection.mutable.HashMap
 import org.scalatest.FunSuite
+import org.scalatest.concurrent.Timeouts._
+import org.scalatest.time.{Span, Millis}
 import spark.SparkContext._
-import spark.rdd.{CoalescedRDD, PartitionPruningRDD}
+import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD}
 
 class RDDSuite extends FunSuite with LocalSparkContext {
 
@@ -100,6 +102,28 @@ class RDDSuite extends FunSuite with LocalSparkContext {
     assert(rdd.collect().toList === List(1, 2, 3, 4))
   }
 
+  test("unpersist RDD") {
+    sc = new SparkContext("local", "test")
+    val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache()
+    rdd.count
+    assert(sc.persistentRdds.isEmpty == false)
+    rdd.unpersist()
+    assert(sc.persistentRdds.isEmpty == true)
+
+    failAfter(Span(3000, Millis)) {
+      try {
+        while (! sc.getRDDStorageInfo.isEmpty) {
+          Thread.sleep(200)
+        }
+      } catch {
+        case e: Exception =>
+          // Do nothing. We might see exceptions because block manager
+          // is racing this thread to remove entries from the driver.
+      }
+    }
+    assert(sc.getRDDStorageInfo.isEmpty == true)
+  }
+
   test("caching with failures") {
     sc = new SparkContext("local", "test")
     val onlySplit = new Partition { override def index: Int = 0 }
@@ -123,6 +147,56 @@ class RDDSuite extends FunSuite with LocalSparkContext {
     assert(rdd.collect().toList === List(1, 2, 3, 4))
   }
 
+  test("empty RDD") {
+    sc = new SparkContext("local", "test")
+    val empty = new EmptyRDD[Int](sc)
+    assert(empty.count === 0)
+    assert(empty.collect().size === 0)
+
+    val thrown = intercept[UnsupportedOperationException]{
+      empty.reduce(_+_)
+    }
+    assert(thrown.getMessage.contains("empty"))
+
+    val emptyKv = new EmptyRDD[(Int, Int)](sc)
+    val rdd = sc.parallelize(1 to 2, 2).map(x => (x, x))
+    assert(rdd.join(emptyKv).collect().size === 0)
+    assert(rdd.rightOuterJoin(emptyKv).collect().size === 0)
+    assert(rdd.leftOuterJoin(emptyKv).collect().size === 2)
+    assert(rdd.cogroup(emptyKv).collect().size === 2)
+    assert(rdd.union(emptyKv).collect().size === 2)
+  }
+
+  test("cogrouped RDDs") {
+    sc = new SparkContext("local", "test")
+    val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2)
+    val rdd2 = sc.makeRDD(Array((1, "one1"), (1, "another one1"), (2, "two1")), 2)
+
+    // Use cogroup function
+    val cogrouped = rdd1.cogroup(rdd2).collectAsMap()
+    assert(cogrouped(1) === (Seq("one", "another one"), Seq("one1", "another one1")))
+    assert(cogrouped(2) === (Seq("two"), Seq("two1")))
+    assert(cogrouped(3) === (Seq("three"), Seq()))
+
+    // Construct CoGroupedRDD directly, with map side combine enabled
+    val cogrouped1 = new CoGroupedRDD[Int](
+      Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]),
+      new HashPartitioner(3),
+      true).collectAsMap()
+    assert(cogrouped1(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1")))
+    assert(cogrouped1(2).toSeq === Seq(Seq("two"), Seq("two1")))
+    assert(cogrouped1(3).toSeq === Seq(Seq("three"), Seq()))
+
+    // Construct CoGroupedRDD directly, with map side combine disabled
+    val cogrouped2 = new CoGroupedRDD[Int](
+      Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]),
+      new HashPartitioner(3),
+      false).collectAsMap()
+    assert(cogrouped2(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1")))
+    assert(cogrouped2(2).toSeq === Seq(Seq("two"), Seq("two1")))
+    assert(cogrouped2(3).toSeq === Seq(Seq("three"), Seq()))
+  }
+
   test("coalesced RDDs") {
     sc = new SparkContext("local", "test")
     val data = sc.parallelize(1 to 10, 10)
@@ -154,6 +228,11 @@ class RDDSuite extends FunSuite with LocalSparkContext {
     assert(coalesced4.collect().toList === (1 to 10).toList)
     assert(coalesced4.glom().collect().map(_.toList).toList ===
       (1 to 10).map(x => List(x)).toList)
+
+    // we can optionally shuffle to keep the upstream parallel
+    val coalesced5 = data.coalesce(1, shuffle = true)
+    assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] !=
+      null)
   }
 
   test("zipped RDDs") {
@@ -178,4 +257,64 @@ class RDDSuite extends FunSuite with LocalSparkContext {
     assert(prunedData.size === 1)
     assert(prunedData(0) === 10)
   }
+
+  test("mapWith") {
+    import java.util.Random
+    sc = new SparkContext("local", "test")
+    val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2)
+    val randoms = ones.mapWith(
+      (index: Int) => new Random(index + 42))
+      {(t: Int, prng: Random) => prng.nextDouble * t}.collect()
+    val prn42_3 = {
+      val prng42 = new Random(42)
+      prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble()
+    }
+    val prn43_3 = {
+      val prng43 = new Random(43)
+      prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble()
+    }
+    assert(randoms(2) === prn42_3)
+    assert(randoms(5) === prn43_3)
+  }
+
+  test("flatMapWith") {
+    import java.util.Random
+    sc = new SparkContext("local", "test")
+    val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2)
+    val randoms = ones.flatMapWith(
+      (index: Int) => new Random(index + 42))
+      {(t: Int, prng: Random) =>
+        val random = prng.nextDouble()
+        Seq(random * t, random * t * 10)}.
+      collect()
+    val prn42_3 = {
+      val prng42 = new Random(42)
+      prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble()
+    }
+    val prn43_3 = {
+      val prng43 = new Random(43)
+      prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble()
+    }
+    assert(randoms(5) === prn42_3 * 10)
+    assert(randoms(11) === prn43_3 * 10)
+  }
+
+  test("filterWith") {
+    import java.util.Random
+    sc = new SparkContext("local", "test")
+    val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2)
+    val sample = ints.filterWith(
+      (index: Int) => new Random(index + 42))
+      {(t: Int, prng: Random) => prng.nextInt(3) == 0}.
+      collect()
+    val checkSample = {
+      val prng42 = new Random(42)
+      val prng43 = new Random(43)
+      Array(1, 2, 3, 4, 5, 6).filter{i =>
+	      if (i < 4) 0 == prng42.nextInt(3)
+	      else 0 == prng43.nextInt(3)}
+    }
+    assert(sample.size === checkSample.size)
+    for (i <- 0 until sample.size) assert(sample(i) === checkSample(i))
+  }
 }
diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..bfaffa953e328764f68c4c28baf276e3543a1a62
--- /dev/null
+++ b/core/src/test/scala/spark/ShuffleNettySuite.scala
@@ -0,0 +1,17 @@
+package spark
+
+import org.scalatest.BeforeAndAfterAll
+
+
+class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll {
+
+  // This test suite should run all tests in ShuffleSuite with Netty shuffle mode.
+
+  override def beforeAll(configMap: Map[String, Any]) {
+    System.setProperty("spark.shuffle.use.netty", "true")
+  }
+
+  override def afterAll(configMap: Map[String, Any]) {
+    System.setProperty("spark.shuffle.use.netty", "false")
+  }
+}
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 8411291b2caa31e86f58bbf17f39fbf68020a669..2b2a90defa4e902a8db7fb5ab2bc13da411b5913 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -272,13 +272,39 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
     }
     // partitionBy so we have a narrow dependency
     val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p)
-    println(sc.runJob(a, (i: Iterator[(Int, String)]) => i.toList).toList)
     // more partitions/no partitioner so a shuffle dependency 
     val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4)
     val c = a.subtract(b)
     assert(c.collect().toSet === Set((1, "a"), (3, "c")))
+    // Ideally we could keep the original partitioner...
+    assert(c.partitioner === None)
+  }
+
+  test("subtractByKey") {
+    sc = new SparkContext("local", "test")
+    val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2)
+    val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4)
+    val c = a.subtractByKey(b)
+    assert(c.collect().toSet === Set((1, "a"), (1, "a")))
+    assert(c.partitions.size === a.partitions.size)
+  }
+
+  test("subtractByKey with narrow dependency") {
+    sc = new SparkContext("local", "test")
+    // use a deterministic partitioner
+    val p = new Partitioner() {
+      def numPartitions = 5
+      def getPartition(key: Any) = key.asInstanceOf[Int]
+    }
+    // partitionBy so we have a narrow dependency
+    val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p)
+    // more partitions/no partitioner so a shuffle dependency 
+    val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4)
+    val c = a.subtractByKey(b)
+    assert(c.collect().toSet === Set((1, "a"), (1, "a")))
     assert(c.partitioner.get === p)
   }
+
 }
 
 object ShuffleSuite {
diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5f60aa75d7f0334a4d99662e04abc9d785c47b4d
--- /dev/null
+++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala
@@ -0,0 +1,34 @@
+package spark
+
+import scala.collection.immutable.NumericRange
+
+import org.scalatest.FunSuite
+import org.scalatest.prop.Checkers
+import org.scalacheck.Arbitrary._
+import org.scalacheck.Gen
+import org.scalacheck.Prop._
+
+import SparkContext._
+
+
+object ZippedPartitionsSuite {
+  def procZippedData(i: Iterator[Int], s: Iterator[String], d: Iterator[Double]) : Iterator[Int] = {
+    Iterator(i.toArray.size, s.toArray.size, d.toArray.size)
+  }
+}
+
+class ZippedPartitionsSuite extends FunSuite with LocalSparkContext {
+  test("print sizes") {
+    sc = new SparkContext("local", "test")
+    val data1 = sc.makeRDD(Array(1, 2, 3, 4), 2)
+    val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2)
+    val data3 = sc.makeRDD(Array(1.0, 2.0), 2)
+
+    val zippedRDD = data1.zipPartitions(ZippedPartitionsSuite.procZippedData, data2, data3)
+
+    val obtainedSizes = zippedRDD.collect()
+    val expectedSizes = Array(2, 3, 1, 2, 3, 1)
+    assert(obtainedSizes.size == 6)
+    assert(obtainedSizes.zip(expectedSizes).forall(x => x._1 == x._2))
+  }
+}
diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6afb0fa9bc77f6663939d9be2cc2ad8ae6c58b43
--- /dev/null
+++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
@@ -0,0 +1,56 @@
+package spark
+
+import org.scalatest.{ BeforeAndAfter, FunSuite }
+import spark.SparkContext._
+import spark.rdd.JdbcRDD
+import java.sql._
+
+class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
+
+  before {
+    Class.forName("org.apache.derby.jdbc.EmbeddedDriver")
+    val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true")
+    try {
+      val create = conn.createStatement
+      create.execute("""
+        CREATE TABLE FOO(
+          ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1),
+          DATA INTEGER
+        )""")
+      create.close
+      val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)")
+      (1 to 100).foreach { i =>
+        insert.setInt(1, i * 2)
+        insert.executeUpdate
+      }
+      insert.close
+    } catch {
+      case e: SQLException if e.getSQLState == "X0Y32" =>
+        // table exists
+    } finally {
+      conn.close
+    }
+  }
+
+  test("basic functionality") {
+    sc = new SparkContext("local", "test")
+    val rdd = new JdbcRDD(
+      sc,
+      () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") },
+      "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?",
+      1, 100, 3,
+      (r: ResultSet) => { r.getInt(1) } ).cache
+
+    assert(rdd.count === 100)
+    assert(rdd.reduce(_+_) === 10100)
+  }
+
+  after {
+    try {
+      DriverManager.getConnection("jdbc:derby:;shutdown=true")
+    } catch {
+      case se: SQLException if se.getSQLState == "XJ015" =>
+        // normal shutdown
+    }
+  }
+}
diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
index 956cc7421c5aac83f100521d851772469a26f0a7..7af749fb5cc34215752f8f7253b24f333ea9b7c6 100644
--- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
@@ -55,10 +55,10 @@ class DummyTaskSetManager(
   override def executorLost(executorId: String, host: String): Unit = {
   }
 
-  override def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = {
+  override def slaveOffer(execId: String, host: String, avaiableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = {
     if (tasksFinished + runningTasks < numTasks) {
       increaseRunningTasks(1)
-      return Some(new TaskDescription(0, stageId.toString, execId, "task 0:0", null))
+      return Some(new TaskDescription(0, execId, "task 0:0", null))
     }
     return None
   }
@@ -113,7 +113,7 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter {
     {
       taskSet.slaveOffer("execId_1", "hostname_1", 1) match {
         case Some(task) =>
-          return task.taskSetId.toInt
+          return taskSet.stageId
         case None => {}
       }
     }
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 6da58a0f6e5fe3beeb6117382960e3c57ab679e9..16554eac6e684e7ca787c55ab8037438d3b8c46c 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -271,7 +271,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     // have the 2nd attempt pass
     complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1))))
     // we can see both result blocks now
-    assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.ip) === Array("hostA", "hostB"))
+    assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB"))
     complete(taskSets(3), Seq((Success, 43)))
     assert(results === Map(0 -> 42, 1 -> 43))
   }
@@ -385,12 +385,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     assert(results === Map(0 -> 42))
   }
 
-  /** Assert that the supplied TaskSet has exactly the given preferredLocations. */
+  /** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */
   private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) {
     assert(locations.size === taskSet.tasks.size)
     for ((expectLocs, taskLocs) <-
             taskSet.tasks.map(_.preferredLocations).zip(locations)) {
-      assert(expectLocs === taskLocs)
+      assert(expectLocs.map(loc => spark.Utils.parseHostPort(loc)._1) === taskLocs)
     }
   }
 
diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..42a87d8b90fe57c111d4d438c8f033207d0fedbd
--- /dev/null
+++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
@@ -0,0 +1,85 @@
+package spark.scheduler
+
+import org.scalatest.FunSuite
+import spark.{SparkContext, LocalSparkContext}
+import scala.collection.mutable
+import org.scalatest.matchers.ShouldMatchers
+import spark.SparkContext._
+
+/**
+ *
+ */
+
+class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
+
+  test("local metrics") {
+    sc = new SparkContext("local[4]", "test")
+    val listener = new SaveStageInfo
+    sc.addSparkListener(listener)
+    sc.addSparkListener(new StatsReportListener)
+    //just to make sure some of the tasks take a noticeable amount of time
+    val w = {i:Int =>
+      if (i == 0)
+        Thread.sleep(100)
+      i
+    }
+
+    val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)}
+    d.count
+    listener.stageInfos.size should be (1)
+
+    val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1")
+
+    val d3 = d.map{i => w(i) -> (0 to (i % 5))}.setName("shuffle input 2")
+
+    val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)}
+    d4.setName("A Cogroup")
+
+    d4.collectAsMap
+
+    listener.stageInfos.size should be (4)
+    listener.stageInfos.foreach {stageInfo =>
+      //small test, so some tasks might take less than 1 millisecond, but average should be greater than 1 ms
+      checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration")
+      checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, stageInfo + " executorRunTime")
+      checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, stageInfo + " executorDeserializeTime")
+      if (stageInfo.stage.rdd.name == d4.name) {
+        checkNonZeroAvg(stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime")
+      }
+
+        stageInfo.taskInfos.foreach{case (taskInfo, taskMetrics) =>
+        taskMetrics.resultSize should be > (0l)
+        if (isStage(stageInfo, Set(d2.name, d3.name), Set(d4.name))) {
+          taskMetrics.shuffleWriteMetrics should be ('defined)
+          taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l)
+        }
+        if (stageInfo.stage.rdd.name == d4.name) {
+          taskMetrics.shuffleReadMetrics should be ('defined)
+          val sm = taskMetrics.shuffleReadMetrics.get
+          sm.totalBlocksFetched should be > (0)
+          sm.localBlocksFetched should be > (0)
+          sm.remoteBlocksFetched should be (0)
+          sm.remoteBytesRead should be (0l)
+          sm.remoteFetchTime should be (0l)
+        }
+      }
+    }
+  }
+
+  def checkNonZeroAvg(m: Traversable[Long], msg: String) {
+    assert(m.sum / m.size.toDouble > 0.0, msg)
+  }
+
+  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
+  }
+
+  class SaveStageInfo extends SparkListener {
+    val stageInfos = mutable.Buffer[StageInfo]()
+    def onStageCompleted(stage: StageCompleted) {
+      stageInfos += stage.stageInfo
+    }
+  }
+
+}
diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
index b8c0f6fb763a9d58251988d12708c7f7776f7454..71d1f0bcc83951f95cf9e5f20f0c94db106c3b08 100644
--- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
@@ -15,6 +15,8 @@ import org.scalatest.time.SpanSugar._
 import spark.JavaSerializer
 import spark.KryoSerializer
 import spark.SizeEstimator
+import spark.Utils
+import spark.util.AkkaUtils
 import spark.util.ByteBufferInputStream
 
 class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
@@ -31,7 +33,11 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   val serializer = new KryoSerializer
 
   before {
-    actorSystem = ActorSystem("test")
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0)
+    this.actorSystem = actorSystem
+    System.setProperty("spark.driver.port", boundPort.toString)
+    System.setProperty("spark.hostPort", "localhost:" + boundPort)
+
     master = new BlockManagerMaster(
       actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true))))
 
@@ -41,9 +47,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true")
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
+    // Set some value ...
+    System.setProperty("spark.hostPort", spark.Utils.localHostName() + ":" + 1111)
   }
 
   after {
+    System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
+
     if (store != null) {
       store.stop()
       store = null
@@ -198,6 +209,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     }
   }
 
+  test("removing rdd") {
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    val a1 = new Array[Byte](400)
+    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("nonrddblock", a3, StorageLevel.MEMORY_ONLY)
+    master.removeRdd(0)
+
+    eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
+      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
+    }
+    eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
+      store.getSingle("nonrddblock") should not be (None)
+      master.getLocations("nonrddblock") should have size (1)
+    }
+  }
+
   test("reregistration on heart beat") {
     val heartBeat = PrivateMethod[Unit]('heartBeat)
     store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ed5b36da73fd16dcb8844bfcc68e5728f7406355
--- /dev/null
+++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala
@@ -0,0 +1,68 @@
+package spark.util
+
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+import scala.collection.mutable.Buffer
+import java.util.NoSuchElementException
+
+class NextIteratorSuite extends FunSuite with ShouldMatchers {
+  test("one iteration") {
+    val i = new StubIterator(Buffer(1))
+    i.hasNext should be === true
+    i.next should be === 1
+    i.hasNext should be === false
+    intercept[NoSuchElementException] { i.next() }
+  }
+  
+  test("two iterations") {
+    val i = new StubIterator(Buffer(1, 2))
+    i.hasNext should be === true
+    i.next should be === 1
+    i.hasNext should be === true
+    i.next should be === 2
+    i.hasNext should be === false
+    intercept[NoSuchElementException] { i.next() }
+  }
+
+  test("empty iteration") {
+    val i = new StubIterator(Buffer())
+    i.hasNext should be === false
+    intercept[NoSuchElementException] { i.next() }
+  }
+
+  test("close is called once for empty iterations") {
+    val i = new StubIterator(Buffer())
+    i.hasNext should be === false
+    i.hasNext should be === false
+    i.closeCalled should be === 1
+  }
+
+  test("close is called once for non-empty iterations") {
+    val i = new StubIterator(Buffer(1, 2))
+    i.next should be === 1
+    i.next should be === 2
+    // close isn't called until we check for the next element
+    i.closeCalled should be === 0
+    i.hasNext should be === false
+    i.closeCalled should be === 1
+    i.hasNext should be === false
+    i.closeCalled should be === 1
+  }
+
+  class StubIterator(ints: Buffer[Int])  extends NextIterator[Int] {
+    var closeCalled = 0
+    
+    override def getNext() = {
+      if (ints.size == 0) {
+        finished = true
+        0
+      } else {
+        ints.remove(0)
+      }
+    }
+
+    override def close() {
+      closeCalled += 1
+    }
+  }
+}
diff --git a/docs/_config.yml b/docs/_config.yml
index f99d5bb376027823a57749e6e5e07b4ac04a69f7..5c135a024215b0bfc697a938a592f7518f121428 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -3,8 +3,8 @@ markdown: kramdown
 
 # These allow the documentation to be updated with nerw releases
 # of Spark, Scala, and Mesos.
-SPARK_VERSION: 0.7.1-SNAPSHOT
-SPARK_VERSION_SHORT: 0.7.1
-SCALA_VERSION: 2.9.2
+SPARK_VERSION: 0.8.0-SNAPSHOT
+SPARK_VERSION_SHORT: 0.8.0
+SCALA_VERSION: 2.9.3
 MESOS_VERSION: 0.9.0-incubating
 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 280ead03232adaa7da876b8016d0e779ef5c8a33..f06ab2d5b08ccf70cbca7c6f7a7ba34d0e0b9945 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -90,6 +90,7 @@
                         <li class="dropdown">
                             <a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a>
                             <ul class="dropdown-menu">
+                                <li><a href="building-with-maven.html">Building Spark with Maven</a></li>
                                 <li><a href="configuration.html">Configuration</a></li>
                                 <li><a href="tuning.html">Tuning Guide</a></li>
                                 <li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index d77e53963c2f7594e5857103bf062c810b861023..c10ae595de02fed60c0bab5d6ad887b796feb86f 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -18,7 +18,7 @@ if ENV['SKIP_API'] != '1'
   # Copy over the scaladoc from each project into the docs directory.
   # This directory will be copied over to _site when `jekyll` command is run.
   projects.each do |project_name|
-    source = "../" + project_name + "/target/scala-2.9.2/api"
+    source = "../" + project_name + "/target/scala-2.9.3/api"
     dest = "api/" + project_name
 
     puts "echo making directory " + dest
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
new file mode 100644
index 0000000000000000000000000000000000000000..04cd79d039ec1b645f41400b7bf86d0cfa80475f
--- /dev/null
+++ b/docs/building-with-maven.md
@@ -0,0 +1,66 @@
+---
+layout: global
+title: Building Spark with Maven
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer.
+
+Building with Maven requires that a Hadoop profile be specified explicitly at the command line, there is no default. There are two profiles to choose from, one for building for Hadoop 1 or Hadoop 2.
+
+for Hadoop 1 (using 0.20.205.0) use:
+
+    $ mvn -Phadoop1 clean install
+
+
+for Hadoop 2 (using 2.0.0-mr1-cdh4.1.1) use:
+
+    $ mvn -Phadoop2 clean install
+
+It uses the scala-maven-plugin which supports incremental and continuous compilation. E.g.
+
+    $ mvn -Phadoop2 scala:cc
+
+…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively.
+
+## Spark Tests in Maven ##
+
+Tests are run by default via the scalatest-maven-plugin. With this you can do things like:
+
+Skip test execution (but not compilation):
+
+    $ mvn -DskipTests -Phadoop2 clean install
+
+To run a specific test suite:
+
+    $ mvn -Phadoop2 -Dsuites=spark.repl.ReplSuite test
+
+
+## Setting up JVM Memory Usage Via Maven ##
+
+You might run into the following errors if you're using a vanilla installation of Maven:
+
+    [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes...
+    [ERROR] PermGen space -> [Help 1]
+
+    [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes...
+    [ERROR] Java heap space -> [Help 1]
+
+To fix these, you can do the following:
+
+    export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=128M"
+
+
+## Using With IntelliJ IDEA ##
+
+This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this.
+
+## Building Spark Debian Packages ##
+
+It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the deb profile:
+
+    $ mvn -Phadoop2,deb clean install
+
+The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions.
diff --git a/docs/index.md b/docs/index.md
index 45facd8e63f32494edb5759f2b2f8eb46421c17c..0c4add45dcd2adb6a5b4783a670afb985a6b8ce0 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -18,10 +18,12 @@ or you will need to set the `SCALA_HOME` environment variable to point
 to where you've installed Scala. Scala must also be accessible through one
 of these methods on slave nodes on your cluster.
 
-Spark uses [Simple Build Tool](https://github.com/harrah/xsbt/wiki), which is bundled with it. To compile the code, go into the top-level Spark directory and run
+Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run
 
     sbt/sbt package
 
+Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](building-with-maven.html).
+
 # Testing the Build
 
 Spark comes with a number of sample programs in the `examples` directory.
@@ -72,6 +74,7 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
 
 **Other documents:**
 
+* [Building Spark With Maven](building-with-maven.html): Build Spark using the Maven build tool
 * [Configuration](configuration.html): customize Spark via its configuration system
 * [Tuning Guide](tuning.html): best practices to optimize performance and memory use
 * [Bagel](bagel-programming-guide.html): an implementation of Google's Pregel on Spark
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 36d024f13ac1807654611c0b01800b4d9039bf61..335643536aac959386cf8a915f5912d393a06d2f 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -53,8 +53,8 @@ scala> textFile.filter(line => line.contains("Spark")).count() // How many lines
 res3: Long = 15
 {% endhighlight %}
 
-## Transformations
-RDD transformations can be used for more complex computations. Let's say we want to find the line with the most words:
+## More On RDD Operations
+RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words:
 
 {% highlight scala %}
 scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a else b)
@@ -111,18 +111,20 @@ We'll create a very simple Spark job in Scala. So simple, in fact, that it's nam
 import spark.SparkContext
 import SparkContext._
 
-object SimpleJob extends Application {
-  val logFile = "/var/log/syslog" // Should be some file on your system
-  val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME",
-    List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar"))
-  val logData = sc.textFile(logFile, 2).cache()
-  val numAs = logData.filter(line => line.contains("a")).count()
-  val numBs = logData.filter(line => line.contains("b")).count()
-  println("Lines with a: %s, Lines with b: %s".format(numAs, numBs))
+object SimpleJob {
+  def main(args: Array[String]) {
+    val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system
+    val sc = new SparkContext("local", "Simple Job", "YOUR_SPARK_HOME",
+      List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar"))
+    val logData = sc.textFile(logFile, 2).cache()
+    val numAs = logData.filter(line => line.contains("a")).count()
+    val numBs = logData.filter(line => line.contains("b")).count()
+    println("Lines with a: %s, Lines with b: %s".format(numAs, numBs))
+  }
 }
 {% endhighlight %}
 
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This job simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
 
 This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds two repositories which host Spark dependencies:
 
@@ -154,7 +156,7 @@ $ find .
 $ sbt package
 $ sbt run
 ...
-Lines with a: 8422, Lines with b: 1836
+Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
 This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS.
@@ -171,7 +173,7 @@ import spark.api.java.function.Function;
 
 public class SimpleJob {
   public static void main(String[] args) {
-    String logFile = "/var/log/syslog"; // Should be some file on your system
+    String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system
     JavaSparkContext sc = new JavaSparkContext("local", "Simple Job",
       "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"});
     JavaRDD<String> logData = sc.textFile(logFile).cache();
@@ -189,7 +191,7 @@ public class SimpleJob {
 }
 {% endhighlight %}
 
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that like in the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide]("java-programming-guide") describes these differences in more detail.
+This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
 
 To build the job, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
 
@@ -237,7 +239,7 @@ Now, we can execute the job using Maven:
 $ mvn package
 $ mvn exec:java -Dexec.mainClass="SimpleJob"
 ...
-Lines with a: 8422, Lines with b: 1836
+Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
 This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS.
@@ -251,7 +253,7 @@ As an example, we'll create a simple Spark job, `SimpleJob.py`:
 """SimpleJob.py"""
 from pyspark import SparkContext
 
-logFile = "/var/log/syslog"  # Should be some file on your system
+logFile = "$YOUR_SPARK_HOME/README.md"  # Should be some file on your system
 sc = SparkContext("local", "Simple job")
 logData = sc.textFile(logFile).cache()
 
@@ -263,9 +265,10 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
 
 
 This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
-Like in the Scala and Java examples, we use a SparkContext to create RDDs.
+Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. 
+As with the Scala and Java examples, we use a SparkContext to create RDDs.
 We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
-For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide).
+For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
 `SimpleJob` is simple enough that we do not need to specify any code dependencies.
 
 We can run this job using the `pyspark` script:
@@ -274,7 +277,7 @@ We can run this job using the `pyspark` script:
 $ cd $SPARK_HOME
 $ ./pyspark SimpleJob.py
 ...
-Lines with a: 8422, Lines with b: 1836
+Lines with a: 46, Lines with b: 23
 {% endhighlight python %}
 
 This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS.
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 6fb81b60043f9b7ecdb8c59c25bdb673e7927c7e..c8cf8ffc35451d7a01ff961686e5c2f3d8418da3 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -4,25 +4,35 @@ title: Launching Spark on YARN
 ---
 
 Experimental support for running over a [YARN (Hadoop
-NextGen)](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html)
-cluster was added to Spark in version 0.6.0.  Because YARN depends on version
-2.0 of the Hadoop libraries, this currently requires checking out a separate
-branch of Spark, called `yarn`, which you can do as follows:
+NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html)
+cluster was added to Spark in version 0.6.0.  This was merged into master as part of 0.7 effort.
+To build spark core with YARN support, please use the hadoop2-yarn profile.
+Ex:  mvn -Phadoop2-yarn clean install
 
-    git clone git://github.com/mesos/spark
-    cd spark
-    git checkout -b yarn --track origin/yarn
+# Building spark core consolidated jar.
+
+Currently, only sbt can buid a consolidated jar which contains the entire spark code - which is required for launching jars on yarn.
+To do this via sbt - though (right now) is a manual process of enabling it in project/SparkBuild.scala.
+Please comment out the
+  HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN
+variables before the line 'For Hadoop 2 YARN support'
+Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support.
+
+Currnetly, it is a TODO to add support for maven assembly.
 
 
 # Preparations
 
-- In order to distribute Spark within the cluster, it must be packaged into a single JAR file. This can be done by running `sbt/sbt assembly`
+- Building spark core assembled jar (see above).
 - 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 package`. 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.
 
 # 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.
+
 The command to launch the YARN Client is as follows:
 
     SPARK_JAR=<SPARK_YAR_FILE> ./run spark.deploy.yarn.Client \
@@ -30,8 +40,11 @@ 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-memory <MEMORY_FOR_MASTER> \
       --worker-memory <MEMORY_PER_WORKER> \
-      --worker-cores <CORES_PER_WORKER>
+      --worker-cores <CORES_PER_WORKER> \
+      --user <hadoop_user> \
+      --queue <queue_name>
 
 For example:
 
@@ -40,8 +53,9 @@ For example:
       --class spark.examples.SparkPi \
       --args standalone \
       --num-workers 3 \
+      --master-memory 4g \
       --worker-memory 2g \
-      --worker-cores 2
+      --worker-cores 1
 
 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.
 
@@ -49,3 +63,5 @@ The above starts a YARN Client programs which periodically polls the Application
 
 - When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above.
 - YARN does not support requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
+- Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster).
+  Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed.
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 2315aadbdf17ce5dcc70d17518b275c496cee001..b0da130fcb3209af880879e217dda221afd461e5 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -67,6 +67,8 @@ The master URL passed to Spark can be in one of the following formats:
 </td></tr>
 </table>
 
+If no master URL is specified, the spark shell defaults to "local".
+
 For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details.
 
 ### Deploying Code on a Cluster
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index b30699cf3df8615b0e9360a751849aed76da3342..f5788dc46776d6709f1991195aefe2e742a35841 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -83,7 +83,7 @@ DStreams support many of the transformations available on normal Spark RDD's:
 <tr>
   <td> <b>groupByKey</b>([<i>numTasks</i>]) </td>
   <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together all the values of each key in the RDDs of the source DStream. <br />
-  <b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
+  <b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
 </td>
 </tr>
 <tr>
@@ -132,7 +132,7 @@ Spark Streaming features windowed computations, which allow you to apply transfo
   <td> <b>groupByKeyAndWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>])
   </td>
   <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together values of each key over batches in a sliding window. <br />
-<b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.</td>
+<b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.</td>
 </tr>
 <tr>
   <td> <b>reduceByKeyAndWindow</b>(<i>func</i>, <i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>]) </td>
diff --git a/docs/tuning.md b/docs/tuning.md
index 843380b9a28829898877dd092018e1b31ddabca5..32c7ab86e9991e11228ecc1f3363654027682f0f 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -49,7 +49,7 @@ Finally, to register your classes with Kryo, create a public class that extends
 {% highlight scala %}
 import com.esotericsoftware.kryo.Kryo
 
-class MyRegistrator extends KryoRegistrator {
+class MyRegistrator extends spark.KryoRegistrator {
   override def registerClasses(kryo: Kryo) {
     kryo.register(classOf[MyClass1])
     kryo.register(classOf[MyClass2])
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 571d27fde66ed21d00aacdda1f587af381e5fefb..7affe6fffc9e1969f45fe88e825c6cd0772f1eca 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -103,7 +103,7 @@ def parse_args():
     parser.print_help()
     sys.exit(1)
   (action, cluster_name) = args
-  if opts.identity_file == None and action in ['launch', 'login']:
+  if opts.identity_file == None and action in ['launch', 'login', 'start']:
     print >> stderr, ("ERROR: The -i or --identity-file argument is " +
                       "required for " + action)
     sys.exit(1)
@@ -540,11 +540,24 @@ def scp(host, opts, local_file, dest_file):
       (opts.identity_file, local_file, opts.user, host, dest_file), shell=True)
 
 
-# Run a command on a host through ssh, throwing an exception if ssh fails
+# Run a command on a host through ssh, retrying up to two times
+# and then throwing an exception if ssh continues to fail.
 def ssh(host, opts, command):
-  subprocess.check_call(
-      "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" %
-      (opts.identity_file, opts.user, host, command), shell=True)
+  tries = 0
+  while True:
+    try:
+      return subprocess.check_call(
+        "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" %
+        (opts.identity_file, opts.user, host, command), shell=True)
+    except subprocess.CalledProcessError as e:
+      if (tries > 2):
+        raise e
+      print "Error connecting to host {0}, sleeping 30".format(e)
+      time.sleep(30)
+      tries = tries + 1
+    
+    
+    
 
 
 # Gets a list of zones to launch instances in
diff --git a/examples/pom.xml b/examples/pom.xml
index 2adeec8786fb378df9b912df0c3b57feee6df141..c42d2bcdb9ed55bcae303fa8844a8827e8a07ba1 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -22,7 +22,7 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>algebird-core_2.9.2</artifactId>
-      <version>0.1.8</version>
+      <version>0.1.11</version>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
@@ -118,5 +118,48 @@
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-streaming</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2-yarn</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>
diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java
new file mode 100644
index 0000000000000000000000000000000000000000..626034eb0d4d3a59f49a45e0a71d90896a095da1
--- /dev/null
+++ b/examples/src/main/java/spark/examples/JavaKMeans.java
@@ -0,0 +1,114 @@
+package spark.examples;
+
+import scala.Tuple2;
+import spark.api.java.JavaPairRDD;
+import spark.api.java.JavaRDD;
+import spark.api.java.JavaSparkContext;
+import spark.api.java.function.Function;
+import spark.api.java.function.PairFunction;
+import spark.util.Vector;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * K-means clustering using Java API.
+ */
+public class JavaKMeans {
+
+  /** Parses numbers split by whitespace to a vector */
+  static Vector parseVector(String line) {
+    String[] splits = line.split(" ");
+    double[] data = new double[splits.length];
+    int i = 0;
+    for (String s : splits)
+      data[i] = Double.parseDouble(splits[i++]);
+    return new Vector(data);
+  }
+
+  /** Computes the vector to which the input vector is closest using squared distance */
+  static int closestPoint(Vector p, List<Vector> centers) {
+    int bestIndex = 0;
+    double closest = Double.POSITIVE_INFINITY;
+    for (int i = 0; i < centers.size(); i++) {
+      double tempDist = p.squaredDist(centers.get(i));
+      if (tempDist < closest) {
+        closest = tempDist;
+        bestIndex = i;
+      }
+    }
+    return bestIndex;
+  }
+
+  /** Computes the mean across all vectors in the input set of vectors */
+  static Vector average(List<Vector> ps) {
+    int numVectors = ps.size();
+    Vector out = new Vector(ps.get(0).elements());
+    // start from i = 1 since we already copied index 0 above
+    for (int i = 1; i < numVectors; i++) {
+      out.addInPlace(ps.get(i));
+    }
+    return out.divide(numVectors);
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length < 4) {
+      System.err.println("Usage: JavaKMeans <master> <file> <k> <convergeDist>");
+      System.exit(1);
+    }
+    JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans",
+      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+    String path = args[1];
+    int K = Integer.parseInt(args[2]);
+    double convergeDist = Double.parseDouble(args[3]);
+
+    JavaRDD<Vector> data = sc.textFile(path).map(
+      new Function<String, Vector>() {
+        @Override
+        public Vector call(String line) throws Exception {
+          return parseVector(line);
+        }
+      }
+    ).cache();
+
+    final List<Vector> centroids = data.takeSample(false, K, 42);
+
+    double tempDist;
+    do {
+      // allocate each vector to closest centroid
+      JavaPairRDD<Integer, Vector> closest = data.map(
+        new PairFunction<Vector, Integer, Vector>() {
+          @Override
+          public Tuple2<Integer, Vector> call(Vector vector) throws Exception {
+            return new Tuple2<Integer, Vector>(
+              closestPoint(vector, centroids), vector);
+          }
+        }
+      );
+
+      // group by cluster id and average the vectors within each cluster to compute centroids
+      JavaPairRDD<Integer, List<Vector>> pointsGroup = closest.groupByKey();
+      Map<Integer, Vector> newCentroids = pointsGroup.mapValues(
+        new Function<List<Vector>, Vector>() {
+          public Vector call(List<Vector> ps) throws Exception {
+            return average(ps);
+          }
+        }).collectAsMap();
+      tempDist = 0.0;
+      for (int i = 0; i < K; i++) {
+        tempDist += centroids.get(i).squaredDist(newCentroids.get(i));
+      }
+      for (Map.Entry<Integer, Vector> t: newCentroids.entrySet()) {
+        centroids.set(t.getKey(), t.getValue());
+      }
+      System.out.println("Finished iteration (delta = " + tempDist + ")");
+    } while (tempDist > convergeDist);
+
+    System.out.println("Final centers:");
+    for (Vector c : centroids)
+      System.out.println(c);
+
+    System.exit(0);
+
+  }
+}
diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java
new file mode 100644
index 0000000000000000000000000000000000000000..6b22e7120c9174ccc602482aac062383d748ebe2
--- /dev/null
+++ b/examples/src/main/java/spark/examples/JavaLogQuery.java
@@ -0,0 +1,114 @@
+package spark.examples;
+
+import com.google.common.collect.Lists;
+import scala.Tuple2;
+import scala.Tuple3;
+import spark.api.java.JavaPairRDD;
+import spark.api.java.JavaRDD;
+import spark.api.java.JavaSparkContext;
+import spark.api.java.function.Function2;
+import spark.api.java.function.PairFunction;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Executes a roll up-style query against Apache logs.
+ */
+public class JavaLogQuery {
+
+  public static List<String> exampleApacheLogs = Lists.newArrayList(
+    "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " +
+      "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " +
+      "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " +
+      ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " +
+      "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " +
+      "62.24.11.25 images.com 1358492167 - Whatup",
+    "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " +
+      "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " +
+      "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " +
+      "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR  " +
+      "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " +
+      "0 73.23.2.15 images.com 1358492557 - Whatup");
+
+  public static Pattern apacheLogRegex = Pattern.compile(
+    "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*");
+
+  /** Tracks the total query count and number of aggregate bytes for a particular group. */
+  public static class Stats implements Serializable {
+
+    private int count;
+    private int numBytes;
+
+    public Stats(int count, int numBytes) {
+      this.count = count;
+      this.numBytes = numBytes;
+    }
+    public Stats merge(Stats other) {
+      return new Stats(count + other.count, numBytes + other.numBytes);
+    }
+
+    public String toString() {
+      return String.format("bytes=%s\tn=%s", numBytes, count);
+    }
+  }
+
+  public static Tuple3<String, String, String> extractKey(String line) {
+    Matcher m = apacheLogRegex.matcher(line);
+    List<String> key = Collections.emptyList();
+    if (m.find()) {
+      String ip = m.group(1);
+      String user = m.group(3);
+      String query = m.group(5);
+      if (!user.equalsIgnoreCase("-")) {
+        return new Tuple3<String, String, String>(ip, user, query);
+      }
+    }
+    return new Tuple3<String, String, String>(null, null, null);
+  }
+
+  public static Stats extractStats(String line) {
+    Matcher m = apacheLogRegex.matcher(line);
+    if (m.find()) {
+      int bytes = Integer.parseInt(m.group(7));
+      return new Stats(1, bytes);
+    }
+    else
+      return new Stats(1, 0);
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length == 0) {
+      System.err.println("Usage: JavaLogQuery <master> [logFile]");
+      System.exit(1);
+    }
+
+    JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
+      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    JavaRDD<String> dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs);
+
+    JavaPairRDD<Tuple3<String, String, String>, Stats> extracted = dataSet.map(new PairFunction<String, Tuple3<String, String, String>, Stats>() {
+      @Override
+      public Tuple2<Tuple3<String, String, String>, Stats> call(String s) throws Exception {
+        return new Tuple2<Tuple3<String, String, String>, Stats>(extractKey(s), extractStats(s));
+      }
+    });
+
+    JavaPairRDD<Tuple3<String, String, String>, Stats> counts = extracted.reduceByKey(new Function2<Stats, Stats, Stats>() {
+      @Override
+      public Stats call(Stats stats, Stats stats2) throws Exception {
+        return stats.merge(stats2);
+      }
+    });
+
+    List<Tuple2<Tuple3<String, String, String>, Stats>> output = counts.collect();
+    for (Tuple2 t : output) {
+      System.out.println(t._1 + "\t" + t._2);
+    }
+    System.exit(0);
+  }
+}
diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java
new file mode 100644
index 0000000000000000000000000000000000000000..a15a967de85dfc505877f5a9d79ec30a481a505e
--- /dev/null
+++ b/examples/src/main/java/spark/examples/JavaSparkPi.java
@@ -0,0 +1,48 @@
+package spark.examples;
+
+import spark.api.java.JavaRDD;
+import spark.api.java.JavaSparkContext;
+import spark.api.java.function.Function;
+import spark.api.java.function.Function2;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Computes an approximation to pi */
+public class JavaSparkPi {
+
+
+  public static void main(String[] args) throws Exception {
+    if (args.length == 0) {
+      System.err.println("Usage: JavaLogQuery <master> [slices]");
+      System.exit(1);
+    }
+
+    JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
+      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2;
+    int n = 100000 * slices;
+    List<Integer> l = new ArrayList<Integer>(n);
+    for (int i = 0; i < n; i++)
+      l.add(i);
+
+    JavaRDD<Integer> dataSet = jsc.parallelize(l, slices);
+
+    int count = dataSet.map(new Function<Integer, Integer>() {
+      @Override
+      public Integer call(Integer integer) throws Exception {
+        double x = Math.random() * 2 - 1;
+        double y = Math.random() * 2 - 1;
+        return (x * x + y * y < 1) ? 1 : 0;
+      }
+    }).reduce(new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer integer, Integer integer2) throws Exception {
+        return integer + integer2;
+      }
+    });
+
+    System.out.println("Pi is roughly " + 4.0 * count / n);
+  }
+}
diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala
index b07e799cef5afcfb855d64da4858326e17c4d8e7..4849f216fb2933faa17542897a584a70d88c2fda 100644
--- a/examples/src/main/scala/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala
@@ -10,73 +10,73 @@ import scala.collection.mutable.HashSet
  * K-means clustering.
  */
 object LocalKMeans {
-	val N = 1000
-	val R = 1000   	// Scaling factor
-	val D = 10
-	val K = 10
-	val convergeDist = 0.001
-	val rand = new Random(42)
-  	
-	def generateData = {
-	    def generatePoint(i: Int) = {
-	      Vector(D, _ => rand.nextDouble * R)
-	    }
-	    Array.tabulate(N)(generatePoint)
-	  }
-	
-	def closestPoint(p: Vector, centers: HashMap[Int, Vector]): Int = {
-		var index = 0
-		var bestIndex = 0
-		var closest = Double.PositiveInfinity
-	
-		for (i <- 1 to centers.size) {
-			val vCurr = centers.get(i).get
-			val tempDist = p.squaredDist(vCurr)
-			if (tempDist < closest) {
-				closest = tempDist
-				bestIndex = i
-			}
-		}
-	
-		return bestIndex
-	}
-
-	def main(args: Array[String]) {
-	  val data = generateData
-		var points = new HashSet[Vector]
-		var kPoints = new HashMap[Int, Vector]
-		var tempDist = 1.0
-		
-		while (points.size < K) {
-			points.add(data(rand.nextInt(N)))
-		}
-		
-		val iter = points.iterator
-		for (i <- 1 to points.size) {
-			kPoints.put(i, iter.next())
-		}
-
-		println("Initial centers: " + kPoints)
-
-		while(tempDist > convergeDist) {
-			var closest = data.map (p => (closestPoint(p, kPoints), (p, 1)))
-			
-			var mappings = closest.groupBy[Int] (x => x._1)
-			
-			var pointStats = mappings.map(pair => pair._2.reduceLeft [(Int, (Vector, Int))] {case ((id1, (x1, y1)), (id2, (x2, y2))) => (id1, (x1 + x2, y1+y2))})
-			
-			var newPoints = pointStats.map {mapping => (mapping._1, mapping._2._1/mapping._2._2)}
-			
-			tempDist = 0.0
-			for (mapping <- newPoints) {
-				tempDist += kPoints.get(mapping._1).get.squaredDist(mapping._2)
-			}
-			
-			for (newP <- newPoints) {
-				kPoints.put(newP._1, newP._2)
-			}
-		}
-
-		println("Final centers: " + kPoints)
-	}
+  val N = 1000
+  val R = 1000    // Scaling factor
+  val D = 10
+  val K = 10
+  val convergeDist = 0.001
+  val rand = new Random(42)
+
+  def generateData = {
+    def generatePoint(i: Int) = {
+      Vector(D, _ => rand.nextDouble * R)
+    }
+    Array.tabulate(N)(generatePoint)
+  }
+
+  def closestPoint(p: Vector, centers: HashMap[Int, Vector]): Int = {
+    var index = 0
+    var bestIndex = 0
+    var closest = Double.PositiveInfinity
+
+    for (i <- 1 to centers.size) {
+      val vCurr = centers.get(i).get
+      val tempDist = p.squaredDist(vCurr)
+      if (tempDist < closest) {
+        closest = tempDist
+        bestIndex = i
+      }
+    }
+
+    return bestIndex
+  }
+
+  def main(args: Array[String]) {
+    val data = generateData
+    var points = new HashSet[Vector]
+    var kPoints = new HashMap[Int, Vector]
+    var tempDist = 1.0
+
+    while (points.size < K) {
+      points.add(data(rand.nextInt(N)))
+    }
+
+    val iter = points.iterator
+    for (i <- 1 to points.size) {
+      kPoints.put(i, iter.next())
+    }
+
+    println("Initial centers: " + kPoints)
+
+    while(tempDist > convergeDist) {
+      var closest = data.map (p => (closestPoint(p, kPoints), (p, 1)))
+
+      var mappings = closest.groupBy[Int] (x => x._1)
+
+      var pointStats = mappings.map(pair => pair._2.reduceLeft [(Int, (Vector, Int))] {case ((id1, (x1, y1)), (id2, (x2, y2))) => (id1, (x1 + x2, y1+y2))})
+
+      var newPoints = pointStats.map {mapping => (mapping._1, mapping._2._1/mapping._2._2)}
+
+      tempDist = 0.0
+      for (mapping <- newPoints) {
+        tempDist += kPoints.get(mapping._1).get.squaredDist(mapping._2)
+      }
+
+      for (newP <- newPoints) {
+        kPoints.put(newP._1, newP._2)
+      }
+    }
+
+    println("Final centers: " + kPoints)
+  }
 }
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
index 92cd81c48742fb7fe1e7c598512d5bd91e0dc5ee..a0aaf609186b74813f010ea419465102ecdfd0d0 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
@@ -8,7 +8,7 @@ object MultiBroadcastTest {
       System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
       System.exit(1)
     }
-    
+
     val sc = new SparkContext(args(0), "Broadcast Test",
       System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
 
@@ -19,7 +19,7 @@ object MultiBroadcastTest {
     for (i <- 0 until arr1.length) {
       arr1(i) = i
     }
-    
+
     var arr2 = new Array[Int](num)
     for (i <- 0 until arr2.length) {
       arr2(i) = i
@@ -30,7 +30,7 @@ object MultiBroadcastTest {
     sc.parallelize(1 to 10, slices).foreach {
       i => println(barr1.value.size + barr2.value.size)
     }
-    
+
     System.exit(0)
   }
 }
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
index 0d17bda004b6e2f1fc4d5b43f703dd11f97c79e8..461b84a2c66232d7a2c01e31b97b9ded377909a2 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
@@ -11,7 +11,7 @@ object SimpleSkewedGroupByTest {
         "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]")
       System.exit(1)
     }  
-    
+
     var numMappers = if (args.length > 1) args(1).toInt else 2
     var numKVPairs = if (args.length > 2) args(2).toInt else 1000
     var valSize = if (args.length > 3) args(3).toInt else 1000
@@ -20,7 +20,7 @@ object SimpleSkewedGroupByTest {
 
     val sc = new SparkContext(args(0), "GroupBy Test",
       System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    
+
     val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
       val ranGen = new Random
       var result = new Array[(Int, Array[Byte])](numKVPairs)
diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
index 83be3fc27b5b0a5c3ad2549f0b443136417d55a6..435675f9de489d65988fded440e91084a820cdf0 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
@@ -10,7 +10,7 @@ object SkewedGroupByTest {
       System.err.println("Usage: GroupByTest <master> [numMappers] [numKVPairs] [KeySize] [numReducers]")
       System.exit(1)
     }  
-    
+
     var numMappers = if (args.length > 1) args(1).toInt else 2
     var numKVPairs = if (args.length > 2) args(2).toInt else 1000
     var valSize = if (args.length > 3) args(3).toInt else 1000
@@ -18,7 +18,7 @@ object SkewedGroupByTest {
 
     val sc = new SparkContext(args(0), "GroupBy Test",
       System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    
+
     val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
       val ranGen = new Random
 
diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
index 0f42f405a058cf9f2a218004b47c66d2330810d6..3d080a02577a38cdd2eeb1cd30a5c5e7b6fed629 100644
--- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
@@ -4,6 +4,8 @@ import java.util.Random
 import scala.math.exp
 import spark.util.Vector
 import spark._
+import spark.deploy.SparkHadoopUtil
+import spark.scheduler.InputFormatInfo
 
 /**
  * Logistic regression based classification.
@@ -32,9 +34,13 @@ object SparkHdfsLR {
       System.err.println("Usage: SparkHdfsLR <master> <file> <iters>")
       System.exit(1)
     }
+    val inputPath = args(1)
+    val conf = SparkHadoopUtil.newConfiguration()
     val sc = new SparkContext(args(0), "SparkHdfsLR",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    val lines = sc.textFile(args(1))
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), 
+      InputFormatInfo.computePreferredLocations(
+          Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath))))
+    val lines = sc.textFile(inputPath)
     val points = lines.map(parsePoint _).cache()
     val ITERATIONS = args(2).toInt
 
diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala
index 7c21ea12fb72430089d0a4166c8b74fac7677277..4161c59fead2046851428f799f1ecbc07b1eedf8 100644
--- a/examples/src/main/scala/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala
@@ -64,6 +64,7 @@ object SparkKMeans {
       for (newP <- newPoints) {
         kPoints(newP._1) = newP._2
       }
+      println("Finished iteration (delta = " + tempDist + ")")
     }
 
     println("Final centers:")
diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala
index 5a31d74444f1c637b835c907c2337d09724f59e3..f598d2ff9c7cdf0565594eb36965fbdea5b27b25 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/spark/examples/SparkPi.scala
@@ -4,6 +4,7 @@ import scala.math.random
 import spark._
 import SparkContext._
 
+/** Computes an approximation to pi */
 object SparkPi {
   def main(args: Array[String]) {
     if (args.length == 0) {
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 483aae452b05ef5a0ef32903cdf53bae41e1a7cb..a9642100e3d4886acd337e82e9ca6af049197410 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -49,7 +49,7 @@ object TwitterAlgebirdCMS {
 
     val users = stream.map(status => status.getUser.getId)
 
-    val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC)
+    val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC)
     var globalCMS = cms.zero
     val mm = new MapMonoid[Long, Int]()
     var globalExact = Map[Long, Int]()
diff --git a/pom.xml b/pom.xml
index 09ad903e6e6ecc4d2706c7b293e1250fc2d578b5..eda18fdd12d333f254da20e8831855b5d8a1d585 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2,8 +2,8 @@
 <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>
   <groupId>org.spark-project</groupId>
-  <artifactId>parent</artifactId>
-  <version>0.7.1-SNAPSHOT</version>
+  <artifactId>spark-parent</artifactId>
+  <version>0.8.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Spark Project Parent POM</name>
   <url>http://spark-project.org/</url>
@@ -51,13 +51,14 @@
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
     <java.version>1.5</java.version>
-    <scala.version>2.9.2</scala.version>
+    <scala.version>2.9.3</scala.version>
     <mesos.version>0.9.0-incubating</mesos.version>
     <akka.version>2.0.3</akka.version>
     <spray.version>1.0-M2.1</spray.version>
     <spray.json.version>1.1.1</spray.json.version>
     <slf4j.version>1.6.1</slf4j.version>
     <cdh.version>4.1.2</cdh.version>
+    <log4j.version>1.2.17</log4j.version>
   </properties>
 
   <repositories>
@@ -237,7 +238,7 @@
       </dependency>
       <dependency>
         <groupId>cc.spray</groupId>
-        <artifactId>spray-json_${scala.version}</artifactId>
+        <artifactId>spray-json_2.9.2</artifactId>
         <version>${spray.json.version}</version>
       </dependency>
       <dependency>
@@ -247,7 +248,7 @@
       </dependency>
       <dependency>
         <groupId>com.github.scala-incubator.io</groupId>
-        <artifactId>scala-io-file_${scala.version}</artifactId>
+        <artifactId>scala-io-file_2.9.2</artifactId>
         <version>0.4.1</version>
       </dependency>
       <dependency>
@@ -255,6 +256,17 @@
         <artifactId>mesos</artifactId>
         <version>${mesos.version}</version>
       </dependency>
+      <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty-all</artifactId>
+        <version>4.0.0.Beta2</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.derby</groupId>
+        <artifactId>derby</artifactId>
+        <version>10.4.2.0</version>
+        <scope>test</scope>
+      </dependency>
 
       <dependency>
         <groupId>org.scala-lang</groupId>
@@ -267,10 +279,16 @@
         <version>${scala.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>log4j</groupId>
+        <artifactId>log4j</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>org.scalatest</groupId>
         <artifactId>scalatest_${scala.version}</artifactId>
-        <version>1.8</version>
+        <version>1.9.1</version>
         <scope>test</scope>
       </dependency>
       <dependency>
@@ -282,7 +300,7 @@
       <dependency>
         <groupId>org.scalacheck</groupId>
         <artifactId>scalacheck_${scala.version}</artifactId>
-        <version>1.9</version>
+        <version>1.10.0</version>
         <scope>test</scope>
       </dependency>
       <dependency>
@@ -410,8 +428,9 @@
           <configuration>
             <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
             <junitxml>.</junitxml>
-            <filereports>WDF TestSuite.txt</filereports>
+            <filereports>${project.build.directory}/SparkTestSuite.txt</filereports>
             <argLine>-Xms64m -Xmx1024m</argLine>
+            <stderr/>
           </configuration>
           <executions>
             <execution>
@@ -505,7 +524,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-
       <properties>
         <hadoop.major.version>1</hadoop.major.version>
       </properties>
@@ -551,5 +569,66 @@
         </dependencies>
       </dependencyManagement>
     </profile>
+
+    <profile>
+      <id>hadoop2-yarn</id>
+      <properties>
+        <hadoop.major.version>2</hadoop.major.version>
+        <!-- 0.23.* is same as 2.0.* - except hardened to run production jobs -->
+        <!-- <yarn.version>0.23.7</yarn.version> -->
+        <yarn.version>2.0.2-alpha</yarn.version>
+      </properties>
+
+      <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>
+          <!-- TODO: check versions, bringover from yarn branch ! -->
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <version>${yarn.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+            <version>${yarn.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-common</artifactId>
+            <version>${yarn.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-client</artifactId>
+            <version>${yarn.version}</version>
+          </dependency>
+          <!-- Specify Avro version because Kafka also has it as a dependency -->
+          <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>1.7.1.cloudera.2</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro-ipc</artifactId>
+            <version>1.7.1.cloudera.2</version>
+          </dependency>
+        </dependencies>
+      </dependencyManagement>
+    </profile>
   </profiles>
 </project>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index d44bf3b5e393755197a82d87e93cea725d084c88..0ea23b446fd31baad7011873b09c183feba169bb 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -1,3 +1,4 @@
+
 import sbt._
 import sbt.Classpaths.publishTask
 import Keys._
@@ -12,16 +13,23 @@ object SparkBuild extends Build {
   // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop.
   val HADOOP_VERSION = "1.0.4"
   val HADOOP_MAJOR_VERSION = "1"
+  val HADOOP_YARN = false
 
   // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2"
   //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1"
   //val HADOOP_MAJOR_VERSION = "2"
+  //val HADOOP_YARN = false
+
+  // For Hadoop 2 YARN support
+  //val HADOOP_VERSION = "2.0.2-alpha"
+  //val HADOOP_MAJOR_VERSION = "2"
+  //val HADOOP_YARN = true
 
   lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming)
 
   lazy val core = Project("core", file("core"), settings = coreSettings)
 
-  lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming)
+  lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core)
 
   lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
 
@@ -35,16 +43,23 @@ object SparkBuild extends Build {
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization := "org.spark-project",
-    version := "0.7.1-SNAPSHOT",
-    scalaVersion := "2.9.2",
-    scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue
+    version := "0.8.0-SNAPSHOT",
+    scalaVersion := "2.9.3",
+    scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"),
     unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath },
     retrieveManaged := true,
     retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]",
     transitiveClassifiers in Scope.GlobalScope := Seq("sources"),
     testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))),
 
-    // shared between both core and streaming.
+    // Fork new JVMs for tests and set Java options for those
+    fork := true,
+    javaOptions += "-Xmx2g",
+
+    // Only allow one test at a time, even across projects, since they run in the same JVM
+    concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
+
+    // Shared between both core and streaming.
     resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
 
     // For Sonatype publishing
@@ -92,13 +107,13 @@ object SparkBuild extends Build {
 */
 
     libraryDependencies ++= Seq(
-      "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011",
-      "org.scalatest" %% "scalatest" % "1.8" % "test",
-      "org.scalacheck" %% "scalacheck" % "1.9" % "test",
-      "com.novocode" % "junit-interface" % "0.8" % "test",
+      "io.netty" % "netty" % "3.5.3.Final",
+      "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
+      "org.scalatest" %% "scalatest" % "1.9.1" % "test",
+      "org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
+      "com.novocode" % "junit-interface" % "0.9" % "test",
       "org.easymock" % "easymock" % "3.1" % "test"
     ),
-    parallelExecution := false,
     /* Workaround for issue #206 (fixed after SBT 0.11.0) */
     watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task,
       const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) },
@@ -114,6 +129,9 @@ object SparkBuild extends Build {
 
   val slf4jVersion = "1.6.1"
 
+  val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson")
+  val excludeNetty = ExclusionRule(organization = "org.jboss.netty")
+
   def coreSettings = sharedSettings ++ Seq(
     name := "spark-core",
     resolvers ++= Seq(
@@ -128,22 +146,48 @@ object SparkBuild extends Build {
       "log4j" % "log4j" % "1.2.16",
       "org.slf4j" % "slf4j-api" % slf4jVersion,
       "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
+      "commons-daemon" % "commons-daemon" % "1.0.10",
       "com.ning" % "compress-lzf" % "0.8.4",
-      "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION,
       "asm" % "asm-all" % "3.3.1",
       "com.google.protobuf" % "protobuf-java" % "2.4.1",
       "de.javakaffee" % "kryo-serializers" % "0.22",
-      "com.typesafe.akka" % "akka-actor" % "2.0.3",
-      "com.typesafe.akka" % "akka-remote" % "2.0.3",
-      "com.typesafe.akka" % "akka-slf4j" % "2.0.3",
+      "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty),
+      "com.typesafe.akka" % "akka-remote" % "2.0.3" excludeAll(excludeNetty),
+      "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty),
       "it.unimi.dsi" % "fastutil" % "6.4.4",
       "colt" % "colt" % "1.2.0",
-      "cc.spray" % "spray-can" % "1.0-M2.1",
-      "cc.spray" % "spray-server" % "1.0-M2.1",
-      "cc.spray" %%  "spray-json" % "1.1.1",
-      "org.apache.mesos" % "mesos" % "0.9.0-incubating"
-    ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq,
-    unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") }
+      "cc.spray" % "spray-can" % "1.0-M2.1" excludeAll(excludeNetty),
+      "cc.spray" % "spray-server" % "1.0-M2.1" excludeAll(excludeNetty),
+      "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty),
+      "org.apache.mesos" % "mesos" % "0.9.0-incubating",
+      "io.netty" % "netty-all" % "4.0.0.Beta2",
+      "org.apache.derby" % "derby" % "10.4.2.0" % "test"
+    ) ++ (
+      if (HADOOP_MAJOR_VERSION == "2") {
+        if (HADOOP_YARN) {
+          Seq(
+            // Exclude rule required for all ?
+            "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty),
+            "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty),
+            "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty),
+            "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty)
+          )
+        } else {
+          Seq(
+            "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty),
+            "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty)
+          )
+        }
+      } else {
+        Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) )
+      }),
+    unmanagedSourceDirectories in Compile <+= baseDirectory{ _ /
+      ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") {
+        "src/hadoop2-yarn/scala"
+      } else {
+        "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala"
+      } )
+    }
   ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings
 
   def rootSettings = sharedSettings ++ Seq(
@@ -157,7 +201,7 @@ object SparkBuild extends Build {
 
   def examplesSettings = sharedSettings ++ Seq(
     name := "spark-examples",
-    libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8")
+    libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11")
   )
 
   def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
@@ -165,16 +209,17 @@ object SparkBuild extends Build {
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
     libraryDependencies ++= Seq(
-      "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
+      "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty),
       "com.github.sgroschupf" % "zkclient" % "0.1",
-      "org.twitter4j" % "twitter4j-stream" % "3.0.3",
-      "com.typesafe.akka" % "akka-zeromq" % "2.0.3"
+      "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
+      "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty)
     )
   ) ++ assemblySettings ++ extraAssemblySettings
 
   def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
     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 "reference.conf" => MergeStrategy.concat
       case _ => MergeStrategy.first
     }
diff --git a/project/build.properties b/project/build.properties
index d4287112c6afb76c00419432dbc7aa79945f09ee..9b860e23c51a6a794509c549acde3ce9ab007233 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1 +1 @@
-sbt.version=0.11.3
+sbt.version=0.12.3
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 4d0e696a113581cad40119265e1e86f67ec0ab9a..d4f244287236343a0d475849bb3d1ff4c3b6d152 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -4,13 +4,13 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release
 
 resolvers += "Spray Repository" at "http://repo.spray.cc/"
 
-addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3")
+addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5")
 
-addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1")
+addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1")
 
-addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0")
+addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0")
 
-addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2")
+addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1")
 
 // For Sonatype publishing
 //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns)
diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py
index 73f7f8fbafcc5d577be823691187bc51940c9a18..7f85a1008e9f440660afb1bffc37ddaaa44849a4 100644
--- a/python/examples/transitive_closure.py
+++ b/python/examples/transitive_closure.py
@@ -24,7 +24,7 @@ if __name__ == "__main__":
             "Usage: PythonTC <master> [<slices>]"
         exit(-1)
     sc = SparkContext(sys.argv[1], "PythonTC")
-    slices = sys.argv[2] if len(sys.argv) > 2 else 2
+    slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2
     tc = sc.parallelize(generateGraph(), slices).cache()
 
     # Linear transitive closure: each round grows paths by one edge,
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 172ed85fab9267b7cfffbec8c0474701c9faf23f..a9fec17a9da69fb4ce1f7153d4ee31e60a6cb2f8 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -399,7 +399,7 @@ class RDD(object):
         >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name)
         >>> from fileinput import input
         >>> from glob import glob
-        >>> ''.join(input(glob(tempFile.name + "/part-0000*")))
+        >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*"))))
         '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n'
         """
         def func(split, iterator):
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index a60028bb53d71220e00cd719522bfd345735854d..7a7280313edb7ab9f1bc5bb218198cf78305e799 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -153,6 +153,61 @@
         </dependency>
       </dependencies>
     </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <properties>
+        <classifier>hadoop2-yarn</classifier>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-bagel</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-examples</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-repl</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-client</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
     <profile>
       <id>deb</id>
       <build>
diff --git a/repl/pom.xml b/repl/pom.xml
index a1b3ccece8b3114c2c982c7b49d3fd985069fc8f..92a2020b48d0b93235d3eeb40c3a8afb9d5f98a8 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -96,13 +96,6 @@
           <classifier>hadoop1</classifier>
           <scope>runtime</scope>
         </dependency>
-        <dependency>
-          <groupId>org.spark-project</groupId>
-          <artifactId>spark-streaming</artifactId>
-          <version>${project.version}</version>
-          <classifier>hadoop1</classifier>
-          <scope>runtime</scope>
-        </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-core</artifactId>
@@ -147,21 +140,85 @@
           <classifier>hadoop2</classifier>
           <scope>runtime</scope>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro-ipc</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <properties>
+        <classifier>hadoop2-yarn</classifier>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-bagel</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-examples</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+          <scope>runtime</scope>
+        </dependency>
         <dependency>
           <groupId>org.spark-project</groupId>
           <artifactId>spark-streaming</artifactId>
           <version>${project.version}</version>
-          <classifier>hadoop2</classifier>
+          <classifier>hadoop2-yarn</classifier>
           <scope>runtime</scope>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-core</artifactId>
+          <artifactId>hadoop-client</artifactId>
           <scope>provided</scope>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
           <scope>provided</scope>
         </dependency>
         <dependency>
@@ -181,7 +238,7 @@
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-jar-plugin</artifactId>
             <configuration>
-              <classifier>hadoop2</classifier>
+              <classifier>hadoop2-yarn</classifier>
             </configuration>
           </plugin>
         </plugins>
diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala
index cd7b5128b24df21611df32598db2d2cc95b74b03..23556dbc8f3b9388c4ce99a080b6785ef844c501 100644
--- a/repl/src/main/scala/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/spark/repl/SparkILoop.scala
@@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
       ____              __  
      / __/__  ___ _____/ /__
     _\ \/ _ \/ _ `/ __/  '_/
-   /___/ .__/\_,_/_/ /_/\_\   version 0.7.1
+   /___/ .__/\_,_/_/ /_/\_\   version 0.8.0
       /_/                  
 """)
     import Properties._
diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala
index 43559b96d3a807a220053264cc3f34fb68af0c07..1c64f9b98d099d618eedffc559f4cfc7114bfa1f 100644
--- a/repl/src/test/scala/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/spark/repl/ReplSuite.scala
@@ -32,6 +32,7 @@ class ReplSuite extends FunSuite {
       interp.sparkContext.stop()
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     return out.toString
   }
   
diff --git a/run b/run
index 2c29cc4a6641cd6f11f80f5485b34ab27528006e..c0065c53f17090aef3ef0627477d6eaf9360de84 100755
--- a/run
+++ b/run
@@ -1,6 +1,6 @@
 #!/bin/bash
 
-SCALA_VERSION=2.9.2
+SCALA_VERSION=2.9.3
 
 # Figure out where the Scala framework is installed
 FWDIR="$(cd `dirname $0`; pwd)"
@@ -22,6 +22,7 @@ fi
 # values for that; it doesn't need a lot
 if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then
   SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
   SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS   # Empty by default
 fi
 
@@ -29,31 +30,32 @@ fi
 # Add java opts for master, worker, executor. The opts maybe null
 case "$1" in
   'spark.deploy.master.Master')
-    SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS"
+    SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_MASTER_OPTS"
     ;;
   'spark.deploy.worker.Worker')
-    SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS"
+    SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_WORKER_OPTS"
     ;;
   'spark.executor.StandaloneExecutorBackend')
-    SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
+    SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
     ;;
   'spark.executor.MesosExecutorBackend')
-    SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
+    SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
     ;;
   'spark.repl.Main')
-    SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS"
+    SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS"
     ;;
 esac
 
 if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
-  if [ `command -v scala` ]; then
-    RUNNER="scala"
+  if [ "$SCALA_HOME" ]; then
+    RUNNER="${SCALA_HOME}/bin/scala"
   else
-    if [ -z "$SCALA_HOME" ]; then
-      echo "SCALA_HOME is not set" >&2
+    if [ `command -v scala` ]; then
+      RUNNER="scala"
+    else
+      echo "SCALA_HOME is not set and scala is not in PATH" >&2
       exit 1
     fi
-    RUNNER="${SCALA_HOME}/bin/scala"
   fi
 else
   if [ `command -v java` ]; then
@@ -83,56 +85,62 @@ export SPARK_MEM
 
 # Set JAVA_OPTS to be able to load native libraries and to set heap size
 JAVA_OPTS="$SPARK_JAVA_OPTS"
-JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM"
+JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
 # Load extra JAVA_OPTS from conf/java-opts, if it exists
 if [ -e $FWDIR/conf/java-opts ] ; then
-  JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`"
+  JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
 fi
 export JAVA_OPTS
 
 CORE_DIR="$FWDIR/core"
 REPL_DIR="$FWDIR/repl"
+REPL_BIN_DIR="$FWDIR/repl-bin"
 EXAMPLES_DIR="$FWDIR/examples"
 BAGEL_DIR="$FWDIR/bagel"
 STREAMING_DIR="$FWDIR/streaming"
 PYSPARK_DIR="$FWDIR/python"
 
 # Exit if the user hasn't compiled Spark
-if [ ! -e "$REPL_DIR/target" ]; then
-  echo "Failed to find Spark classes in $REPL_DIR/target" >&2
+if [ ! -e "$CORE_DIR/target" ]; then
+  echo "Failed to find Spark classes in $CORE_DIR/target" >&2
   echo "You need to compile Spark before running this program" >&2
   exit 1
 fi
 
+if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then
+  echo "Failed to find Spark classes in $REPL_DIR/target" >&2
+  echo "You need to compile Spark repl module before running this program" >&2
+  exit 1
+fi
+
 # Build up classpath
 CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$FWDIR/conf"
-CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH="$CLASSPATH:$FWDIR/conf"
+CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
 if [ -n "$SPARK_TESTING" ] ; then
-  CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
-fi
-CLASSPATH+=":$CORE_DIR/src/main/resources"
-CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes"
-CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
-CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
-CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
+  CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
+fi
+CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
+CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
 if [ -e "$FWDIR/lib_managed" ]; then
-  CLASSPATH+=":$FWDIR/lib_managed/jars/*"
-  CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
+  CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
+  CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
 fi
-CLASSPATH+=":$REPL_DIR/lib/*"
-if [ -e repl-bin/target ]; then
-  for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
-    CLASSPATH+=":$jar"
+CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
+if [ -e $REPL_BIN_DIR/target ]; then
+  for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
+    CLASSPATH="$CLASSPATH:$jar"
   done
 fi
-CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
 for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
-  CLASSPATH+=":$jar"
+  CLASSPATH="$CLASSPATH:$jar"
 done
-export CLASSPATH # Needed for spark-shell
 
 # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
 # to avoid the -sources and -doc packages that are built by publish-local.
@@ -145,6 +153,17 @@ if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then
   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar`
 fi
 
+# Add hadoop conf dir - else FileSystem.*, etc fail !
+# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+# the configurtion files.
+if [ "x" != "x$HADOOP_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
+fi
+if [ "x" != "x$YARN_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
+fi
+
+
 # Figure out whether to run our class with java or with the scala launcher.
 # In most cases, we'd prefer to execute our process with java because scala
 # creates a shell script as the parent of its Java process, which makes it
@@ -154,11 +173,12 @@ fi
 if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
   EXTRA_ARGS=""     # Java options will be passed to scala as JAVA_OPTS
 else
-  CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-library.jar"
-  CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-compiler.jar"
-  CLASSPATH+=":$SCALA_LIBRARY_PATH/jline.jar"
+  CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
+  CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
+  CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
   # The JVM doesn't read JAVA_OPTS by default so we need to pass it in
   EXTRA_ARGS="$JAVA_OPTS"
 fi
 
+export CLASSPATH # Needed for spark-shell
 exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@"
diff --git a/run2.cmd b/run2.cmd
index cb20a4b7a2f9a228cf66febbba429e5d39ecb60b..c6f43dde5b260aac08ee722e69dce1dd752229c8 100644
--- a/run2.cmd
+++ b/run2.cmd
@@ -1,6 +1,6 @@
 @echo off
 
-set SCALA_VERSION=2.9.2
+set SCALA_VERSION=2.9.3
 
 rem Figure out where the Spark framework is installed
 set FWDIR=%~dp0
@@ -21,6 +21,7 @@ set RUNNING_DAEMON=0
 if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
+set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
 if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
 if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
 
@@ -62,6 +63,19 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\*
 set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\*
 set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
 
+rem Add hadoop conf dir - else FileSystem.*, etc fail
+rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+rem the configurtion files.
+if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
+  set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
+:no_hadoop_conf_dir
+
+if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
+  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
+:no_yarn_conf_dir
+
+
+
 rem Figure out the JAR file that our examples were packaged into.
 rem First search in the build path from SBT:
 for %%d in ("examples/target/scala-%SCALA_VERSION%/spark-examples*.jar") do (
diff --git a/sbt/sbt b/sbt/sbt
index 8f426d18e892facbc84fe1fe47edc8bc3a0f24ea..850c58e1e9745db4833f58748df11943c4a2b5f0 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then
 fi
 export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd)
 export SPARK_TESTING=1  # To put test classes on classpath
-java -Xmx1200M -XX:MaxPermSize=250m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"
+java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"
diff --git a/streaming/pom.xml b/streaming/pom.xml
index d1a766aeacf454d6966f46626ef550597f9fb9d3..08ff3e2ae12f48480d8d0526e6131254d127031e 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -3,8 +3,8 @@
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.spark-project</groupId>
-    <artifactId>parent</artifactId>
-    <version>0.7.1-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -149,5 +149,42 @@
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>hadoop2-yarn</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2-yarn</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <scope>provided</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2-yarn</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index e303e33e5e4014e7b252b491edc9d5090dc5e88f..66e67cbfa1d786931580a7379e6aa0b10cb80d5b 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -38,11 +38,20 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
 private[streaming]
 class CheckpointWriter(checkpointDir: String) extends Logging {
   val file = new Path(checkpointDir, "graph")
+  // The file to which we actually write - and then "move" to file.
+  private val writeFile = new Path(file.getParent, file.getName + ".next")
+  private val bakFile = new Path(file.getParent, file.getName + ".bk")
+
+  private var stopped = false
+
   val conf = new Configuration()
   var fs = file.getFileSystem(conf)
   val maxAttempts = 3
   val executor = Executors.newFixedThreadPool(1)
 
+  // Removed code which validates whether there is only one CheckpointWriter per path 'file' since 
+  // I did not notice any errors - reintroduce it ?
+
   class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
     def run() {
       var attempts = 0
@@ -51,15 +60,17 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
         attempts += 1
         try {
           logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
-          if (fs.exists(file)) {
-            val bkFile = new Path(file.getParent, file.getName + ".bk")
-            FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
-            logDebug("Moved existing checkpoint file to " + bkFile)
-          }
-          val fos = fs.create(file)
+          // This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast.
+          val fos = fs.create(writeFile)
           fos.write(bytes)
           fos.close()
-          fos.close()
+          if (fs.exists(file) && fs.rename(file, bakFile)) {
+            logDebug("Moved existing checkpoint file to " + bakFile)
+          }
+          // paranoia
+          fs.delete(file, false)
+          fs.rename(writeFile, file)
+
           val finishTime = System.currentTimeMillis();
           logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
             "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
@@ -84,7 +95,15 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
   }
 
   def stop() {
+    synchronized {
+      if (stopped) return ;
+      stopped = true
+    }
     executor.shutdown()
+    val startTime = System.currentTimeMillis()
+    val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS)
+    val endTime = System.currentTimeMillis()
+    logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.")
   }
 }
 
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
index adb7f3a24d25f6fcbd1453c2e75b56b9a22d10b4..3b331956f5973c07af145939d556baa7875a72a0 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -54,8 +54,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
         throw new Exception("Batch duration already set as " + batchDuration +
           ". cannot set it again.")
       }
+      batchDuration = duration
     }
-    batchDuration = duration
   }
 
   def remember(duration: Duration) {
diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
index 6b310bc0b611c7a6260042e3ddc555a59bed28a2..da224ad6f718cbb8a29e1e6f976923f1bfacf6d9 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
@@ -28,7 +28,7 @@ class QueueInputDStream[T: ClassManifest](
     }
     if (buffer.size > 0) {
       if (oneAtATime) {
-        Some(buffer.first)
+        Some(buffer.head)
       } else {
         Some(new UnionRDD(ssc.sc, buffer.toSeq))
       }
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
index 4af839ad7f03d1e75dd19715a2cb0785a9a91628..1408af0afa5018545fd9ec1db61df9182499d095 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
@@ -2,6 +2,7 @@ package spark.streaming.dstream
 
 import spark.streaming.StreamingContext
 import spark.storage.StorageLevel
+import spark.util.NextIterator
 
 import java.io._
 import java.net.Socket
@@ -59,45 +60,18 @@ object SocketReceiver  {
    */
   def bytesToLines(inputStream: InputStream): Iterator[String] = {
     val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8"))
-
-    val iterator = new Iterator[String] {
-      var gotNext = false
-      var finished = false
-      var nextValue: String = null
-
-      private def getNext() {
-        try {
-          nextValue = dataInputStream.readLine()
-          if (nextValue == null) {
-            finished = true
-          }
-        }
-        gotNext = true
-      }
-
-      override def hasNext: Boolean = {
-        if (!finished) {
-          if (!gotNext) {
-            getNext()
-            if (finished) {
-              dataInputStream.close()
-            }
-          }
+    new NextIterator[String] {
+      protected override def getNext() = {
+        val nextValue = dataInputStream.readLine()
+        if (nextValue == null) {
+          finished = true
         }
-        !finished
+        nextValue
       }
 
-      override def next(): String = {
-        if (finished) {
-          throw new NoSuchElementException("End of stream")
-        }
-        if (!gotNext) {
-          getNext()
-        }
-        gotNext = false
-        nextValue
+      protected override def close() {
+        dataInputStream.close()
       }
     }
-    iterator
   }
 }
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
index f673e5be15485b839d5f894912d3290eecc21637..426a9b6f71d88852aa62d4fa920018a4a344a667 100644
--- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
@@ -159,6 +159,7 @@ object MasterFailureTest extends Logging {
 
     // Setup the streaming computation with the given operation
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map())
     ssc.checkpoint(checkpointDir.toString)
     val inputStream = ssc.textFileStream(testDir.toString)
@@ -205,6 +206,7 @@ object MasterFailureTest extends Logging {
         // (iii) Its not timed out yet
         System.clearProperty("spark.streaming.clock")
         System.clearProperty("spark.driver.port")
+        System.clearProperty("spark.hostPort")
         ssc.start()
         val startTime = System.currentTimeMillis()
         while (!killed && !isLastOutputGenerated && !isTimedOut) {
@@ -357,13 +359,16 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
         // Write the data to a local file and then move it to the target test directory
         val localFile = new File(localTestDir, (i+1).toString)
         val hadoopFile = new Path(testDir, (i+1).toString)
+        val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString)
         FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
         var tries = 0
 	var done = false
         while (!done && tries < maxTries) {
           tries += 1
           try {
-            fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+            // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+            fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
+            fs.rename(tempHadoopFile, hadoopFile)
 	    done = true
 	  } catch {
 	    case ioe: IOException => { 
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
index d8b987ec860bbcd3ea4d08d8a02a7bd213c7fb9f..bd0b0e74c1d9ddc651c704e0860225144e3648b3 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
@@ -5,7 +5,7 @@ import spark.util.{RateLimitedOutputStream, IntParam}
 import java.net.ServerSocket
 import spark.{Logging, KryoSerializer}
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-import io.Source
+import scala.io.Source
 import java.io.IOException
 
 /**
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
index 8fce91853c77eed0f5d3e5e48aa2e93f13988e3e..e7352deb81da3a140710d8fb0c223e50121ed7b1 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -6,13 +6,16 @@ import util.ManualClock
 
 class BasicOperationsSuite extends TestSuiteBase {
 
-  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
-
   override def framework() = "BasicOperationsSuite"
 
+  before {
+    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+  }
+
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   test("map") {
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
index cac86deeaf3492cb298c26d9812cc82e71fd83cf..607dea77ec24b6bfd90d59571b702b4933239f91 100644
--- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -31,6 +31,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   var ssc: StreamingContext = null
@@ -325,6 +326,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     )
     ssc = new StreamingContext(checkpointDir)
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
     ssc.start()
     val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches)
     // the first element will be re-processed data of the last batch before restart
@@ -350,4 +352,4 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
     outputStream.output
   }
-}
\ No newline at end of file
+}
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
index 4d33857b25ed5c1ecf4b3fdf783db5cc61214c89..0acb6db6f2e484c213ea0bc15d291fb6f8a2dab6 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -30,15 +30,18 @@ import com.google.common.io.Files
 
 class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
 
-  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
-
   val testPort = 9999
 
   override def checkpointDir = "checkpoint"
 
+  before {
+    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+  }
+
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
 
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
index 1b66f3bda20ad0f112295e529f4a3f1419c167ed..80d827706f63c2cfdee90db2fe6f1bb5d114dd41 100644
--- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -16,6 +16,7 @@ class WindowOperationsSuite extends TestSuiteBase {
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
+    System.clearProperty("spark.hostPort")
   }
 
   val largerSlideInput = Seq(